diff --git a/.gitignore b/.gitignore index 4f177c82ae5e0..7ec8d45e12c6b 100644 --- a/.gitignore +++ b/.gitignore @@ -19,6 +19,7 @@ conf/spark-env.sh conf/streaming-env.sh conf/log4j.properties conf/spark-defaults.conf +conf/hive-site.xml docs/_site docs/api target/ @@ -50,9 +51,11 @@ unit-tests.log rat-results.txt scalastyle.txt conf/*.conf +scalastyle-output.xml # For Hive metastore_db/ metastore/ warehouse/ TempStatsStore/ +sql/hive-thriftserver/test_warehouses diff --git a/.rat-excludes b/.rat-excludes index 372bc2587ccc3..eaefef1b0aa2e 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -25,6 +25,7 @@ log4j-defaults.properties bootstrap-tooltip.js jquery-1.11.1.min.js sorttable.js +.*avsc .*txt .*json .*data @@ -55,3 +56,4 @@ dist/* .*ipr .*iws logs +.*scalastyle-output.xml diff --git a/LICENSE b/LICENSE index 383f079df8c8b..e9a1153fdc5db 100644 --- a/LICENSE +++ b/LICENSE @@ -272,7 +272,7 @@ SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. ======================================================================== -For Py4J (python/lib/py4j0.7.egg and files in assembly/lib/net/sf/py4j): +For Py4J (python/lib/py4j-0.8.2.1-src.zip) ======================================================================== Copyright (c) 2009-2011, Barthelemy Dagenais All rights reserved. @@ -442,7 +442,7 @@ Written by Pavel Binko, Dino Ferrero Merlino, Wolfgang Hoschek, Tony Johnson, An ======================================================================== -Fo SnapTree: +For SnapTree: ======================================================================== SNAPTREE LICENSE @@ -482,6 +482,24 @@ OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. +======================================================================== +For Timsort (core/src/main/java/org/apache/spark/util/collection/Sorter.java): +======================================================================== +Copyright (C) 2008 The Android Open Source Project + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. + + ======================================================================== BSD-style licenses ======================================================================== @@ -514,7 +532,7 @@ The following components are provided under a BSD-style license. See project lin (New BSD license) Protocol Buffer Java API (org.spark-project.protobuf:protobuf-java:2.4.1-shaded - http://code.google.com/p/protobuf) (The BSD License) Fortran to Java ARPACK (net.sourceforge.f2j:arpack_combined_all:0.1 - http://f2j.sourceforge.net) (The BSD License) xmlenc Library (xmlenc:xmlenc:0.52 - http://xmlenc.sourceforge.net) - (The New BSD License) Py4J (net.sf.py4j:py4j:0.8.1 - http://py4j.sourceforge.net/) + (The New BSD License) Py4J (net.sf.py4j:py4j:0.8.2.1 - http://py4j.sourceforge.net/) (Two-clause BSD-style license) JUnit-Interface (com.novocode:junit-interface:0.10 - http://github.com/szeiger/junit-interface/) (ISC/BSD License) jbcrypt (org.mindrot:jbcrypt:0.3m - http://www.mindrot.org/) @@ -531,3 +549,4 @@ The following components are provided under the MIT License. See project link fo (MIT License) pyrolite (org.spark-project:pyrolite:2.0.1 - http://pythonhosted.org/Pyro4/) (MIT License) scopt (com.github.scopt:scopt_2.10:3.2.0 - https://github.com/scopt/scopt) (The MIT License) Mockito (org.mockito:mockito-all:1.8.5 - http://www.mockito.org) + (MIT License) jquery (https://jquery.org/license/) diff --git a/README.md b/README.md index 6211a5889a3f5..a1a48f5bd0819 100644 --- a/README.md +++ b/README.md @@ -1,6 +1,13 @@ # Apache Spark -Lightning-Fast Cluster Computing - +Spark is a fast and general cluster computing system for Big Data. It provides +high-level APIs in Scala, Java, and Python, and an optimized engine that +supports general computation graphs for data analysis. It also supports a +rich set of higher-level tools including Spark SQL for SQL and structured +data processing, MLLib for machine learning, GraphX for graph processing, +and Spark Streaming. + + ## Online Documentation @@ -69,29 +76,28 @@ can be run using: Spark uses the Hadoop core library to talk to HDFS and other Hadoop-supported storage systems. Because the protocols have changed in different versions of Hadoop, you must build Spark against the same version that your cluster runs. -You can change the version by setting the `SPARK_HADOOP_VERSION` environment -when building Spark. +You can change the version by setting `-Dhadoop.version` when building Spark. For Apache Hadoop versions 1.x, Cloudera CDH MRv1, and other Hadoop versions without YARN, use: # Apache Hadoop 1.2.1 - $ SPARK_HADOOP_VERSION=1.2.1 sbt/sbt assembly + $ sbt/sbt -Dhadoop.version=1.2.1 assembly # Cloudera CDH 4.2.0 with MapReduce v1 - $ SPARK_HADOOP_VERSION=2.0.0-mr1-cdh4.2.0 sbt/sbt assembly + $ sbt/sbt -Dhadoop.version=2.0.0-mr1-cdh4.2.0 assembly For Apache Hadoop 2.2.X, 2.1.X, 2.0.X, 0.23.x, Cloudera CDH MRv2, and other Hadoop versions -with YARN, also set `SPARK_YARN=true`: +with YARN, also set `-Pyarn`: # Apache Hadoop 2.0.5-alpha - $ SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_YARN=true sbt/sbt assembly + $ sbt/sbt -Dhadoop.version=2.0.5-alpha -Pyarn assembly # Cloudera CDH 4.2.0 with MapReduce v2 - $ SPARK_HADOOP_VERSION=2.0.0-cdh4.2.0 SPARK_YARN=true sbt/sbt assembly + $ sbt/sbt -Dhadoop.version=2.0.0-cdh4.2.0 -Pyarn assembly # Apache Hadoop 2.2.X and newer - $ SPARK_HADOOP_VERSION=2.2.0 SPARK_YARN=true sbt/sbt assembly + $ sbt/sbt -Dhadoop.version=2.2.0 -Pyarn assembly When developing a Spark application, specify the Hadoop version by adding the "hadoop-client" artifact to your project's dependencies. For example, if you're @@ -109,6 +115,15 @@ If your project is built with Maven, add this to your POM file's ` +## A Note About Thrift JDBC server and CLI for Spark SQL + +Spark SQL supports Thrift JDBC server and CLI. +See sql-programming-guide.md for more information about those features. +You can use those features by setting `-Phive-thriftserver` when building Spark as follows. + + $ sbt/sbt -Phive-thriftserver assembly + + ## Configuration Please refer to the [Configuration guide](http://spark.apache.org/docs/latest/configuration.html) diff --git a/assembly/pom.xml b/assembly/pom.xml index 0c60b66c3daca..703f15925bc44 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -32,12 +32,14 @@ pom + assembly scala-${scala.binary.version} spark-assembly-${project.version}-hadoop${hadoop.version}.jar ${project.build.directory}/${spark.jar.dir}/${spark.jar.basename} spark /usr/share/spark root + 744 @@ -163,6 +165,16 @@ + + hive-thriftserver + + + org.apache.spark + spark-hive-thriftserver_${scala.binary.version} + ${project.version} + + + spark-ganglia-lgpl @@ -275,7 +287,7 @@ ${deb.user} ${deb.user} ${deb.install.path}/bin - 744 + ${deb.bin.filemode} diff --git a/bagel/pom.xml b/bagel/pom.xml index c8e39a415af28..bd51b112e26fa 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -27,6 +27,9 @@ org.apache.spark spark-bagel_2.10 + + bagel + jar Spark Project Bagel http://spark.apache.org/ diff --git a/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala b/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala index 70a99b33d753c..ef0bb2ac13f08 100644 --- a/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala +++ b/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala @@ -72,6 +72,7 @@ object Bagel extends Logging { var verts = vertices var msgs = messages var noActivity = false + var lastRDD: RDD[(K, (V, Array[M]))] = null do { logInfo("Starting superstep " + superstep + ".") val startTime = System.currentTimeMillis @@ -83,6 +84,10 @@ object Bagel extends Logging { val superstep_ = superstep // Create a read-only copy of superstep for capture in closure val (processed, numMsgs, numActiveVerts) = comp[K, V, M, C](sc, grouped, compute(_, _, aggregated, superstep_), storageLevel) + if (lastRDD != null) { + lastRDD.unpersist(false) + } + lastRDD = processed val timeTaken = System.currentTimeMillis - startTime logInfo("Superstep %d took %d s".format(superstep, timeTaken / 1000)) diff --git a/bin/beeline b/bin/beeline new file mode 100755 index 0000000000000..1bda4dba50605 --- /dev/null +++ b/bin/beeline @@ -0,0 +1,30 @@ +#!/usr/bin/env bash + +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# +# Shell script for starting BeeLine + +# Enter posix mode for bash +set -o posix + +# Figure out where Spark is installed +FWDIR="$(cd `dirname $0`/..; pwd)" + +CLASS="org.apache.hive.beeline.BeeLine" +exec "$FWDIR/bin/spark-class" $CLASS "$@" diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index e81e8c060cb98..16b794a1592e8 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -52,6 +52,7 @@ if [ -n "$SPARK_PREPEND_CLASSES" ]; then CLASSPATH="$CLASSPATH:$FWDIR/sql/catalyst/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/sql/core/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/sql/hive/target/scala-$SCALA_VERSION/classes" + CLASSPATH="$CLASSPATH:$FWDIR/sql/hive-thriftserver/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/yarn/stable/target/scala-$SCALA_VERSION/classes" fi diff --git a/bin/pyspark b/bin/pyspark index 69b056fe28f2c..01d42025c978e 100755 --- a/bin/pyspark +++ b/bin/pyspark @@ -23,12 +23,18 @@ FWDIR="$(cd `dirname $0`/..; pwd)" # Export this as SPARK_HOME export SPARK_HOME="$FWDIR" +source $FWDIR/bin/utils.sh + SCALA_VERSION=2.10 -if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then +function usage() { echo "Usage: ./bin/pyspark [options]" 1>&2 $FWDIR/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 exit 0 +} + +if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then + usage fi # Exit if the user hasn't compiled Spark @@ -52,7 +58,7 @@ export PYSPARK_PYTHON # Add the PySpark classes to the Python path: export PYTHONPATH=$SPARK_HOME/python/:$PYTHONPATH -export PYTHONPATH=$SPARK_HOME/python/lib/py4j-0.8.1-src.zip:$PYTHONPATH +export PYTHONPATH=$SPARK_HOME/python/lib/py4j-0.8.2.1-src.zip:$PYTHONPATH # Load the PySpark shell.py script when ./pyspark is used interactively: export OLD_PYTHONSTARTUP=$PYTHONSTARTUP @@ -66,10 +72,11 @@ fi # 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. - +SUBMIT_USAGE_FUNCTION=usage +gatherSparkSubmitOpts "$@" PYSPARK_SUBMIT_ARGS="" whitespace="[[:space:]]" -for i in "$@"; do +for i in "${SUBMISSION_OPTS[@]}"; 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" @@ -90,7 +97,10 @@ fi 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 "$@" + primary=$1 + shift + gatherSparkSubmitOpts "$@" + exec $FWDIR/bin/spark-submit "${SUBMISSION_OPTS[@]}" $primary "${APPLICATION_OPTS[@]}" else # Only use ipython if no command line arguments were provided [SPARK-1134] if [[ "$IPYTHON" = "1" ]]; then diff --git a/bin/pyspark2.cmd b/bin/pyspark2.cmd index 0ef9eea95342e..2c4b08af8d4c3 100644 --- a/bin/pyspark2.cmd +++ b/bin/pyspark2.cmd @@ -45,7 +45,7 @@ rem Figure out which Python to use. 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 PYTHONPATH=%FWDIR%python\lib\py4j-0.8.2.1-src.zip;%PYTHONPATH% set OLD_PYTHONSTARTUP=%PYTHONSTARTUP% set PYTHONSTARTUP=%FWDIR%python\pyspark\shell.py diff --git a/bin/run-example b/bin/run-example index 942706d733122..68a35702eddd3 100755 --- a/bin/run-example +++ b/bin/run-example @@ -29,7 +29,8 @@ if [ -n "$1" ]; then else echo "Usage: ./bin/run-example [example-args]" 1>&2 echo " - set MASTER=XX to use a specific master" 1>&2 - echo " - can use abbreviated example class name (e.g. SparkPi, mllib.LinearRegression)" 1>&2 + echo " - can use abbreviated example class name relative to com.apache.spark.examples" 1>&2 + echo " (e.g. SparkPi, mllib.LinearRegression, streaming.KinesisWordCountASL)" 1>&2 exit 1 fi diff --git a/bin/run-example2.cmd b/bin/run-example2.cmd index eadedd7fa61ff..b29bf90c64e90 100644 --- a/bin/run-example2.cmd +++ b/bin/run-example2.cmd @@ -32,7 +32,8 @@ rem Test that an argument was given if not "x%1"=="x" goto arg_given 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) + echo - can use abbreviated example class name relative to com.apache.spark.examples + echo (e.g. SparkPi, mllib.LinearRegression, streaming.KinesisWordCountASL) goto exit :arg_given diff --git a/bin/spark-class b/bin/spark-class index 04fa52c6756b1..3f6beca5becf0 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -110,9 +110,9 @@ export JAVA_OPTS TOOLS_DIR="$FWDIR"/tools SPARK_TOOLS_JAR="" -if [ -e "$TOOLS_DIR"/target/scala-$SCALA_VERSION/*assembly*[0-9Tg].jar ]; then +if [ -e "$TOOLS_DIR"/target/scala-$SCALA_VERSION/spark-tools*[0-9Tg].jar ]; then # Use the JAR from the SBT build - export SPARK_TOOLS_JAR=`ls "$TOOLS_DIR"/target/scala-$SCALA_VERSION/*assembly*[0-9Tg].jar` + export SPARK_TOOLS_JAR=`ls "$TOOLS_DIR"/target/scala-$SCALA_VERSION/spark-tools*[0-9Tg].jar` fi if [ -e "$TOOLS_DIR"/target/spark-tools*[0-9Tg].jar ]; then # Use the JAR from the Maven build diff --git a/bin/spark-shell b/bin/spark-shell index 850e9507ec38f..8b7ccd7439551 100755 --- a/bin/spark-shell +++ b/bin/spark-shell @@ -31,13 +31,21 @@ set -o posix ## Global script variables FWDIR="$(cd `dirname $0`/..; pwd)" +function usage() { + echo "Usage: ./bin/spark-shell [options]" + $FWDIR/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 + exit 0 +} + if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then - echo "Usage: ./bin/spark-shell [options]" - $FWDIR/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 - exit 0 + usage fi -function main(){ +source $FWDIR/bin/utils.sh +SUBMIT_USAGE_FUNCTION=usage +gatherSparkSubmitOpts "$@" + +function main() { if $cygwin; then # Workaround for issue involving JLine and Cygwin # (see http://sourceforge.net/p/jline/bugs/40/). @@ -46,11 +54,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-shell "$@" --class org.apache.spark.repl.Main + $FWDIR/bin/spark-submit --class org.apache.spark.repl.Main "${SUBMISSION_OPTS[@]}" spark-shell "${APPLICATION_OPTS[@]}" stty icanon echo > /dev/null 2>&1 else export SPARK_SUBMIT_OPTS - $FWDIR/bin/spark-submit spark-shell "$@" --class org.apache.spark.repl.Main + $FWDIR/bin/spark-submit --class org.apache.spark.repl.Main "${SUBMISSION_OPTS[@]}" spark-shell "${APPLICATION_OPTS[@]}" fi } diff --git a/bin/spark-shell.cmd b/bin/spark-shell.cmd index 4b9708a8c03f3..2ee60b4e2a2b3 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-shell %* --class org.apache.spark.repl.Main +cmd /V /E /C %SPARK_HOME%\bin\spark-submit.cmd --class org.apache.spark.repl.Main %* spark-shell diff --git a/bin/spark-sql b/bin/spark-sql new file mode 100755 index 0000000000000..564f1f419060f --- /dev/null +++ b/bin/spark-sql @@ -0,0 +1,94 @@ +#!/usr/bin/env bash + +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# +# Shell script for starting the Spark SQL CLI + +# Enter posix mode for bash +set -o posix + +CLASS="org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver" + +# Figure out where Spark is installed +FWDIR="$(cd `dirname $0`/..; pwd)" + +function usage { + echo "Usage: ./bin/spark-sql [options] [cli option]" + pattern="usage" + pattern+="\|Spark assembly has been built with Hive" + pattern+="\|NOTE: SPARK_PREPEND_CLASSES is set" + pattern+="\|Spark Command: " + pattern+="\|--help" + pattern+="\|=======" + + $FWDIR/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 + echo + echo "CLI options:" + $FWDIR/bin/spark-class $CLASS --help 2>&1 | grep -v "$pattern" 1>&2 +} + +function ensure_arg_number { + arg_number=$1 + at_least=$2 + + if [[ $arg_number -lt $at_least ]]; then + usage + exit 1 + fi +} + +if [[ "$@" = --help ]] || [[ "$@" = -h ]]; then + usage + exit 0 +fi + +CLI_ARGS=() +SUBMISSION_ARGS=() + +while (($#)); do + case $1 in + -d | --define | --database | -f | -h | --hiveconf | --hivevar | -i | -p) + ensure_arg_number $# 2 + CLI_ARGS+=("$1"); shift + CLI_ARGS+=("$1"); shift + ;; + + -e) + ensure_arg_number $# 2 + CLI_ARGS+=("$1"); shift + CLI_ARGS+=("$1"); shift + ;; + + -s | --silent) + CLI_ARGS+=("$1"); shift + ;; + + -v | --verbose) + # Both SparkSubmit and SparkSQLCLIDriver recognizes -v | --verbose + CLI_ARGS+=("$1") + SUBMISSION_ARGS+=("$1"); shift + ;; + + *) + SUBMISSION_ARGS+=("$1"); shift + ;; + esac +done + +exec "$FWDIR"/bin/spark-submit --class $CLASS "${SUBMISSION_ARGS[@]}" spark-internal "${CLI_ARGS[@]}" diff --git a/bin/utils.sh b/bin/utils.sh new file mode 100644 index 0000000000000..0804b1ed9f231 --- /dev/null +++ b/bin/utils.sh @@ -0,0 +1,59 @@ +#!/usr/bin/env bash + +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Gather all all spark-submit options into SUBMISSION_OPTS +function gatherSparkSubmitOpts() { + + if [ -z "$SUBMIT_USAGE_FUNCTION" ]; then + echo "Function for printing usage of $0 is not set." 1>&2 + echo "Please set usage function to shell variable 'SUBMIT_USAGE_FUNCTION' in $0" 1>&2 + exit 1 + fi + + # NOTE: If you add or remove spark-sumbmit options, + # modify NOT ONLY this script but also SparkSubmitArgument.scala + SUBMISSION_OPTS=() + APPLICATION_OPTS=() + while (($#)); do + case "$1" in + --master | --deploy-mode | --class | --name | --jars | --py-files | --files | \ + --conf | --properties-file | --driver-memory | --driver-java-options | \ + --driver-library-path | --driver-class-path | --executor-memory | --driver-cores | \ + --total-executor-cores | --executor-cores | --queue | --num-executors | --archives) + if [[ $# -lt 2 ]]; then + "$SUBMIT_USAGE_FUNCTION" + exit 1; + fi + SUBMISSION_OPTS+=("$1"); shift + SUBMISSION_OPTS+=("$1"); shift + ;; + + --verbose | -v | --supervise) + SUBMISSION_OPTS+=("$1"); shift + ;; + + *) + APPLICATION_OPTS+=("$1"); shift + ;; + esac + done + + export SUBMISSION_OPTS + export APPLICATION_OPTS +} diff --git a/core/pom.xml b/core/pom.xml index 6abf8480d5da0..6d8be37037729 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -27,6 +27,9 @@ org.apache.spark spark-core_2.10 + + core + jar Spark Project Core http://spark.apache.org/ @@ -111,6 +114,10 @@ org.xerial.snappy snappy-java + + net.jpountz.lz4 + lz4 + com.twitter chill_${scala.binary.version} @@ -143,7 +150,7 @@ org.json4s json4s-jackson_${scala.binary.version} - 3.2.6 + 3.2.10 colt @@ -185,8 +192,8 @@ org.tachyonproject - tachyon - 0.4.1-thrift + tachyon-client + 0.5.0 org.apache.hadoop @@ -255,6 +262,11 @@ asm test + + junit + junit + test + com.novocode junit-interface @@ -268,7 +280,7 @@ net.sf.py4j py4j - 0.8.1 + 0.8.2.1 diff --git a/core/src/main/java/org/apache/spark/network/netty/FileClient.java b/core/src/main/java/org/apache/spark/network/netty/FileClient.java deleted file mode 100644 index 0d31894d6ec7a..0000000000000 --- a/core/src/main/java/org/apache/spark/network/netty/FileClient.java +++ /dev/null @@ -1,100 +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.network.netty; - -import java.util.concurrent.TimeUnit; - -import io.netty.bootstrap.Bootstrap; -import io.netty.channel.Channel; -import io.netty.channel.ChannelOption; -import io.netty.channel.EventLoopGroup; -import io.netty.channel.oio.OioEventLoopGroup; -import io.netty.channel.socket.oio.OioSocketChannel; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -class FileClient { - - private static final Logger LOG = LoggerFactory.getLogger(FileClient.class.getName()); - - private final FileClientHandler handler; - private Channel channel = null; - private Bootstrap bootstrap = null; - private EventLoopGroup group = null; - private final int connectTimeout; - private final int sendTimeout = 60; // 1 min - - FileClient(FileClientHandler handler, int connectTimeout) { - this.handler = handler; - this.connectTimeout = connectTimeout; - } - - public void init() { - group = new OioEventLoopGroup(); - bootstrap = new Bootstrap(); - bootstrap.group(group) - .channel(OioSocketChannel.class) - .option(ChannelOption.SO_KEEPALIVE, true) - .option(ChannelOption.TCP_NODELAY, true) - .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, connectTimeout) - .handler(new FileClientChannelInitializer(handler)); - } - - public void connect(String host, int port) { - try { - // Start the connection attempt. - channel = bootstrap.connect(host, port).sync().channel(); - // ChannelFuture cf = channel.closeFuture(); - //cf.addListener(new ChannelCloseListener(this)); - } catch (InterruptedException e) { - LOG.warn("FileClient interrupted while trying to connect", e); - close(); - } - } - - public void waitForClose() { - try { - channel.closeFuture().sync(); - } catch (InterruptedException e) { - LOG.warn("FileClient interrupted", e); - } - } - - public void sendRequest(String file) { - //assert(file == null); - //assert(channel == null); - try { - // Should be able to send the message to network link channel. - boolean bSent = channel.writeAndFlush(file + "\r\n").await(sendTimeout, TimeUnit.SECONDS); - if (!bSent) { - throw new RuntimeException("Failed to send"); - } - } catch (InterruptedException e) { - LOG.error("Error", e); - } - } - - public void close() { - if (group != null) { - group.shutdownGracefully(); - group = null; - bootstrap = null; - } - } -} diff --git a/core/src/main/java/org/apache/spark/network/netty/FileServer.java b/core/src/main/java/org/apache/spark/network/netty/FileServer.java deleted file mode 100644 index c93425e2787dc..0000000000000 --- a/core/src/main/java/org/apache/spark/network/netty/FileServer.java +++ /dev/null @@ -1,111 +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.network.netty; - -import java.net.InetSocketAddress; - -import io.netty.bootstrap.ServerBootstrap; -import io.netty.channel.ChannelFuture; -import io.netty.channel.ChannelOption; -import io.netty.channel.EventLoopGroup; -import io.netty.channel.oio.OioEventLoopGroup; -import io.netty.channel.socket.oio.OioServerSocketChannel; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Server that accept the path of a file an echo back its content. - */ -class FileServer { - - private static final Logger LOG = LoggerFactory.getLogger(FileServer.class.getName()); - - private EventLoopGroup bossGroup = null; - private EventLoopGroup workerGroup = null; - private ChannelFuture channelFuture = null; - private int port = 0; - - FileServer(PathResolver pResolver, int port) { - InetSocketAddress addr = new InetSocketAddress(port); - - // Configure the server. - bossGroup = new OioEventLoopGroup(); - workerGroup = new OioEventLoopGroup(); - - ServerBootstrap bootstrap = new ServerBootstrap(); - bootstrap.group(bossGroup, workerGroup) - .channel(OioServerSocketChannel.class) - .option(ChannelOption.SO_BACKLOG, 100) - .option(ChannelOption.SO_RCVBUF, 1500) - .childHandler(new FileServerChannelInitializer(pResolver)); - // Start the server. - channelFuture = bootstrap.bind(addr); - try { - // Get the address we bound to. - InetSocketAddress boundAddress = - ((InetSocketAddress) channelFuture.sync().channel().localAddress()); - this.port = boundAddress.getPort(); - } catch (InterruptedException ie) { - this.port = 0; - } - } - - /** - * Start the file server asynchronously in a new thread. - */ - public void start() { - Thread blockingThread = new Thread() { - @Override - public void run() { - try { - channelFuture.channel().closeFuture().sync(); - LOG.info("FileServer exiting"); - } catch (InterruptedException e) { - LOG.error("File server start got interrupted", e); - } - // NOTE: bootstrap is shutdown in stop() - } - }; - blockingThread.setDaemon(true); - blockingThread.start(); - } - - public int getPort() { - return port; - } - - public void stop() { - // Close the bound channel. - if (channelFuture != null) { - channelFuture.channel().close().awaitUninterruptibly(); - channelFuture = null; - } - - // Shutdown event groups - if (bossGroup != null) { - bossGroup.shutdownGracefully(); - bossGroup = null; - } - - if (workerGroup != null) { - workerGroup.shutdownGracefully(); - workerGroup = null; - } - // TODO: Shutdown all accepted channels as well ? - } -} diff --git a/core/src/main/java/org/apache/spark/network/netty/FileServerHandler.java b/core/src/main/java/org/apache/spark/network/netty/FileServerHandler.java deleted file mode 100644 index c0133e19c7f79..0000000000000 --- a/core/src/main/java/org/apache/spark/network/netty/FileServerHandler.java +++ /dev/null @@ -1,83 +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.network.netty; - -import java.io.File; -import java.io.FileInputStream; - -import io.netty.channel.ChannelHandlerContext; -import io.netty.channel.SimpleChannelInboundHandler; -import io.netty.channel.DefaultFileRegion; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.spark.storage.BlockId; -import org.apache.spark.storage.FileSegment; - -class FileServerHandler extends SimpleChannelInboundHandler { - - private static final Logger LOG = LoggerFactory.getLogger(FileServerHandler.class.getName()); - - private final PathResolver pResolver; - - FileServerHandler(PathResolver pResolver){ - this.pResolver = pResolver; - } - - @Override - public void channelRead0(ChannelHandlerContext ctx, String blockIdString) { - BlockId blockId = BlockId.apply(blockIdString); - FileSegment fileSegment = pResolver.getBlockLocation(blockId); - // if getBlockLocation returns null, close the channel - if (fileSegment == null) { - //ctx.close(); - return; - } - File file = fileSegment.file(); - if (file.exists()) { - if (!file.isFile()) { - ctx.write(new FileHeader(0, blockId).buffer()); - ctx.flush(); - return; - } - long length = fileSegment.length(); - if (length > Integer.MAX_VALUE || length <= 0) { - ctx.write(new FileHeader(0, blockId).buffer()); - ctx.flush(); - return; - } - int len = (int) length; - ctx.write((new FileHeader(len, blockId)).buffer()); - try { - ctx.write(new DefaultFileRegion(new FileInputStream(file) - .getChannel(), fileSegment.offset(), fileSegment.length())); - } catch (Exception e) { - LOG.error("Exception: ", e); - } - } else { - ctx.write(new FileHeader(0, blockId).buffer()); - } - ctx.flush(); - } - - @Override - public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { - LOG.error("Exception: ", cause); - ctx.close(); - } -} diff --git a/core/src/main/java/org/apache/spark/util/collection/Sorter.java b/core/src/main/java/org/apache/spark/util/collection/Sorter.java new file mode 100644 index 0000000000000..64ad18c0e463a --- /dev/null +++ b/core/src/main/java/org/apache/spark/util/collection/Sorter.java @@ -0,0 +1,915 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.collection; + +import java.util.Comparator; + +/** + * A port of the Android Timsort class, which utilizes a "stable, adaptive, iterative mergesort." + * See the method comment on sort() for more details. + * + * This has been kept in Java with the original style in order to match very closely with the + * Anroid source code, and thus be easy to verify correctness. + * + * The purpose of the port is to generalize the interface to the sort to accept input data formats + * besides simple arrays where every element is sorted individually. For instance, the AppendOnlyMap + * uses this to sort an Array with alternating elements of the form [key, value, key, value]. + * This generalization comes with minimal overhead -- see SortDataFormat for more information. + */ +class Sorter { + + /** + * This is the minimum sized sequence that will be merged. Shorter + * sequences will be lengthened by calling binarySort. If the entire + * array is less than this length, no merges will be performed. + * + * This constant should be a power of two. It was 64 in Tim Peter's C + * implementation, but 32 was empirically determined to work better in + * this implementation. In the unlikely event that you set this constant + * to be a number that's not a power of two, you'll need to change the + * minRunLength computation. + * + * If you decrease this constant, you must change the stackLen + * computation in the TimSort constructor, or you risk an + * ArrayOutOfBounds exception. See listsort.txt for a discussion + * of the minimum stack length required as a function of the length + * of the array being sorted and the minimum merge sequence length. + */ + private static final int MIN_MERGE = 32; + + private final SortDataFormat s; + + public Sorter(SortDataFormat sortDataFormat) { + this.s = sortDataFormat; + } + + /** + * A stable, adaptive, iterative mergesort that requires far fewer than + * n lg(n) comparisons when running on partially sorted arrays, while + * offering performance comparable to a traditional mergesort when run + * on random arrays. Like all proper mergesorts, this sort is stable and + * runs O(n log n) time (worst case). In the worst case, this sort requires + * temporary storage space for n/2 object references; in the best case, + * it requires only a small constant amount of space. + * + * This implementation was adapted from Tim Peters's list sort for + * Python, which is described in detail here: + * + * http://svn.python.org/projects/python/trunk/Objects/listsort.txt + * + * Tim's C code may be found here: + * + * http://svn.python.org/projects/python/trunk/Objects/listobject.c + * + * The underlying techniques are described in this paper (and may have + * even earlier origins): + * + * "Optimistic Sorting and Information Theoretic Complexity" + * Peter McIlroy + * SODA (Fourth Annual ACM-SIAM Symposium on Discrete Algorithms), + * pp 467-474, Austin, Texas, 25-27 January 1993. + * + * While the API to this class consists solely of static methods, it is + * (privately) instantiable; a TimSort instance holds the state of an ongoing + * sort, assuming the input array is large enough to warrant the full-blown + * TimSort. Small arrays are sorted in place, using a binary insertion sort. + * + * @author Josh Bloch + */ + void sort(Buffer a, int lo, int hi, Comparator c) { + assert c != null; + + int nRemaining = hi - lo; + if (nRemaining < 2) + return; // Arrays of size 0 and 1 are always sorted + + // If array is small, do a "mini-TimSort" with no merges + if (nRemaining < MIN_MERGE) { + int initRunLen = countRunAndMakeAscending(a, lo, hi, c); + binarySort(a, lo, hi, lo + initRunLen, c); + return; + } + + /** + * March over the array once, left to right, finding natural runs, + * extending short natural runs to minRun elements, and merging runs + * to maintain stack invariant. + */ + SortState sortState = new SortState(a, c, hi - lo); + int minRun = minRunLength(nRemaining); + do { + // Identify next run + int runLen = countRunAndMakeAscending(a, lo, hi, c); + + // If run is short, extend to min(minRun, nRemaining) + if (runLen < minRun) { + int force = nRemaining <= minRun ? nRemaining : minRun; + binarySort(a, lo, lo + force, lo + runLen, c); + runLen = force; + } + + // Push run onto pending-run stack, and maybe merge + sortState.pushRun(lo, runLen); + sortState.mergeCollapse(); + + // Advance to find next run + lo += runLen; + nRemaining -= runLen; + } while (nRemaining != 0); + + // Merge all remaining runs to complete sort + assert lo == hi; + sortState.mergeForceCollapse(); + assert sortState.stackSize == 1; + } + + /** + * Sorts the specified portion of the specified array using a binary + * insertion sort. This is the best method for sorting small numbers + * of elements. It requires O(n log n) compares, but O(n^2) data + * movement (worst case). + * + * If the initial part of the specified range is already sorted, + * this method can take advantage of it: the method assumes that the + * elements from index {@code lo}, inclusive, to {@code start}, + * exclusive are already sorted. + * + * @param a the array in which a range is to be sorted + * @param lo the index of the first element in the range to be sorted + * @param hi the index after the last element in the range to be sorted + * @param start the index of the first element in the range that is + * not already known to be sorted ({@code lo <= start <= hi}) + * @param c comparator to used for the sort + */ + @SuppressWarnings("fallthrough") + private void binarySort(Buffer a, int lo, int hi, int start, Comparator c) { + assert lo <= start && start <= hi; + if (start == lo) + start++; + + Buffer pivotStore = s.allocate(1); + for ( ; start < hi; start++) { + s.copyElement(a, start, pivotStore, 0); + K pivot = s.getKey(pivotStore, 0); + + // Set left (and right) to the index where a[start] (pivot) belongs + int left = lo; + int right = start; + assert left <= right; + /* + * Invariants: + * pivot >= all in [lo, left). + * pivot < all in [right, start). + */ + while (left < right) { + int mid = (left + right) >>> 1; + if (c.compare(pivot, s.getKey(a, mid)) < 0) + right = mid; + else + left = mid + 1; + } + assert left == right; + + /* + * The invariants still hold: pivot >= all in [lo, left) and + * pivot < all in [left, start), so pivot belongs at left. Note + * that if there are elements equal to pivot, left points to the + * first slot after them -- that's why this sort is stable. + * Slide elements over to make room for pivot. + */ + int n = start - left; // The number of elements to move + // Switch is just an optimization for arraycopy in default case + switch (n) { + case 2: s.copyElement(a, left + 1, a, left + 2); + case 1: s.copyElement(a, left, a, left + 1); + break; + default: s.copyRange(a, left, a, left + 1, n); + } + s.copyElement(pivotStore, 0, a, left); + } + } + + /** + * Returns the length of the run beginning at the specified position in + * the specified array and reverses the run if it is descending (ensuring + * that the run will always be ascending when the method returns). + * + * A run is the longest ascending sequence with: + * + * a[lo] <= a[lo + 1] <= a[lo + 2] <= ... + * + * or the longest descending sequence with: + * + * a[lo] > a[lo + 1] > a[lo + 2] > ... + * + * For its intended use in a stable mergesort, the strictness of the + * definition of "descending" is needed so that the call can safely + * reverse a descending sequence without violating stability. + * + * @param a the array in which a run is to be counted and possibly reversed + * @param lo index of the first element in the run + * @param hi index after the last element that may be contained in the run. + It is required that {@code lo < hi}. + * @param c the comparator to used for the sort + * @return the length of the run beginning at the specified position in + * the specified array + */ + private int countRunAndMakeAscending(Buffer a, int lo, int hi, Comparator c) { + assert lo < hi; + int runHi = lo + 1; + if (runHi == hi) + return 1; + + // Find end of run, and reverse range if descending + if (c.compare(s.getKey(a, runHi++), s.getKey(a, lo)) < 0) { // Descending + while (runHi < hi && c.compare(s.getKey(a, runHi), s.getKey(a, runHi - 1)) < 0) + runHi++; + reverseRange(a, lo, runHi); + } else { // Ascending + while (runHi < hi && c.compare(s.getKey(a, runHi), s.getKey(a, runHi - 1)) >= 0) + runHi++; + } + + return runHi - lo; + } + + /** + * Reverse the specified range of the specified array. + * + * @param a the array in which a range is to be reversed + * @param lo the index of the first element in the range to be reversed + * @param hi the index after the last element in the range to be reversed + */ + private void reverseRange(Buffer a, int lo, int hi) { + hi--; + while (lo < hi) { + s.swap(a, lo, hi); + lo++; + hi--; + } + } + + /** + * Returns the minimum acceptable run length for an array of the specified + * length. Natural runs shorter than this will be extended with + * {@link #binarySort}. + * + * Roughly speaking, the computation is: + * + * If n < MIN_MERGE, return n (it's too small to bother with fancy stuff). + * Else if n is an exact power of 2, return MIN_MERGE/2. + * Else return an int k, MIN_MERGE/2 <= k <= MIN_MERGE, such that n/k + * is close to, but strictly less than, an exact power of 2. + * + * For the rationale, see listsort.txt. + * + * @param n the length of the array to be sorted + * @return the length of the minimum run to be merged + */ + private int minRunLength(int n) { + assert n >= 0; + int r = 0; // Becomes 1 if any 1 bits are shifted off + while (n >= MIN_MERGE) { + r |= (n & 1); + n >>= 1; + } + return n + r; + } + + private class SortState { + + /** + * The Buffer being sorted. + */ + private final Buffer a; + + /** + * Length of the sort Buffer. + */ + private final int aLength; + + /** + * The comparator for this sort. + */ + private final Comparator c; + + /** + * When we get into galloping mode, we stay there until both runs win less + * often than MIN_GALLOP consecutive times. + */ + private static final int MIN_GALLOP = 7; + + /** + * This controls when we get *into* galloping mode. It is initialized + * to MIN_GALLOP. The mergeLo and mergeHi methods nudge it higher for + * random data, and lower for highly structured data. + */ + private int minGallop = MIN_GALLOP; + + /** + * Maximum initial size of tmp array, which is used for merging. The array + * can grow to accommodate demand. + * + * Unlike Tim's original C version, we do not allocate this much storage + * when sorting smaller arrays. This change was required for performance. + */ + private static final int INITIAL_TMP_STORAGE_LENGTH = 256; + + /** + * Temp storage for merges. + */ + private Buffer tmp; // Actual runtime type will be Object[], regardless of T + + /** + * Length of the temp storage. + */ + private int tmpLength = 0; + + /** + * A stack of pending runs yet to be merged. Run i starts at + * address base[i] and extends for len[i] elements. It's always + * true (so long as the indices are in bounds) that: + * + * runBase[i] + runLen[i] == runBase[i + 1] + * + * so we could cut the storage for this, but it's a minor amount, + * and keeping all the info explicit simplifies the code. + */ + private int stackSize = 0; // Number of pending runs on stack + private final int[] runBase; + private final int[] runLen; + + /** + * Creates a TimSort instance to maintain the state of an ongoing sort. + * + * @param a the array to be sorted + * @param c the comparator to determine the order of the sort + */ + private SortState(Buffer a, Comparator c, int len) { + this.aLength = len; + this.a = a; + this.c = c; + + // Allocate temp storage (which may be increased later if necessary) + tmpLength = len < 2 * INITIAL_TMP_STORAGE_LENGTH ? len >>> 1 : INITIAL_TMP_STORAGE_LENGTH; + tmp = s.allocate(tmpLength); + + /* + * Allocate runs-to-be-merged stack (which cannot be expanded). The + * stack length requirements are described in listsort.txt. The C + * version always uses the same stack length (85), but this was + * measured to be too expensive when sorting "mid-sized" arrays (e.g., + * 100 elements) in Java. Therefore, we use smaller (but sufficiently + * large) stack lengths for smaller arrays. The "magic numbers" in the + * computation below must be changed if MIN_MERGE is decreased. See + * the MIN_MERGE declaration above for more information. + */ + int stackLen = (len < 120 ? 5 : + len < 1542 ? 10 : + len < 119151 ? 19 : 40); + runBase = new int[stackLen]; + runLen = new int[stackLen]; + } + + /** + * Pushes the specified run onto the pending-run stack. + * + * @param runBase index of the first element in the run + * @param runLen the number of elements in the run + */ + private void pushRun(int runBase, int runLen) { + this.runBase[stackSize] = runBase; + this.runLen[stackSize] = runLen; + stackSize++; + } + + /** + * Examines the stack of runs waiting to be merged and merges adjacent runs + * until the stack invariants are reestablished: + * + * 1. runLen[i - 3] > runLen[i - 2] + runLen[i - 1] + * 2. runLen[i - 2] > runLen[i - 1] + * + * This method is called each time a new run is pushed onto the stack, + * so the invariants are guaranteed to hold for i < stackSize upon + * entry to the method. + */ + private void mergeCollapse() { + while (stackSize > 1) { + int n = stackSize - 2; + if (n > 0 && runLen[n-1] <= runLen[n] + runLen[n+1]) { + if (runLen[n - 1] < runLen[n + 1]) + n--; + mergeAt(n); + } else if (runLen[n] <= runLen[n + 1]) { + mergeAt(n); + } else { + break; // Invariant is established + } + } + } + + /** + * Merges all runs on the stack until only one remains. This method is + * called once, to complete the sort. + */ + private void mergeForceCollapse() { + while (stackSize > 1) { + int n = stackSize - 2; + if (n > 0 && runLen[n - 1] < runLen[n + 1]) + n--; + mergeAt(n); + } + } + + /** + * Merges the two runs at stack indices i and i+1. Run i must be + * the penultimate or antepenultimate run on the stack. In other words, + * i must be equal to stackSize-2 or stackSize-3. + * + * @param i stack index of the first of the two runs to merge + */ + private void mergeAt(int i) { + assert stackSize >= 2; + assert i >= 0; + assert i == stackSize - 2 || i == stackSize - 3; + + int base1 = runBase[i]; + int len1 = runLen[i]; + int base2 = runBase[i + 1]; + int len2 = runLen[i + 1]; + assert len1 > 0 && len2 > 0; + assert base1 + len1 == base2; + + /* + * Record the length of the combined runs; if i is the 3rd-last + * run now, also slide over the last run (which isn't involved + * in this merge). The current run (i+1) goes away in any case. + */ + runLen[i] = len1 + len2; + if (i == stackSize - 3) { + runBase[i + 1] = runBase[i + 2]; + runLen[i + 1] = runLen[i + 2]; + } + stackSize--; + + /* + * Find where the first element of run2 goes in run1. Prior elements + * in run1 can be ignored (because they're already in place). + */ + int k = gallopRight(s.getKey(a, base2), a, base1, len1, 0, c); + assert k >= 0; + base1 += k; + len1 -= k; + if (len1 == 0) + return; + + /* + * Find where the last element of run1 goes in run2. Subsequent elements + * in run2 can be ignored (because they're already in place). + */ + len2 = gallopLeft(s.getKey(a, base1 + len1 - 1), a, base2, len2, len2 - 1, c); + assert len2 >= 0; + if (len2 == 0) + return; + + // Merge remaining runs, using tmp array with min(len1, len2) elements + if (len1 <= len2) + mergeLo(base1, len1, base2, len2); + else + mergeHi(base1, len1, base2, len2); + } + + /** + * Locates the position at which to insert the specified key into the + * specified sorted range; if the range contains an element equal to key, + * returns the index of the leftmost equal element. + * + * @param key the key whose insertion point to search for + * @param a the array in which to search + * @param base the index of the first element in the range + * @param len the length of the range; must be > 0 + * @param hint the index at which to begin the search, 0 <= hint < n. + * The closer hint is to the result, the faster this method will run. + * @param c the comparator used to order the range, and to search + * @return the int k, 0 <= k <= n such that a[b + k - 1] < key <= a[b + k], + * pretending that a[b - 1] is minus infinity and a[b + n] is infinity. + * In other words, key belongs at index b + k; or in other words, + * the first k elements of a should precede key, and the last n - k + * should follow it. + */ + private int gallopLeft(K key, Buffer a, int base, int len, int hint, Comparator c) { + assert len > 0 && hint >= 0 && hint < len; + int lastOfs = 0; + int ofs = 1; + if (c.compare(key, s.getKey(a, base + hint)) > 0) { + // Gallop right until a[base+hint+lastOfs] < key <= a[base+hint+ofs] + int maxOfs = len - hint; + while (ofs < maxOfs && c.compare(key, s.getKey(a, base + hint + ofs)) > 0) { + lastOfs = ofs; + ofs = (ofs << 1) + 1; + if (ofs <= 0) // int overflow + ofs = maxOfs; + } + if (ofs > maxOfs) + ofs = maxOfs; + + // Make offsets relative to base + lastOfs += hint; + ofs += hint; + } else { // key <= a[base + hint] + // Gallop left until a[base+hint-ofs] < key <= a[base+hint-lastOfs] + final int maxOfs = hint + 1; + while (ofs < maxOfs && c.compare(key, s.getKey(a, base + hint - ofs)) <= 0) { + lastOfs = ofs; + ofs = (ofs << 1) + 1; + if (ofs <= 0) // int overflow + ofs = maxOfs; + } + if (ofs > maxOfs) + ofs = maxOfs; + + // Make offsets relative to base + int tmp = lastOfs; + lastOfs = hint - ofs; + ofs = hint - tmp; + } + assert -1 <= lastOfs && lastOfs < ofs && ofs <= len; + + /* + * Now a[base+lastOfs] < key <= a[base+ofs], so key belongs somewhere + * to the right of lastOfs but no farther right than ofs. Do a binary + * search, with invariant a[base + lastOfs - 1] < key <= a[base + ofs]. + */ + lastOfs++; + while (lastOfs < ofs) { + int m = lastOfs + ((ofs - lastOfs) >>> 1); + + if (c.compare(key, s.getKey(a, base + m)) > 0) + lastOfs = m + 1; // a[base + m] < key + else + ofs = m; // key <= a[base + m] + } + assert lastOfs == ofs; // so a[base + ofs - 1] < key <= a[base + ofs] + return ofs; + } + + /** + * Like gallopLeft, except that if the range contains an element equal to + * key, gallopRight returns the index after the rightmost equal element. + * + * @param key the key whose insertion point to search for + * @param a the array in which to search + * @param base the index of the first element in the range + * @param len the length of the range; must be > 0 + * @param hint the index at which to begin the search, 0 <= hint < n. + * The closer hint is to the result, the faster this method will run. + * @param c the comparator used to order the range, and to search + * @return the int k, 0 <= k <= n such that a[b + k - 1] <= key < a[b + k] + */ + private int gallopRight(K key, Buffer a, int base, int len, int hint, Comparator c) { + assert len > 0 && hint >= 0 && hint < len; + + int ofs = 1; + int lastOfs = 0; + if (c.compare(key, s.getKey(a, base + hint)) < 0) { + // Gallop left until a[b+hint - ofs] <= key < a[b+hint - lastOfs] + int maxOfs = hint + 1; + while (ofs < maxOfs && c.compare(key, s.getKey(a, base + hint - ofs)) < 0) { + lastOfs = ofs; + ofs = (ofs << 1) + 1; + if (ofs <= 0) // int overflow + ofs = maxOfs; + } + if (ofs > maxOfs) + ofs = maxOfs; + + // Make offsets relative to b + int tmp = lastOfs; + lastOfs = hint - ofs; + ofs = hint - tmp; + } else { // a[b + hint] <= key + // Gallop right until a[b+hint + lastOfs] <= key < a[b+hint + ofs] + int maxOfs = len - hint; + while (ofs < maxOfs && c.compare(key, s.getKey(a, base + hint + ofs)) >= 0) { + lastOfs = ofs; + ofs = (ofs << 1) + 1; + if (ofs <= 0) // int overflow + ofs = maxOfs; + } + if (ofs > maxOfs) + ofs = maxOfs; + + // Make offsets relative to b + lastOfs += hint; + ofs += hint; + } + assert -1 <= lastOfs && lastOfs < ofs && ofs <= len; + + /* + * Now a[b + lastOfs] <= key < a[b + ofs], so key belongs somewhere to + * the right of lastOfs but no farther right than ofs. Do a binary + * search, with invariant a[b + lastOfs - 1] <= key < a[b + ofs]. + */ + lastOfs++; + while (lastOfs < ofs) { + int m = lastOfs + ((ofs - lastOfs) >>> 1); + + if (c.compare(key, s.getKey(a, base + m)) < 0) + ofs = m; // key < a[b + m] + else + lastOfs = m + 1; // a[b + m] <= key + } + assert lastOfs == ofs; // so a[b + ofs - 1] <= key < a[b + ofs] + return ofs; + } + + /** + * Merges two adjacent runs in place, in a stable fashion. The first + * element of the first run must be greater than the first element of the + * second run (a[base1] > a[base2]), and the last element of the first run + * (a[base1 + len1-1]) must be greater than all elements of the second run. + * + * For performance, this method should be called only when len1 <= len2; + * its twin, mergeHi should be called if len1 >= len2. (Either method + * may be called if len1 == len2.) + * + * @param base1 index of first element in first run to be merged + * @param len1 length of first run to be merged (must be > 0) + * @param base2 index of first element in second run to be merged + * (must be aBase + aLen) + * @param len2 length of second run to be merged (must be > 0) + */ + private void mergeLo(int base1, int len1, int base2, int len2) { + assert len1 > 0 && len2 > 0 && base1 + len1 == base2; + + // Copy first run into temp array + Buffer a = this.a; // For performance + Buffer tmp = ensureCapacity(len1); + s.copyRange(a, base1, tmp, 0, len1); + + int cursor1 = 0; // Indexes into tmp array + int cursor2 = base2; // Indexes int a + int dest = base1; // Indexes int a + + // Move first element of second run and deal with degenerate cases + s.copyElement(a, cursor2++, a, dest++); + if (--len2 == 0) { + s.copyRange(tmp, cursor1, a, dest, len1); + return; + } + if (len1 == 1) { + s.copyRange(a, cursor2, a, dest, len2); + s.copyElement(tmp, cursor1, a, dest + len2); // Last elt of run 1 to end of merge + return; + } + + Comparator c = this.c; // Use local variable for performance + int minGallop = this.minGallop; // " " " " " + outer: + while (true) { + int count1 = 0; // Number of times in a row that first run won + int count2 = 0; // Number of times in a row that second run won + + /* + * Do the straightforward thing until (if ever) one run starts + * winning consistently. + */ + do { + assert len1 > 1 && len2 > 0; + if (c.compare(s.getKey(a, cursor2), s.getKey(tmp, cursor1)) < 0) { + s.copyElement(a, cursor2++, a, dest++); + count2++; + count1 = 0; + if (--len2 == 0) + break outer; + } else { + s.copyElement(tmp, cursor1++, a, dest++); + count1++; + count2 = 0; + if (--len1 == 1) + break outer; + } + } while ((count1 | count2) < minGallop); + + /* + * One run is winning so consistently that galloping may be a + * huge win. So try that, and continue galloping until (if ever) + * neither run appears to be winning consistently anymore. + */ + do { + assert len1 > 1 && len2 > 0; + count1 = gallopRight(s.getKey(a, cursor2), tmp, cursor1, len1, 0, c); + if (count1 != 0) { + s.copyRange(tmp, cursor1, a, dest, count1); + dest += count1; + cursor1 += count1; + len1 -= count1; + if (len1 <= 1) // len1 == 1 || len1 == 0 + break outer; + } + s.copyElement(a, cursor2++, a, dest++); + if (--len2 == 0) + break outer; + + count2 = gallopLeft(s.getKey(tmp, cursor1), a, cursor2, len2, 0, c); + if (count2 != 0) { + s.copyRange(a, cursor2, a, dest, count2); + dest += count2; + cursor2 += count2; + len2 -= count2; + if (len2 == 0) + break outer; + } + s.copyElement(tmp, cursor1++, a, dest++); + if (--len1 == 1) + break outer; + minGallop--; + } while (count1 >= MIN_GALLOP | count2 >= MIN_GALLOP); + if (minGallop < 0) + minGallop = 0; + minGallop += 2; // Penalize for leaving gallop mode + } // End of "outer" loop + this.minGallop = minGallop < 1 ? 1 : minGallop; // Write back to field + + if (len1 == 1) { + assert len2 > 0; + s.copyRange(a, cursor2, a, dest, len2); + s.copyElement(tmp, cursor1, a, dest + len2); // Last elt of run 1 to end of merge + } else if (len1 == 0) { + throw new IllegalArgumentException( + "Comparison method violates its general contract!"); + } else { + assert len2 == 0; + assert len1 > 1; + s.copyRange(tmp, cursor1, a, dest, len1); + } + } + + /** + * Like mergeLo, except that this method should be called only if + * len1 >= len2; mergeLo should be called if len1 <= len2. (Either method + * may be called if len1 == len2.) + * + * @param base1 index of first element in first run to be merged + * @param len1 length of first run to be merged (must be > 0) + * @param base2 index of first element in second run to be merged + * (must be aBase + aLen) + * @param len2 length of second run to be merged (must be > 0) + */ + private void mergeHi(int base1, int len1, int base2, int len2) { + assert len1 > 0 && len2 > 0 && base1 + len1 == base2; + + // Copy second run into temp array + Buffer a = this.a; // For performance + Buffer tmp = ensureCapacity(len2); + s.copyRange(a, base2, tmp, 0, len2); + + int cursor1 = base1 + len1 - 1; // Indexes into a + int cursor2 = len2 - 1; // Indexes into tmp array + int dest = base2 + len2 - 1; // Indexes into a + + // Move last element of first run and deal with degenerate cases + s.copyElement(a, cursor1--, a, dest--); + if (--len1 == 0) { + s.copyRange(tmp, 0, a, dest - (len2 - 1), len2); + return; + } + if (len2 == 1) { + dest -= len1; + cursor1 -= len1; + s.copyRange(a, cursor1 + 1, a, dest + 1, len1); + s.copyElement(tmp, cursor2, a, dest); + return; + } + + Comparator c = this.c; // Use local variable for performance + int minGallop = this.minGallop; // " " " " " + outer: + while (true) { + int count1 = 0; // Number of times in a row that first run won + int count2 = 0; // Number of times in a row that second run won + + /* + * Do the straightforward thing until (if ever) one run + * appears to win consistently. + */ + do { + assert len1 > 0 && len2 > 1; + if (c.compare(s.getKey(tmp, cursor2), s.getKey(a, cursor1)) < 0) { + s.copyElement(a, cursor1--, a, dest--); + count1++; + count2 = 0; + if (--len1 == 0) + break outer; + } else { + s.copyElement(tmp, cursor2--, a, dest--); + count2++; + count1 = 0; + if (--len2 == 1) + break outer; + } + } while ((count1 | count2) < minGallop); + + /* + * One run is winning so consistently that galloping may be a + * huge win. So try that, and continue galloping until (if ever) + * neither run appears to be winning consistently anymore. + */ + do { + assert len1 > 0 && len2 > 1; + count1 = len1 - gallopRight(s.getKey(tmp, cursor2), a, base1, len1, len1 - 1, c); + if (count1 != 0) { + dest -= count1; + cursor1 -= count1; + len1 -= count1; + s.copyRange(a, cursor1 + 1, a, dest + 1, count1); + if (len1 == 0) + break outer; + } + s.copyElement(tmp, cursor2--, a, dest--); + if (--len2 == 1) + break outer; + + count2 = len2 - gallopLeft(s.getKey(a, cursor1), tmp, 0, len2, len2 - 1, c); + if (count2 != 0) { + dest -= count2; + cursor2 -= count2; + len2 -= count2; + s.copyRange(tmp, cursor2 + 1, a, dest + 1, count2); + if (len2 <= 1) // len2 == 1 || len2 == 0 + break outer; + } + s.copyElement(a, cursor1--, a, dest--); + if (--len1 == 0) + break outer; + minGallop--; + } while (count1 >= MIN_GALLOP | count2 >= MIN_GALLOP); + if (minGallop < 0) + minGallop = 0; + minGallop += 2; // Penalize for leaving gallop mode + } // End of "outer" loop + this.minGallop = minGallop < 1 ? 1 : minGallop; // Write back to field + + if (len2 == 1) { + assert len1 > 0; + dest -= len1; + cursor1 -= len1; + s.copyRange(a, cursor1 + 1, a, dest + 1, len1); + s.copyElement(tmp, cursor2, a, dest); // Move first elt of run2 to front of merge + } else if (len2 == 0) { + throw new IllegalArgumentException( + "Comparison method violates its general contract!"); + } else { + assert len1 == 0; + assert len2 > 0; + s.copyRange(tmp, 0, a, dest - (len2 - 1), len2); + } + } + + /** + * Ensures that the external array tmp has at least the specified + * number of elements, increasing its size if necessary. The size + * increases exponentially to ensure amortized linear time complexity. + * + * @param minCapacity the minimum required capacity of the tmp array + * @return tmp, whether or not it grew + */ + private Buffer ensureCapacity(int minCapacity) { + if (tmpLength < minCapacity) { + // Compute smallest power of 2 > minCapacity + int newSize = minCapacity; + newSize |= newSize >> 1; + newSize |= newSize >> 2; + newSize |= newSize >> 4; + newSize |= newSize >> 8; + newSize |= newSize >> 16; + newSize++; + + if (newSize < 0) // Not bloody likely! + newSize = minCapacity; + else + newSize = Math.min(newSize, aLength >>> 1); + + tmp = s.allocate(newSize); + tmpLength = newSize; + } + return tmp; + } + } +} 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 7448af87fcf38..445110d63e184 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 @@ -81,7 +81,9 @@ table.sortable thead { span.kill-link { margin-right: 2px; + margin-left: 20px; color: gray; + float: right; } span.kill-link a { diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala index 9c55bfbb47626..12f2fe031cb1d 100644 --- a/core/src/main/scala/org/apache/spark/Accumulators.scala +++ b/core/src/main/scala/org/apache/spark/Accumulators.scala @@ -36,15 +36,21 @@ import org.apache.spark.serializer.JavaSerializer * * @param initialValue initial value of accumulator * @param param helper object defining how to add elements of type `R` and `T` + * @param name human-readable name for use in Spark's web UI * @tparam R the full accumulated data (result type) * @tparam T partial data that can be added in */ class Accumulable[R, T] ( @transient initialValue: R, - param: AccumulableParam[R, T]) + param: AccumulableParam[R, T], + val name: Option[String]) extends Serializable { - val id = Accumulators.newId + def this(@transient initialValue: R, param: AccumulableParam[R, T]) = + this(initialValue, param, None) + + val id: Long = Accumulators.newId + @transient private var value_ = initialValue // Current value on master val zero = param.zero(initialValue) // Zero value to be passed to workers private var deserialized = false @@ -219,8 +225,10 @@ GrowableAccumulableParam[R <% Growable[T] with TraversableOnce[T] with Serializa * @param param helper object defining how to add elements of type `T` * @tparam T result type */ -class Accumulator[T](@transient initialValue: T, param: AccumulatorParam[T]) - extends Accumulable[T,T](initialValue, param) +class Accumulator[T](@transient initialValue: T, param: AccumulatorParam[T], name: Option[String]) + extends Accumulable[T,T](initialValue, param, name) { + def this(initialValue: T, param: AccumulatorParam[T]) = this(initialValue, param, None) +} /** * A simpler version of [[org.apache.spark.AccumulableParam]] where the only data type you can add @@ -281,4 +289,7 @@ private object Accumulators { } } } + + def stringifyPartialValue(partialValue: Any) = "%s".format(partialValue) + def stringifyValue(value: Any) = "%s".format(value) } diff --git a/core/src/main/scala/org/apache/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala index 59fdf659c9e11..79c9c451d273d 100644 --- a/core/src/main/scala/org/apache/spark/Aggregator.scala +++ b/core/src/main/scala/org/apache/spark/Aggregator.scala @@ -55,22 +55,24 @@ case class Aggregator[K, V, C] ( combiners.iterator } else { val combiners = new ExternalAppendOnlyMap[K, V, C](createCombiner, mergeValue, mergeCombiners) - while (iter.hasNext) { - val (k, v) = iter.next() - combiners.insert(k, v) + combiners.insertAll(iter) + // Update task metrics if context is not null + // TODO: Make context non optional in a future release + Option(context).foreach { c => + c.taskMetrics.memoryBytesSpilled += combiners.memoryBytesSpilled + c.taskMetrics.diskBytesSpilled += combiners.diskBytesSpilled } - // TODO: Make this non optional in a future release - Option(context).foreach(c => c.taskMetrics.memoryBytesSpilled = combiners.memoryBytesSpilled) - Option(context).foreach(c => c.taskMetrics.diskBytesSpilled = combiners.diskBytesSpilled) combiners.iterator } } @deprecated("use combineCombinersByKey with TaskContext argument", "0.9.0") - def combineCombinersByKey(iter: Iterator[(K, C)]) : Iterator[(K, C)] = + def combineCombinersByKey(iter: Iterator[_ <: Product2[K, C]]) : Iterator[(K, C)] = combineCombinersByKey(iter, null) - def combineCombinersByKey(iter: Iterator[(K, C)], context: TaskContext) : Iterator[(K, C)] = { + def combineCombinersByKey(iter: Iterator[_ <: Product2[K, C]], context: TaskContext) + : Iterator[(K, C)] = + { if (!externalSorting) { val combiners = new AppendOnlyMap[K,C] var kc: Product2[K, C] = null @@ -85,12 +87,15 @@ case class Aggregator[K, V, C] ( } else { val combiners = new ExternalAppendOnlyMap[K, C, C](identity, mergeCombiners, mergeCombiners) while (iter.hasNext) { - val (k, c) = iter.next() - combiners.insert(k, c) + val pair = iter.next() + combiners.insert(pair._1, pair._2) + } + // Update task metrics if context is not null + // TODO: Make context non-optional in a future release + Option(context).foreach { c => + c.taskMetrics.memoryBytesSpilled += combiners.memoryBytesSpilled + c.taskMetrics.diskBytesSpilled += combiners.diskBytesSpilled } - // TODO: Make this non optional in a future release - Option(context).foreach(c => c.taskMetrics.memoryBytesSpilled = combiners.memoryBytesSpilled) - Option(context).foreach(c => c.taskMetrics.diskBytesSpilled = combiners.diskBytesSpilled) combiners.iterator } } diff --git a/core/src/main/scala/org/apache/spark/CacheManager.scala b/core/src/main/scala/org/apache/spark/CacheManager.scala index 8f867686a0443..5ddda4d6953fa 100644 --- a/core/src/main/scala/org/apache/spark/CacheManager.scala +++ b/core/src/main/scala/org/apache/spark/CacheManager.scala @@ -17,9 +17,9 @@ package org.apache.spark -import scala.collection.mutable.{ArrayBuffer, HashSet} +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer -import org.apache.spark.executor.InputMetrics import org.apache.spark.rdd.RDD import org.apache.spark.storage._ @@ -30,7 +30,7 @@ import org.apache.spark.storage._ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { /** Keys of RDD partitions that are being computed/loaded. */ - private val loading = new HashSet[RDDBlockId]() + private val loading = new mutable.HashSet[RDDBlockId] /** Gets or computes an RDD partition. Used by RDD.iterator() when an RDD is cached. */ def getOrCompute[T]( @@ -118,21 +118,29 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { } /** - * Cache the values of a partition, keeping track of any updates in the storage statuses - * of other blocks along the way. + * Cache the values of a partition, keeping track of any updates in the storage statuses of + * other blocks along the way. + * + * The effective storage level refers to the level that actually specifies BlockManager put + * behavior, not the level originally specified by the user. This is mainly for forcing a + * MEMORY_AND_DISK partition to disk if there is not enough room to unroll the partition, + * while preserving the the original semantics of the RDD as specified by the application. */ private def putInBlockManager[T]( key: BlockId, values: Iterator[T], - storageLevel: StorageLevel, - updatedBlocks: ArrayBuffer[(BlockId, BlockStatus)]): Iterator[T] = { - - if (!storageLevel.useMemory) { - /* This RDD is not to be cached in memory, so we can just pass the computed values - * as an iterator directly to the BlockManager, rather than first fully unrolling - * it in memory. The latter option potentially uses much more memory and risks OOM - * exceptions that can be avoided. */ - updatedBlocks ++= blockManager.put(key, values, storageLevel, tellMaster = true) + level: StorageLevel, + updatedBlocks: ArrayBuffer[(BlockId, BlockStatus)], + effectiveStorageLevel: Option[StorageLevel] = None): Iterator[T] = { + + val putLevel = effectiveStorageLevel.getOrElse(level) + if (!putLevel.useMemory) { + /* + * This RDD is not to be cached in memory, so we can just pass the computed values as an + * iterator directly to the BlockManager rather than first fully unrolling it in memory. + */ + updatedBlocks ++= + blockManager.putIterator(key, values, level, tellMaster = true, effectiveStorageLevel) blockManager.get(key) match { case Some(v) => v.data.asInstanceOf[Iterator[T]] case None => @@ -140,14 +148,36 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { throw new BlockException(key, s"Block manager failed to return cached value for $key!") } } else { - /* This RDD is to be cached in memory. In this case we cannot pass the computed values + /* + * This RDD is to be cached in memory. In this case we cannot pass the computed values * to the BlockManager as an iterator and expect to read it back later. This is because - * we may end up dropping a partition from memory store before getting it back, e.g. - * when the entirety of the RDD does not fit in memory. */ - val elements = new ArrayBuffer[Any] - elements ++= values - updatedBlocks ++= blockManager.put(key, elements, storageLevel, tellMaster = true) - elements.iterator.asInstanceOf[Iterator[T]] + * we may end up dropping a partition from memory store before getting it back. + * + * In addition, we must be careful to not unroll the entire partition in memory at once. + * Otherwise, we may cause an OOM exception if the JVM does not have enough space for this + * single partition. Instead, we unroll the values cautiously, potentially aborting and + * dropping the partition to disk if applicable. + */ + blockManager.memoryStore.unrollSafely(key, values, updatedBlocks) match { + case Left(arr) => + // We have successfully unrolled the entire partition, so cache it in memory + updatedBlocks ++= + blockManager.putArray(key, arr, level, tellMaster = true, effectiveStorageLevel) + arr.iterator.asInstanceOf[Iterator[T]] + case Right(it) => + // There is not enough space to cache this partition in memory + logWarning(s"Not enough space to cache partition $key in memory! " + + s"Free memory is ${blockManager.memoryStore.freeMemory} bytes.") + val returnValues = it.asInstanceOf[Iterator[T]] + if (putLevel.useDisk) { + logWarning(s"Persisting partition $key to disk instead.") + val diskOnlyLevel = StorageLevel(useDisk = true, useMemory = false, + useOffHeap = false, deserialized = false, putLevel.replication) + putInBlockManager[T](key, returnValues, level, updatedBlocks, Some(diskOnlyLevel)) + } else { + returnValues + } + } } } diff --git a/core/src/main/scala/org/apache/spark/ContextCleaner.scala b/core/src/main/scala/org/apache/spark/ContextCleaner.scala index bf3c3a6ceb5ef..3848734d6f639 100644 --- a/core/src/main/scala/org/apache/spark/ContextCleaner.scala +++ b/core/src/main/scala/org/apache/spark/ContextCleaner.scala @@ -66,10 +66,15 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { /** * Whether the cleaning thread will block on cleanup tasks. - * This is set to true only for tests. + * + * Due to SPARK-3015, this is set to true by default. This is intended to be only a temporary + * workaround for the issue, which is ultimately caused by the way the BlockManager actors + * issue inter-dependent blocking Akka messages to each other at high frequencies. This happens, + * for instance, when the driver performs a GC and cleans up all broadcast blocks that are no + * longer in scope. */ private val blockOnCleanupTasks = sc.conf.getBoolean( - "spark.cleaner.referenceTracking.blocking", false) + "spark.cleaner.referenceTracking.blocking", true) @volatile private var stopped = false @@ -174,9 +179,6 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { private def blockManagerMaster = sc.env.blockManager.master private def broadcastManager = sc.env.broadcastManager private def mapOutputTrackerMaster = sc.env.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster] - - // Used for testing. These methods explicitly blocks until cleanup is completed - // to ensure that more reliable testing. } private object ContextCleaner { diff --git a/core/src/main/scala/org/apache/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala index 09a60571238ea..ab2594cfc02eb 100644 --- a/core/src/main/scala/org/apache/spark/Dependency.scala +++ b/core/src/main/scala/org/apache/spark/Dependency.scala @@ -27,29 +27,35 @@ import org.apache.spark.shuffle.ShuffleHandle * Base class for dependencies. */ @DeveloperApi -abstract class Dependency[T](val rdd: RDD[T]) extends Serializable +abstract class Dependency[T] extends Serializable { + def rdd: RDD[T] +} /** * :: DeveloperApi :: - * Base class for dependencies where each partition of the parent RDD is used by at most one - * partition of the child RDD. Narrow dependencies allow for pipelined execution. + * Base class for dependencies where each partition of the child RDD depends on a small number + * of partitions of the parent RDD. Narrow dependencies allow for pipelined execution. */ @DeveloperApi -abstract class NarrowDependency[T](rdd: RDD[T]) extends Dependency(rdd) { +abstract class NarrowDependency[T](_rdd: RDD[T]) extends Dependency[T] { /** * Get the parent partitions for a child partition. * @param partitionId a partition of the child RDD * @return the partitions of the parent RDD that the child partition depends upon */ def getParents(partitionId: Int): Seq[Int] + + override def rdd: RDD[T] = _rdd } /** * :: DeveloperApi :: - * Represents a dependency on the output of a shuffle stage. - * @param rdd the parent RDD + * Represents a dependency on the output of a shuffle stage. Note that in the case of shuffle, + * the RDD is transient since we don't need it on the executor side. + * + * @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 None, * the default serializer, as specified by `spark.serializer` config option, will @@ -57,20 +63,22 @@ abstract class NarrowDependency[T](rdd: RDD[T]) extends Dependency(rdd) { */ @DeveloperApi class ShuffleDependency[K, V, C]( - @transient rdd: RDD[_ <: Product2[K, V]], + @transient _rdd: RDD[_ <: Product2[K, V]], val partitioner: Partitioner, val serializer: Option[Serializer] = None, val keyOrdering: Option[Ordering[K]] = None, val aggregator: Option[Aggregator[K, V, C]] = None, val mapSideCombine: Boolean = false) - extends Dependency(rdd.asInstanceOf[RDD[Product2[K, V]]]) { + extends Dependency[Product2[K, V]] { + + override def rdd = _rdd.asInstanceOf[RDD[Product2[K, V]]] - val shuffleId: Int = rdd.context.newShuffleId() + val shuffleId: Int = _rdd.context.newShuffleId() - val shuffleHandle: ShuffleHandle = rdd.context.env.shuffleManager.registerShuffle( - shuffleId, rdd.partitions.size, this) + val shuffleHandle: ShuffleHandle = _rdd.context.env.shuffleManager.registerShuffle( + shuffleId, _rdd.partitions.size, this) - rdd.sparkContext.cleaner.foreach(_.registerShuffleForCleanup(this)) + _rdd.sparkContext.cleaner.foreach(_.registerShuffleForCleanup(this)) } diff --git a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala new file mode 100644 index 0000000000000..83ae57b7f1516 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark + +import akka.actor.Actor +import org.apache.spark.executor.TaskMetrics +import org.apache.spark.storage.BlockManagerId +import org.apache.spark.scheduler.TaskScheduler +import org.apache.spark.util.ActorLogReceive + +/** + * A heartbeat from executors to the driver. This is a shared message used by several internal + * components to convey liveness or execution information for in-progress tasks. + */ +private[spark] case class Heartbeat( + executorId: String, + taskMetrics: Array[(Long, TaskMetrics)], // taskId -> TaskMetrics + blockManagerId: BlockManagerId) + +private[spark] case class HeartbeatResponse(reregisterBlockManager: Boolean) + +/** + * Lives in the driver to receive heartbeats from executors.. + */ +private[spark] class HeartbeatReceiver(scheduler: TaskScheduler) + extends Actor with ActorLogReceive with Logging { + + override def receiveWithLogging = { + case Heartbeat(executorId, taskMetrics, blockManagerId) => + val response = HeartbeatResponse( + !scheduler.executorHeartbeatReceived(executorId, taskMetrics, blockManagerId)) + sender ! response + } +} diff --git a/core/src/main/scala/org/apache/spark/HttpFileServer.scala b/core/src/main/scala/org/apache/spark/HttpFileServer.scala index 0e3750fdde415..edc3889c9ae51 100644 --- a/core/src/main/scala/org/apache/spark/HttpFileServer.scala +++ b/core/src/main/scala/org/apache/spark/HttpFileServer.scala @@ -23,7 +23,10 @@ import com.google.common.io.Files import org.apache.spark.util.Utils -private[spark] class HttpFileServer(securityManager: SecurityManager) extends Logging { +private[spark] class HttpFileServer( + securityManager: SecurityManager, + requestedPort: Int = 0) + extends Logging { var baseDir : File = null var fileDir : File = null @@ -38,7 +41,7 @@ private[spark] class HttpFileServer(securityManager: SecurityManager) extends Lo fileDir.mkdir() jarDir.mkdir() logInfo("HTTP File server directory is " + baseDir) - httpServer = new HttpServer(baseDir, securityManager) + httpServer = new HttpServer(baseDir, securityManager, requestedPort, "HTTP file server") httpServer.start() serverUri = httpServer.uri logDebug("HTTP file server started at: " + serverUri) diff --git a/core/src/main/scala/org/apache/spark/HttpServer.scala b/core/src/main/scala/org/apache/spark/HttpServer.scala index 7e9b517f901a2..912558d0cab7d 100644 --- a/core/src/main/scala/org/apache/spark/HttpServer.scala +++ b/core/src/main/scala/org/apache/spark/HttpServer.scala @@ -21,7 +21,7 @@ import java.io.File import org.eclipse.jetty.util.security.{Constraint, Password} import org.eclipse.jetty.security.authentication.DigestAuthenticator -import org.eclipse.jetty.security.{ConstraintMapping, ConstraintSecurityHandler, HashLoginService, SecurityHandler} +import org.eclipse.jetty.security.{ConstraintMapping, ConstraintSecurityHandler, HashLoginService} import org.eclipse.jetty.server.Server import org.eclipse.jetty.server.bio.SocketConnector @@ -41,48 +41,68 @@ private[spark] class ServerStateException(message: String) extends Exception(mes * as well as classes created by the interpreter when the user types in code. This is just a wrapper * around a Jetty server. */ -private[spark] class HttpServer(resourceBase: File, securityManager: SecurityManager) - extends Logging { +private[spark] class HttpServer( + resourceBase: File, + securityManager: SecurityManager, + requestedPort: Int = 0, + serverName: String = "HTTP server") + extends Logging { + private var server: Server = null - private var port: Int = -1 + private var port: Int = requestedPort def start() { if (server != null) { throw new ServerStateException("Server is already started") } else { logInfo("Starting HTTP Server") - server = new Server() - val connector = new SocketConnector - connector.setMaxIdleTime(60*1000) - connector.setSoLingerTime(-1) - connector.setPort(0) - server.addConnector(connector) - - val threadPool = new QueuedThreadPool - threadPool.setDaemon(true) - server.setThreadPool(threadPool) - val resHandler = new ResourceHandler - resHandler.setResourceBase(resourceBase.getAbsolutePath) - - val handlerList = new HandlerList - handlerList.setHandlers(Array(resHandler, new DefaultHandler)) - - if (securityManager.isAuthenticationEnabled()) { - logDebug("HttpServer is using security") - val sh = setupSecurityHandler(securityManager) - // make sure we go through security handler to get resources - sh.setHandler(handlerList) - server.setHandler(sh) - } else { - logDebug("HttpServer is not using security") - server.setHandler(handlerList) - } - - server.start() - port = server.getConnectors()(0).getLocalPort() + val (actualServer, actualPort) = + Utils.startServiceOnPort[Server](requestedPort, doStart, serverName) + server = actualServer + port = actualPort } } + /** + * Actually start the HTTP server on the given port. + * + * Note that this is only best effort in the sense that we may end up binding to a nearby port + * in the event of port collision. Return the bound server and the actual port used. + */ + private def doStart(startPort: Int): (Server, Int) = { + val server = new Server() + val connector = new SocketConnector + connector.setMaxIdleTime(60 * 1000) + connector.setSoLingerTime(-1) + connector.setPort(startPort) + server.addConnector(connector) + + val threadPool = new QueuedThreadPool + threadPool.setDaemon(true) + server.setThreadPool(threadPool) + val resHandler = new ResourceHandler + resHandler.setResourceBase(resourceBase.getAbsolutePath) + + val handlerList = new HandlerList + handlerList.setHandlers(Array(resHandler, new DefaultHandler)) + + if (securityManager.isAuthenticationEnabled()) { + logDebug("HttpServer is using security") + val sh = setupSecurityHandler(securityManager) + // make sure we go through security handler to get resources + sh.setHandler(handlerList) + server.setHandler(sh) + } else { + logDebug("HttpServer is not using security") + server.setHandler(handlerList) + } + + server.start() + val actualPort = server.getConnectors()(0).getLocalPort + + (server, actualPort) + } + /** * Setup Jetty to the HashLoginService using a single user with our * shared secret. Configure it to use DIGEST-MD5 authentication so that the password @@ -134,7 +154,7 @@ private[spark] class HttpServer(resourceBase: File, securityManager: SecurityMan if (server == null) { throw new ServerStateException("Server is not started") } else { - return "http://" + Utils.localIpAddress + ":" + port + "http://" + Utils.localIpAddress + ":" + port } } } diff --git a/core/src/main/scala/org/apache/spark/InterruptibleIterator.scala b/core/src/main/scala/org/apache/spark/InterruptibleIterator.scala index f40baa8e43592..5c262bcbddf76 100644 --- a/core/src/main/scala/org/apache/spark/InterruptibleIterator.scala +++ b/core/src/main/scala/org/apache/spark/InterruptibleIterator.scala @@ -33,7 +33,7 @@ class InterruptibleIterator[+T](val context: TaskContext, val delegate: Iterator // is allowed. The assumption is that Thread.interrupted does not have a memory fence in read // (just a volatile field in C), while context.interrupted is a volatile in the JVM, which // introduces an expensive read fence. - if (context.interrupted) { + if (context.isInterrupted) { throw new TaskKilledException } else { delegate.hasNext diff --git a/core/src/main/scala/org/apache/spark/Logging.scala b/core/src/main/scala/org/apache/spark/Logging.scala index 50d8e93e1f0d7..d4f2624061e35 100644 --- a/core/src/main/scala/org/apache/spark/Logging.scala +++ b/core/src/main/scala/org/apache/spark/Logging.scala @@ -39,16 +39,17 @@ trait Logging { // be serialized and used on another machine @transient private var log_ : Logger = null + // Method to get the logger name for this object + protected def logName = { + // Ignore trailing $'s in the class names for Scala objects + this.getClass.getName.stripSuffix("$") + } + // Method to get or create the logger for this object protected def log: Logger = { if (log_ == null) { initializeIfNecessary() - var className = this.getClass.getName - // Ignore trailing $'s in the class names for Scala objects - if (className.endsWith("$")) { - className = className.substring(0, className.length - 1) - } - log_ = LoggerFactory.getLogger(className) + log_ = LoggerFactory.getLogger(logName) } log_ } @@ -110,23 +111,27 @@ trait Logging { } private def initializeLogging() { - // If Log4j is being used, but is not initialized, load a default properties file - val binder = StaticLoggerBinder.getSingleton - val usingLog4j = binder.getLoggerFactoryClassStr.endsWith("Log4jLoggerFactory") - val log4jInitialized = LogManager.getRootLogger.getAllAppenders.hasMoreElements - if (!log4jInitialized && usingLog4j) { + // Don't use a logger in here, as this is itself occurring during initialization of a logger + // If Log4j 1.2 is being used, but is not initialized, load a default properties file + val binderClass = StaticLoggerBinder.getSingleton.getLoggerFactoryClassStr + // This distinguishes the log4j 1.2 binding, currently + // org.slf4j.impl.Log4jLoggerFactory, from the log4j 2.0 binding, currently + // org.apache.logging.slf4j.Log4jLoggerFactory + val usingLog4j12 = "org.slf4j.impl.Log4jLoggerFactory".equals(binderClass) + val log4j12Initialized = LogManager.getRootLogger.getAllAppenders.hasMoreElements + if (!log4j12Initialized && usingLog4j12) { val defaultLogProps = "org/apache/spark/log4j-defaults.properties" Option(Utils.getSparkClassLoader.getResource(defaultLogProps)) match { case Some(url) => PropertyConfigurator.configure(url) - log.info(s"Using Spark's default log4j profile: $defaultLogProps") + System.err.println(s"Using Spark's default log4j profile: $defaultLogProps") case None => System.err.println(s"Spark was unable to load $defaultLogProps") } } Logging.initialized = true - // Force a call into slf4j to initialize it. Avoids this happening from mutliple threads + // Force a call into slf4j to initialize it. Avoids this happening from multiple threads // and triggering this: http://mailman.qos.ch/pipermail/slf4j-dev/2010-April/002956.html log } diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 894091761485d..51705c895a55c 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -38,10 +38,10 @@ private[spark] case object StopMapOutputTracker extends MapOutputTrackerMessage /** Actor class for MapOutputTrackerMaster */ private[spark] class MapOutputTrackerMasterActor(tracker: MapOutputTrackerMaster, conf: SparkConf) - extends Actor with Logging { + extends Actor with ActorLogReceive with Logging { val maxAkkaFrameSize = AkkaUtils.maxFrameSizeBytes(conf) - def receive = { + override def receiveWithLogging = { case GetMapOutputStatuses(shuffleId: Int) => val hostPort = sender.path.address.hostPort logInfo("Asked to send map output locations for shuffle " + shuffleId + " to " + hostPort) diff --git a/core/src/main/scala/org/apache/spark/Partitioner.scala b/core/src/main/scala/org/apache/spark/Partitioner.scala index ec99648a8488a..37053bb6f37ad 100644 --- a/core/src/main/scala/org/apache/spark/Partitioner.scala +++ b/core/src/main/scala/org/apache/spark/Partitioner.scala @@ -19,11 +19,15 @@ package org.apache.spark import java.io.{IOException, ObjectInputStream, ObjectOutputStream} -import scala.reflect.ClassTag +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer +import scala.reflect.{ClassTag, classTag} +import scala.util.hashing.byteswap32 -import org.apache.spark.rdd.RDD +import org.apache.spark.rdd.{PartitionPruningRDD, RDD} import org.apache.spark.serializer.JavaSerializer import org.apache.spark.util.{CollectionsUtils, Utils} +import org.apache.spark.util.random.{XORShiftRandom, SamplingUtils} /** * An object that defines how the elements in a key-value pair RDD are partitioned by key. @@ -103,26 +107,49 @@ class RangePartitioner[K : Ordering : ClassTag, V]( private var ascending: Boolean = true) extends Partitioner { + // We allow partitions = 0, which happens when sorting an empty RDD under the default settings. + require(partitions >= 0, s"Number of partitions cannot be negative but found $partitions.") + private var ordering = implicitly[Ordering[K]] // An array of upper bounds for the first (partitions - 1) partitions private var rangeBounds: Array[K] = { - if (partitions == 1) { - Array() + if (partitions <= 1) { + Array.empty } else { - val rddSize = rdd.count() - val maxSampleSize = partitions * 20.0 - val frac = math.min(maxSampleSize / math.max(rddSize, 1), 1.0) - val rddSample = rdd.sample(false, frac, 1).map(_._1).collect().sorted - if (rddSample.length == 0) { - Array() + // This is the sample size we need to have roughly balanced output partitions, capped at 1M. + val sampleSize = math.min(20.0 * partitions, 1e6) + // Assume the input partitions are roughly balanced and over-sample a little bit. + val sampleSizePerPartition = math.ceil(3.0 * sampleSize / rdd.partitions.size).toInt + val (numItems, sketched) = RangePartitioner.sketch(rdd.map(_._1), sampleSizePerPartition) + if (numItems == 0L) { + Array.empty } else { - val bounds = new Array[K](partitions - 1) - for (i <- 0 until partitions - 1) { - val index = (rddSample.length - 1) * (i + 1) / partitions - bounds(i) = rddSample(index) + // If a partition contains much more than the average number of items, we re-sample from it + // to ensure that enough items are collected from that partition. + val fraction = math.min(sampleSize / math.max(numItems, 1L), 1.0) + val candidates = ArrayBuffer.empty[(K, Float)] + val imbalancedPartitions = mutable.Set.empty[Int] + sketched.foreach { case (idx, n, sample) => + if (fraction * n > sampleSizePerPartition) { + imbalancedPartitions += idx + } else { + // The weight is 1 over the sampling probability. + val weight = (n.toDouble / sample.size).toFloat + for (key <- sample) { + candidates += ((key, weight)) + } + } + } + if (imbalancedPartitions.nonEmpty) { + // Re-sample imbalanced partitions with the desired sampling probability. + val imbalanced = new PartitionPruningRDD(rdd.map(_._1), imbalancedPartitions.contains) + val seed = byteswap32(-rdd.id - 1) + val reSampled = imbalanced.sample(withReplacement = false, fraction, seed).collect() + val weight = (1.0 / fraction).toFloat + candidates ++= reSampled.map(x => (x, weight)) } - bounds + RangePartitioner.determineBounds(candidates, partitions) } } } @@ -134,8 +161,8 @@ class RangePartitioner[K : Ordering : ClassTag, V]( def getPartition(key: Any): Int = { val k = key.asInstanceOf[K] var partition = 0 - if (rangeBounds.length < 1000) { - // If we have less than 100 partitions naive search + if (rangeBounds.length <= 128) { + // If we have less than 128 partitions naive search while (partition < rangeBounds.length && ordering.gt(k, rangeBounds(partition))) { partition += 1 } @@ -212,3 +239,67 @@ class RangePartitioner[K : Ordering : ClassTag, V]( } } } + +private[spark] object RangePartitioner { + + /** + * Sketches the input RDD via reservoir sampling on each partition. + * + * @param rdd the input RDD to sketch + * @param sampleSizePerPartition max sample size per partition + * @return (total number of items, an array of (partitionId, number of items, sample)) + */ + def sketch[K:ClassTag]( + rdd: RDD[K], + sampleSizePerPartition: Int): (Long, Array[(Int, Int, Array[K])]) = { + val shift = rdd.id + // val classTagK = classTag[K] // to avoid serializing the entire partitioner object + val sketched = rdd.mapPartitionsWithIndex { (idx, iter) => + val seed = byteswap32(idx ^ (shift << 16)) + val (sample, n) = SamplingUtils.reservoirSampleAndCount( + iter, sampleSizePerPartition, seed) + Iterator((idx, n, sample)) + }.collect() + val numItems = sketched.map(_._2.toLong).sum + (numItems, sketched) + } + + /** + * Determines the bounds for range partitioning from candidates with weights indicating how many + * items each represents. Usually this is 1 over the probability used to sample this candidate. + * + * @param candidates unordered candidates with weights + * @param partitions number of partitions + * @return selected bounds + */ + def determineBounds[K:Ordering:ClassTag]( + candidates: ArrayBuffer[(K, Float)], + partitions: Int): Array[K] = { + val ordering = implicitly[Ordering[K]] + val ordered = candidates.sortBy(_._1) + val numCandidates = ordered.size + val sumWeights = ordered.map(_._2.toDouble).sum + val step = sumWeights / partitions + var cumWeight = 0.0 + var target = step + val bounds = ArrayBuffer.empty[K] + var i = 0 + var j = 0 + var previousBound = Option.empty[K] + while ((i < numCandidates) && (j < partitions - 1)) { + val (key, weight) = ordered(i) + cumWeight += weight + if (cumWeight > target) { + // Skip duplicate values. + if (previousBound.isEmpty || ordering.gt(key, previousBound.get)) { + bounds += key + target += step + j += 1 + previousBound = Some(key) + } + } + i += 1 + } + bounds.toArray + } +} diff --git a/core/src/main/scala/org/apache/spark/SecurityManager.scala b/core/src/main/scala/org/apache/spark/SecurityManager.scala index 74aa441619bd2..25c2c9fc6af7c 100644 --- a/core/src/main/scala/org/apache/spark/SecurityManager.scala +++ b/core/src/main/scala/org/apache/spark/SecurityManager.scala @@ -41,10 +41,19 @@ import org.apache.spark.deploy.SparkHadoopUtil * secure the UI if it has data that other users should not be allowed to see. The javax * servlet filter specified by the user can authenticate the user and then once the user * is logged in, Spark can compare that user versus the view acls to make sure they are - * authorized to view the UI. The configs 'spark.ui.acls.enable' and 'spark.ui.view.acls' + * authorized to view the UI. The configs 'spark.acls.enable' and 'spark.ui.view.acls' * control the behavior of the acls. Note that the person who started the application * always has view access to the UI. * + * Spark has a set of modify acls (`spark.modify.acls`) that controls which users have permission + * to modify a single application. This would include things like killing the application. By + * default the person who started the application has modify access. For modify access through + * the UI, you must have a filter that does authentication in place for the modify acls to work + * properly. + * + * Spark also has a set of admin acls (`spark.admin.acls`) which is a set of users/administrators + * who always have permission to view or modify the Spark application. + * * Spark does not currently support encryption after authentication. * * At this point spark has multiple communication protocols that need to be secured and @@ -137,18 +146,32 @@ private[spark] class SecurityManager(sparkConf: SparkConf) extends Logging { private val sparkSecretLookupKey = "sparkCookie" private val authOn = sparkConf.getBoolean("spark.authenticate", false) - private var uiAclsOn = sparkConf.getBoolean("spark.ui.acls.enable", false) + // keep spark.ui.acls.enable for backwards compatibility with 1.0 + private var aclsOn = sparkConf.getOption("spark.acls.enable").getOrElse( + sparkConf.get("spark.ui.acls.enable", "false")).toBoolean + + // admin acls should be set before view or modify acls + private var adminAcls: Set[String] = + stringToSet(sparkConf.get("spark.admin.acls", "")) private var viewAcls: Set[String] = _ + + // list of users who have permission to modify the application. This should + // apply to both UI and CLI for things like killing the application. + private var modifyAcls: Set[String] = _ + // always add the current user and SPARK_USER to the viewAcls - private val defaultAclUsers = Seq[String](System.getProperty("user.name", ""), + private val defaultAclUsers = Set[String](System.getProperty("user.name", ""), Option(System.getenv("SPARK_USER")).getOrElse("")) + setViewAcls(defaultAclUsers, sparkConf.get("spark.ui.view.acls", "")) + setModifyAcls(defaultAclUsers, sparkConf.get("spark.modify.acls", "")) private val secretKey = generateSecretKey() logInfo("SecurityManager: authentication " + (if (authOn) "enabled" else "disabled") + - "; ui acls " + (if (uiAclsOn) "enabled" else "disabled") + - "; users with view permissions: " + viewAcls.toString()) + "; ui acls " + (if (aclsOn) "enabled" else "disabled") + + "; users with view permissions: " + viewAcls.toString() + + "; users with modify permissions: " + modifyAcls.toString()) // Set our own authenticator to properly negotiate user/password for HTTP connections. // This is needed by the HTTP client fetching from the HttpServer. Put here so its @@ -169,18 +192,51 @@ private[spark] class SecurityManager(sparkConf: SparkConf) extends Logging { ) } - private[spark] def setViewAcls(defaultUsers: Seq[String], allowedUsers: String) { - viewAcls = (defaultUsers ++ allowedUsers.split(',')).map(_.trim()).filter(!_.isEmpty).toSet + /** + * Split a comma separated String, filter out any empty items, and return a Set of strings + */ + private def stringToSet(list: String): Set[String] = { + list.split(',').map(_.trim).filter(!_.isEmpty).toSet + } + + /** + * Admin acls should be set before the view or modify acls. If you modify the admin + * acls you should also set the view and modify acls again to pick up the changes. + */ + def setViewAcls(defaultUsers: Set[String], allowedUsers: String) { + viewAcls = (adminAcls ++ defaultUsers ++ stringToSet(allowedUsers)) logInfo("Changing view acls to: " + viewAcls.mkString(",")) } - private[spark] def setViewAcls(defaultUser: String, allowedUsers: String) { - setViewAcls(Seq[String](defaultUser), allowedUsers) + def setViewAcls(defaultUser: String, allowedUsers: String) { + setViewAcls(Set[String](defaultUser), allowedUsers) + } + + def getViewAcls: String = viewAcls.mkString(",") + + /** + * Admin acls should be set before the view or modify acls. If you modify the admin + * acls you should also set the view and modify acls again to pick up the changes. + */ + def setModifyAcls(defaultUsers: Set[String], allowedUsers: String) { + modifyAcls = (adminAcls ++ defaultUsers ++ stringToSet(allowedUsers)) + logInfo("Changing modify acls to: " + modifyAcls.mkString(",")) + } + + def getModifyAcls: String = modifyAcls.mkString(",") + + /** + * Admin acls should be set before the view or modify acls. If you modify the admin + * acls you should also set the view and modify acls again to pick up the changes. + */ + def setAdminAcls(adminUsers: String) { + adminAcls = stringToSet(adminUsers) + logInfo("Changing admin acls to: " + adminAcls.mkString(",")) } - private[spark] def setUIAcls(aclSetting: Boolean) { - uiAclsOn = aclSetting - logInfo("Changing acls enabled to: " + uiAclsOn) + def setAcls(aclSetting: Boolean) { + aclsOn = aclSetting + logInfo("Changing acls enabled to: " + aclsOn) } /** @@ -224,22 +280,39 @@ private[spark] class SecurityManager(sparkConf: SparkConf) extends Logging { * Check to see if Acls for the UI are enabled * @return true if UI authentication is enabled, otherwise false */ - def uiAclsEnabled(): Boolean = uiAclsOn + def aclsEnabled(): Boolean = aclsOn /** * Checks the given user against the view acl list to see if they have - * authorization to view the UI. If the UI acls must are disabled - * via spark.ui.acls.enable, all users have view access. + * authorization to view the UI. If the UI acls are disabled + * via spark.acls.enable, all users have view access. If the user is null + * it is assumed authentication is off and all users have access. * * @param user to see if is authorized * @return true is the user has permission, otherwise false */ def checkUIViewPermissions(user: String): Boolean = { - logDebug("user=" + user + " uiAclsEnabled=" + uiAclsEnabled() + " viewAcls=" + + logDebug("user=" + user + " aclsEnabled=" + aclsEnabled() + " viewAcls=" + viewAcls.mkString(",")) - if (uiAclsEnabled() && (user != null) && (!viewAcls.contains(user))) false else true + if (aclsEnabled() && (user != null) && (!viewAcls.contains(user))) false else true } + /** + * Checks the given user against the modify acl list to see if they have + * authorization to modify the application. If the UI acls are disabled + * via spark.acls.enable, all users have modify access. If the user is null + * it is assumed authentication isn't turned on and all users have access. + * + * @param user to see if is authorized + * @return true is the user has permission, otherwise false + */ + def checkModifyPermissions(user: String): Boolean = { + logDebug("user=" + user + " aclsEnabled=" + aclsEnabled() + " modifyAcls=" + + modifyAcls.mkString(",")) + if (aclsEnabled() && (user != null) && (!modifyAcls.contains(user))) false else true + } + + /** * Check to see if authentication for the Spark communication protocols is enabled * @return true if authentication is enabled, otherwise false diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 8ce4b91cae8ae..13f0bff7ee507 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -40,6 +40,8 @@ import scala.collection.mutable.HashMap */ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { + import SparkConf._ + /** Create a SparkConf that loads defaults from system properties and the classpath */ def this() = this(true) @@ -198,7 +200,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { * * E.g. spark.akka.option.x.y.x = "value" */ - getAll.filter {case (k, v) => k.startsWith("akka.")} + getAll.filter { case (k, _) => isAkkaConf(k) } /** Does the configuration contain a given parameter? */ def contains(key: String): Boolean = settings.contains(key) @@ -236,6 +238,20 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { } } + // Validate memory fractions + val memoryKeys = Seq( + "spark.storage.memoryFraction", + "spark.shuffle.memoryFraction", + "spark.shuffle.safetyFraction", + "spark.storage.unrollFraction", + "spark.storage.safetyFraction") + for (key <- memoryKeys) { + val value = getDouble(key, 0.5) + if (value > 1 || value < 0) { + throw new IllegalArgumentException("$key should be between 0 and 1 (was '$value').") + } + } + // Check for legacy configs sys.env.get("SPARK_JAVA_OPTS").foreach { value => val warning = @@ -292,3 +308,29 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { settings.toArray.sorted.map{case (k, v) => k + "=" + v}.mkString("\n") } } + +private[spark] object SparkConf { + /** + * Return whether the given config is an akka config (e.g. akka.actor.provider). + * Note that this does not include spark-specific akka configs (e.g. spark.akka.timeout). + */ + def isAkkaConf(name: String): Boolean = name.startsWith("akka.") + + /** + * Return whether the given config should be passed to an executor on start-up. + * + * Certain akka and authentication configs are required of the executor when it connects to + * the scheduler, while the rest of the spark configs can be inherited from the driver later. + */ + def isExecutorStartupConf(name: String): Boolean = { + isAkkaConf(name) || + name.startsWith("spark.akka") || + name.startsWith("spark.auth") || + isSparkPortConf(name) + } + + /** + * Return whether the given config is a Spark port config. + */ + def isSparkPortConf(name: String): Boolean = name.startsWith("spark.") && name.endsWith(".port") +} diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 8819e73d17fb2..e132955f0f850 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -36,6 +36,7 @@ import org.apache.hadoop.mapred.{FileInputFormat, InputFormat, JobConf, Sequence import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat, Job => NewHadoopJob} import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFormat} import org.apache.mesos.MesosNativeLibrary +import akka.actor.Props import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.broadcast.Broadcast @@ -47,7 +48,7 @@ import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, SparkDeploySchedulerBackend, SimrSchedulerBackend} import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} import org.apache.spark.scheduler.local.LocalBackend -import org.apache.spark.storage.{BlockManagerSource, RDDInfo, StorageStatus, StorageUtils} +import org.apache.spark.storage._ import org.apache.spark.ui.SparkUI import org.apache.spark.util.{CallSite, ClosureCleaner, MetadataCleaner, MetadataCleanerType, TimeStampedWeakValueHashMap, Utils} @@ -289,7 +290,7 @@ 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 => + 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. @@ -307,6 +308,8 @@ class SparkContext(config: SparkConf) extends Logging { // Create and start the scheduler private[spark] var taskScheduler = SparkContext.createTaskScheduler(this, master) + private val heartbeatReceiver = env.actorSystem.actorOf( + Props(new HeartbeatReceiver(taskScheduler)), "HeartbeatReceiver") @volatile private[spark] var dagScheduler: DAGScheduler = _ try { dagScheduler = new DAGScheduler(this) @@ -455,7 +458,7 @@ class SparkContext(config: SparkConf) extends Logging { /** Distribute a local Scala collection to form an RDD, with one or more * location preferences (hostnames of Spark nodes) for each object. * Create a new partition for each collection item. */ - def makeRDD[T: ClassTag](seq: Seq[(T, Seq[String])]): RDD[T] = { + def makeRDD[T: ClassTag](seq: Seq[(T, Seq[String])]): RDD[T] = { val indexToPrefs = seq.zipWithIndex.map(t => (t._2, t._1._2)).toMap new ParallelCollectionRDD[T](this, seq.map(_._1), seq.size, indexToPrefs) } @@ -757,6 +760,15 @@ class SparkContext(config: SparkConf) extends Logging { def accumulator[T](initialValue: T)(implicit param: AccumulatorParam[T]) = new Accumulator(initialValue, param) + /** + * Create an [[org.apache.spark.Accumulator]] variable of a given type, with a name for display + * in the Spark UI. Tasks can "add" values to the accumulator using the `+=` method. Only the + * driver can access the accumulator's `value`. + */ + def accumulator[T](initialValue: T, name: String)(implicit param: AccumulatorParam[T]) = { + new Accumulator(initialValue, param, Some(name)) + } + /** * Create an [[org.apache.spark.Accumulable]] shared variable, to which tasks can add values * with `+=`. Only the driver can access the accumuable's `value`. @@ -766,6 +778,16 @@ class SparkContext(config: SparkConf) extends Logging { def accumulable[T, R](initialValue: T)(implicit param: AccumulableParam[T, R]) = new Accumulable(initialValue, param) + /** + * Create an [[org.apache.spark.Accumulable]] shared variable, with a name for display in the + * Spark UI. Tasks can add values to the accumuable using the `+=` operator. Only the driver can + * access the accumuable's `value`. + * @tparam T accumulator type + * @tparam R type that can be added to the accumulator + */ + def accumulable[T, R](initialValue: T, name: String)(implicit param: AccumulableParam[T, R]) = + new Accumulable(initialValue, param, Some(name)) + /** * Create an accumulator from a "mutable collection" type. * @@ -840,7 +862,9 @@ class SparkContext(config: SparkConf) extends Logging { */ @DeveloperApi def getRDDStorageInfo: Array[RDDInfo] = { - StorageUtils.rddInfoFromStorageStatus(getExecutorStorageStatus, this) + val rddInfos = persistentRdds.values.map(RDDInfo.fromRdd).toArray + StorageUtils.updateRddInfo(rddInfos, getExecutorStorageStatus) + rddInfos.filter(_.isCached) } /** @@ -990,15 +1014,15 @@ class SparkContext(config: SparkConf) extends Logging { val dagSchedulerCopy = dagScheduler dagScheduler = null if (dagSchedulerCopy != null) { + env.metricsSystem.report() metadataCleaner.cancel() + env.actorSystem.stop(heartbeatReceiver) cleaner.foreach(_.stop()) dagSchedulerCopy.stop() taskScheduler = null // TODO: Cache.stop()? env.stop() SparkEnv.set(null) - ShuffleMapTask.clearCache() - ResultTask.clearCache() listenerBus.stop() eventLogger.foreach(_.stop()) logInfo("Successfully stopped SparkContext") @@ -1037,7 +1061,7 @@ class SparkContext(config: SparkConf) extends Logging { */ private[spark] def getCallSite(): CallSite = { Option(getLocalProperty("externalCallSite")) match { - case Some(callSite) => CallSite(callSite, long = "") + case Some(callSite) => CallSite(callSite, longForm = "") case None => Utils.getCallSite } } @@ -1059,11 +1083,12 @@ class SparkContext(config: SparkConf) extends Logging { } val callSite = getCallSite val cleanedFunc = clean(func) - logInfo("Starting job: " + callSite.short) + logInfo("Starting job: " + callSite.shortForm) val start = System.nanoTime dagScheduler.runJob(rdd, cleanedFunc, partitions, callSite, allowLocal, resultHandler, localProperties.get) - logInfo("Job finished: " + callSite.short + ", took " + (System.nanoTime - start) / 1e9 + " s") + logInfo( + "Job finished: " + callSite.shortForm + ", took " + (System.nanoTime - start) / 1e9 + " s") rdd.doCheckpoint() } @@ -1144,11 +1169,12 @@ class SparkContext(config: SparkConf) extends Logging { evaluator: ApproximateEvaluator[U, R], timeout: Long): PartialResult[R] = { val callSite = getCallSite - logInfo("Starting job: " + callSite.short) + logInfo("Starting job: " + callSite.shortForm) val start = System.nanoTime val result = dagScheduler.runApproximateJob(rdd, func, evaluator, callSite, timeout, localProperties.get) - logInfo("Job finished: " + callSite.short + ", took " + (System.nanoTime - start) / 1e9 + " s") + logInfo( + "Job finished: " + callSite.shortForm + ", took " + (System.nanoTime - start) / 1e9 + " s") result } @@ -1203,10 +1229,10 @@ class SparkContext(config: SparkConf) extends Logging { /** * Clean a closure to make it ready to serialized and send to tasks * (removes unreferenced variables in $outer's, updates REPL variables) - * If checkSerializable is set, clean will also proactively - * check to see if f is serializable and throw a SparkException + * If checkSerializable is set, clean will also proactively + * check to see if f is serializable and throw a SparkException * if not. - * + * * @param f the closure to clean * @param checkSerializable whether or not to immediately check f for serializability * @throws SparkException if checkSerializable is set but f is not @@ -1451,9 +1477,9 @@ object SparkContext extends Logging { /** Creates a task scheduler based on a given master URL. Extracted for testing. */ private def createTaskScheduler(sc: SparkContext, master: String): TaskScheduler = { // Regular expression used for local[N] and local[*] master formats - val LOCAL_N_REGEX = """local\[([0-9\*]+)\]""".r + val LOCAL_N_REGEX = """local\[([0-9]+|\*)\]""".r // Regular expression for local[N, maxRetries], used in tests with failing tasks - val LOCAL_N_FAILURES_REGEX = """local\[([0-9]+)\s*,\s*([0-9]+)\]""".r + val LOCAL_N_FAILURES_REGEX = """local\[([0-9]+|\*)\s*,\s*([0-9]+)\]""".r // Regular expression for simulating a Spark cluster of [N, cores, memory] locally val LOCAL_CLUSTER_REGEX = """local-cluster\[\s*([0-9]+)\s*,\s*([0-9]+)\s*,\s*([0-9]+)\s*]""".r // Regular expression for connecting to Spark deploy clusters @@ -1483,8 +1509,12 @@ object SparkContext extends Logging { scheduler case LOCAL_N_FAILURES_REGEX(threads, maxFailures) => + def localCpuCount = Runtime.getRuntime.availableProcessors() + // local[*, M] means the number of cores on the computer with M failures + // local[N, M] means exactly N threads with M failures + val threadCount = if (threads == "*") localCpuCount else threads.toInt val scheduler = new TaskSchedulerImpl(sc, maxFailures.toInt, isLocal = true) - val backend = new LocalBackend(scheduler, threads.toInt) + val backend = new LocalBackend(scheduler, threadCount) scheduler.initialize(backend) scheduler @@ -1531,7 +1561,16 @@ object SparkContext extends Logging { throw new SparkException("YARN mode not available ?", e) } } - val backend = new CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem) + val backend = try { + val clazz = + Class.forName("org.apache.spark.scheduler.cluster.YarnClusterSchedulerBackend") + val cons = clazz.getConstructor(classOf[TaskSchedulerImpl], classOf[SparkContext]) + cons.newInstance(scheduler, sc).asInstanceOf[CoarseGrainedSchedulerBackend] + } catch { + case e: Exception => { + throw new SparkException("YARN mode not available ?", e) + } + } scheduler.initialize(backend) scheduler diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 8f70744d804d9..fc36e37c53f5e 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -18,10 +18,10 @@ package org.apache.spark import java.io.File +import java.net.Socket import scala.collection.JavaConversions._ import scala.collection.mutable -import scala.concurrent.Await import scala.util.Properties import akka.actor._ @@ -34,7 +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.shuffle.{ShuffleMemoryManager, ShuffleManager} import org.apache.spark.storage._ import org.apache.spark.util.{AkkaUtils, Utils} @@ -65,12 +65,9 @@ class SparkEnv ( val httpFileServer: HttpFileServer, val sparkFilesDir: String, val metricsSystem: MetricsSystem, + val shuffleMemoryManager: ShuffleMemoryManager, val conf: SparkConf) extends Logging { - // A mapping of thread ID to amount of memory used for shuffle in bytes - // All accesses should be manually synchronized - val shuffleMemoryMap = mutable.HashMap[Long, Long]() - private val pythonWorkers = mutable.HashMap[(String, Map[String, String]), PythonWorkerFactory]() // A general, soft-reference map for metadata needed during HadoopRDD split computation @@ -102,10 +99,10 @@ class SparkEnv ( } private[spark] - def destroyPythonWorker(pythonExec: String, envVars: Map[String, String]) { + def destroyPythonWorker(pythonExec: String, envVars: Map[String, String], worker: Socket) { synchronized { val key = (pythonExec, envVars) - pythonWorkers(key).stop() + pythonWorkers.get(key).foreach(_.stopWorker(worker)) } } } @@ -153,17 +150,15 @@ object SparkEnv extends Logging { val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, port, conf = conf, securityManager = securityManager) - // Bit of a hack: If this is the driver and our port was 0 (meaning bind to any free port), - // figure out which port number Akka actually bound to and set spark.driver.port to it. - if (isDriver && port == 0) { - conf.set("spark.driver.port", boundPort.toString) + // Figure out which port Akka actually bound to in case the original port is 0 or occupied. + // This is so that we tell the executors the correct port to connect to. + if (isDriver) { + conf.set("spark.driver.port", boundPort.toString) } - // Create an instance of the class named by the given Java system property, or by - // defaultClassName if the property is not set, and return it as a T - def instantiateClass[T](propertyName: String, defaultClassName: String): T = { - val name = conf.get(propertyName, defaultClassName) - val cls = Class.forName(name, true, Utils.getContextOrSparkClassLoader) + // Create an instance of the class with the given name, possibly initializing it with our conf + def instantiateClass[T](className: String): T = { + val cls = Class.forName(className, true, Utils.getContextOrSparkClassLoader) // Look for a constructor taking a SparkConf and a boolean isDriver, then one taking just // SparkConf, then one taking no arguments try { @@ -181,11 +176,17 @@ object SparkEnv extends Logging { } } - val serializer = instantiateClass[Serializer]( + // Create an instance of the class named by the given SparkConf property, or defaultClassName + // if the property is not set, possibly initializing it with our conf + def instantiateClassFromConf[T](propertyName: String, defaultClassName: String): T = { + instantiateClass[T](conf.get(propertyName, defaultClassName)) + } + + val serializer = instantiateClassFromConf[Serializer]( "spark.serializer", "org.apache.spark.serializer.JavaSerializer") logDebug(s"Using serializer: ${serializer.getClass}") - val closureSerializer = instantiateClass[Serializer]( + val closureSerializer = instantiateClassFromConf[Serializer]( "spark.closure.serializer", "org.apache.spark.serializer.JavaSerializer") def registerOrLookup(name: String, newActor: => Actor): ActorRef = { @@ -193,13 +194,7 @@ object SparkEnv extends Logging { logInfo("Registering " + name) actorSystem.actorOf(Props(newActor), name = name) } else { - val driverHost: String = conf.get("spark.driver.host", "localhost") - val driverPort: Int = conf.getInt("spark.driver.port", 7077) - Utils.checkHost(driverHost, "Expected hostname") - val url = s"akka.tcp://spark@$driverHost:$driverPort/user/$name" - val timeout = AkkaUtils.lookupTimeout(conf) - logInfo(s"Connecting to $name: $url") - Await.result(actorSystem.actorSelection(url).resolveOne(timeout), timeout) + AkkaUtils.makeDriverRef(name, conf, actorSystem) } } @@ -215,12 +210,22 @@ object SparkEnv extends Logging { "MapOutputTracker", new MapOutputTrackerMasterActor(mapOutputTracker.asInstanceOf[MapOutputTrackerMaster], conf)) + // Let the user specify short names for shuffle managers + val shortShuffleMgrNames = Map( + "hash" -> "org.apache.spark.shuffle.hash.HashShuffleManager", + "sort" -> "org.apache.spark.shuffle.sort.SortShuffleManager") + val shuffleMgrName = conf.get("spark.shuffle.manager", "hash") + val shuffleMgrClass = shortShuffleMgrNames.getOrElse(shuffleMgrName.toLowerCase, shuffleMgrName) + val shuffleManager = instantiateClass[ShuffleManager](shuffleMgrClass) + + val shuffleMemoryManager = new ShuffleMemoryManager(conf) + val blockManagerMaster = new BlockManagerMaster(registerOrLookup( "BlockManagerMaster", new BlockManagerMasterActor(isLocal, conf, listenerBus)), conf) val blockManager = new BlockManager(executorId, actorSystem, blockManagerMaster, - serializer, conf, securityManager, mapOutputTracker) + serializer, conf, securityManager, mapOutputTracker, shuffleManager) val connectionManager = blockManager.connectionManager @@ -230,7 +235,8 @@ object SparkEnv extends Logging { val httpFileServer = if (isDriver) { - val server = new HttpFileServer(securityManager) + val fileServerPort = conf.getInt("spark.fileserver.port", 0) + val server = new HttpFileServer(securityManager, fileServerPort) server.initialize() conf.set("spark.fileserver.uri", server.serverUri) server @@ -254,9 +260,6 @@ 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 " + @@ -278,6 +281,7 @@ object SparkEnv extends Logging { httpFileServer, sparkFilesDir, metricsSystem, + shuffleMemoryManager, conf) } diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala index 51f40c339d13c..2b99b8a5af250 100644 --- a/core/src/main/scala/org/apache/spark/TaskContext.scala +++ b/core/src/main/scala/org/apache/spark/TaskContext.scala @@ -21,10 +21,18 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.TaskMetrics +import org.apache.spark.util.TaskCompletionListener + /** * :: DeveloperApi :: * Contextual information about a task which can be read or mutated during execution. + * + * @param stageId stage id + * @param partitionId index of the partition + * @param attemptId the number of attempts to execute this task + * @param runningLocally whether the task is running locally in the driver JVM + * @param taskMetrics performance metrics of the task */ @DeveloperApi class TaskContext( @@ -39,13 +47,45 @@ class TaskContext( def splitId = partitionId // List of callback functions to execute when the task completes. - @transient private val onCompleteCallbacks = new ArrayBuffer[() => Unit] + @transient private val onCompleteCallbacks = new ArrayBuffer[TaskCompletionListener] // Whether the corresponding task has been killed. - @volatile var interrupted: Boolean = false + @volatile private var interrupted: Boolean = false + + // Whether the task has completed. + @volatile private var completed: Boolean = false + + /** Checks whether the task has completed. */ + def isCompleted: Boolean = completed - // Whether the task has completed, before the onCompleteCallbacks are executed. - @volatile var completed: Boolean = false + /** Checks whether the task has been killed. */ + def isInterrupted: Boolean = interrupted + + // TODO: Also track whether the task has completed successfully or with exception. + + /** + * Add a (Java friendly) listener to be executed on task completion. + * This will be called in all situation - success, failure, or cancellation. + * + * An example use is for HadoopRDD to register a callback to close the input stream. + */ + def addTaskCompletionListener(listener: TaskCompletionListener): this.type = { + onCompleteCallbacks += listener + this + } + + /** + * Add a listener in the form of a Scala closure to be executed on task completion. + * This will be called in all situation - success, failure, or cancellation. + * + * An example use is for HadoopRDD to register a callback to close the input stream. + */ + def addTaskCompletionListener(f: TaskContext => Unit): this.type = { + onCompleteCallbacks += new TaskCompletionListener { + override def onTaskCompletion(context: TaskContext): Unit = f(context) + } + this + } /** * Add a callback function to be executed on task completion. An example use @@ -53,13 +93,22 @@ class TaskContext( * Will be called in any situation - success, failure, or cancellation. * @param f Callback function. */ + @deprecated("use addTaskCompletionListener", "1.1.0") def addOnCompleteCallback(f: () => Unit) { - onCompleteCallbacks += f + onCompleteCallbacks += new TaskCompletionListener { + override def onTaskCompletion(context: TaskContext): Unit = f() + } } - def executeOnCompleteCallbacks() { + /** Marks the task as completed and triggers the listeners. */ + private[spark] def markTaskCompleted(): Unit = { completed = true // Process complete callbacks in the reverse order of registration - onCompleteCallbacks.reverse.foreach { _() } + onCompleteCallbacks.reverse.foreach { _.onTaskCompletion(this) } + } + + /** Marks the task for interruption, i.e. cancellation. */ + private[spark] def markInterrupted(): Unit = { + interrupted = true } } diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala index df42d679b4699..8f0c5e78416c2 100644 --- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala +++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala @@ -20,6 +20,7 @@ package org.apache.spark import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.TaskMetrics import org.apache.spark.storage.BlockManagerId +import org.apache.spark.util.Utils /** * :: DeveloperApi :: @@ -88,10 +89,7 @@ case class ExceptionFailure( stackTrace: Array[StackTraceElement], metrics: Option[TaskMetrics]) extends TaskFailedReason { - override def toErrorString: String = { - val stackTraceString = if (stackTrace == null) "null" else stackTrace.mkString("\n") - s"$className ($description}\n$stackTraceString" - } + override def toErrorString: String = Utils.exceptionString(className, description, stackTrace) } /** diff --git a/core/src/main/scala/org/apache/spark/TestUtils.scala b/core/src/main/scala/org/apache/spark/TestUtils.scala index 885c6829a2d72..8ca731038e528 100644 --- a/core/src/main/scala/org/apache/spark/TestUtils.scala +++ b/core/src/main/scala/org/apache/spark/TestUtils.scala @@ -92,8 +92,8 @@ private[spark] object TestUtils { def createCompiledClass(className: String, destDir: File, value: String = ""): File = { val compiler = ToolProvider.getSystemJavaCompiler val sourceFile = new JavaSourceFromString(className, - "public class " + className + " { @Override public String toString() { " + - "return \"" + value + "\";}}") + "public class " + className + " implements java.io.Serializable {" + + " @Override public String toString() { return \"" + value + "\"; }}") // Calling this outputs a class file in pwd. It's easier to just rename the file than // build a custom FileManager that controls the output location. diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaHadoopRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaHadoopRDD.scala new file mode 100644 index 0000000000000..0ae0b4ec042e2 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/api/java/JavaHadoopRDD.scala @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.api.java + +import scala.collection.JavaConversions._ +import scala.reflect.ClassTag + +import org.apache.hadoop.mapred.InputSplit + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.api.java.JavaSparkContext._ +import org.apache.spark.api.java.function.{Function2 => JFunction2} +import org.apache.spark.rdd.HadoopRDD + +@DeveloperApi +class JavaHadoopRDD[K, V](rdd: HadoopRDD[K, V]) + (implicit override val kClassTag: ClassTag[K], implicit override val vClassTag: ClassTag[V]) + extends JavaPairRDD[K, V](rdd) { + + /** Maps over a partition, providing the InputSplit that was used as the base of the partition. */ + @DeveloperApi + def mapPartitionsWithInputSplit[R]( + f: JFunction2[InputSplit, java.util.Iterator[(K, V)], java.util.Iterator[R]], + preservesPartitioning: Boolean = false): JavaRDD[R] = { + new JavaRDD(rdd.mapPartitionsWithInputSplit((a, b) => f.call(a, asJavaIterator(b)), + preservesPartitioning)(fakeClassTag))(fakeClassTag) + } +} diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaNewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaNewHadoopRDD.scala new file mode 100644 index 0000000000000..ec4f3964d75e0 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/api/java/JavaNewHadoopRDD.scala @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.api.java + +import scala.collection.JavaConversions._ +import scala.reflect.ClassTag + +import org.apache.hadoop.mapreduce.InputSplit + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.api.java.JavaSparkContext._ +import org.apache.spark.api.java.function.{Function2 => JFunction2} +import org.apache.spark.rdd.NewHadoopRDD + +@DeveloperApi +class JavaNewHadoopRDD[K, V](rdd: NewHadoopRDD[K, V]) + (implicit override val kClassTag: ClassTag[K], implicit override val vClassTag: ClassTag[V]) + extends JavaPairRDD[K, V](rdd) { + + /** Maps over a partition, providing the InputSplit that was used as the base of the partition. */ + @DeveloperApi + def mapPartitionsWithInputSplit[R]( + f: JFunction2[InputSplit, java.util.Iterator[(K, V)], java.util.Iterator[R]], + preservesPartitioning: Boolean = false): JavaRDD[R] = { + new JavaRDD(rdd.mapPartitionsWithInputSplit((a, b) => f.call(a, asJavaIterator(b)), + preservesPartitioning)(fakeClassTag))(fakeClassTag) + } +} 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 4f3081433a542..feeb6c02caa78 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 @@ -17,7 +17,7 @@ package org.apache.spark.api.java -import java.util.{Comparator, List => JList} +import java.util.{Comparator, List => JList, Map => JMap} import java.lang.{Iterable => JIterable} import scala.collection.JavaConversions._ @@ -122,13 +122,74 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) */ def sample(withReplacement: Boolean, fraction: Double): JavaPairRDD[K, V] = sample(withReplacement, fraction, Utils.random.nextLong) - + /** * Return a sampled subset of this RDD. */ def sample(withReplacement: Boolean, fraction: Double, seed: Long): JavaPairRDD[K, V] = new JavaPairRDD[K, V](rdd.sample(withReplacement, fraction, seed)) + /** + * Return a subset of this RDD sampled by key (via stratified sampling). + * + * Create a sample of this RDD using variable sampling rates for different keys as specified by + * `fractions`, a key to sampling rate map, via simple random sampling with one pass over the + * RDD, to produce a sample of size that's approximately equal to the sum of + * math.ceil(numItems * samplingRate) over all key values. + */ + def sampleByKey(withReplacement: Boolean, + fractions: JMap[K, Double], + seed: Long): JavaPairRDD[K, V] = + new JavaPairRDD[K, V](rdd.sampleByKey(withReplacement, fractions, seed)) + + /** + * Return a subset of this RDD sampled by key (via stratified sampling). + * + * Create a sample of this RDD using variable sampling rates for different keys as specified by + * `fractions`, a key to sampling rate map, via simple random sampling with one pass over the + * RDD, to produce a sample of size that's approximately equal to the sum of + * math.ceil(numItems * samplingRate) over all key values. + * + * Use Utils.random.nextLong as the default seed for the random number generator. + */ + def sampleByKey(withReplacement: Boolean, + fractions: JMap[K, Double]): JavaPairRDD[K, V] = + sampleByKey(withReplacement, fractions, Utils.random.nextLong) + + /** + * ::Experimental:: + * Return a subset of this RDD sampled by key (via stratified sampling) containing exactly + * math.ceil(numItems * samplingRate) for each stratum (group of pairs with the same key). + * + * This method differs from [[sampleByKey]] in that we make additional passes over the RDD to + * create a sample size that's exactly equal to the sum of math.ceil(numItems * samplingRate) + * over all key values with a 99.99% confidence. When sampling without replacement, we need one + * additional pass over the RDD to guarantee sample size; when sampling with replacement, we need + * two additional passes. + */ + @Experimental + def sampleByKeyExact(withReplacement: Boolean, + fractions: JMap[K, Double], + seed: Long): JavaPairRDD[K, V] = + new JavaPairRDD[K, V](rdd.sampleByKeyExact(withReplacement, fractions, seed)) + + /** + * ::Experimental:: + * Return a subset of this RDD sampled by key (via stratified sampling) containing exactly + * math.ceil(numItems * samplingRate) for each stratum (group of pairs with the same key). + * + * This method differs from [[sampleByKey]] in that we make additional passes over the RDD to + * create a sample size that's exactly equal to the sum of math.ceil(numItems * samplingRate) + * over all key values with a 99.99% confidence. When sampling without replacement, we need one + * additional pass over the RDD to guarantee sample size; when sampling with replacement, we need + * two additional passes. + * + * Use Utils.random.nextLong as the default seed for the random number generator. + */ + @Experimental + def sampleByKeyExact(withReplacement: Boolean, fractions: JMap[K, Double]): JavaPairRDD[K, V] = + sampleByKeyExact(withReplacement, fractions, Utils.random.nextLong) + /** * Return the union of this RDD and another one. Any identical elements will appear multiple * times (use `.distinct()` to eliminate them). @@ -716,6 +777,17 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) sortByKey(comp, ascending) } + /** + * Sort the RDD by key, so that each partition contains a sorted range of the elements. Calling + * `collect` or `save` on the resulting RDD will return or output an ordered list of records + * (in the `save` case, they will be written to multiple `part-X` files in the filesystem, in + * order of the keys). + */ + def sortByKey(ascending: Boolean, numPartitions: Int): JavaPairRDD[K, V] = { + val comp = com.google.common.collect.Ordering.natural().asInstanceOf[Comparator[K]] + sortByKey(comp, ascending, numPartitions) + } + /** * Sort the RDD by key, so that each partition contains a sorted range of the elements. Calling * `collect` or `save` on the resulting RDD will return or output an ordered list of records 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 1e0493c4855e0..e0a4815940db3 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 @@ -34,7 +34,7 @@ import org.apache.spark._ import org.apache.spark.SparkContext.{DoubleAccumulatorParam, IntAccumulatorParam} import org.apache.spark.api.java.JavaSparkContext.fakeClassTag import org.apache.spark.broadcast.Broadcast -import org.apache.spark.rdd.RDD +import org.apache.spark.rdd.{EmptyRDD, HadoopRDD, NewHadoopRDD, RDD} /** * A Java-friendly version of [[org.apache.spark.SparkContext]] that returns @@ -112,6 +112,9 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork def startTime: java.lang.Long = sc.startTime + /** The version of Spark on which this application is running. */ + def version: String = sc.version + /** Default level of parallelism to use when not given by user (e.g. parallelize and makeRDD). */ def defaultParallelism: java.lang.Integer = sc.defaultParallelism @@ -132,6 +135,13 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork sc.parallelize(JavaConversions.asScalaBuffer(list), numSlices) } + /** Get an RDD that has no partitions or elements. */ + def emptyRDD[T]: JavaRDD[T] = { + implicit val ctag: ClassTag[T] = fakeClassTag + JavaRDD.fromRDD(new EmptyRDD[T](sc)) + } + + /** Distribute a local Scala collection to form an RDD. */ def parallelize[T](list: java.util.List[T]): JavaRDD[T] = parallelize(list, sc.defaultParallelism) @@ -284,7 +294,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork ): JavaPairRDD[K, V] = { implicit val ctagK: ClassTag[K] = ClassTag(keyClass) implicit val ctagV: ClassTag[V] = ClassTag(valueClass) - new JavaPairRDD(sc.hadoopRDD(conf, inputFormatClass, keyClass, valueClass, minPartitions)) + val rdd = sc.hadoopRDD(conf, inputFormatClass, keyClass, valueClass, minPartitions) + new JavaHadoopRDD(rdd.asInstanceOf[HadoopRDD[K, V]]) } /** @@ -304,7 +315,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork ): JavaPairRDD[K, V] = { implicit val ctagK: ClassTag[K] = ClassTag(keyClass) implicit val ctagV: ClassTag[V] = ClassTag(valueClass) - new JavaPairRDD(sc.hadoopRDD(conf, inputFormatClass, keyClass, valueClass)) + val rdd = sc.hadoopRDD(conf, inputFormatClass, keyClass, valueClass) + new JavaHadoopRDD(rdd.asInstanceOf[HadoopRDD[K, V]]) } /** Get an RDD for a Hadoop file with an arbitrary InputFormat. @@ -323,7 +335,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork ): JavaPairRDD[K, V] = { implicit val ctagK: ClassTag[K] = ClassTag(keyClass) implicit val ctagV: ClassTag[V] = ClassTag(valueClass) - new JavaPairRDD(sc.hadoopFile(path, inputFormatClass, keyClass, valueClass, minPartitions)) + val rdd = sc.hadoopFile(path, inputFormatClass, keyClass, valueClass, minPartitions) + new JavaHadoopRDD(rdd.asInstanceOf[HadoopRDD[K, V]]) } /** Get an RDD for a Hadoop file with an arbitrary InputFormat @@ -341,8 +354,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork ): JavaPairRDD[K, V] = { implicit val ctagK: ClassTag[K] = ClassTag(keyClass) implicit val ctagV: ClassTag[V] = ClassTag(valueClass) - new JavaPairRDD(sc.hadoopFile(path, - inputFormatClass, keyClass, valueClass)) + val rdd = sc.hadoopFile(path, inputFormatClass, keyClass, valueClass) + new JavaHadoopRDD(rdd.asInstanceOf[HadoopRDD[K, V]]) } /** @@ -362,7 +375,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork conf: Configuration): JavaPairRDD[K, V] = { implicit val ctagK: ClassTag[K] = ClassTag(kClass) implicit val ctagV: ClassTag[V] = ClassTag(vClass) - new JavaPairRDD(sc.newAPIHadoopFile(path, fClass, kClass, vClass, conf)) + val rdd = sc.newAPIHadoopFile(path, fClass, kClass, vClass, conf) + new JavaNewHadoopRDD(rdd.asInstanceOf[NewHadoopRDD[K, V]]) } /** @@ -381,7 +395,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork vClass: Class[V]): JavaPairRDD[K, V] = { implicit val ctagK: ClassTag[K] = ClassTag(kClass) implicit val ctagV: ClassTag[V] = ClassTag(vClass) - new JavaPairRDD(sc.newAPIHadoopRDD(conf, fClass, kClass, vClass)) + val rdd = sc.newAPIHadoopRDD(conf, fClass, kClass, vClass) + new JavaNewHadoopRDD(rdd.asInstanceOf[NewHadoopRDD[K, V]]) } /** Build the union of two or more RDDs. */ @@ -414,6 +429,16 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork def intAccumulator(initialValue: Int): Accumulator[java.lang.Integer] = sc.accumulator(initialValue)(IntAccumulatorParam).asInstanceOf[Accumulator[java.lang.Integer]] + /** + * Create an [[org.apache.spark.Accumulator]] integer variable, which tasks can "add" values + * to using the `add` method. Only the master can access the accumulator's `value`. + * + * This version supports naming the accumulator for display in Spark's web UI. + */ + def intAccumulator(initialValue: Int, name: String): Accumulator[java.lang.Integer] = + sc.accumulator(initialValue, name)(IntAccumulatorParam) + .asInstanceOf[Accumulator[java.lang.Integer]] + /** * Create an [[org.apache.spark.Accumulator]] double variable, which tasks can "add" values * to using the `add` method. Only the master can access the accumulator's `value`. @@ -421,12 +446,31 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork def doubleAccumulator(initialValue: Double): Accumulator[java.lang.Double] = sc.accumulator(initialValue)(DoubleAccumulatorParam).asInstanceOf[Accumulator[java.lang.Double]] + /** + * Create an [[org.apache.spark.Accumulator]] double variable, which tasks can "add" values + * to using the `add` method. Only the master can access the accumulator's `value`. + * + * This version supports naming the accumulator for display in Spark's web UI. + */ + def doubleAccumulator(initialValue: Double, name: String): Accumulator[java.lang.Double] = + sc.accumulator(initialValue, name)(DoubleAccumulatorParam) + .asInstanceOf[Accumulator[java.lang.Double]] + /** * Create an [[org.apache.spark.Accumulator]] integer variable, which tasks can "add" values * to using the `add` method. Only the master can access the accumulator's `value`. */ def accumulator(initialValue: Int): Accumulator[java.lang.Integer] = intAccumulator(initialValue) + /** + * Create an [[org.apache.spark.Accumulator]] integer variable, which tasks can "add" values + * to using the `add` method. Only the master can access the accumulator's `value`. + * + * This version supports naming the accumulator for display in Spark's web UI. + */ + def accumulator(initialValue: Int, name: String): Accumulator[java.lang.Integer] = + intAccumulator(initialValue, name) + /** * Create an [[org.apache.spark.Accumulator]] double variable, which tasks can "add" values * to using the `add` method. Only the master can access the accumulator's `value`. @@ -434,6 +478,16 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork def accumulator(initialValue: Double): Accumulator[java.lang.Double] = doubleAccumulator(initialValue) + + /** + * Create an [[org.apache.spark.Accumulator]] double variable, which tasks can "add" values + * to using the `add` method. Only the master can access the accumulator's `value`. + * + * This version supports naming the accumulator for display in Spark's web UI. + */ + def accumulator(initialValue: Double, name: String): Accumulator[java.lang.Double] = + doubleAccumulator(initialValue, name) + /** * Create an [[org.apache.spark.Accumulator]] variable of a given type, which tasks can "add" * values to using the `add` method. Only the master can access the accumulator's `value`. @@ -441,6 +495,16 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork def accumulator[T](initialValue: T, accumulatorParam: AccumulatorParam[T]): Accumulator[T] = sc.accumulator(initialValue)(accumulatorParam) + /** + * Create an [[org.apache.spark.Accumulator]] variable of a given type, which tasks can "add" + * values to using the `add` method. Only the master can access the accumulator's `value`. + * + * This version supports naming the accumulator for display in Spark's web UI. + */ + def accumulator[T](initialValue: T, name: String, accumulatorParam: AccumulatorParam[T]) + : Accumulator[T] = + sc.accumulator(initialValue, name)(accumulatorParam) + /** * Create an [[org.apache.spark.Accumulable]] shared variable of the given type, to which tasks * can "add" values with `add`. Only the master can access the accumuable's `value`. @@ -448,6 +512,16 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork def accumulable[T, R](initialValue: T, param: AccumulableParam[T, R]): Accumulable[T, R] = sc.accumulable(initialValue)(param) + /** + * Create an [[org.apache.spark.Accumulable]] shared variable of the given type, to which tasks + * can "add" values with `add`. Only the master can access the accumuable's `value`. + * + * This version supports naming the accumulator for display in Spark's web UI. + */ + def accumulable[T, R](initialValue: T, name: String, param: AccumulableParam[T, R]) + : Accumulable[T, R] = + sc.accumulable(initialValue, name)(param) + /** * Broadcast a read-only variable to the cluster, returning a * [[org.apache.spark.broadcast.Broadcast]] object for reading it in distributed functions. 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 index adaa1ef6cf9ff..49dc95f349eac 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala @@ -17,8 +17,10 @@ package org.apache.spark.api.python +import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD -import org.apache.spark.Logging +import org.apache.spark.util.Utils +import org.apache.spark.{Logging, SerializableWritable, SparkException} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.io._ import scala.util.{Failure, Success, Try} @@ -31,16 +33,17 @@ import org.apache.spark.annotation.Experimental * transformation code by overriding the convert method. */ @Experimental -trait Converter[T, U] extends Serializable { +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] = { + def getInstance(converterClass: Option[String], + defaultConverter: Converter[Any, Any]): Converter[Any, Any] = { converterClass.map { cc => Try { - val c = Class.forName(cc).newInstance().asInstanceOf[Converter[Any, Any]] + val c = Utils.classForName(cc).newInstance().asInstanceOf[Converter[Any, Any]] logInfo(s"Loaded converter: $cc") c } match { @@ -49,7 +52,7 @@ private[python] object Converter extends Logging { logError(s"Failed to load converter: $cc") throw err } - }.getOrElse { new DefaultConverter } + }.getOrElse { defaultConverter } } } @@ -57,7 +60,9 @@ private[python] object Converter extends Logging { * 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] { +private[python] class WritableToJavaConverter( + conf: Broadcast[SerializableWritable[Configuration]], + batchSize: Int) extends Converter[Any, Any] { /** * Converts a [[org.apache.hadoop.io.Writable]] to the underlying primitive, String or @@ -72,17 +77,30 @@ private[python] class DefaultConverter extends Converter[Any, Any] { case fw: FloatWritable => fw.get() case t: Text => t.toString case bw: BooleanWritable => bw.get() - case byw: BytesWritable => byw.getBytes + case byw: BytesWritable => + val bytes = new Array[Byte](byw.getLength) + System.arraycopy(byw.getBytes(), 0, bytes, 0, byw.getLength) + bytes 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 aw: ArrayWritable => + // Due to erasure, all arrays appear as Object[] and they get pickled to Python tuples. + // Since we can't determine element types for empty arrays, we will not attempt to + // convert to primitive arrays (which get pickled to Python arrays). Users may want + // write custom converters for arrays if they know the element types a priori. + aw.get().map(convertWritable(_)) + case mw: MapWritable => + val map = new java.util.HashMap[Any, Any]() + mw.foreach { case (k, v) => + map.put(convertWritable(k), convertWritable(v)) + } + map + case w: Writable => + if (batchSize > 1) WritableUtils.clone(w, conf.value.value) else w case other => other } } - def convert(obj: Any): Any = { + override def convert(obj: Any): Any = { obj match { case writable: Writable => convertWritable(writable) @@ -92,6 +110,47 @@ private[python] class DefaultConverter extends Converter[Any, Any] { } } +/** + * A converter that converts common types to [[org.apache.hadoop.io.Writable]]. Note that array + * types are not supported since the user needs to subclass [[org.apache.hadoop.io.ArrayWritable]] + * to set the type properly. See [[org.apache.spark.api.python.DoubleArrayWritable]] and + * [[org.apache.spark.api.python.DoubleArrayToWritableConverter]] for an example. They are used in + * PySpark RDD `saveAsNewAPIHadoopFile` doctest. + */ +private[python] class JavaToWritableConverter extends Converter[Any, Writable] { + + /** + * Converts common data types to [[org.apache.hadoop.io.Writable]]. Note that array types are not + * supported out-of-the-box. + */ + private def convertToWritable(obj: Any): Writable = { + import collection.JavaConversions._ + obj match { + case i: java.lang.Integer => new IntWritable(i) + case d: java.lang.Double => new DoubleWritable(d) + case l: java.lang.Long => new LongWritable(l) + case f: java.lang.Float => new FloatWritable(f) + case s: java.lang.String => new Text(s) + case b: java.lang.Boolean => new BooleanWritable(b) + case aob: Array[Byte] => new BytesWritable(aob) + case null => NullWritable.get() + case map: java.util.Map[_, _] => + val mapWritable = new MapWritable() + map.foreach { case (k, v) => + mapWritable.put(convertToWritable(k), convertToWritable(v)) + } + mapWritable + case other => throw new SparkException( + s"Data of type ${other.getClass.getName} cannot be used") + } + } + + override def convert(obj: Any): Writable = obj match { + case writable: Writable => writable + case other => convertToWritable(other) + } +} + /** Utilities for working with Python objects <-> Hadoop-related objects */ private[python] object PythonHadoopUtil { @@ -118,7 +177,7 @@ private[python] object PythonHadoopUtil { /** * 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)] + * [[org.apache.hadoop.io.Writable]], into an RDD of base types, or vice versa. */ def convertRDD[K, V](rdd: RDD[(K, V)], keyConverter: Converter[Any, Any], 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 462e09466bfa6..851862856d67b 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 @@ -23,22 +23,25 @@ import java.nio.charset.Charset import java.util.{List => JList, ArrayList => JArrayList, Map => JMap, Collections} import scala.collection.JavaConversions._ +import scala.language.existentials import scala.reflect.ClassTag -import scala.util.Try +import scala.util.{Try, Success, Failure} 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.hadoop.io.compress.CompressionCodec +import org.apache.hadoop.mapred.{InputFormat, OutputFormat, JobConf} +import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat, OutputFormat => NewOutputFormat} import org.apache.spark._ +import org.apache.spark.SparkContext._ import org.apache.spark.api.java.{JavaSparkContext, JavaPairRDD, JavaRDD} import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD import org.apache.spark.util.Utils -private[spark] class PythonRDD[T: ClassTag]( - parent: RDD[T], +private[spark] class PythonRDD( + parent: RDD[_], command: Array[Byte], envVars: JMap[String, String], pythonIncludes: JList[String], @@ -57,12 +60,15 @@ private[spark] class PythonRDD[T: ClassTag]( override def compute(split: Partition, context: TaskContext): Iterator[Array[Byte]] = { val startTime = System.currentTimeMillis val env = SparkEnv.get + val localdir = env.blockManager.diskBlockManager.localDirs.map( + f => f.getPath()).mkString(",") + envVars += ("SPARK_LOCAL_DIR" -> localdir) // it's also used in monitor thread val worker: Socket = env.createPythonWorker(pythonExec, envVars.toMap) // Start a thread to feed the process input from our parent's iterator val writerThread = new WriterThread(env, worker, split, context) - context.addOnCompleteCallback { () => + context.addTaskCompletionListener { context => writerThread.shutdownOnTaskCompletion() // Cleanup the worker socket. This will also cause the Python worker to exit. @@ -131,7 +137,7 @@ private[spark] class PythonRDD[T: ClassTag]( } } catch { - case e: Exception if context.interrupted => + case e: Exception if context.isInterrupted => logDebug("Exception thrown after task interruption", e) throw new TaskKilledException @@ -170,7 +176,7 @@ private[spark] class PythonRDD[T: ClassTag]( /** Terminates the writer thread, ignoring any exceptions that may occur due to cleanup. */ def shutdownOnTaskCompletion() { - assert(context.completed) + assert(context.isCompleted) this.interrupt() } @@ -203,7 +209,7 @@ private[spark] class PythonRDD[T: ClassTag]( PythonRDD.writeIteratorToStream(parent.iterator(split, context), dataOut) dataOut.flush() } catch { - case e: Exception if context.completed || context.interrupted => + case e: Exception if context.isCompleted || context.isInterrupted => logDebug("Exception thrown after task completion (likely due to cleanup)", e) case e: Exception => @@ -229,13 +235,13 @@ private[spark] class PythonRDD[T: ClassTag]( override def run() { // Kill the worker if it is interrupted, checking until task completion. // TODO: This has a race condition if interruption occurs, as completed may still become true. - while (!context.interrupted && !context.completed) { + while (!context.isInterrupted && !context.isCompleted) { Thread.sleep(2000) } - if (!context.completed) { + if (!context.isCompleted) { try { logWarning("Incomplete task interrupted: Attempting to kill Python Worker") - env.destroyPythonWorker(pythonExec, envVars.toMap) + env.destroyPythonWorker(pythonExec, envVars.toMap, worker) } catch { case e: Exception => logError("Exception when trying to kill worker", e) @@ -252,7 +258,7 @@ private class PythonException(msg: String, cause: Exception) extends RuntimeExce * Form an RDD[(Array[Byte], Array[Byte])] from key-value pairs returned from Python. * This is used by PySpark's shuffle operations. */ -private class PairwiseRDD(prev: RDD[Array[Byte]]) extends +private[spark] class PairwiseRDD(prev: RDD[Array[Byte]]) extends RDD[(Long, Array[Byte])](prev) { override def getPartitions = prev.partitions override def compute(split: Partition, context: TaskContext) = @@ -309,6 +315,14 @@ private[spark] object PythonRDD extends Logging { JavaRDD.fromRDD(sc.sc.parallelize(objs, parallelism)) } + def readBroadcastFromFile(sc: JavaSparkContext, filename: String): Broadcast[Array[Byte]] = { + val file = new DataInputStream(new FileInputStream(filename)) + val length = file.readInt() + val obj = new Array[Byte](length) + file.readFully(obj) + sc.broadcast(obj) + } + def writeIteratorToStream[T](iter: Iterator[T], dataOut: DataOutputStream) { // The right way to implement this would be to use TypeTags to get the full // type of T. Since I don't want to introduce breaking changes throughout the @@ -362,19 +376,17 @@ private[spark] object PythonRDD extends Logging { valueClassMaybeNull: String, keyConverterClass: String, valueConverterClass: String, - minSplits: Int) = { + minSplits: Int, + batchSize: 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 kc = Utils.classForName(keyClass).asInstanceOf[Class[K]] + val vc = Utils.classForName(valueClass).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)) + val confBroadcasted = sc.sc.broadcast(new SerializableWritable(sc.hadoopConfiguration())) + val converted = convertRDD(rdd, keyConverterClass, valueConverterClass, + new WritableToJavaConverter(confBroadcasted, batchSize)) + JavaRDD.fromRDD(SerDeUtil.pairRDDToPython(converted, batchSize)) } /** @@ -391,17 +403,16 @@ private[spark] object PythonRDD extends Logging { 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) + confAsMap: java.util.HashMap[String, String], + batchSize: Int) = { + val mergedConf = getMergedConf(confAsMap, sc.hadoopConfiguration()) 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)) + val confBroadcasted = sc.sc.broadcast(new SerializableWritable(mergedConf)) + val converted = convertRDD(rdd, keyConverterClass, valueConverterClass, + new WritableToJavaConverter(confBroadcasted, batchSize)) + JavaRDD.fromRDD(SerDeUtil.pairRDDToPython(converted, batchSize)) } /** @@ -418,15 +429,16 @@ private[spark] object PythonRDD extends Logging { valueClass: String, keyConverterClass: String, valueConverterClass: String, - confAsMap: java.util.HashMap[String, String]) = { + confAsMap: java.util.HashMap[String, String], + batchSize: Int) = { 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)) + val confBroadcasted = sc.sc.broadcast(new SerializableWritable(conf)) + val converted = convertRDD(rdd, keyConverterClass, valueConverterClass, + new WritableToJavaConverter(confBroadcasted, batchSize)) + JavaRDD.fromRDD(SerDeUtil.pairRDDToPython(converted, batchSize)) } private def newAPIHadoopRDDFromClassNames[K, V, F <: NewInputFormat[K, V]]( @@ -436,18 +448,14 @@ private[spark] object PythonRDD extends Logging { 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) { + val kc = Utils.classForName(keyClass).asInstanceOf[Class[K]] + val vc = Utils.classForName(valueClass).asInstanceOf[Class[V]] + val fc = Utils.classForName(inputFormatClass).asInstanceOf[Class[F]] + 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 } /** @@ -464,17 +472,16 @@ private[spark] object PythonRDD extends Logging { 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) + confAsMap: java.util.HashMap[String, String], + batchSize: Int) = { + val mergedConf = getMergedConf(confAsMap, sc.hadoopConfiguration()) 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)) + val confBroadcasted = sc.sc.broadcast(new SerializableWritable(mergedConf)) + val converted = convertRDD(rdd, keyConverterClass, valueConverterClass, + new WritableToJavaConverter(confBroadcasted, batchSize)) + JavaRDD.fromRDD(SerDeUtil.pairRDDToPython(converted, batchSize)) } /** @@ -491,15 +498,16 @@ private[spark] object PythonRDD extends Logging { valueClass: String, keyConverterClass: String, valueConverterClass: String, - confAsMap: java.util.HashMap[String, String]) = { + confAsMap: java.util.HashMap[String, String], + batchSize: Int) = { 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)) + val confBroadcasted = sc.sc.broadcast(new SerializableWritable(conf)) + val converted = convertRDD(rdd, keyConverterClass, valueConverterClass, + new WritableToJavaConverter(confBroadcasted, batchSize)) + JavaRDD.fromRDD(SerDeUtil.pairRDDToPython(converted, batchSize)) } private def hadoopRDDFromClassNames[K, V, F <: InputFormat[K, V]]( @@ -509,18 +517,14 @@ private[spark] object PythonRDD extends Logging { 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) { + val kc = Utils.classForName(keyClass).asInstanceOf[Class[K]] + val vc = Utils.classForName(valueClass).asInstanceOf[Class[V]] + val fc = Utils.classForName(inputFormatClass).asInstanceOf[Class[F]] + 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) { @@ -540,24 +544,199 @@ private[spark] object PythonRDD extends Logging { file.close() } + private def getMergedConf(confAsMap: java.util.HashMap[String, String], + baseConf: Configuration): Configuration = { + val conf = PythonHadoopUtil.mapToConf(confAsMap) + PythonHadoopUtil.mergeConfs(baseConf, conf) + } + + private def inferKeyValueTypes[K, V](rdd: RDD[(K, V)], keyConverterClass: String = null, + valueConverterClass: String = null): (Class[_], Class[_]) = { + // Peek at an element to figure out key/value types. Since Writables are not serializable, + // we cannot call first() on the converted RDD. Instead, we call first() on the original RDD + // and then convert locally. + val (key, value) = rdd.first() + val (kc, vc) = getKeyValueConverters(keyConverterClass, valueConverterClass, + new JavaToWritableConverter) + (kc.convert(key).getClass, vc.convert(value).getClass) + } + + private def getKeyValueTypes(keyClass: String, valueClass: String): + Option[(Class[_], Class[_])] = { + for { + k <- Option(keyClass) + v <- Option(valueClass) + } yield (Utils.classForName(k), Utils.classForName(v)) + } + + private def getKeyValueConverters(keyConverterClass: String, valueConverterClass: String, + defaultConverter: Converter[Any, Any]): (Converter[Any, Any], Converter[Any, Any]) = { + val keyConverter = Converter.getInstance(Option(keyConverterClass), defaultConverter) + val valueConverter = Converter.getInstance(Option(valueConverterClass), defaultConverter) + (keyConverter, valueConverter) + } + + /** + * Convert an RDD of key-value pairs from internal types to serializable types suitable for + * output, or vice versa. + */ + private def convertRDD[K, V](rdd: RDD[(K, V)], + keyConverterClass: String, + valueConverterClass: String, + defaultConverter: Converter[Any, Any]): RDD[(Any, Any)] = { + val (kc, vc) = getKeyValueConverters(keyConverterClass, valueConverterClass, + defaultConverter) + PythonHadoopUtil.convertRDD(rdd, kc, vc) + } + + /** + * Output a Python RDD of key-value pairs as a Hadoop SequenceFile using the Writable types + * we convert from the RDD's key and value types. Note that keys and values can't be + * [[org.apache.hadoop.io.Writable]] types already, since Writables are not Java + * `Serializable` and we can't peek at them. The `path` can be on any Hadoop file system. + */ + def saveAsSequenceFile[K, V, C <: CompressionCodec]( + pyRDD: JavaRDD[Array[Byte]], + batchSerialized: Boolean, + path: String, + compressionCodecClass: String) = { + saveAsHadoopFile( + pyRDD, batchSerialized, path, "org.apache.hadoop.mapred.SequenceFileOutputFormat", + null, null, null, null, new java.util.HashMap(), compressionCodecClass) + } + + /** + * Output a Python RDD of key-value pairs to any Hadoop file system, using old Hadoop + * `OutputFormat` in mapred package. Keys and values are converted to suitable output + * types using either user specified converters or, if not specified, + * [[org.apache.spark.api.python.JavaToWritableConverter]]. Post-conversion types + * `keyClass` and `valueClass` are automatically inferred if not specified. The passed-in + * `confAsMap` is merged with the default Hadoop conf associated with the SparkContext of + * this RDD. + */ + def saveAsHadoopFile[K, V, F <: OutputFormat[_, _], C <: CompressionCodec]( + pyRDD: JavaRDD[Array[Byte]], + batchSerialized: Boolean, + path: String, + outputFormatClass: String, + keyClass: String, + valueClass: String, + keyConverterClass: String, + valueConverterClass: String, + confAsMap: java.util.HashMap[String, String], + compressionCodecClass: String) = { + val rdd = SerDeUtil.pythonToPairRDD(pyRDD, batchSerialized) + val (kc, vc) = getKeyValueTypes(keyClass, valueClass).getOrElse( + inferKeyValueTypes(rdd, keyConverterClass, valueConverterClass)) + val mergedConf = getMergedConf(confAsMap, pyRDD.context.hadoopConfiguration) + val codec = Option(compressionCodecClass).map(Utils.classForName(_).asInstanceOf[Class[C]]) + val converted = convertRDD(rdd, keyConverterClass, valueConverterClass, + new JavaToWritableConverter) + val fc = Utils.classForName(outputFormatClass).asInstanceOf[Class[F]] + converted.saveAsHadoopFile(path, kc, vc, fc, new JobConf(mergedConf), codec=codec) + } + /** - * Convert an RDD of serialized Python dictionaries to Scala Maps - * TODO: Support more Python types. + * Output a Python RDD of key-value pairs to any Hadoop file system, using new Hadoop + * `OutputFormat` in mapreduce package. Keys and values are converted to suitable output + * types using either user specified converters or, if not specified, + * [[org.apache.spark.api.python.JavaToWritableConverter]]. Post-conversion types + * `keyClass` and `valueClass` are automatically inferred if not specified. The passed-in + * `confAsMap` is merged with the default Hadoop conf associated with the SparkContext of + * this RDD. */ + def saveAsNewAPIHadoopFile[K, V, F <: NewOutputFormat[_, _]]( + pyRDD: JavaRDD[Array[Byte]], + batchSerialized: Boolean, + path: String, + outputFormatClass: String, + keyClass: String, + valueClass: String, + keyConverterClass: String, + valueConverterClass: String, + confAsMap: java.util.HashMap[String, String]) = { + val rdd = SerDeUtil.pythonToPairRDD(pyRDD, batchSerialized) + val (kc, vc) = getKeyValueTypes(keyClass, valueClass).getOrElse( + inferKeyValueTypes(rdd, keyConverterClass, valueConverterClass)) + val mergedConf = getMergedConf(confAsMap, pyRDD.context.hadoopConfiguration) + val converted = convertRDD(rdd, keyConverterClass, valueConverterClass, + new JavaToWritableConverter) + val fc = Utils.classForName(outputFormatClass).asInstanceOf[Class[F]] + converted.saveAsNewAPIHadoopFile(path, kc, vc, fc, mergedConf) + } + + /** + * Output a Python RDD of key-value pairs to any Hadoop file system, using a Hadoop conf + * converted from the passed-in `confAsMap`. The conf should set relevant output params ( + * e.g., output path, output format, etc), in the same way as it would be configured for + * a Hadoop MapReduce job. Both old and new Hadoop OutputFormat APIs are supported + * (mapred vs. mapreduce). Keys/values are converted for output using either user specified + * converters or, by default, [[org.apache.spark.api.python.JavaToWritableConverter]]. + */ + def saveAsHadoopDataset[K, V]( + pyRDD: JavaRDD[Array[Byte]], + batchSerialized: Boolean, + confAsMap: java.util.HashMap[String, String], + keyConverterClass: String, + valueConverterClass: String, + useNewAPI: Boolean) = { + val conf = PythonHadoopUtil.mapToConf(confAsMap) + val converted = convertRDD(SerDeUtil.pythonToPairRDD(pyRDD, batchSerialized), + keyConverterClass, valueConverterClass, new JavaToWritableConverter) + if (useNewAPI) { + converted.saveAsNewAPIHadoopDataset(conf) + } else { + converted.saveAsHadoopDataset(new JobConf(conf)) + } + } + + + /** + * Convert an RDD of serialized Python dictionaries to Scala Maps (no recursive conversions). + */ + @deprecated("PySpark does not use it anymore", "1.1") def pythonToJavaMap(pyRDD: JavaRDD[Array[Byte]]): JavaRDD[Map[String, _]] = { pyRDD.rdd.mapPartitions { iter => val unpickle = new Unpickler - // TODO: Figure out why flatMap is necessay for pyspark iter.flatMap { row => unpickle.loads(row) match { - case objs: java.util.ArrayList[JMap[String, _] @unchecked] => objs.map(_.toMap) - // Incase the partition doesn't have a collection + // in case of objects are pickled in batch mode + case objs: JArrayList[JMap[String, _] @unchecked] => objs.map(_.toMap) + // not in batch mode case obj: JMap[String @unchecked, _] => Seq(obj.toMap) } } } } + /** + * Convert an RDD of serialized Python tuple to Array (no recursive conversions). + * It is only used by pyspark.sql. + */ + def pythonToJavaArray(pyRDD: JavaRDD[Array[Byte]], batched: Boolean): JavaRDD[Array[_]] = { + + def toArray(obj: Any): Array[_] = { + obj match { + case objs: JArrayList[_] => + objs.toArray + case obj if obj.getClass.isArray => + obj.asInstanceOf[Array[_]].toArray + } + } + + pyRDD.rdd.mapPartitions { iter => + val unpickle = new Unpickler + iter.flatMap { row => + val obj = unpickle.loads(row) + if (batched) { + obj.asInstanceOf[JArrayList[_]].map(toArray) + } else { + Seq(toArray(obj)) + } + } + }.toJavaRDD() + } + /** * Convert and RDD of Java objects to and RDD of serialized Python objects, that is usable by * PySpark. @@ -588,19 +767,30 @@ private class PythonAccumulatorParam(@transient serverHost: String, serverPort: val bufferSize = SparkEnv.get.conf.getInt("spark.buffer.size", 65536) + /** + * We try to reuse a single Socket to transfer accumulator updates, as they are all added + * by the DAGScheduler's single-threaded actor anyway. + */ + @transient var socket: Socket = _ + + def openSocket(): Socket = synchronized { + if (socket == null || socket.isClosed) { + socket = new Socket(serverHost, serverPort) + } + socket + } + override def zero(value: JList[Array[Byte]]): JList[Array[Byte]] = new JArrayList override def addInPlace(val1: JList[Array[Byte]], val2: JList[Array[Byte]]) - : JList[Array[Byte]] = { + : JList[Array[Byte]] = synchronized { if (serverHost == null) { // This happens on the worker node, where we just want to remember all the updates val1.addAll(val2) val1 } else { // This happens on the master, where we pass the updates to Python through a socket - val socket = new Socket(serverHost, serverPort) - // SPARK-2282: Immediately reuse closed sockets because we create one per task. - socket.setReuseAddress(true) + val socket = openSocket() val in = socket.getInputStream val out = new DataOutputStream(new BufferedOutputStream(socket.getOutputStream, bufferSize)) out.writeInt(val2.size) @@ -614,7 +804,6 @@ private class PythonAccumulatorParam(@transient serverHost: String, serverPort: if (byteRead == -1) { throw new SparkException("EOF reached before Python server acknowledged") } - socket.close() null } } diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala b/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala index 6d3e257c4d5df..52c70712eea3d 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala @@ -29,7 +29,7 @@ private[spark] object PythonUtils { val pythonPath = new ArrayBuffer[String] for (sparkHome <- sys.env.get("SPARK_HOME")) { pythonPath += Seq(sparkHome, "python").mkString(File.separator) - pythonPath += Seq(sparkHome, "python", "lib", "py4j-0.8.1-src.zip").mkString(File.separator) + pythonPath += Seq(sparkHome, "python", "lib", "py4j-0.8.2.1-src.zip").mkString(File.separator) } pythonPath ++= SparkContext.jarOfObject(this) pythonPath.mkString(File.pathSeparator) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala index 759cbe2c46c52..bf716a8ab025b 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala @@ -17,9 +17,11 @@ package org.apache.spark.api.python -import java.io.{DataInputStream, InputStream, OutputStreamWriter} +import java.lang.Runtime +import java.io.{DataOutputStream, DataInputStream, InputStream, OutputStreamWriter} import java.net.{InetAddress, ServerSocket, Socket, SocketException} +import scala.collection.mutable import scala.collection.JavaConversions._ import org.apache.spark._ @@ -39,6 +41,9 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String var daemon: Process = null val daemonHost = InetAddress.getByAddress(Array(127, 0, 0, 1)) var daemonPort: Int = 0 + var daemonWorkers = new mutable.WeakHashMap[Socket, Int]() + + var simpleWorkers = new mutable.WeakHashMap[Socket, Process]() val pythonPath = PythonUtils.mergePythonPaths( PythonUtils.sparkPythonPath, @@ -58,19 +63,31 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String * to avoid the high cost of forking from Java. This currently only works on UNIX-based systems. */ private def createThroughDaemon(): Socket = { + + def createSocket(): Socket = { + val socket = new Socket(daemonHost, daemonPort) + val pid = new DataInputStream(socket.getInputStream).readInt() + if (pid < 0) { + throw new IllegalStateException("Python daemon failed to launch worker with code " + pid) + } + daemonWorkers.put(socket, pid) + socket + } + synchronized { // Start the daemon if it hasn't been started startDaemon() // Attempt to connect, restart and retry once if it fails try { - new Socket(daemonHost, daemonPort) + createSocket() } catch { case exc: SocketException => - logWarning("Python daemon unexpectedly quit, attempting to restart") + logWarning("Failed to open socket to Python daemon:", exc) + logWarning("Assuming that daemon unexpectedly quit, attempting to restart") stopDaemon() startDaemon() - new Socket(daemonHost, daemonPort) + createSocket() } } } @@ -101,7 +118,9 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String // Wait for it to connect to our socket serverSocket.setSoTimeout(10000) try { - return serverSocket.accept() + val socket = serverSocket.accept() + simpleWorkers.put(socket, worker) + return socket } catch { case e: Exception => throw new SparkException("Python worker did not connect back in time", e) @@ -183,19 +202,40 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String private def stopDaemon() { synchronized { - // Request shutdown of existing daemon by sending SIGTERM - if (daemon != null) { - daemon.destroy() + if (useDaemon) { + // Request shutdown of existing daemon by sending SIGTERM + if (daemon != null) { + daemon.destroy() + } + + daemon = null + daemonPort = 0 + } else { + simpleWorkers.mapValues(_.destroy()) } - - daemon = null - daemonPort = 0 } } def stop() { stopDaemon() } + + def stopWorker(worker: Socket) { + if (useDaemon) { + if (daemon != null) { + daemonWorkers.get(worker).foreach { pid => + // tell daemon to kill worker by pid + val output = new DataOutputStream(daemon.getOutputStream) + output.writeInt(pid) + output.flush() + daemon.getOutputStream.flush() + } + } + } else { + simpleWorkers.get(worker).foreach(_.destroy()) + } + worker.close() + } } private object PythonWorkerFactory { 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 index 9a012e7254901..efc9009c088a8 100644 --- a/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala +++ b/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala @@ -17,13 +17,14 @@ 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.collection.JavaConversions._ import scala.util.Failure -import net.razorvine.pickle.Pickler +import scala.util.Try +import net.razorvine.pickle.{Unpickler, Pickler} + +import org.apache.spark.{Logging, SparkException} +import org.apache.spark.rdd.RDD /** Utilities for serialization / deserialization between Python and Java, using Pickle. */ private[python] object SerDeUtil extends Logging { @@ -65,20 +66,52 @@ private[python] object SerDeUtil extends Logging { * 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]] = { + def pairRDDToPython(rdd: RDD[(Any, Any)], batchSize: Int): 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)) + val cleaned = iter.map { case (k, v) => + val key = if (keyFailed) k.toString else k + val value = if (valueFailed) v.toString else v + Array[Any](key, value) + } + if (batchSize > 1) { + cleaned.grouped(batchSize).map(batched => pickle.dumps(seqAsJavaList(batched))) + } else { + cleaned.map(pickle.dumps(_)) + } + } + } + + /** + * Convert an RDD of serialized Python tuple (K, V) to RDD[(K, V)]. + */ + def pythonToPairRDD[K, V](pyRDD: RDD[Array[Byte]], batchSerialized: Boolean): RDD[(K, V)] = { + def isPair(obj: Any): Boolean = { + Option(obj.getClass.getComponentType).map(!_.isPrimitive).getOrElse(false) && + obj.asInstanceOf[Array[_]].length == 2 + } + pyRDD.mapPartitions { iter => + val unpickle = new Unpickler + val unpickled = + if (batchSerialized) { + iter.flatMap { batch => + unpickle.loads(batch) match { + case objs: java.util.List[_] => collectionAsScalaIterable(objs) + case other => throw new SparkException( + s"Unexpected type ${other.getClass.getName} for batch serialized Python RDD") + } + } } else { - pickle.dumps(Array(k, v)) + iter.map(unpickle.loads(_)) } + unpickled.map { + case obj if isPair(obj) => + // we only accept (K, V) + val arr = obj.asInstanceOf[Array[_]] + (arr.head.asInstanceOf[K], arr.last.asInstanceOf[V]) + case other => throw new SparkException( + s"RDD element of type ${other.getClass.getName} cannot be used") } } } 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 index f0e3fb9aff5a0..d11db978b842e 100644 --- a/core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTestDataGenerator.scala +++ b/core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTestDataGenerator.scala @@ -17,15 +17,16 @@ 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 java.nio.charset.Charset + +import org.apache.hadoop.io._ import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat import org.apache.spark.api.java.JavaSparkContext +import org.apache.spark.{SparkContext, SparkException} /** - * A class to test MsgPack serialization on the Scala side, that will be deserialized + * A class to test Pyrolite serialization on the Scala side, that will be deserialized * in Python * @param str * @param int @@ -54,7 +55,13 @@ case class TestWritable(var str: String, var int: Int, var double: Double) exten } } -class TestConverter extends Converter[Any, Any] { +private[python] class TestInputKeyConverter extends Converter[Any, Any] { + override def convert(obj: Any) = { + obj.asInstanceOf[IntWritable].get().toChar + } +} + +private[python] class TestInputValueConverter extends Converter[Any, Any] { import collection.JavaConversions._ override def convert(obj: Any) = { val m = obj.asInstanceOf[MapWritable] @@ -62,6 +69,38 @@ class TestConverter extends Converter[Any, Any] { } } +private[python] class TestOutputKeyConverter extends Converter[Any, Any] { + override def convert(obj: Any) = { + new Text(obj.asInstanceOf[Int].toString) + } +} + +private[python] class TestOutputValueConverter extends Converter[Any, Any] { + import collection.JavaConversions._ + override def convert(obj: Any) = { + new DoubleWritable(obj.asInstanceOf[java.util.Map[Double, _]].keySet().head) + } +} + +private[python] class DoubleArrayWritable extends ArrayWritable(classOf[DoubleWritable]) + +private[python] class DoubleArrayToWritableConverter extends Converter[Any, Writable] { + override def convert(obj: Any) = obj match { + case arr if arr.getClass.isArray && arr.getClass.getComponentType == classOf[Double] => + val daw = new DoubleArrayWritable + daw.set(arr.asInstanceOf[Array[Double]].map(new DoubleWritable(_))) + daw + case other => throw new SparkException(s"Data of type $other is not supported") + } +} + +private[python] class WritableToDoubleArrayConverter extends Converter[Any, Array[Double]] { + override def convert(obj: Any): Array[Double] = obj match { + case daw : DoubleArrayWritable => daw.get().map(_.asInstanceOf[DoubleWritable].get()) + case other => throw new SparkException(s"Data of type $other is not supported") + } +} + /** * This object contains method to generate SequenceFile test data and write it to a * given directory (probably a temp directory) @@ -97,7 +136,8 @@ object WriteInputFormatTestDataGenerator { 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) + sc.parallelize(intKeys.map{ case (k, v) => (k, v.getBytes(Charset.forName("UTF-8"))) } + ).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) => @@ -106,19 +146,20 @@ object WriteInputFormatTestDataGenerator { // Create test data for ArrayWritable val data = Seq( - (1, Array(1.0, 2.0, 3.0)), + (1, Array()), (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) + val va = new DoubleArrayWritable + va.set(v.map(new DoubleWritable(_))) + (new IntWritable(k), va) + }.saveAsNewAPIHadoopFile[SequenceFileOutputFormat[IntWritable, DoubleArrayWritable]](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")), + (1, Map()), (2, Map(1.0 -> "cc")), (3, Map(2.0 -> "dd")), (2, Map(1.0 -> "aa")), @@ -126,9 +167,9 @@ object WriteInputFormatTestDataGenerator { ) 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)) + m.foreach { case (k, v) => + mw.put(new DoubleWritable(k), new Text(v)) + } (new IntWritable(i), mw) }.saveAsSequenceFile(mapPath) diff --git a/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala index 76956f6a345d1..15fd30e65761d 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala @@ -106,23 +106,23 @@ abstract class Broadcast[T: ClassTag](val id: Long) extends Serializable { * Actually get the broadcasted value. Concrete implementations of Broadcast class must * define their own way to get the value. */ - private[spark] def getValue(): T + protected def getValue(): T /** * Actually unpersist the broadcasted value on the executors. Concrete implementations of * Broadcast class must define their own logic to unpersist their own data. */ - private[spark] def doUnpersist(blocking: Boolean) + protected def doUnpersist(blocking: Boolean) /** * Actually destroy all data and metadata related to this broadcast variable. * Implementation of Broadcast class must define their own logic to destroy their own * state. */ - private[spark] def doDestroy(blocking: Boolean) + protected def doDestroy(blocking: Boolean) /** Check if this broadcast is valid. If not valid, exception is thrown. */ - private[spark] def assertValid() { + protected def assertValid() { if (!_isValid) { throw new SparkException("Attempted to use %s after it has been destroyed!".format(toString)) } diff --git a/core/src/main/scala/org/apache/spark/broadcast/BroadcastManager.scala b/core/src/main/scala/org/apache/spark/broadcast/BroadcastManager.scala index c88be6aba6901..8f8a0b11f9f2e 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/BroadcastManager.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/BroadcastManager.scala @@ -39,7 +39,7 @@ private[spark] class BroadcastManager( synchronized { if (!initialized) { val broadcastFactoryClass = - conf.get("spark.broadcast.factory", "org.apache.spark.broadcast.HttpBroadcastFactory") + conf.get("spark.broadcast.factory", "org.apache.spark.broadcast.TorrentBroadcastFactory") broadcastFactory = Class.forName(broadcastFactoryClass).newInstance.asInstanceOf[BroadcastFactory] diff --git a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala index 4f6cabaff2b99..942dc7d7eac87 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala @@ -40,9 +40,9 @@ private[spark] class HttpBroadcast[T: ClassTag]( @transient var value_ : T, isLocal: Boolean, id: Long) extends Broadcast[T](id) with Logging with Serializable { - def getValue = value_ + override protected def getValue() = value_ - val blockId = BroadcastBlockId(id) + private val blockId = BroadcastBlockId(id) /* * Broadcasted data is also stored in the BlockManager of the driver. The BlockManagerMaster @@ -60,14 +60,14 @@ private[spark] class HttpBroadcast[T: ClassTag]( /** * Remove all persisted state associated with this HTTP broadcast on the executors. */ - def doUnpersist(blocking: Boolean) { + override protected def doUnpersist(blocking: Boolean) { HttpBroadcast.unpersist(id, removeFromDriver = false, blocking) } /** * Remove all persisted state associated with this HTTP broadcast on the executors and driver. */ - def doDestroy(blocking: Boolean) { + override protected def doDestroy(blocking: Boolean) { HttpBroadcast.unpersist(id, removeFromDriver = true, blocking) } @@ -102,7 +102,7 @@ private[spark] class HttpBroadcast[T: ClassTag]( } } -private[spark] object HttpBroadcast extends Logging { +private[broadcast] object HttpBroadcast extends Logging { private var initialized = false private var broadcastDir: File = null private var compress: Boolean = false @@ -152,7 +152,8 @@ private[spark] object HttpBroadcast extends Logging { private def createServer(conf: SparkConf) { broadcastDir = Utils.createTempDir(Utils.getLocalDir(conf)) - server = new HttpServer(broadcastDir, securityManager) + val broadcastPort = conf.getInt("spark.broadcast.port", 0) + server = new HttpServer(broadcastDir, securityManager, broadcastPort, "HTTP broadcast server") server.start() serverUri = server.uri logInfo("Broadcast server started at " + serverUri) @@ -160,7 +161,7 @@ private[spark] object HttpBroadcast extends Logging { def getFile(id: Long) = new File(broadcastDir, BroadcastBlockId(id).name) - def write(id: Long, value: Any) { + private def write(id: Long, value: Any) { val file = getFile(id) val out: OutputStream = { if (compress) { @@ -176,7 +177,7 @@ private[spark] object HttpBroadcast extends Logging { files += file } - def read[T: ClassTag](id: Long): T = { + private def read[T: ClassTag](id: Long): T = { logDebug("broadcast read server: " + serverUri + " id: broadcast-" + id) val url = serverUri + "/" + BroadcastBlockId(id).name diff --git a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcastFactory.scala b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcastFactory.scala index d5a031e2bbb59..c7ef02d572a19 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcastFactory.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcastFactory.scala @@ -27,21 +27,21 @@ import org.apache.spark.{SecurityManager, SparkConf} * [[org.apache.spark.broadcast.HttpBroadcast]] for more details about this mechanism. */ class HttpBroadcastFactory extends BroadcastFactory { - def initialize(isDriver: Boolean, conf: SparkConf, securityMgr: SecurityManager) { + override def initialize(isDriver: Boolean, conf: SparkConf, securityMgr: SecurityManager) { HttpBroadcast.initialize(isDriver, conf, securityMgr) } - def newBroadcast[T: ClassTag](value_ : T, isLocal: Boolean, id: Long) = + override def newBroadcast[T: ClassTag](value_ : T, isLocal: Boolean, id: Long) = new HttpBroadcast[T](value_, isLocal, id) - def stop() { HttpBroadcast.stop() } + override def stop() { HttpBroadcast.stop() } /** * Remove all persisted state associated with the HTTP broadcast with the given ID. * @param removeFromDriver Whether to remove state from the driver * @param blocking Whether to block until unbroadcasted */ - def unbroadcast(id: Long, removeFromDriver: Boolean, blocking: Boolean) { + override def unbroadcast(id: Long, removeFromDriver: Boolean, blocking: Boolean) { HttpBroadcast.unpersist(id, removeFromDriver, blocking) } } diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala index 734de37ba115d..fe73456ef8fad 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala @@ -17,15 +17,15 @@ package org.apache.spark.broadcast -import java.io.{ByteArrayInputStream, ObjectInputStream, ObjectOutputStream} +import java.io.{ByteArrayOutputStream, ByteArrayInputStream, InputStream, + ObjectInputStream, ObjectOutputStream, OutputStream} import scala.reflect.ClassTag -import scala.math import scala.util.Random import org.apache.spark.{Logging, SparkConf, SparkEnv, SparkException} +import org.apache.spark.io.CompressionCodec import org.apache.spark.storage.{BroadcastBlockId, StorageLevel} -import org.apache.spark.util.Utils /** * A [[org.apache.spark.broadcast.Broadcast]] implementation that uses a BitTorrent-like @@ -49,19 +49,19 @@ private[spark] class TorrentBroadcast[T: ClassTag]( @transient var value_ : T, isLocal: Boolean, id: Long) extends Broadcast[T](id) with Logging with Serializable { - def getValue = value_ + override protected def getValue() = value_ - val broadcastId = BroadcastBlockId(id) + private val broadcastId = BroadcastBlockId(id) TorrentBroadcast.synchronized { SparkEnv.get.blockManager.putSingle( broadcastId, value_, StorageLevel.MEMORY_AND_DISK, tellMaster = false) } - @transient var arrayOfBlocks: Array[TorrentBlock] = null - @transient var totalBlocks = -1 - @transient var totalBytes = -1 - @transient var hasBlocks = 0 + @transient private var arrayOfBlocks: Array[TorrentBlock] = null + @transient private var totalBlocks = -1 + @transient private var totalBytes = -1 + @transient private var hasBlocks = 0 if (!isLocal) { sendBroadcast() @@ -70,7 +70,7 @@ private[spark] class TorrentBroadcast[T: ClassTag]( /** * Remove all persisted state associated with this Torrent broadcast on the executors. */ - def doUnpersist(blocking: Boolean) { + override protected def doUnpersist(blocking: Boolean) { TorrentBroadcast.unpersist(id, removeFromDriver = false, blocking) } @@ -78,11 +78,11 @@ private[spark] class TorrentBroadcast[T: ClassTag]( * Remove all persisted state associated with this Torrent broadcast on the executors * and driver. */ - def doDestroy(blocking: Boolean) { + override protected def doDestroy(blocking: Boolean) { TorrentBroadcast.unpersist(id, removeFromDriver = true, blocking) } - def sendBroadcast() { + private def sendBroadcast() { val tInfo = TorrentBroadcast.blockifyObject(value_) totalBlocks = tInfo.totalBlocks totalBytes = tInfo.totalBytes @@ -159,7 +159,7 @@ private[spark] class TorrentBroadcast[T: ClassTag]( hasBlocks = 0 } - def receiveBroadcast(): Boolean = { + private def receiveBroadcast(): Boolean = { // Receive meta-info about the size of broadcast data, // the number of chunks it is divided into, etc. val metaId = BroadcastBlockId(id, "meta") @@ -211,15 +211,19 @@ private[spark] class TorrentBroadcast[T: ClassTag]( } -private[spark] object TorrentBroadcast extends Logging { +private[broadcast] object TorrentBroadcast extends Logging { private lazy val BLOCK_SIZE = conf.getInt("spark.broadcast.blockSize", 4096) * 1024 private var initialized = false private var conf: SparkConf = null + private var compress: Boolean = false + private var compressionCodec: CompressionCodec = null def initialize(_isDriver: Boolean, conf: SparkConf) { TorrentBroadcast.conf = conf // TODO: we might have to fix it in tests synchronized { if (!initialized) { + compress = conf.getBoolean("spark.broadcast.compress", true) + compressionCodec = CompressionCodec.createCodec(conf) initialized = true } } @@ -229,8 +233,13 @@ private[spark] object TorrentBroadcast extends Logging { initialized = false } - def blockifyObject[T](obj: T): TorrentInfo = { - val byteArray = Utils.serialize[T](obj) + def blockifyObject[T: ClassTag](obj: T): TorrentInfo = { + val bos = new ByteArrayOutputStream() + val out: OutputStream = if (compress) compressionCodec.compressedOutputStream(bos) else bos + val ser = SparkEnv.get.serializer.newInstance() + val serOut = ser.serializeStream(out) + serOut.writeObject[T](obj).close() + val byteArray = bos.toByteArray val bais = new ByteArrayInputStream(byteArray) var blockNum = byteArray.length / BLOCK_SIZE @@ -256,7 +265,7 @@ private[spark] object TorrentBroadcast extends Logging { info } - def unBlockifyObject[T]( + def unBlockifyObject[T: ClassTag]( arrayOfBlocks: Array[TorrentBlock], totalBytes: Int, totalBlocks: Int): T = { @@ -265,24 +274,35 @@ private[spark] object TorrentBroadcast extends Logging { System.arraycopy(arrayOfBlocks(i).byteArray, 0, retByteArray, i * BLOCK_SIZE, arrayOfBlocks(i).byteArray.length) } - Utils.deserialize[T](retByteArray, Thread.currentThread.getContextClassLoader) + + val in: InputStream = { + val arrIn = new ByteArrayInputStream(retByteArray) + if (compress) compressionCodec.compressedInputStream(arrIn) else arrIn + } + val ser = SparkEnv.get.serializer.newInstance() + val serIn = ser.deserializeStream(in) + val obj = serIn.readObject[T]() + serIn.close() + obj } /** * Remove all persisted blocks associated with this torrent broadcast on the executors. * If removeFromDriver is true, also remove these persisted blocks on the driver. */ - def unpersist(id: Long, removeFromDriver: Boolean, blocking: Boolean) = synchronized { - SparkEnv.get.blockManager.master.removeBroadcast(id, removeFromDriver, blocking) + def unpersist(id: Long, removeFromDriver: Boolean, blocking: Boolean) = { + synchronized { + SparkEnv.get.blockManager.master.removeBroadcast(id, removeFromDriver, blocking) + } } } -private[spark] case class TorrentBlock( +private[broadcast] case class TorrentBlock( blockID: Int, byteArray: Array[Byte]) extends Serializable -private[spark] case class TorrentInfo( +private[broadcast] case class TorrentInfo( @transient arrayOfBlocks: Array[TorrentBlock], totalBlocks: Int, totalBytes: Int) diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcastFactory.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcastFactory.scala index 1de8396a0e17f..ad0f701d7a98f 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcastFactory.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcastFactory.scala @@ -28,21 +28,21 @@ import org.apache.spark.{SecurityManager, SparkConf} */ class TorrentBroadcastFactory extends BroadcastFactory { - def initialize(isDriver: Boolean, conf: SparkConf, securityMgr: SecurityManager) { + override def initialize(isDriver: Boolean, conf: SparkConf, securityMgr: SecurityManager) { TorrentBroadcast.initialize(isDriver, conf) } - def newBroadcast[T: ClassTag](value_ : T, isLocal: Boolean, id: Long) = + override def newBroadcast[T: ClassTag](value_ : T, isLocal: Boolean, id: Long) = new TorrentBroadcast[T](value_, isLocal, id) - def stop() { TorrentBroadcast.stop() } + override def stop() { TorrentBroadcast.stop() } /** * Remove all persisted state associated with the torrent broadcast with the given ID. * @param removeFromDriver Whether to remove state from the driver. * @param blocking Whether to block until unbroadcasted */ - def unbroadcast(id: Long, removeFromDriver: Boolean, blocking: Boolean) { + override def unbroadcast(id: Long, removeFromDriver: Boolean, blocking: Boolean) { TorrentBroadcast.unpersist(id, removeFromDriver, blocking) } } diff --git a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala index 86305d2ea8a09..65a1a8fd7e929 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala @@ -22,7 +22,6 @@ private[spark] class ApplicationDescription( val maxCores: Option[Int], val memoryPerSlave: Int, val command: Command, - val sparkHome: Option[String], var appUiUrl: String, val eventLogDir: Option[String] = None) extends Serializable { 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 c371dc3a51c73..065ddda50e65e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/Client.scala +++ b/core/src/main/scala/org/apache/spark/deploy/Client.scala @@ -17,8 +17,6 @@ package org.apache.spark.deploy -import scala.collection.JavaConversions._ -import scala.collection.mutable.Map import scala.concurrent._ import akka.actor._ @@ -29,12 +27,14 @@ import org.apache.log4j.{Level, Logger} import org.apache.spark.{Logging, SecurityManager, SparkConf} import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.{DriverState, Master} -import org.apache.spark.util.{AkkaUtils, Utils} +import org.apache.spark.util.{ActorLogReceive, AkkaUtils, Utils} /** * Proxy that relays messages to the driver. */ -private class ClientActor(driverArgs: ClientArguments, conf: SparkConf) extends Actor with Logging { +private class ClientActor(driverArgs: ClientArguments, conf: SparkConf) + extends Actor with ActorLogReceive with Logging { + var masterActor: ActorSelection = _ val timeout = AkkaUtils.askTimeout(conf) @@ -50,9 +50,6 @@ private class ClientActor(driverArgs: ClientArguments, conf: SparkConf) extends // TODO: We could add an env variable here and intercept it in `sc.addJar` that would // truncate filesystem paths similar to what YARN does. For now, we just require // people call `addJar` assuming the jar is in the same directory. - val env = Map[String, String]() - System.getenv().foreach{case (k, v) => env(k) = v} - val mainClass = "org.apache.spark.deploy.worker.DriverWrapper" val classPathConf = "spark.driver.extraClassPath" @@ -65,10 +62,13 @@ private class ClientActor(driverArgs: ClientArguments, conf: SparkConf) extends cp.split(java.io.File.pathSeparator) } - val javaOptionsConf = "spark.driver.extraJavaOptions" - val javaOpts = sys.props.get(javaOptionsConf) + val extraJavaOptsConf = "spark.driver.extraJavaOptions" + val extraJavaOpts = sys.props.get(extraJavaOptsConf) + .map(Utils.splitCommandString).getOrElse(Seq.empty) + val sparkJavaOpts = Utils.sparkJavaOpts(conf) + val javaOpts = sparkJavaOpts ++ extraJavaOpts val command = new Command(mainClass, Seq("{{WORKER_URL}}", driverArgs.mainClass) ++ - driverArgs.driverOptions, env, classPathEntries, libraryPathEntries, javaOpts) + driverArgs.driverOptions, sys.env, classPathEntries, libraryPathEntries, javaOpts) val driverDescription = new DriverDescription( driverArgs.jarUrl, @@ -109,13 +109,14 @@ private class ClientActor(driverArgs: ClientArguments, conf: SparkConf) extends // Exception, if present statusResponse.exception.map { e => println(s"Exception from cluster was: $e") + e.printStackTrace() System.exit(-1) } System.exit(0) } } - override def receive = { + override def receiveWithLogging = { case SubmitDriverResponse(success, driverId, message) => println(message) @@ -141,8 +142,10 @@ private class ClientActor(driverArgs: ClientArguments, conf: SparkConf) extends */ object Client { def main(args: Array[String]) { - println("WARNING: This client is deprecated and will be removed in a future version of Spark.") - println("Use ./bin/spark-submit with \"--master spark://host:port\"") + if (!sys.props.contains("SPARK_SUBMIT")) { + println("WARNING: This client is deprecated and will be removed in a future version of Spark") + println("Use ./bin/spark-submit with \"--master spark://host:port\"") + } val conf = new SparkConf() val driverArgs = new ClientArguments(args) @@ -154,8 +157,6 @@ object Client { conf.set("akka.loglevel", driverArgs.logLevel.toString.replace("WARN", "WARNING")) Logger.getRootLogger.setLevel(driverArgs.logLevel) - // TODO: See if we can initialize akka so return messages are sent back using the same TCP - // flow. Else, this (sadly) requires the DriverClient be routable from the Master. val (actorSystem, _) = AkkaUtils.createActorSystem( "driverClient", Utils.localHostName(), 0, conf, new SecurityManager(conf)) diff --git a/core/src/main/scala/org/apache/spark/deploy/Command.scala b/core/src/main/scala/org/apache/spark/deploy/Command.scala index 32f3ba385084f..a2b263544c6a2 100644 --- a/core/src/main/scala/org/apache/spark/deploy/Command.scala +++ b/core/src/main/scala/org/apache/spark/deploy/Command.scala @@ -25,5 +25,5 @@ private[spark] case class Command( environment: Map[String, String], classPathEntries: Seq[String], libraryPathEntries: Seq[String], - extraJavaOptions: Option[String] = None) { + javaOpts: Seq[String]) { } diff --git a/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala index c4f5e294a393e..696f32a6f5730 100644 --- a/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala @@ -56,7 +56,6 @@ private[spark] object JsonProtocol { ("cores" -> obj.maxCores) ~ ("memoryperslave" -> obj.memoryPerSlave) ~ ("user" -> obj.user) ~ - ("sparkhome" -> obj.sparkHome) ~ ("command" -> obj.command.toString) } 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 b050dccb6d57f..318509a67a36f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -27,25 +27,43 @@ import org.apache.spark.executor.ExecutorURLClassLoader import org.apache.spark.util.Utils /** - * Scala code behind the spark-submit script. The script handles setting up the classpath with - * relevant Spark dependencies and provides a layer over the different cluster managers and deploy - * modes that Spark supports. + * Main gateway of launching a Spark application. + * + * This program handles setting up the classpath with relevant Spark dependencies and provides + * a layer over the different cluster managers and deploy modes that Spark supports. */ object SparkSubmit { + + // Cluster managers private val YARN = 1 private val STANDALONE = 2 private val MESOS = 4 private val LOCAL = 8 private val ALL_CLUSTER_MGRS = YARN | STANDALONE | MESOS | LOCAL - private var clusterManager: Int = LOCAL + // Deploy modes + private val CLIENT = 1 + private val CLUSTER = 2 + private val ALL_DEPLOY_MODES = CLIENT | CLUSTER - /** - * Special primary resource names that represent shells rather than application jars. - */ + // A special jar name that indicates the class being run is inside of Spark itself, and therefore + // no user jar is needed. + private val SPARK_INTERNAL = "spark-internal" + + // Special primary resource names that represent shells rather than application jars. private val SPARK_SHELL = "spark-shell" private val PYSPARK_SHELL = "pyspark-shell" + // Exposed for testing + private[spark] var exitFn: () => Unit = () => System.exit(-1) + private[spark] var printStream: PrintStream = System.err + private[spark] def printWarning(str: String) = printStream.println("Warning: " + str) + private[spark] def printErrorAndExit(str: String) = { + printStream.println("Error: " + str) + printStream.println("Run with --help for usage help or --verbose for debug output") + exitFn() + } + def main(args: Array[String]) { val appArgs = new SparkSubmitArguments(args) if (appArgs.verbose) { @@ -55,88 +73,78 @@ object SparkSubmit { launch(childArgs, classpath, sysProps, mainClass, appArgs.verbose) } - // Exposed for testing - private[spark] var printStream: PrintStream = System.err - private[spark] var exitFn: () => Unit = () => System.exit(-1) - - private[spark] def printErrorAndExit(str: String) = { - printStream.println("Error: " + str) - printStream.println("Run with --help for usage help or --verbose for debug output") - exitFn() - } - private[spark] def printWarning(str: String) = printStream.println("Warning: " + str) - /** - * @return a tuple containing the arguments for the child, a list of classpath - * entries for the child, a list of system properties, a list of env vars - * and the main class for the child + * @return a tuple containing + * (1) the arguments for the child process, + * (2) a list of classpath entries for the child, + * (3) a list of system properties and env vars, and + * (4) the main class for the child */ 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")) { - clusterManager = YARN - } else if (args.master.startsWith("spark")) { - clusterManager = STANDALONE - } else if (args.master.startsWith("mesos")) { - clusterManager = MESOS - } else { - printErrorAndExit("Master must start with yarn, mesos, spark, or local") - } - // Because "yarn-cluster" and "yarn-client" encapsulate both the master - // and deploy mode, we have some logic to infer the master and deploy mode - // from each other if only one is specified, or exit early if they are at odds. - if (args.deployMode == null && - (args.master == "yarn-standalone" || args.master == "yarn-cluster")) { - args.deployMode = "cluster" - } - if (args.deployMode == "cluster" && args.master == "yarn-client") { - printErrorAndExit("Deploy mode \"cluster\" and master \"yarn-client\" are not compatible") - } - if (args.deployMode == "client" && - (args.master == "yarn-standalone" || args.master == "yarn-cluster")) { - printErrorAndExit("Deploy mode \"client\" and master \"" + args.master - + "\" are not compatible") - } - if (args.deployMode == "cluster" && args.master.startsWith("yarn")) { - args.master = "yarn-cluster" - } - if (args.deployMode != "cluster" && args.master.startsWith("yarn")) { - args.master = "yarn-client" - } - - val deployOnCluster = Option(args.deployMode).getOrElse("client") == "cluster" - - val childClasspath = new ArrayBuffer[String]() + // Values to return val childArgs = new ArrayBuffer[String]() + val childClasspath = new ArrayBuffer[String]() val sysProps = new HashMap[String, String]() var childMainClass = "" - val isPython = args.isPython - val isYarnCluster = clusterManager == YARN && deployOnCluster + // Set the cluster manager + val clusterManager: Int = args.master match { + case m if m.startsWith("yarn") => YARN + case m if m.startsWith("spark") => STANDALONE + case m if m.startsWith("mesos") => MESOS + case m if m.startsWith("local") => LOCAL + case _ => printErrorAndExit("Master must start with yarn, spark, mesos, or local"); -1 + } - // For mesos, only client mode is supported - if (clusterManager == MESOS && deployOnCluster) { - printErrorAndExit("Cluster deploy mode is currently not supported for Mesos clusters.") + // Set the deploy mode; default is client mode + var deployMode: Int = args.deployMode match { + case "client" | null => CLIENT + case "cluster" => CLUSTER + case _ => printErrorAndExit("Deploy mode must be either client or cluster"); -1 } - // For standalone, only client mode is supported - if (clusterManager == STANDALONE && deployOnCluster) { - printErrorAndExit("Cluster deploy mode is currently not supported for standalone clusters.") + // Because "yarn-cluster" and "yarn-client" encapsulate both the master + // and deploy mode, we have some logic to infer the master and deploy mode + // from each other if only one is specified, or exit early if they are at odds. + if (clusterManager == YARN) { + if (args.master == "yarn-standalone") { + printWarning("\"yarn-standalone\" is deprecated. Use \"yarn-cluster\" instead.") + args.master = "yarn-cluster" + } + (args.master, args.deployMode) match { + case ("yarn-cluster", null) => + deployMode = CLUSTER + case ("yarn-cluster", "client") => + printErrorAndExit("Client deploy mode is not compatible with master \"yarn-cluster\"") + case ("yarn-client", "cluster") => + printErrorAndExit("Cluster deploy mode is not compatible with master \"yarn-client\"") + case (_, mode) => + args.master = "yarn-" + Option(mode).getOrElse("client") + } + + // Make sure YARN is included in our build if we're trying to use it + if (!Utils.classIsLoadable("org.apache.spark.deploy.yarn.Client") && !Utils.isTesting) { + printErrorAndExit( + "Could not load YARN classes. " + + "This copy of Spark may not have been compiled with YARN support.") + } } - // For shells, only client mode is applicable - if (isShell(args.primaryResource) && deployOnCluster) { - printErrorAndExit("Cluster deploy mode is not applicable to Spark shells.") + // The following modes are not supported or applicable + (clusterManager, deployMode) match { + case (MESOS, CLUSTER) => + printErrorAndExit("Cluster deploy mode is currently not supported for Mesos clusters.") + case (_, CLUSTER) if args.isPython => + printErrorAndExit("Cluster deploy mode is currently not supported for python applications.") + case (_, CLUSTER) if isShell(args.primaryResource) => + printErrorAndExit("Cluster deploy mode is not applicable to Spark shells.") + case _ => } // If we're running a python app, set the main class to our specific python runner - if (isPython) { - if (deployOnCluster) { - printErrorAndExit("Cluster deploy mode is currently not supported for python.") - } + if (args.isPython) { if (args.primaryResource == PYSPARK_SHELL) { args.mainClass = "py4j.GatewayServer" args.childArgs = ArrayBuffer("--die-on-broken-pipe", "0") @@ -152,120 +160,123 @@ object SparkSubmit { 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 (isUserJar(args.primaryResource)) { - childClasspath += args.primaryResource - } - } else if (clusterManager == YARN) { - childMainClass = "org.apache.spark.deploy.yarn.Client" - childArgs += ("--jar", args.primaryResource) - childArgs += ("--class", args.mainClass) - } - - // Make sure YARN is included in our build if we're trying to use it - if (clusterManager == YARN) { - if (!Utils.classIsLoadable("org.apache.spark.deploy.yarn.Client") && !Utils.isTesting) { - printErrorAndExit("Could not load YARN classes. " + - "This copy of Spark may not have been compiled with YARN support.") - } - } - // Special flag to avoid deprecation warnings at the client sysProps("SPARK_SUBMIT") = "true" // A list of rules to map each argument to system properties or command-line options in // each deploy mode; we iterate through these below val options = List[OptionAssigner]( - OptionAssigner(args.master, ALL_CLUSTER_MGRS, false, sysProp = "spark.master"), - OptionAssigner(args.name, ALL_CLUSTER_MGRS, false, sysProp = "spark.app.name"), - OptionAssigner(args.name, YARN, true, clOption = "--name", sysProp = "spark.app.name"), - OptionAssigner(args.driverExtraClassPath, STANDALONE | YARN, true, + + // All cluster managers + OptionAssigner(args.master, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES, sysProp = "spark.master"), + OptionAssigner(args.name, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES, sysProp = "spark.app.name"), + OptionAssigner(args.jars, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES, sysProp = "spark.jars"), + + // Standalone cluster only + OptionAssigner(args.driverMemory, STANDALONE, CLUSTER, clOption = "--memory"), + OptionAssigner(args.driverCores, STANDALONE, CLUSTER, clOption = "--cores"), + + // Yarn client only + OptionAssigner(args.queue, YARN, CLIENT, sysProp = "spark.yarn.queue"), + OptionAssigner(args.numExecutors, YARN, CLIENT, sysProp = "spark.executor.instances"), + OptionAssigner(args.executorCores, YARN, CLIENT, sysProp = "spark.executor.cores"), + OptionAssigner(args.files, YARN, CLIENT, sysProp = "spark.yarn.dist.files"), + OptionAssigner(args.archives, YARN, CLIENT, sysProp = "spark.yarn.dist.archives"), + + // Yarn cluster only + OptionAssigner(args.name, YARN, CLUSTER, clOption = "--name"), + OptionAssigner(args.driverMemory, YARN, CLUSTER, clOption = "--driver-memory"), + OptionAssigner(args.queue, YARN, CLUSTER, clOption = "--queue"), + OptionAssigner(args.numExecutors, YARN, CLUSTER, clOption = "--num-executors"), + OptionAssigner(args.executorMemory, YARN, CLUSTER, clOption = "--executor-memory"), + OptionAssigner(args.executorCores, YARN, CLUSTER, clOption = "--executor-cores"), + OptionAssigner(args.files, YARN, CLUSTER, clOption = "--files"), + OptionAssigner(args.archives, YARN, CLUSTER, clOption = "--archives"), + OptionAssigner(args.jars, YARN, CLUSTER, clOption = "--addJars"), + + // Other options + OptionAssigner(args.driverExtraClassPath, STANDALONE | YARN, CLUSTER, sysProp = "spark.driver.extraClassPath"), - OptionAssigner(args.driverExtraJavaOptions, STANDALONE | YARN, true, + OptionAssigner(args.driverExtraJavaOptions, STANDALONE | YARN, CLUSTER, sysProp = "spark.driver.extraJavaOptions"), - OptionAssigner(args.driverExtraLibraryPath, STANDALONE | YARN, true, + OptionAssigner(args.driverExtraLibraryPath, STANDALONE | YARN, CLUSTER, sysProp = "spark.driver.extraLibraryPath"), - OptionAssigner(args.driverMemory, YARN, true, clOption = "--driver-memory"), - OptionAssigner(args.driverMemory, STANDALONE, true, clOption = "--memory"), - OptionAssigner(args.driverCores, STANDALONE, true, clOption = "--cores"), - OptionAssigner(args.queue, YARN, true, clOption = "--queue"), - OptionAssigner(args.queue, YARN, false, sysProp = "spark.yarn.queue"), - OptionAssigner(args.numExecutors, YARN, true, clOption = "--num-executors"), - OptionAssigner(args.numExecutors, YARN, false, sysProp = "spark.executor.instances"), - OptionAssigner(args.executorMemory, YARN, true, clOption = "--executor-memory"), - OptionAssigner(args.executorMemory, STANDALONE | MESOS | YARN, false, + OptionAssigner(args.executorMemory, STANDALONE | MESOS | YARN, ALL_DEPLOY_MODES, sysProp = "spark.executor.memory"), - OptionAssigner(args.executorCores, YARN, true, clOption = "--executor-cores"), - OptionAssigner(args.executorCores, YARN, false, sysProp = "spark.executor.cores"), - OptionAssigner(args.totalExecutorCores, STANDALONE | MESOS, false, + OptionAssigner(args.totalExecutorCores, STANDALONE | MESOS, ALL_DEPLOY_MODES, sysProp = "spark.cores.max"), - OptionAssigner(args.files, YARN, false, sysProp = "spark.yarn.dist.files"), - OptionAssigner(args.files, YARN, true, clOption = "--files"), - OptionAssigner(args.files, LOCAL | STANDALONE | MESOS, false, sysProp = "spark.files"), - OptionAssigner(args.files, LOCAL | STANDALONE | MESOS, true, sysProp = "spark.files"), - OptionAssigner(args.archives, YARN, false, sysProp = "spark.yarn.dist.archives"), - OptionAssigner(args.archives, YARN, true, clOption = "--archives"), - OptionAssigner(args.jars, YARN, true, clOption = "--addJars"), - OptionAssigner(args.jars, ALL_CLUSTER_MGRS, false, sysProp = "spark.jars") + OptionAssigner(args.files, LOCAL | STANDALONE | MESOS, ALL_DEPLOY_MODES, + sysProp = "spark.files") ) - // For client mode make any added jars immediately visible on the classpath - if (args.jars != null && !deployOnCluster) { - for (jar <- args.jars.split(",")) { - childClasspath += jar + // In client mode, launch the application main class directly + // In addition, add the main application jar and any added jars (if any) to the classpath + if (deployMode == CLIENT) { + childMainClass = args.mainClass + if (isUserJar(args.primaryResource)) { + childClasspath += args.primaryResource } + if (args.jars != null) { childClasspath ++= args.jars.split(",") } + if (args.childArgs != null) { childArgs ++= args.childArgs } } + // Map all arguments to command-line options or system properties for our chosen mode for (opt <- options) { - if (opt.value != null && deployOnCluster == opt.deployOnCluster && + if (opt.value != null && + (deployMode & opt.deployMode) != 0 && (clusterManager & opt.clusterManager) != 0) { - if (opt.clOption != null) { - childArgs += (opt.clOption, opt.value) - } - if (opt.sysProp != null) { - sysProps.put(opt.sysProp, opt.value) - } + if (opt.clOption != null) { childArgs += (opt.clOption, opt.value) } + if (opt.sysProp != null) { sysProps.put(opt.sysProp, opt.value) } } } // Add the application jar automatically so the user doesn't have to call sc.addJar // For YARN cluster mode, the jar is already distributed on each node as "app.jar" // 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()) + val isYarnCluster = clusterManager == YARN && deployMode == CLUSTER + if (!isYarnCluster && !args.isPython) { + var jars = sysProps.get("spark.jars").map(x => x.split(",").toSeq).getOrElse(Seq.empty) if (isUserJar(args.primaryResource)) { jars = jars ++ Seq(args.primaryResource) } sysProps.put("spark.jars", jars.mkString(",")) } - // Standalone cluster specific configurations - if (deployOnCluster && clusterManager == STANDALONE) { + // In standalone-cluster mode, use Client as a wrapper around the user class + if (clusterManager == STANDALONE && deployMode == CLUSTER) { + childMainClass = "org.apache.spark.deploy.Client" if (args.supervise) { childArgs += "--supervise" } - childMainClass = "org.apache.spark.deploy.Client" childArgs += "launch" childArgs += (args.master, args.primaryResource, args.mainClass) + if (args.childArgs != null) { + childArgs ++= args.childArgs + } } - // Arguments to be passed to user program - if (args.childArgs != null) { - if (!deployOnCluster || clusterManager == STANDALONE) { - childArgs ++= args.childArgs - } else if (clusterManager == YARN) { - for (arg <- args.childArgs) { - childArgs += ("--arg", arg) - } + // In yarn-cluster mode, use yarn.Client as a wrapper around the user class + if (clusterManager == YARN && deployMode == CLUSTER) { + childMainClass = "org.apache.spark.deploy.yarn.Client" + if (args.primaryResource != SPARK_INTERNAL) { + childArgs += ("--jar", args.primaryResource) + } + childArgs += ("--class", args.mainClass) + if (args.childArgs != null) { + args.childArgs.foreach { arg => childArgs += ("--arg", arg) } } } + // Properties given with --conf are superceded by other options, but take precedence over + // properties in the defaults file. + for ((k, v) <- args.sparkProperties) { + sysProps.getOrElseUpdate(k, v) + } + // Read from default spark properties, if any for ((k, v) <- args.getDefaultSparkProperties) { - if (!sysProps.contains(k)) sysProps(k) = v + sysProps.getOrElseUpdate(k, v) } (childArgs, childClasspath, sysProps, childMainClass) @@ -328,7 +339,7 @@ object SparkSubmit { * Return whether the given primary resource represents a user jar. */ private def isUserJar(primaryResource: String): Boolean = { - !isShell(primaryResource) && !isPython(primaryResource) + !isShell(primaryResource) && !isPython(primaryResource) && !isInternal(primaryResource) } /** @@ -345,6 +356,10 @@ object SparkSubmit { primaryResource.endsWith(".py") || primaryResource == PYSPARK_SHELL } + private[spark] def isInternal(primaryResource: String): Boolean = { + primaryResource == SPARK_INTERNAL + } + /** * Merge a sequence of comma-separated file lists, some of which may be null to indicate * no files, into a single comma-separated string. @@ -364,6 +379,6 @@ object SparkSubmit { private[spark] case class OptionAssigner( value: String, clusterManager: Int, - deployOnCluster: Boolean, + deployMode: Int, clOption: String = null, sysProp: String = null) 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 57655aa4c32b1..d545f58c5da7e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -55,9 +55,10 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { var verbose: Boolean = false var isPython: Boolean = false var pyFiles: String = null + val sparkProperties: HashMap[String, String] = new HashMap[String, String]() parseOpts(args.toList) - loadDefaults() + mergeSparkProperties() checkRequiredArguments() /** Return default present in the currently defined defaults file. */ @@ -78,10 +79,23 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { defaultProperties } - /** Fill in any undefined values based on the current properties file or built-in defaults. */ - private def loadDefaults(): Unit = { - + /** + * Fill in any undefined values based on the default properties file or options passed in through + * the '--conf' flag. + */ + private def mergeSparkProperties(): Unit = { // Use common defaults file, if not specified by user + if (propertiesFile == null) { + sys.env.get("SPARK_CONF_DIR").foreach { sparkConfDir => + val sep = File.separator + val defaultPath = s"${sparkConfDir}${sep}spark-defaults.conf" + val file = new File(defaultPath) + if (file.exists()) { + propertiesFile = file.getAbsolutePath + } + } + } + if (propertiesFile == null) { sys.env.get("SPARK_HOME").foreach { sparkHome => val sep = File.separator @@ -93,18 +107,20 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { } } - val defaultProperties = getDefaultSparkProperties + val properties = getDefaultSparkProperties + properties.putAll(sparkProperties) + // Use properties file as fallback for values which have a direct analog to // arguments in this script. - master = Option(master).getOrElse(defaultProperties.get("spark.master").orNull) + master = Option(master).getOrElse(properties.get("spark.master").orNull) executorMemory = Option(executorMemory) - .getOrElse(defaultProperties.get("spark.executor.memory").orNull) + .getOrElse(properties.get("spark.executor.memory").orNull) executorCores = Option(executorCores) - .getOrElse(defaultProperties.get("spark.executor.cores").orNull) + .getOrElse(properties.get("spark.executor.cores").orNull) totalExecutorCores = Option(totalExecutorCores) - .getOrElse(defaultProperties.get("spark.cores.max").orNull) - name = Option(name).getOrElse(defaultProperties.get("spark.app.name").orNull) - jars = Option(jars).getOrElse(defaultProperties.get("spark.jars").orNull) + .getOrElse(properties.get("spark.cores.max").orNull) + name = Option(name).getOrElse(properties.get("spark.app.name").orNull) + jars = Option(jars).getOrElse(properties.get("spark.jars").orNull) // This supports env vars in older versions of Spark master = Option(master).getOrElse(System.getenv("MASTER")) @@ -177,6 +193,7 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { | executorCores $executorCores | totalExecutorCores $totalExecutorCores | propertiesFile $propertiesFile + | extraSparkProperties $sparkProperties | driverMemory $driverMemory | driverCores $driverCores | driverExtraClassPath $driverExtraClassPath @@ -202,10 +219,15 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { /** Fill in values by parsing user options. */ private def parseOpts(opts: Seq[String]): Unit = { + val EQ_SEPARATED_OPT="""(--[^=]+)=(.+)""".r + // Delineates parsing of Spark options from parsing of user options. - var inSparkOpts = true parse(opts) + /** + * NOTE: If you add or remove spark-submit options, + * modify NOT ONLY this file but also utils.sh + */ def parse(opts: Seq[String]): Unit = opts match { case ("--name") :: value :: tail => name = value @@ -290,6 +312,13 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { jars = Utils.resolveURIs(value) parse(tail) + case ("--conf" | "-c") :: value :: tail => + value.split("=", 2).toSeq match { + case Seq(k, v) => sparkProperties(k) = v + case _ => SparkSubmit.printErrorAndExit(s"Spark config without '=': $value") + } + parse(tail) + case ("--help" | "-h") :: tail => printUsageAndExit(0) @@ -297,33 +326,21 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { verbose = true parse(tail) + case EQ_SEPARATED_OPT(opt, value) :: tail => + parse(opt :: value :: tail) + + case value :: tail if value.startsWith("-") => + SparkSubmit.printErrorAndExit(s"Unrecognized option '$value'.") + case value :: tail => - if (inSparkOpts) { - value match { - // convert --foo=bar to --foo bar - case v if v.startsWith("--") && v.contains("=") && v.split("=").size == 2 => - val parts = v.split("=") - parse(Seq(parts(0), parts(1)) ++ tail) - case v if v.startsWith("-") => - val errMessage = s"Unrecognized option '$value'." - SparkSubmit.printErrorAndExit(errMessage) - case v => - primaryResource = - if (!SparkSubmit.isShell(v)) { - Utils.resolveURI(v).toString - } else { - v - } - inSparkOpts = false - isPython = SparkSubmit.isPython(v) - parse(tail) - } - } else { - if (!value.isEmpty) { - childArgs += value + primaryResource = + if (!SparkSubmit.isShell(value) && !SparkSubmit.isInternal(value)) { + Utils.resolveURI(value).toString + } else { + value } - parse(tail) - } + isPython = SparkSubmit.isPython(value) + childArgs ++= tail case Nil => } @@ -349,6 +366,8 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { | on the PYTHONPATH for Python apps. | --files FILES Comma-separated list of files to be placed in the working | directory of each executor. + | + | --conf PROP=VALUE Arbitrary Spark configuration property. | --properties-file FILE Path to a file from which to load extra properties. If not | specified, this will look for conf/spark-defaults.conf. | diff --git a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala index d38e9e79204c2..32790053a6be8 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala @@ -30,7 +30,7 @@ import org.apache.spark.{Logging, SparkConf, SparkException} import org.apache.spark.deploy.{ApplicationDescription, ExecutorState} import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.Master -import org.apache.spark.util.{Utils, AkkaUtils} +import org.apache.spark.util.{ActorLogReceive, Utils, AkkaUtils} /** * Interface allowing applications to speak with a Spark deploy cluster. Takes a master URL, @@ -56,7 +56,7 @@ private[spark] class AppClient( var registered = false var activeMasterUrl: String = null - class ClientActor extends Actor with Logging { + class ClientActor extends Actor with ActorLogReceive with Logging { var master: ActorSelection = null var alreadyDisconnected = false // To avoid calling listener.disconnected() multiple times var alreadyDead = false // To avoid calling listener.dead() multiple times @@ -119,7 +119,7 @@ private[spark] class AppClient( .contains(remoteUrl.hostPort) } - override def receive = { + override def receiveWithLogging = { case RegisteredApplication(appId_, masterUrl) => appId = appId_ registered = true diff --git a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala index e15a87bd38fda..88a0862b96afe 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala @@ -46,11 +46,10 @@ private[spark] object TestClient { def main(args: Array[String]) { val url = args(0) val conf = new SparkConf - val (actorSystem, port) = AkkaUtils.createActorSystem("spark", Utils.localIpAddress, 0, + val (actorSystem, _) = AkkaUtils.createActorSystem("spark", Utils.localIpAddress, 0, conf = conf, securityManager = new SecurityManager(conf)) - val desc = new ApplicationDescription( - "TestClient", Some(1), 512, Command("spark.deploy.client.TestExecutor", Seq(), Map(), Seq(), - Seq()), Some("dummy-spark-home"), "ignored") + val desc = new ApplicationDescription("TestClient", Some(1), 512, + Command("spark.deploy.client.TestExecutor", Seq(), Map(), Seq(), Seq(), Seq()), "ignored") val listener = new TestListener val client = new AppClient(actorSystem, Array(url), desc, listener, new SparkConf) client.start() diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index a8c9ac072449f..cc06540ee0647 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -36,11 +36,11 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis conf.getInt("spark.history.updateInterval", 10)) * 1000 private val logDir = conf.get("spark.history.fs.logDirectory", null) - if (logDir == null) { - throw new IllegalArgumentException("Logging directory must be specified.") - } + private val resolvedLogDir = Option(logDir) + .map { d => Utils.resolveURI(d) } + .getOrElse { throw new IllegalArgumentException("Logging directory must be specified.") } - private val fs = Utils.getHadoopFileSystem(logDir) + private val fs = Utils.getHadoopFileSystem(resolvedLogDir) // A timestamp of when the disk was last accessed to check for log updates private var lastLogCheckTimeMs = -1L @@ -76,14 +76,14 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis private def initialize() { // Validate the log directory. - val path = new Path(logDir) + val path = new Path(resolvedLogDir) if (!fs.exists(path)) { throw new IllegalArgumentException( - "Logging directory specified does not exist: %s".format(logDir)) + "Logging directory specified does not exist: %s".format(resolvedLogDir)) } if (!fs.getFileStatus(path).isDir) { throw new IllegalArgumentException( - "Logging directory specified is not a directory: %s".format(logDir)) + "Logging directory specified is not a directory: %s".format(resolvedLogDir)) } checkForLogs() @@ -95,15 +95,16 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis override def getAppUI(appId: String): SparkUI = { try { - val appLogDir = fs.getFileStatus(new Path(logDir, appId)) - loadAppInfo(appLogDir, true)._2 + val appLogDir = fs.getFileStatus(new Path(resolvedLogDir.toString, appId)) + val (_, ui) = loadAppInfo(appLogDir, renderUI = true) + ui } catch { case e: FileNotFoundException => null } } override def getConfig(): Map[String, String] = - Map(("Event Log Location" -> logDir)) + Map("Event Log Location" -> resolvedLogDir.toString) /** * Builds the application list based on the current contents of the log directory. @@ -114,14 +115,14 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis lastLogCheckTimeMs = getMonotonicTimeMs() logDebug("Checking for logs. Time is now %d.".format(lastLogCheckTimeMs)) try { - val logStatus = fs.listStatus(new Path(logDir)) + val logStatus = fs.listStatus(new Path(resolvedLogDir)) val logDirs = if (logStatus != null) logStatus.filter(_.isDir).toSeq else Seq[FileStatus]() - val logInfos = logDirs.filter { - dir => fs.isFile(new Path(dir.getPath(), EventLoggingListener.APPLICATION_COMPLETE)) + val logInfos = logDirs.filter { dir => + fs.isFile(new Path(dir.getPath, EventLoggingListener.APPLICATION_COMPLETE)) } val currentApps = Map[String, ApplicationHistoryInfo]( - appList.map(app => (app.id -> app)):_*) + appList.map(app => app.id -> app):_*) // For any application that either (i) is not listed or (ii) has changed since the last time // the listing was created (defined by the log dir's modification time), load the app's info. @@ -131,7 +132,8 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis val curr = currentApps.getOrElse(dir.getPath().getName(), null) if (curr == null || curr.lastUpdated < getModificationTime(dir)) { try { - newApps += loadAppInfo(dir, false)._1 + val (app, _) = loadAppInfo(dir, renderUI = false) + newApps += app } catch { case e: Exception => logError(s"Failed to load app info from directory $dir.") } @@ -159,9 +161,9 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis * @return A 2-tuple `(app info, ui)`. `ui` will be null if `renderUI` is false. */ private def loadAppInfo(logDir: FileStatus, renderUI: Boolean) = { - val elogInfo = EventLoggingListener.parseLoggingInfo(logDir.getPath(), fs) val path = logDir.getPath val appId = path.getName + val elogInfo = EventLoggingListener.parseLoggingInfo(path, fs) val replayBus = new ReplayListenerBus(elogInfo.logPaths, fs, elogInfo.compressionCodec) val appListener = new ApplicationEventListener replayBus.addListener(appListener) @@ -169,7 +171,8 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis val ui: SparkUI = if (renderUI) { val conf = this.conf.clone() val appSecManager = new SecurityManager(conf) - new SparkUI(conf, appSecManager, replayBus, appId, "/history/" + appId) + new SparkUI(conf, appSecManager, replayBus, appId, + HistoryServer.UI_PATH_PREFIX + s"/$appId") // Do not call ui.bind() to avoid creating a new server for each application } else { null @@ -186,7 +189,9 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis if (ui != null) { val uiAclsEnabled = conf.getBoolean("spark.history.ui.acls.enable", false) - ui.getSecurityManager.setUIAcls(uiAclsEnabled) + ui.getSecurityManager.setAcls(uiAclsEnabled) + // make sure to set admin acls before view acls so properly picked up + ui.getSecurityManager.setAdminAcls(appListener.adminAcls) ui.getSecurityManager.setViewAcls(appListener.sparkUser, appListener.viewAcls) } (appInfo, ui) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala index a958c837c2ff6..c4ef8b63b0071 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala @@ -45,7 +45,7 @@ private[spark] class HistoryPage(parent: HistoryServer) extends WebUIPage("") {
    - { providerConfig.map(e =>
  • {e._1}: {e._2}
  • ) } + {providerConfig.map { case (k, v) =>
  • {k}: {v}
  • }}
{ if (allApps.size > 0) { @@ -75,7 +75,7 @@ private[spark] class HistoryPage(parent: HistoryServer) extends WebUIPage("") { "Last Updated") private def appRow(info: ApplicationHistoryInfo): Seq[Node] = { - val uiAddress = "/history/" + info.id + val uiAddress = HistoryServer.UI_PATH_PREFIX + s"/${info.id}" val startTime = UIUtils.formatDate(info.startTime) val endTime = UIUtils.formatDate(info.endTime) val duration = UIUtils.formatDuration(info.endTime - info.startTime) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index 56b38ddfc9313..d1a64c1912cb8 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -25,9 +25,9 @@ import org.eclipse.jetty.servlet.{ServletContextHandler, ServletHolder} import org.apache.spark.{Logging, SecurityManager, SparkConf} import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.ui.{WebUI, SparkUI, UIUtils} +import org.apache.spark.ui.{SparkUI, UIUtils, WebUI} import org.apache.spark.ui.JettyUtils._ -import org.apache.spark.util.{SignalLogger, Utils} +import org.apache.spark.util.SignalLogger /** * A web server that renders SparkUIs of completed applications. @@ -114,7 +114,7 @@ class HistoryServer( attachHandler(createStaticHandler(SparkUI.STATIC_RESOURCE_DIR, "/static")) val contextHandler = new ServletContextHandler - contextHandler.setContextPath("/history") + contextHandler.setContextPath(HistoryServer.UI_PATH_PREFIX) contextHandler.addServlet(new ServletHolder(loaderServlet), "/*") attachHandler(contextHandler) } @@ -172,10 +172,12 @@ class HistoryServer( object HistoryServer extends Logging { private val conf = new SparkConf + val UI_PATH_PREFIX = "/history" + def main(argStrings: Array[String]) { SignalLogger.register(log) initSecurity() - val args = new HistoryServerArguments(conf, argStrings) + new HistoryServerArguments(conf, argStrings) val securityManager = new SecurityManager(conf) val providerName = conf.getOption("spark.history.provider") diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala index be9361b754fc3..25fc76c23e0fb 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala @@ -18,7 +18,6 @@ package org.apache.spark.deploy.history import org.apache.spark.SparkConf -import org.apache.spark.util.Utils /** * Command-line parser for the master. @@ -32,6 +31,7 @@ private[spark] class HistoryServerArguments(conf: SparkConf, args: Array[String] args match { case ("--dir" | "-d") :: value :: tail => logDir = value + conf.set("spark.history.fs.logDirectory", value) parse(tail) case ("--help" | "-h") :: tail => @@ -42,9 +42,6 @@ private[spark] class HistoryServerArguments(conf: SparkConf, args: Array[String] case _ => printUsageAndExit(1) } - if (logDir != null) { - conf.set("spark.history.fs.logDirectory", logDir) - } } private def printUsageAndExit(exitCode: Int) { 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 72d0589689e71..d3674427b1271 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 @@ -46,6 +46,11 @@ private[spark] class ApplicationInfo( init() + private def readObject(in: java.io.ObjectInputStream): Unit = { + in.defaultReadObject() + init() + } + private def init() { state = ApplicationState.WAITING executors = new mutable.HashMap[Int, ExecutorInfo] diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationSource.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationSource.scala index c87b66f047dc8..38db02cd2421b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationSource.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationSource.scala @@ -22,8 +22,8 @@ import com.codahale.metrics.{Gauge, MetricRegistry} import org.apache.spark.metrics.source.Source class ApplicationSource(val application: ApplicationInfo) extends Source { - val metricRegistry = new MetricRegistry() - val sourceName = "%s.%s.%s".format("application", application.desc.name, + override val metricRegistry = new MetricRegistry() + override val sourceName = "%s.%s.%s".format("application", application.desc.name, System.currentTimeMillis()) metricRegistry.register(MetricRegistry.name("status"), new Gauge[String] { 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 a304102a49086..cfa2c028a807b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -17,6 +17,7 @@ package org.apache.spark.deploy.master +import java.net.URLEncoder import java.text.SimpleDateFormat import java.util.Date @@ -30,25 +31,25 @@ import akka.actor._ import akka.pattern.ask import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} import akka.serialization.SerializationExtension -import org.apache.hadoop.fs.FileSystem import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkException} import org.apache.spark.deploy.{ApplicationDescription, DriverDescription, ExecutorState} import org.apache.spark.deploy.DeployMessages._ +import org.apache.spark.deploy.history.HistoryServer import org.apache.spark.deploy.master.DriverState.DriverState import org.apache.spark.deploy.master.MasterMessages._ import org.apache.spark.deploy.master.ui.MasterWebUI import org.apache.spark.metrics.MetricsSystem import org.apache.spark.scheduler.{EventLoggingListener, ReplayListenerBus} import org.apache.spark.ui.SparkUI -import org.apache.spark.util.{AkkaUtils, SignalLogger, Utils} +import org.apache.spark.util.{ActorLogReceive, AkkaUtils, SignalLogger, Utils} private[spark] class Master( host: String, port: Int, webUiPort: Int, val securityMgr: SecurityManager) - extends Actor with Logging { + extends Actor with ActorLogReceive with Logging { import context.dispatcher // to use Akka's scheduler.schedule() @@ -57,6 +58,7 @@ private[spark] class Master( def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs val WORKER_TIMEOUT = conf.getLong("spark.worker.timeout", 60) * 1000 val RETAINED_APPLICATIONS = conf.getInt("spark.deploy.retainedApplications", 200) + val RETAINED_DRIVERS = conf.getInt("spark.deploy.retainedDrivers", 200) val REAPER_ITERATIONS = conf.getInt("spark.dead.worker.persistence", 15) val RECOVERY_DIR = conf.get("spark.deploy.recoveryDirectory", "") val RECOVERY_MODE = conf.get("spark.deploy.recoveryMode", "NONE") @@ -72,9 +74,7 @@ private[spark] class Master( val waitingApps = new ArrayBuffer[ApplicationInfo] val completedApps = new ArrayBuffer[ApplicationInfo] var nextAppNumber = 0 - val appIdToUI = new HashMap[String, SparkUI] - val fileSystemsUsed = new HashSet[FileSystem] val drivers = new HashSet[DriverInfo] val completedDrivers = new ArrayBuffer[DriverInfo] @@ -154,19 +154,20 @@ private[spark] class Master( } override def postStop() { + masterMetricsSystem.report() + applicationMetricsSystem.report() // prevent the CompleteRecovery message sending to restarted master if (recoveryCompletionTask != null) { recoveryCompletionTask.cancel() } webUi.stop() - fileSystemsUsed.foreach(_.close()) masterMetricsSystem.stop() applicationMetricsSystem.stop() persistenceEngine.close() context.stop(leaderElectionAgent) } - override def receive = { + override def receiveWithLogging = { case ElectedLeader => { val (storedApps, storedDrivers, storedWorkers) = persistenceEngine.readPersistedData() state = if (storedApps.isEmpty && storedDrivers.isEmpty && storedWorkers.isEmpty) { @@ -644,10 +645,7 @@ private[spark] class Master( waitingApps -= app // If application events are logged, use them to rebuild the UI - if (!rebuildSparkUI(app)) { - // Avoid broken links if the UI is not reconstructed - app.desc.appUiUrl = "" - } + rebuildSparkUI(app) for (exec <- app.executors.values) { exec.worker.removeExecutor(exec) @@ -669,29 +667,49 @@ private[spark] class Master( */ def rebuildSparkUI(app: ApplicationInfo): Boolean = { val appName = app.desc.name - val eventLogDir = app.desc.eventLogDir.getOrElse { return false } + val notFoundBasePath = HistoryServer.UI_PATH_PREFIX + "/not-found" + val eventLogDir = app.desc.eventLogDir.getOrElse { + // Event logging is not enabled for this application + app.desc.appUiUrl = notFoundBasePath + return false + } val fileSystem = Utils.getHadoopFileSystem(eventLogDir) val eventLogInfo = EventLoggingListener.parseLoggingInfo(eventLogDir, fileSystem) val eventLogPaths = eventLogInfo.logPaths val compressionCodec = eventLogInfo.compressionCodec - if (!eventLogPaths.isEmpty) { - try { - val replayBus = new ReplayListenerBus(eventLogPaths, fileSystem, compressionCodec) - val ui = new SparkUI( - new SparkConf, replayBus, appName + " (completed)", "/history/" + app.id) - replayBus.replay() - app.desc.appUiUrl = ui.basePath - appIdToUI(app.id) = ui - webUi.attachSparkUI(ui) - return true - } catch { - case e: Exception => - logError("Exception in replaying log for application %s (%s)".format(appName, app.id), e) - } - } else { - logWarning("Application %s (%s) has no valid logs: %s".format(appName, app.id, eventLogDir)) + + if (eventLogPaths.isEmpty) { + // Event logging is enabled for this application, but no event logs are found + val title = s"Application history not found (${app.id})" + var msg = s"No event logs found for application $appName in $eventLogDir." + logWarning(msg) + msg += " Did you specify the correct logging directory?" + msg = URLEncoder.encode(msg, "UTF-8") + app.desc.appUiUrl = notFoundBasePath + s"?msg=$msg&title=$title" + return false + } + + try { + val replayBus = new ReplayListenerBus(eventLogPaths, fileSystem, compressionCodec) + val ui = new SparkUI(new SparkConf, replayBus, appName + " (completed)", + HistoryServer.UI_PATH_PREFIX + s"/${app.id}") + replayBus.replay() + appIdToUI(app.id) = ui + webUi.attachSparkUI(ui) + // Application UI is successfully rebuilt, so link the Master UI to it + app.desc.appUiUrl = ui.basePath + true + } catch { + case e: Exception => + // Relay exception message to application UI page + val title = s"Application history load error (${app.id})" + val exception = URLEncoder.encode(Utils.exceptionString(e), "UTF-8") + var msg = s"Exception in replaying log for application $appName!" + logError(msg, e) + msg = URLEncoder.encode(msg, "UTF-8") + app.desc.appUiUrl = notFoundBasePath + s"?msg=$msg&exception=$exception&title=$title" + false } - false } /** Generate a new app ID given a app's submission date */ @@ -744,11 +762,16 @@ private[spark] class Master( case Some(driver) => logInfo(s"Removing driver: $driverId") drivers -= driver + if (completedDrivers.size >= RETAINED_DRIVERS) { + val toRemove = math.max(RETAINED_DRIVERS / 10, 1) + completedDrivers.trimStart(toRemove) + } completedDrivers += driver persistenceEngine.removeDriver(driver) driver.state = finalState driver.exception = exception driver.worker.foreach(w => w.removeDriver(driver)) + schedule() case None => logWarning(s"Asked to remove unknown driver: $driverId") } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala b/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala index a87781fb93850..4b0dbbe543d3f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala @@ -38,8 +38,8 @@ private[spark] class MasterArguments(args: Array[String], conf: SparkConf) { if (System.getenv("SPARK_MASTER_WEBUI_PORT") != null) { webUiPort = System.getenv("SPARK_MASTER_WEBUI_PORT").toInt } - if (conf.contains("master.ui.port")) { - webUiPort = conf.get("master.ui.port").toInt + if (conf.contains("spark.master.ui.port")) { + webUiPort = conf.get("spark.master.ui.port").toInt } parse(args.toList) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/MasterSource.scala b/core/src/main/scala/org/apache/spark/deploy/master/MasterSource.scala index 36c1b87b7f684..9c3f79f1244b7 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/MasterSource.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/MasterSource.scala @@ -22,8 +22,8 @@ import com.codahale.metrics.{Gauge, MetricRegistry} import org.apache.spark.metrics.source.Source private[spark] class MasterSource(val master: Master) extends Source { - val metricRegistry = new MetricRegistry() - val sourceName = "master" + override val metricRegistry = new MetricRegistry() + override val sourceName = "master" // Gauge for worker numbers in cluster metricRegistry.register(MetricRegistry.name("workers"), new Gauge[Int] { 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 34fa1429c86de..4588c130ef439 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 @@ -28,7 +28,7 @@ import org.json4s.JValue 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} +import org.apache.spark.ui.{UIUtils, WebUIPage} import org.apache.spark.util.Utils private[spark] class ApplicationPage(parent: MasterWebUI) extends WebUIPage("app") { diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/HistoryNotFoundPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/HistoryNotFoundPage.scala new file mode 100644 index 0000000000000..d8daff3e7fb9c --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/HistoryNotFoundPage.scala @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy.master.ui + +import java.net.URLDecoder +import javax.servlet.http.HttpServletRequest + +import scala.xml.Node + +import org.apache.spark.ui.{UIUtils, WebUIPage} + +private[spark] class HistoryNotFoundPage(parent: MasterWebUI) + extends WebUIPage("history/not-found") { + + /** + * Render a page that conveys failure in loading application history. + * + * This accepts 3 HTTP parameters: + * msg = message to display to the user + * title = title of the page + * exception = detailed description of the exception in loading application history (if any) + * + * Parameters "msg" and "exception" are assumed to be UTF-8 encoded. + */ + def render(request: HttpServletRequest): Seq[Node] = { + val titleParam = request.getParameter("title") + val msgParam = request.getParameter("msg") + val exceptionParam = request.getParameter("exception") + + // If no parameters are specified, assume the user did not enable event logging + val defaultTitle = "Event logging is not enabled" + val defaultContent = +
+
+ No event logs were found for this application! To + enable event logging, + set spark.eventLog.enabled to true and + spark.eventLog.dir to the directory to which your + event logs are written. +
+
+ + val title = Option(titleParam).getOrElse(defaultTitle) + val content = Option(msgParam) + .map { msg => URLDecoder.decode(msg, "UTF-8") } + .map { msg => +
+
{msg}
+
++ + Option(exceptionParam) + .map { e => URLDecoder.decode(e, "UTF-8") } + .map { e =>
{e}
} + .getOrElse(Seq.empty) + }.getOrElse(defaultContent) + + UIUtils.basicSparkPage(content, title) + } +} diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala index a18b39fc95d64..d86ec1e03e45c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala @@ -21,14 +21,14 @@ import org.apache.spark.Logging import org.apache.spark.deploy.master.Master import org.apache.spark.ui.{SparkUI, WebUI} import org.apache.spark.ui.JettyUtils._ -import org.apache.spark.util.{AkkaUtils, Utils} +import org.apache.spark.util.AkkaUtils /** * Web UI server for the standalone master. */ private[spark] class MasterWebUI(val master: Master, requestedPort: Int) - extends WebUI(master.securityMgr, requestedPort, master.conf) with Logging { + extends WebUI(master.securityMgr, requestedPort, master.conf, name = "MasterUI") with Logging { val masterActorRef = master.self val timeout = AkkaUtils.askTimeout(master.conf) @@ -38,6 +38,7 @@ class MasterWebUI(val master: Master, requestedPort: Int) /** Initialize all components of the server. */ def initialize() { attachPage(new ApplicationPage(this)) + attachPage(new HistoryNotFoundPage(this)) attachPage(new MasterPage(this)) attachHandler(createStaticHandler(MasterWebUI.STATIC_RESOURCE_DIR, "/static")) master.masterMetricsSystem.getServletHandlers.foreach(attachHandler) 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 4af5bc3afad6c..687e492a0d6fc 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 @@ -47,7 +47,6 @@ object CommandUtils extends Logging { */ def buildJavaOpts(command: Command, memory: Int, sparkHome: String): Seq[String] = { val memoryOpts = Seq(s"-Xms${memory}M", s"-Xmx${memory}M") - val extraOpts = command.extraJavaOptions.map(Utils.splitCommandString).getOrElse(Seq()) // Exists for backwards compatibility with older Spark versions val workerLocalOpts = Option(getenv("SPARK_JAVA_OPTS")).map(Utils.splitCommandString) @@ -62,7 +61,7 @@ object CommandUtils extends Logging { val joined = command.libraryPathEntries.mkString(File.pathSeparator) Seq(s"-Djava.library.path=$joined") } else { - Seq() + Seq() } val permGenOpt = Seq("-XX:MaxPermSize=128m") @@ -71,11 +70,11 @@ object CommandUtils extends Logging { val ext = if (System.getProperty("os.name").startsWith("Windows")) ".cmd" else ".sh" val classPath = Utils.executeAndGetOutput( Seq(sparkHome + "/bin/compute-classpath" + ext), - extraEnvironment=command.environment) + extraEnvironment = command.environment) val userClassPath = command.classPathEntries ++ Seq(classPath) Seq("-cp", userClassPath.filterNot(_.isEmpty).mkString(File.pathSeparator)) ++ - permGenOpt ++ libraryOpts ++ extraOpts ++ workerLocalOpts ++ memoryOpts + permGenOpt ++ libraryOpts ++ workerLocalOpts ++ command.javaOpts ++ memoryOpts } /** Spawn a thread that will redirect a given stream to a file */ diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala index 662d37871e7a6..5caaf6bea3575 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala @@ -36,6 +36,7 @@ import org.apache.spark.deploy.master.DriverState.DriverState /** * Manages the execution of one driver, including automatically restarting the driver on failure. + * This is currently only used in standalone cluster deploy mode. */ private[spark] class DriverRunner( val driverId: String, @@ -81,7 +82,7 @@ private[spark] class DriverRunner( driverDesc.command.environment, classPath, driverDesc.command.libraryPathEntries, - driverDesc.command.extraJavaOptions) + driverDesc.command.javaOpts) val command = CommandUtils.buildCommandSeq(newCommand, driverDesc.mem, sparkHome.getAbsolutePath) launchDriver(command, driverDesc.command.environment, driverDir, driverDesc.supervise) 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 467317dd9b44c..7be89f9aff0f3 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 @@ -30,6 +30,7 @@ import org.apache.spark.util.logging.FileAppender /** * Manages the execution of one executor process. + * This is currently only used in standalone mode. */ private[spark] class ExecutorRunner( val appId: String, @@ -72,7 +73,7 @@ private[spark] class ExecutorRunner( } /** - * kill executor process, wait for exit and notify worker to update resource status + * Kill executor process, wait for exit and notify worker to update resource status. * * @param message the exception message which caused the executor's death */ @@ -114,10 +115,13 @@ private[spark] class ExecutorRunner( } def getCommandSeq = { - val command = Command(appDesc.command.mainClass, - appDesc.command.arguments.map(substituteVariables) ++ Seq(appId), appDesc.command.environment, - appDesc.command.classPathEntries, appDesc.command.libraryPathEntries, - appDesc.command.extraJavaOptions) + val command = Command( + appDesc.command.mainClass, + appDesc.command.arguments.map(substituteVariables) ++ Seq(appId), + appDesc.command.environment, + appDesc.command.classPathEntries, + appDesc.command.libraryPathEntries, + appDesc.command.javaOpts) CommandUtils.buildCommandSeq(command, memory, sparkHome.getAbsolutePath) } 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 ce425443051b0..80fde7e4b2624 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 @@ -34,7 +34,7 @@ import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.{DriverState, Master} import org.apache.spark.deploy.worker.ui.WorkerWebUI import org.apache.spark.metrics.MetricsSystem -import org.apache.spark.util.{AkkaUtils, SignalLogger, Utils} +import org.apache.spark.util.{ActorLogReceive, AkkaUtils, SignalLogger, Utils} /** * @param masterUrls Each url should look like spark://host:port. @@ -51,7 +51,7 @@ private[spark] class Worker( workDirPath: String = null, val conf: SparkConf, val securityMgr: SecurityManager) - extends Actor with Logging { + extends Actor with ActorLogReceive with Logging { import context.dispatcher Utils.checkHost(host, "Expected hostname") @@ -71,7 +71,7 @@ private[spark] class Worker( // TTL for app folders/data; after TTL expires it will be cleaned up val APP_DATA_RETENTION_SECS = conf.getLong("spark.worker.cleanup.appDataTtl", 7 * 24 * 3600) - + val testing: Boolean = sys.props.contains("spark.testing") val masterLock: Object = new Object() var master: ActorSelection = null var masterAddress: Address = null @@ -81,7 +81,13 @@ private[spark] class Worker( @volatile var registered = false @volatile var connected = false val workerId = generateWorkerId() - val sparkHome = new File(Option(System.getenv("SPARK_HOME")).getOrElse(".")) + val sparkHome = + if (testing) { + assert(sys.props.contains("spark.test.home"), "spark.test.home is not set!") + new File(sys.props("spark.test.home")) + } else { + new File(sys.env.get("SPARK_HOME").getOrElse(".")) + } var workDir: File = null val executors = new HashMap[String, ExecutorRunner] val finishedExecutors = new HashMap[String, ExecutorRunner] @@ -130,7 +136,7 @@ private[spark] class Worker( logInfo("Spark home: " + sparkHome) createWorkDir() context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) - webUi = new WorkerWebUI(this, workDir, Some(webUiPort)) + webUi = new WorkerWebUI(this, workDir, webUiPort) webUi.bind() registerWithMaster() @@ -181,7 +187,7 @@ private[spark] class Worker( } } - override def receive = { + override def receiveWithLogging = { case RegisteredWorker(masterUrl, masterWebUiUrl) => logInfo("Successfully registered with master " + masterUrl) registered = true @@ -233,9 +239,7 @@ private[spark] class Worker( try { logInfo("Asked to launch executor %s/%d for %s".format(appId, execId, appDesc.name)) val manager = new ExecutorRunner(appId, execId, appDesc, cores_, memory_, - self, workerId, host, - appDesc.sparkHome.map(userSparkHome => new File(userSparkHome)).getOrElse(sparkHome), - workDir, akkaUrl, conf, ExecutorState.RUNNING) + self, workerId, host, sparkHome, workDir, akkaUrl, conf, ExecutorState.RUNNING) executors(appId + "/" + execId) = manager manager.start() coresUsed += cores_ @@ -357,6 +361,7 @@ private[spark] class Worker( } override def postStop() { + metricsSystem.report() registrationRetryTimer.foreach(_.cancel()) executors.values.foreach(_.kill()) drivers.values.foreach(_.kill()) @@ -368,7 +373,8 @@ private[spark] class Worker( private[spark] object Worker extends Logging { def main(argStrings: Array[String]) { SignalLogger.register(log) - val args = new WorkerArguments(argStrings) + val conf = new SparkConf + val args = new WorkerArguments(argStrings, conf) val (actorSystem, _) = startSystemAndActor(args.host, args.port, args.webUiPort, args.cores, args.memory, args.masters, args.workDir) actorSystem.awaitTermination() diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala index dc5158102054e..1e295aaa48c30 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala @@ -20,11 +20,12 @@ package org.apache.spark.deploy.worker import java.lang.management.ManagementFactory import org.apache.spark.util.{IntParam, MemoryParam, Utils} +import org.apache.spark.SparkConf /** * Command-line parser for the worker. */ -private[spark] class WorkerArguments(args: Array[String]) { +private[spark] class WorkerArguments(args: Array[String], conf: SparkConf) { var host = Utils.localHostName() var port = 0 var webUiPort = 8081 @@ -46,6 +47,9 @@ private[spark] class WorkerArguments(args: Array[String]) { if (System.getenv("SPARK_WORKER_WEBUI_PORT") != null) { webUiPort = System.getenv("SPARK_WORKER_WEBUI_PORT").toInt } + if (conf.contains("spark.worker.ui.port")) { + webUiPort = conf.get("spark.worker.ui.port").toInt + } if (System.getenv("SPARK_WORKER_DIR") != null) { workDir = System.getenv("SPARK_WORKER_DIR") } diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerSource.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerSource.scala index b7ddd8c816cbc..df1e01b23b932 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerSource.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerSource.scala @@ -22,8 +22,8 @@ import com.codahale.metrics.{Gauge, MetricRegistry} import org.apache.spark.metrics.source.Source private[spark] class WorkerSource(val worker: Worker) extends Source { - val sourceName = "worker" - val metricRegistry = new MetricRegistry() + override val sourceName = "worker" + override val metricRegistry = new MetricRegistry() metricRegistry.register(MetricRegistry.name("executors"), new Gauge[Int] { override def getValue: Int = worker.executors.size diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala index 530c147000904..6d0d0bbe5ecec 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala @@ -22,13 +22,15 @@ import akka.remote.{AssociatedEvent, AssociationErrorEvent, AssociationEvent, Di import org.apache.spark.Logging import org.apache.spark.deploy.DeployMessages.SendHeartbeat +import org.apache.spark.util.ActorLogReceive /** * Actor which connects to a worker process and terminates the JVM if the connection is severed. * Provides fate sharing between a worker and its associated child processes. */ -private[spark] class WorkerWatcher(workerUrl: String) extends Actor - with Logging { +private[spark] class WorkerWatcher(workerUrl: String) + extends Actor with ActorLogReceive with Logging { + override def preStart() { context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) @@ -48,7 +50,7 @@ private[spark] class WorkerWatcher(workerUrl: String) extends Actor def exitNonZero() = if (isTesting) isShutDown = true else System.exit(-1) - override def receive = { + override def receiveWithLogging = { case AssociatedEvent(localAddress, remoteAddress, inbound) if isWorker(remoteAddress) => logInfo(s"Successfully connected to $workerUrl") 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 b389cb546de6c..ecb358c399819 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 @@ -17,7 +17,6 @@ package org.apache.spark.deploy.worker.ui -import java.io.File import javax.servlet.http.HttpServletRequest import scala.xml.Node @@ -25,7 +24,7 @@ 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} +import org.apache.spark.util.logging.RollingFileAppender private[spark] class LogPage(parent: WorkerWebUI) extends WebUIPage("logPage") with Logging { private val worker = parent.worker @@ -64,11 +63,11 @@ private[spark] class LogPage(parent: WorkerWebUI) extends WebUIPage("logPage") w val offset = Option(request.getParameter("offset")).map(_.toLong) val byteLength = Option(request.getParameter("byteLength")).map(_.toInt).getOrElse(defaultBytes) - val (logDir, params) = (appId, executorId, driverId) match { + val (logDir, params, pageName) = (appId, executorId, driverId) match { case (Some(a), Some(e), None) => - (s"${workDir.getPath}/$a/$e/", s"appId=$a&executorId=$e") + (s"${workDir.getPath}/$a/$e/", s"appId=$a&executorId=$e", s"$a/$e") case (None, None, Some(d)) => - (s"${workDir.getPath}/$d/", s"driverId=$d") + (s"${workDir.getPath}/$d/", s"driverId=$d", d) case _ => throw new Exception("Request must specify either application or driver identifiers") } @@ -120,7 +119,7 @@ private[spark] class LogPage(parent: WorkerWebUI) extends WebUIPage("logPage") w
- UIUtils.basicSparkPage(content, logType + " log page for " + appId.getOrElse("unknown app")) + UIUtils.basicSparkPage(content, logType + " log page for " + pageName) } /** Get the part of the log files given the offset and desired length of bytes */ diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala index 0ad2edba2227f..b07942a9ca729 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala @@ -22,6 +22,7 @@ import javax.servlet.http.HttpServletRequest import org.apache.spark.{Logging, SparkConf} import org.apache.spark.deploy.worker.Worker +import org.apache.spark.deploy.worker.ui.WorkerWebUI._ import org.apache.spark.ui.{SparkUI, WebUI} import org.apache.spark.ui.JettyUtils._ import org.apache.spark.util.AkkaUtils @@ -33,8 +34,8 @@ private[spark] class WorkerWebUI( val worker: Worker, val workDir: File, - port: Option[Int] = None) - extends WebUI(worker.securityMgr, WorkerWebUI.getUIPort(port, worker.conf), worker.conf) + requestedPort: Int) + extends WebUI(worker.securityMgr, requestedPort, worker.conf, name = "WorkerUI") with Logging { val timeout = AkkaUtils.askTimeout(worker.conf) @@ -54,10 +55,5 @@ class WorkerWebUI( } private[spark] object WorkerWebUI { - val DEFAULT_PORT = 8081 val STATIC_RESOURCE_BASE = SparkUI.STATIC_RESOURCE_DIR - - def getUIPort(requestedPort: Option[Int], conf: SparkConf): Int = { - requestedPort.getOrElse(conf.getInt("worker.ui.port", WorkerWebUI.DEFAULT_PORT)) - } } 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 8d31bd05fdbec..13af5b6f5812d 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -31,14 +31,15 @@ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.worker.WorkerWatcher import org.apache.spark.scheduler.TaskDescription import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ -import org.apache.spark.util.{AkkaUtils, SignalLogger, Utils} +import org.apache.spark.util.{ActorLogReceive, AkkaUtils, SignalLogger, Utils} private[spark] class CoarseGrainedExecutorBackend( driverUrl: String, executorId: String, hostPort: String, cores: Int, - sparkProperties: Seq[(String, String)]) extends Actor with ExecutorBackend with Logging { + sparkProperties: Seq[(String, String)]) + extends Actor with ActorLogReceive with ExecutorBackend with Logging { Utils.checkHostPort(hostPort, "Expected hostport") @@ -52,7 +53,7 @@ private[spark] class CoarseGrainedExecutorBackend( context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) } - override def receive = { + override def receiveWithLogging = { case RegisteredExecutor => logInfo("Successfully registered with driver") // Make this host instead of hostPort ? @@ -71,7 +72,7 @@ private[spark] class CoarseGrainedExecutorBackend( 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) + executor.launchTask(this, taskDesc.taskId, taskDesc.name, taskDesc.serializedTask) } case KillTask(taskId, _, interruptThread) => @@ -88,6 +89,7 @@ private[spark] class CoarseGrainedExecutorBackend( case StopExecutor => logInfo("Driver commanded a shutdown") + executor.stop() context.stop(self) context.system.shutdown() } @@ -98,8 +100,13 @@ private[spark] class CoarseGrainedExecutorBackend( } private[spark] object CoarseGrainedExecutorBackend extends Logging { - def run(driverUrl: String, executorId: String, hostname: String, cores: Int, - workerUrl: Option[String]) { + + private def run( + driverUrl: String, + executorId: String, + hostname: String, + cores: Int, + workerUrl: Option[String]) { SignalLogger.register(log) @@ -109,8 +116,9 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { // Bootstrap to fetch the driver's Spark properties. val executorConf = new SparkConf + val port = executorConf.getInt("spark.executor.port", 0) val (fetcher, _) = AkkaUtils.createActorSystem( - "driverPropsFetcher", hostname, 0, executorConf, new SecurityManager(executorConf)) + "driverPropsFetcher", hostname, port, executorConf, new SecurityManager(executorConf)) val driver = fetcher.actorSelection(driverUrl) val timeout = AkkaUtils.askTimeout(executorConf) val fut = Patterns.ask(driver, RetrieveSparkProps, timeout) @@ -120,7 +128,7 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { // Create a new ActorSystem using driver's Spark properties to run the backend. val driverConf = new SparkConf().setAll(props) val (actorSystem, boundPort) = AkkaUtils.createActorSystem( - "sparkExecutor", hostname, 0, driverConf, new SecurityManager(driverConf)) + "sparkExecutor", hostname, port, driverConf, new SecurityManager(driverConf)) // set it val sparkHostPort = hostname + ":" + boundPort actorSystem.actorOf( diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 4d3ba11633bf5..fb3f7bd54bbfa 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -23,7 +23,7 @@ import java.nio.ByteBuffer import java.util.concurrent._ import scala.collection.JavaConversions._ -import scala.collection.mutable.HashMap +import scala.collection.mutable.{ArrayBuffer, HashMap} import org.apache.spark._ import org.apache.spark.scheduler._ @@ -48,6 +48,8 @@ private[spark] class Executor( private val EMPTY_BYTE_BUFFER = ByteBuffer.wrap(new Array[Byte](0)) + @volatile private var isStopped = false + // No ip or host:port - just hostname Utils.checkHost(slaveHostname, "Expected executed slave to be a hostname") // must not have port specified. @@ -97,6 +99,9 @@ private[spark] class Executor( private val urlClassLoader = createClassLoader() private val replClassLoader = addReplClassLoaderIfNeeded(urlClassLoader) + // Set the classloader for serializer + env.serializer.setDefaultClassLoader(urlClassLoader) + // Akka's message frame size. If task result is bigger than this, we use the block manager // to send the result back. private val akkaFrameSize = AkkaUtils.maxFrameSizeBytes(conf) @@ -107,8 +112,11 @@ private[spark] class Executor( // Maintains the list of running tasks. private val runningTasks = new ConcurrentHashMap[Long, TaskRunner] - def launchTask(context: ExecutorBackend, taskId: Long, serializedTask: ByteBuffer) { - val tr = new TaskRunner(context, taskId, serializedTask) + startDriverHeartbeater() + + def launchTask( + context: ExecutorBackend, taskId: Long, taskName: String, serializedTask: ByteBuffer) { + val tr = new TaskRunner(context, taskId, taskName, serializedTask) runningTasks.put(taskId, tr) threadPool.execute(tr) } @@ -120,6 +128,12 @@ private[spark] class Executor( } } + def stop() { + env.metricsSystem.report() + isStopped = true + threadPool.shutdown() + } + /** Get the Yarn approved local directories. */ private def getYarnLocalDirs(): String = { // Hadoop 0.23 and 2.x have different Environment variable names for the @@ -135,14 +149,16 @@ private[spark] class Executor( localDirs } - class TaskRunner(execBackend: ExecutorBackend, taskId: Long, serializedTask: ByteBuffer) + class TaskRunner( + execBackend: ExecutorBackend, val taskId: Long, taskName: String, serializedTask: ByteBuffer) extends Runnable { @volatile private var killed = false - @volatile private var task: Task[Any] = _ + @volatile var task: Task[Any] = _ + @volatile var attemptedTask: Option[Task[Any]] = None def kill(interruptThread: Boolean) { - logInfo("Executor is trying to kill task " + taskId) + logInfo(s"Executor is trying to kill $taskName (TID $taskId)") killed = true if (task != null) { task.kill(interruptThread) @@ -154,9 +170,8 @@ private[spark] class Executor( SparkEnv.set(env) Thread.currentThread.setContextClassLoader(replClassLoader) val ser = SparkEnv.get.closureSerializer.newInstance() - logInfo("Running task ID " + taskId) + logInfo(s"Running $taskName (TID $taskId)") execBackend.statusUpdate(taskId, TaskState.RUNNING, EMPTY_BYTE_BUFFER) - var attemptedTask: Option[Task[Any]] = None var taskStart: Long = 0 def gcTime = ManagementFactory.getGarbageCollectorMXBeans.map(_.getCollectionTime).sum val startGCTime = gcTime @@ -198,7 +213,6 @@ private[spark] class Executor( val afterSerialization = System.currentTimeMillis() for (m <- task.metrics) { - m.hostname = Utils.localHostName() m.executorDeserializeTime = taskStart - startTime m.executorRunTime = taskFinish - taskStart m.jvmGCTime = gcTime - startGCTime @@ -207,25 +221,30 @@ private[spark] class Executor( val accumUpdates = Accumulators.values - val directResult = new DirectTaskResult(valueBytes, accumUpdates, - task.metrics.getOrElse(null)) + val directResult = new DirectTaskResult(valueBytes, accumUpdates, task.metrics.orNull) val serializedDirectResult = ser.serialize(directResult) - logInfo("Serialized size of result for " + taskId + " is " + serializedDirectResult.limit) - val serializedResult = { - if (serializedDirectResult.limit >= akkaFrameSize - AkkaUtils.reservedSizeBytes) { - logInfo("Storing result for " + taskId + " in local BlockManager") + val resultSize = serializedDirectResult.limit + + // directSend = sending directly back to the driver + val (serializedResult, directSend) = { + if (resultSize >= akkaFrameSize - AkkaUtils.reservedSizeBytes) { val blockId = TaskResultBlockId(taskId) env.blockManager.putBytes( blockId, serializedDirectResult, StorageLevel.MEMORY_AND_DISK_SER) - ser.serialize(new IndirectTaskResult[Any](blockId)) + (ser.serialize(new IndirectTaskResult[Any](blockId)), false) } else { - logInfo("Sending result for " + taskId + " directly to driver") - serializedDirectResult + (serializedDirectResult, true) } } execBackend.statusUpdate(taskId, TaskState.FINISHED, serializedResult) - logInfo("Finished task ID " + taskId) + + if (directSend) { + logInfo(s"Finished $taskName (TID $taskId). $resultSize bytes result sent to driver") + } else { + logInfo( + s"Finished $taskName (TID $taskId). $resultSize bytes result sent via BlockManager)") + } } catch { case ffe: FetchFailedException => { val reason = ffe.toTaskEndReason @@ -233,7 +252,7 @@ private[spark] class Executor( } case _: TaskKilledException | _: InterruptedException if task.killed => { - logInfo("Executor killed task " + taskId) + logInfo(s"Executor killed $taskName (TID $taskId)") execBackend.statusUpdate(taskId, TaskState.KILLED, ser.serialize(TaskKilled)) } @@ -241,7 +260,7 @@ private[spark] class Executor( // Attempt to exit cleanly by informing the driver of our failure. // If anything goes wrong (or this was a fatal exception), we will delegate to // the default uncaught exception handler, which will terminate the Executor. - logError("Exception in task ID " + taskId, t) + logError(s"Exception in $taskName (TID $taskId)", t) val serviceTime = System.currentTimeMillis() - taskStart val metrics = attemptedTask.flatMap(t => t.metrics) @@ -249,7 +268,7 @@ private[spark] class Executor( m.executorRunTime = serviceTime m.jvmGCTime = gcTime - startGCTime } - val reason = ExceptionFailure(t.getClass.getName, t.toString, t.getStackTrace, metrics) + val reason = ExceptionFailure(t.getClass.getName, t.getMessage, t.getStackTrace, metrics) execBackend.statusUpdate(taskId, TaskState.FAILED, ser.serialize(reason)) // Don't forcibly exit unless the exception was inherently fatal, to avoid @@ -259,11 +278,10 @@ private[spark] class Executor( } } } finally { - // TODO: Unregister shuffle memory only for ResultTask - val shuffleMemoryMap = env.shuffleMemoryMap - shuffleMemoryMap.synchronized { - shuffleMemoryMap.remove(Thread.currentThread().getId) - } + // Release memory used by this thread for shuffles + env.shuffleMemoryManager.releaseMemoryForThisThread() + // Release memory used by this thread for unrolling blocks + env.blockManager.memoryStore.releaseUnrollMemoryForThisThread() runningTasks.remove(taskId) } } @@ -341,4 +359,43 @@ private[spark] class Executor( } } } + + def startDriverHeartbeater() { + val interval = conf.getInt("spark.executor.heartbeatInterval", 10000) + val timeout = AkkaUtils.lookupTimeout(conf) + val retryAttempts = AkkaUtils.numRetries(conf) + val retryIntervalMs = AkkaUtils.retryWaitMs(conf) + val heartbeatReceiverRef = AkkaUtils.makeDriverRef("HeartbeatReceiver", conf, env.actorSystem) + + val t = new Thread() { + override def run() { + // Sleep a random interval so the heartbeats don't end up in sync + Thread.sleep(interval + (math.random * interval).asInstanceOf[Int]) + + while (!isStopped) { + val tasksMetrics = new ArrayBuffer[(Long, TaskMetrics)]() + for (taskRunner <- runningTasks.values()) { + if (!taskRunner.attemptedTask.isEmpty) { + Option(taskRunner.task).flatMap(_.metrics).foreach { metrics => + metrics.updateShuffleReadMetrics + tasksMetrics += ((taskRunner.taskId, metrics)) + } + } + } + + val message = Heartbeat(executorId, tasksMetrics.toArray, env.blockManager.blockManagerId) + val response = AkkaUtils.askWithReply[HeartbeatResponse](message, heartbeatReceiverRef, + retryAttempts, retryIntervalMs, timeout) + if (response.reregisterBlockManager) { + logWarning("Told to re-register on heartbeat") + env.blockManager.reregister() + } + Thread.sleep(interval) + } + } + } + t.setDaemon(true) + t.setName("Driver Heartbeater") + t.start() + } } diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala index 0ed52cfe9df61..d6721586566c2 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala @@ -35,9 +35,10 @@ private[spark] class ExecutorSource(val executor: Executor, executorId: String) }) } - val metricRegistry = new MetricRegistry() + override val metricRegistry = new MetricRegistry() + // TODO: It would be nice to pass the application name here - val sourceName = "executor.%s".format(executorId) + override val sourceName = "executor.%s".format(executorId) // Gauge for executor thread pool's actively executing task counts metricRegistry.register(MetricRegistry.name("threadpool", "activeTasks"), new Gauge[Int] { diff --git a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala index 2232e6237bf26..a42c8b43bbf7f 100644 --- a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala @@ -64,7 +64,7 @@ private[spark] class MesosExecutorBackend if (executor == null) { logError("Received launchTask but executor was null") } else { - executor.launchTask(this, taskId, taskInfo.getData.asReadOnlyByteBuffer) + executor.launchTask(this, taskId, taskInfo.getName, taskInfo.getData.asReadOnlyByteBuffer) } } diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index ac73288442a74..99a88c13456df 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -17,12 +17,22 @@ package org.apache.spark.executor +import scala.collection.mutable.ArrayBuffer + import org.apache.spark.annotation.DeveloperApi import org.apache.spark.storage.{BlockId, BlockStatus} /** * :: DeveloperApi :: * Metrics tracked during the execution of a task. + * + * This class is used to house metrics both for in-progress and completed tasks. In executors, + * both the task thread and the heartbeat thread write to the TaskMetrics. The heartbeat thread + * reads it to send in-progress metrics, and the task thread reads it to send metrics along with + * the completed task. + * + * So, when adding new fields, take into consideration that the whole object can be serialized for + * shipping off at any time to consumers of the SparkListener interface. */ @DeveloperApi class TaskMetrics extends Serializable { @@ -73,9 +83,26 @@ class TaskMetrics extends Serializable { var inputMetrics: Option[InputMetrics] = None /** - * If this task reads from shuffle output, metrics on getting shuffle data will be collected here + * If this task reads from shuffle output, metrics on getting shuffle data will be collected here. + * This includes read metrics aggregated over all the task's shuffle dependencies. + */ + private var _shuffleReadMetrics: Option[ShuffleReadMetrics] = None + + def shuffleReadMetrics = _shuffleReadMetrics + + /** + * This should only be used when recreating TaskMetrics, not when updating read metrics in + * executors. + */ + private[spark] def setShuffleReadMetrics(shuffleReadMetrics: Option[ShuffleReadMetrics]) { + _shuffleReadMetrics = shuffleReadMetrics + } + + /** + * ShuffleReadMetrics per dependency for collecting independently while task is in progress. */ - var shuffleReadMetrics: Option[ShuffleReadMetrics] = None + @transient private lazy val depsShuffleReadMetrics: ArrayBuffer[ShuffleReadMetrics] = + new ArrayBuffer[ShuffleReadMetrics]() /** * If this task writes to shuffle output, metrics on the written shuffle data will be collected @@ -87,6 +114,33 @@ class TaskMetrics extends Serializable { * Storage statuses of any blocks that have been updated as a result of this task. */ var updatedBlocks: Option[Seq[(BlockId, BlockStatus)]] = None + + /** + * A task may have multiple shuffle readers for multiple dependencies. To avoid synchronization + * issues from readers in different threads, in-progress tasks use a ShuffleReadMetrics for each + * dependency, and merge these metrics before reporting them to the driver. This method returns + * a ShuffleReadMetrics for a dependency and registers it for merging later. + */ + private [spark] def createShuffleReadMetricsForDependency(): ShuffleReadMetrics = synchronized { + val readMetrics = new ShuffleReadMetrics() + depsShuffleReadMetrics += readMetrics + readMetrics + } + + /** + * Aggregates shuffle read metrics for all registered dependencies into shuffleReadMetrics. + */ + private[spark] def updateShuffleReadMetrics() = synchronized { + val merged = new ShuffleReadMetrics() + for (depMetrics <- depsShuffleReadMetrics) { + merged.fetchWaitTime += depMetrics.fetchWaitTime + merged.localBlocksFetched += depMetrics.localBlocksFetched + merged.remoteBlocksFetched += depMetrics.remoteBlocksFetched + merged.remoteBytesRead += depMetrics.remoteBytesRead + merged.shuffleFinishTime = math.max(merged.shuffleFinishTime, depMetrics.shuffleFinishTime) + } + _shuffleReadMetrics = Some(merged) + } } private[spark] object TaskMetrics { @@ -126,12 +180,12 @@ class ShuffleReadMetrics extends Serializable { /** * Absolute time when this task finished reading shuffle data */ - var shuffleFinishTime: Long = _ + var shuffleFinishTime: Long = -1 /** * Number of blocks fetched in this shuffle by this task (remote or local) */ - var totalBlocksFetched: Int = _ + def totalBlocksFetched: Int = remoteBlocksFetched + localBlocksFetched /** * Number of remote blocks fetched in this shuffle by this task @@ -165,10 +219,10 @@ class ShuffleWriteMetrics extends Serializable { /** * Number of bytes written for the shuffle by this task */ - var shuffleBytesWritten: Long = _ + @volatile var shuffleBytesWritten: Long = _ /** * Time the task spent blocking on writes to disk or buffer cache, in nanoseconds */ - var shuffleWriteTime: Long = _ + @volatile var shuffleWriteTime: Long = _ } diff --git a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala index 4b0fe1ab82999..ef9c43ecf14f6 100644 --- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala +++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala @@ -20,6 +20,7 @@ package org.apache.spark.io import java.io.{InputStream, OutputStream} import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream} +import net.jpountz.lz4.{LZ4BlockInputStream, LZ4BlockOutputStream} import org.xerial.snappy.{SnappyInputStream, SnappyOutputStream} import org.apache.spark.SparkConf @@ -45,17 +46,45 @@ trait CompressionCodec { private[spark] object CompressionCodec { + + private val shortCompressionCodecNames = Map( + "lz4" -> classOf[LZ4CompressionCodec].getName, + "lzf" -> classOf[LZFCompressionCodec].getName, + "snappy" -> classOf[SnappyCompressionCodec].getName) + def createCodec(conf: SparkConf): CompressionCodec = { createCodec(conf, conf.get("spark.io.compression.codec", DEFAULT_COMPRESSION_CODEC)) } def createCodec(conf: SparkConf, codecName: String): CompressionCodec = { - val ctor = Class.forName(codecName, true, Utils.getContextOrSparkClassLoader) + val codecClass = shortCompressionCodecNames.getOrElse(codecName.toLowerCase, codecName) + val ctor = Class.forName(codecClass, true, Utils.getContextOrSparkClassLoader) .getConstructor(classOf[SparkConf]) ctor.newInstance(conf).asInstanceOf[CompressionCodec] } - val DEFAULT_COMPRESSION_CODEC = classOf[LZFCompressionCodec].getName + val DEFAULT_COMPRESSION_CODEC = "snappy" +} + + +/** + * :: DeveloperApi :: + * LZ4 implementation of [[org.apache.spark.io.CompressionCodec]]. + * Block size can be configured by `spark.io.compression.lz4.block.size`. + * + * Note: The wire protocol for this codec is not guaranteed to be compatible across versions + * of Spark. This is intended for use as an internal compression utility within a single Spark + * application. + */ +@DeveloperApi +class LZ4CompressionCodec(conf: SparkConf) extends CompressionCodec { + + override def compressedOutputStream(s: OutputStream): OutputStream = { + val blockSize = conf.getInt("spark.io.compression.lz4.block.size", 32768) + new LZ4BlockOutputStream(s, blockSize) + } + + override def compressedInputStream(s: InputStream): InputStream = new LZ4BlockInputStream(s) } @@ -81,7 +110,7 @@ class LZFCompressionCodec(conf: SparkConf) extends CompressionCodec { /** * :: DeveloperApi :: * Snappy implementation of [[org.apache.spark.io.CompressionCodec]]. - * Block size can be configured by spark.io.compression.snappy.block.size. + * Block size can be configured by `spark.io.compression.snappy.block.size`. * * Note: The wire protocol for this codec is not guaranteed to be compatible across versions * of Spark. This is intended for use as an internal compression utility within a single Spark diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala index 651511da1b7fe..6ef817d0e587e 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala @@ -91,6 +91,10 @@ private[spark] class MetricsSystem private (val instance: String, sinks.foreach(_.stop) } + def report(): Unit = { + sinks.foreach(_.report()) + } + def registerSource(source: Source) { sources += source try { diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/ConsoleSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/ConsoleSink.scala index 05852f1f98993..81b9056b40fb8 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/ConsoleSink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/ConsoleSink.scala @@ -57,5 +57,9 @@ private[spark] class ConsoleSink(val property: Properties, val registry: MetricR override def stop() { reporter.stop() } + + override def report() { + reporter.report() + } } diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala index 542dce65366b2..9d5f2ae9328ad 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala @@ -66,5 +66,9 @@ private[spark] class CsvSink(val property: Properties, val registry: MetricRegis override def stop() { reporter.stop() } + + override def report() { + reporter.report() + } } diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala index aeb4ad44a0647..d7b5f5c40efae 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala @@ -81,4 +81,8 @@ private[spark] class GraphiteSink(val property: Properties, val registry: Metric override def stop() { reporter.stop() } + + override def report() { + reporter.report() + } } diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/JmxSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/JmxSink.scala index ed27234b4e760..2588fe2c9edb8 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/JmxSink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/JmxSink.scala @@ -35,4 +35,6 @@ private[spark] class JmxSink(val property: Properties, val registry: MetricRegis reporter.stop() } + override def report() { } + } diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala b/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala index 571539ba5e467..2f65bc8b46609 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala @@ -57,4 +57,6 @@ private[spark] class MetricsServlet(val property: Properties, val registry: Metr override def start() { } override def stop() { } + + override def report() { } } diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/Sink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/Sink.scala index 6f2b5a06027ea..0d83d8c425ca4 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/Sink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/Sink.scala @@ -20,4 +20,5 @@ package org.apache.spark.metrics.sink private[spark] trait Sink { def start: Unit def stop: Unit + def report(): Unit } diff --git a/core/src/main/scala/org/apache/spark/metrics/source/JvmSource.scala b/core/src/main/scala/org/apache/spark/metrics/source/JvmSource.scala index f865f9648a91e..635bff2cd7ec8 100644 --- a/core/src/main/scala/org/apache/spark/metrics/source/JvmSource.scala +++ b/core/src/main/scala/org/apache/spark/metrics/source/JvmSource.scala @@ -21,12 +21,9 @@ import com.codahale.metrics.MetricRegistry import com.codahale.metrics.jvm.{GarbageCollectorMetricSet, MemoryUsageGaugeSet} private[spark] class JvmSource extends Source { - val sourceName = "jvm" - val metricRegistry = new MetricRegistry() + override val sourceName = "jvm" + override val metricRegistry = new MetricRegistry() - val gcMetricSet = new GarbageCollectorMetricSet - val memGaugeSet = new MemoryUsageGaugeSet - - metricRegistry.registerAll(gcMetricSet) - metricRegistry.registerAll(memGaugeSet) + metricRegistry.registerAll(new GarbageCollectorMetricSet) + metricRegistry.registerAll(new MemoryUsageGaugeSet) } diff --git a/core/src/main/scala/org/apache/spark/network/BufferMessage.scala b/core/src/main/scala/org/apache/spark/network/BufferMessage.scala index 04df2f3b0d696..af35f1fc3e459 100644 --- a/core/src/main/scala/org/apache/spark/network/BufferMessage.scala +++ b/core/src/main/scala/org/apache/spark/network/BufferMessage.scala @@ -48,7 +48,7 @@ class BufferMessage(id_ : Int, val buffers: ArrayBuffer[ByteBuffer], var ackId: val security = if (isSecurityNeg) 1 else 0 if (size == 0 && !gotChunkForSendingOnce) { val newChunk = new MessageChunk( - new MessageChunkHeader(typ, id, 0, 0, ackId, security, senderAddress), null) + new MessageChunkHeader(typ, id, 0, 0, ackId, hasError, security, senderAddress), null) gotChunkForSendingOnce = true return Some(newChunk) } @@ -66,7 +66,8 @@ class BufferMessage(id_ : Int, val buffers: ArrayBuffer[ByteBuffer], var ackId: } buffer.position(buffer.position + newBuffer.remaining) val newChunk = new MessageChunk(new MessageChunkHeader( - typ, id, size, newBuffer.remaining, ackId, security, senderAddress), newBuffer) + typ, id, size, newBuffer.remaining, ackId, + hasError, security, senderAddress), newBuffer) gotChunkForSendingOnce = true return Some(newChunk) } @@ -88,7 +89,7 @@ class BufferMessage(id_ : Int, val buffers: ArrayBuffer[ByteBuffer], var ackId: val newBuffer = buffer.slice().limit(chunkSize).asInstanceOf[ByteBuffer] buffer.position(buffer.position + newBuffer.remaining) val newChunk = new MessageChunk(new MessageChunkHeader( - typ, id, size, newBuffer.remaining, ackId, security, senderAddress), newBuffer) + typ, id, size, newBuffer.remaining, ackId, hasError, security, senderAddress), newBuffer) return Some(newChunk) } None 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 8a1cdb812962e..e77d762bdf221 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala @@ -17,10 +17,12 @@ package org.apache.spark.network +import java.io.IOException import java.nio._ import java.nio.channels._ import java.nio.channels.spi._ import java.net._ +import java.util.{Timer, TimerTask} import java.util.concurrent.atomic.AtomicInteger import java.util.concurrent.{LinkedBlockingDeque, TimeUnit, ThreadPoolExecutor} @@ -38,37 +40,53 @@ import scala.language.postfixOps import org.apache.spark._ import org.apache.spark.util.{SystemClock, Utils} -private[spark] class ConnectionManager(port: Int, conf: SparkConf, - securityManager: SecurityManager) extends Logging { - +private[spark] class ConnectionManager( + port: Int, + conf: SparkConf, + securityManager: SecurityManager, + name: String = "Connection manager") + extends Logging { + + /** + * Used by sendMessageReliably to track messages being sent. + * @param message the message that was sent + * @param connectionManagerId the connection manager that sent this message + * @param completionHandler callback that's invoked when the send has completed or failed + */ class MessageStatus( val message: Message, val connectionManagerId: ConnectionManagerId, completionHandler: MessageStatus => Unit) { + /** This is non-None if message has been ack'd */ var ackMessage: Option[Message] = None - var attempted = false - var acked = false - def markDone() { completionHandler(this) } + def markDone(ackMessage: Option[Message]) { + this.ackMessage = ackMessage + completionHandler(this) + } } private val selector = SelectorProvider.provider.openSelector() + private val ackTimeoutMonitor = new Timer("AckTimeoutMonitor", true) // default to 30 second timeout waiting for authentication private val authTimeout = conf.getInt("spark.core.connection.auth.wait.timeout", 30) + private val ackTimeout = conf.getInt("spark.core.connection.ack.wait.timeout", 60) private val handleMessageExecutor = new ThreadPoolExecutor( conf.getInt("spark.core.connection.handler.threads.min", 20), conf.getInt("spark.core.connection.handler.threads.max", 60), conf.getInt("spark.core.connection.handler.threads.keepalive", 60), TimeUnit.SECONDS, - new LinkedBlockingDeque[Runnable]()) + new LinkedBlockingDeque[Runnable](), + Utils.namedThreadFactory("handle-message-executor")) private val handleReadWriteExecutor = new ThreadPoolExecutor( conf.getInt("spark.core.connection.io.threads.min", 4), conf.getInt("spark.core.connection.io.threads.max", 32), conf.getInt("spark.core.connection.io.threads.keepalive", 60), TimeUnit.SECONDS, - new LinkedBlockingDeque[Runnable]()) + new LinkedBlockingDeque[Runnable](), + Utils.namedThreadFactory("handle-read-write-executor")) // Use a different, yet smaller, thread pool - infrequently used with very short lived tasks : // which should be executed asap @@ -76,7 +94,8 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, conf.getInt("spark.core.connection.connect.threads.min", 1), conf.getInt("spark.core.connection.connect.threads.max", 8), conf.getInt("spark.core.connection.connect.threads.keepalive", 60), TimeUnit.SECONDS, - new LinkedBlockingDeque[Runnable]()) + new LinkedBlockingDeque[Runnable](), + Utils.namedThreadFactory("handle-connect-executor")) private val serverChannel = ServerSocketChannel.open() // used to track the SendingConnections waiting to do SASL negotiation @@ -102,7 +121,11 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, serverChannel.socket.setReuseAddress(true) serverChannel.socket.setReceiveBufferSize(256 * 1024) - serverChannel.socket.bind(new InetSocketAddress(port)) + private def startService(port: Int): (ServerSocketChannel, Int) = { + serverChannel.socket.bind(new InetSocketAddress(port)) + (serverChannel, serverChannel.socket.getLocalPort) + } + Utils.startServiceOnPort[ServerSocketChannel](port, startService, name) serverChannel.register(selector, SelectionKey.OP_ACCEPT) val id = new ConnectionManagerId(Utils.localHostName, serverChannel.socket.getLocalPort) @@ -431,11 +454,7 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, messageStatuses.values.filter(_.connectionManagerId == sendingConnectionManagerId) .foreach(status => { logInfo("Notifying " + status) - status.synchronized { - status.attempted = true - status.acked = false - status.markDone() - } + status.markDone(None) }) messageStatuses.retain((i, status) => { @@ -464,11 +483,7 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, for (s <- messageStatuses.values if s.connectionManagerId == sendingConnectionManagerId) { logInfo("Notifying " + s) - s.synchronized { - s.attempted = true - s.acked = false - s.markDone() - } + s.markDone(None) } messageStatuses.retain((i, status) => { @@ -536,13 +551,13 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, val securityMsgResp = SecurityMessage.fromResponse(replyToken, securityMsg.getConnectionId.toString) val message = securityMsgResp.toBufferMessage - if (message == null) throw new Exception("Error creating security message") + if (message == null) throw new IOException("Error creating security message") sendSecurityMessage(waitingConn.getRemoteConnectionManagerId(), message) } catch { case e: Exception => { logError("Error handling sasl client authentication", e) waitingConn.close() - throw new Exception("Error evaluating sasl response: " + e) + throw new IOException("Error evaluating sasl response: ", e) } } } @@ -638,46 +653,59 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, } } if (bufferMessage.hasAckId()) { - val sentMessageStatus = messageStatuses.synchronized { + messageStatuses.synchronized { messageStatuses.get(bufferMessage.ackId) match { case Some(status) => { messageStatuses -= bufferMessage.ackId - status + status.markDone(Some(message)) } case None => { - throw new Exception("Could not find reference for received ack message " + - message.id) + /** + * We can fall down on this code because of following 2 cases + * + * (1) Invalid ack sent due to buggy code. + * + * (2) Late-arriving ack for a SendMessageStatus + * To avoid unwilling late-arriving ack + * caused by long pause like GC, you can set + * larger value than default to spark.core.connection.ack.wait.timeout + */ + logWarning(s"Could not find reference for received ack Message ${message.id}") } } } - sentMessageStatus.synchronized { - sentMessageStatus.ackMessage = Some(message) - sentMessageStatus.attempted = true - sentMessageStatus.acked = true - sentMessageStatus.markDone() - } } else { - val ackMessage = if (onReceiveCallback != null) { - logDebug("Calling back") - onReceiveCallback(bufferMessage, connectionManagerId) - } else { - logDebug("Not calling back as callback is null") - None - } + var ackMessage : Option[Message] = None + try { + ackMessage = if (onReceiveCallback != null) { + logDebug("Calling back") + onReceiveCallback(bufferMessage, connectionManagerId) + } else { + logDebug("Not calling back as callback is null") + None + } - if (ackMessage.isDefined) { - if (!ackMessage.get.isInstanceOf[BufferMessage]) { - logDebug("Response to " + bufferMessage + " is not a buffer message, it is of type " - + ackMessage.get.getClass) - } else if (!ackMessage.get.asInstanceOf[BufferMessage].hasAckId) { - logDebug("Response to " + bufferMessage + " does not have ack id set") - ackMessage.get.asInstanceOf[BufferMessage].ackId = bufferMessage.id + if (ackMessage.isDefined) { + if (!ackMessage.get.isInstanceOf[BufferMessage]) { + logDebug("Response to " + bufferMessage + " is not a buffer message, it is of type " + + ackMessage.get.getClass) + } else if (!ackMessage.get.asInstanceOf[BufferMessage].hasAckId) { + logDebug("Response to " + bufferMessage + " does not have ack id set") + ackMessage.get.asInstanceOf[BufferMessage].ackId = bufferMessage.id + } } + } catch { + case e: Exception => { + logError(s"Exception was thrown while processing message", e) + val m = Message.createBufferMessage(bufferMessage.id) + m.hasError = true + ackMessage = Some(m) + } + } finally { + sendMessage(connectionManagerId, ackMessage.getOrElse { + Message.createBufferMessage(bufferMessage.id) + }) } - - sendMessage(connectionManagerId, ackMessage.getOrElse { - Message.createBufferMessage(bufferMessage.id) - }) } } case _ => throw new Exception("Unknown type message received") @@ -789,11 +817,7 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, case Some(msgStatus) => { messageStatuses -= message.id logInfo("Notifying " + msgStatus.connectionManagerId) - msgStatus.synchronized { - msgStatus.attempted = true - msgStatus.acked = false - msgStatus.markDone() - } + msgStatus.markDone(None) } case None => { logError("no messageStatus for failed message id: " + message.id) @@ -812,28 +836,57 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, selector.wakeup() } + /** + * Send a message and block until an acknowldgment is received or an error occurs. + * @param connectionManagerId the message's destination + * @param message the message being sent + * @return a Future that either returns the acknowledgment message or captures an exception. + */ def sendMessageReliably(connectionManagerId: ConnectionManagerId, message: Message) - : Future[Option[Message]] = { - val promise = Promise[Option[Message]] - val status = new MessageStatus( - message, connectionManagerId, s => promise.success(s.ackMessage)) + : Future[Message] = { + val promise = Promise[Message]() + + val timeoutTask = new TimerTask { + override def run(): Unit = { + messageStatuses.synchronized { + messageStatuses.remove(message.id).foreach ( s => { + promise.failure( + new IOException(s"sendMessageReliably failed because ack " + + "was not received within ${ackTimeout} sec")) + }) + } + } + } + + val status = new MessageStatus(message, connectionManagerId, s => { + timeoutTask.cancel() + s.ackMessage match { + case None => // Indicates a failure where we either never sent or never got ACK'd + promise.failure(new IOException("sendMessageReliably failed without being ACK'd")) + case Some(ackMessage) => + if (ackMessage.hasError) { + promise.failure( + new IOException("sendMessageReliably failed with ACK that signalled a remote error")) + } else { + promise.success(ackMessage) + } + } + }) messageStatuses.synchronized { messageStatuses += ((message.id, status)) } + + ackTimeoutMonitor.schedule(timeoutTask, ackTimeout * 1000) sendMessage(connectionManagerId, message) promise.future } - def sendMessageReliablySync(connectionManagerId: ConnectionManagerId, - message: Message): Option[Message] = { - Await.result(sendMessageReliably(connectionManagerId, message), Duration.Inf) - } - def onReceiveMessage(callback: (Message, ConnectionManagerId) => Option[Message]) { onReceiveCallback = callback } def stop() { + ackTimeoutMonitor.cancel() selectorThread.interrupt() selectorThread.join() selector.close() @@ -851,6 +904,7 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, private[spark] object ConnectionManager { + import ExecutionContext.Implicits.global def main(args: Array[String]) { val conf = new SparkConf @@ -885,7 +939,7 @@ private[spark] object ConnectionManager { (0 until count).map(i => { val bufferMessage = Message.createBufferMessage(buffer.duplicate) - manager.sendMessageReliablySync(manager.id, bufferMessage) + Await.result(manager.sendMessageReliably(manager.id, bufferMessage), Duration.Inf) }) println("--------------------------") println() @@ -906,8 +960,10 @@ private[spark] object ConnectionManager { val bufferMessage = Message.createBufferMessage(buffer.duplicate) manager.sendMessageReliably(manager.id, bufferMessage) }).foreach(f => { - val g = Await.result(f, 1 second) - if (!g.isDefined) println("Failed") + f.onFailure { + case e => println("Failed due to " + e) + } + Await.ready(f, 1 second) }) val finishTime = System.currentTimeMillis @@ -941,8 +997,10 @@ private[spark] object ConnectionManager { val bufferMessage = Message.createBufferMessage(buffers(count - 1 - i).duplicate) manager.sendMessageReliably(manager.id, bufferMessage) }).foreach(f => { - val g = Await.result(f, 1 second) - if (!g.isDefined) println("Failed") + f.onFailure { + case e => println("Failed due to " + e) + } + Await.ready(f, 1 second) }) val finishTime = System.currentTimeMillis @@ -971,8 +1029,10 @@ private[spark] object ConnectionManager { val bufferMessage = Message.createBufferMessage(buffer.duplicate) manager.sendMessageReliably(manager.id, bufferMessage) }).foreach(f => { - val g = Await.result(f, 1 second) - if (!g.isDefined) println("Failed") + f.onFailure { + case e => println("Failed due to " + e) + } + Await.ready(f, 1 second) }) val finishTime = System.currentTimeMillis Thread.sleep(1000) diff --git a/core/src/main/scala/org/apache/spark/network/Message.scala b/core/src/main/scala/org/apache/spark/network/Message.scala index 7caccfdbb44f9..04ea50f62918c 100644 --- a/core/src/main/scala/org/apache/spark/network/Message.scala +++ b/core/src/main/scala/org/apache/spark/network/Message.scala @@ -28,6 +28,7 @@ private[spark] abstract class Message(val typ: Long, val id: Int) { var startTime = -1L var finishTime = -1L var isSecurityNeg = false + var hasError = false def size: Int @@ -87,6 +88,7 @@ private[spark] object Message { case BUFFER_MESSAGE => new BufferMessage(header.id, ArrayBuffer(ByteBuffer.allocate(header.totalSize)), header.other) } + newMessage.hasError = header.hasError newMessage.senderAddress = header.address newMessage } diff --git a/core/src/main/scala/org/apache/spark/network/MessageChunkHeader.scala b/core/src/main/scala/org/apache/spark/network/MessageChunkHeader.scala index ead663ede7a1c..f3ecca5f992e0 100644 --- a/core/src/main/scala/org/apache/spark/network/MessageChunkHeader.scala +++ b/core/src/main/scala/org/apache/spark/network/MessageChunkHeader.scala @@ -27,6 +27,7 @@ private[spark] class MessageChunkHeader( val totalSize: Int, val chunkSize: Int, val other: Int, + val hasError: Boolean, val securityNeg: Int, val address: InetSocketAddress) { lazy val buffer = { @@ -41,6 +42,7 @@ private[spark] class MessageChunkHeader( putInt(totalSize). putInt(chunkSize). putInt(other). + put(if (hasError) 1.asInstanceOf[Byte] else 0.asInstanceOf[Byte]). putInt(securityNeg). putInt(ip.size). put(ip). @@ -56,7 +58,7 @@ private[spark] class MessageChunkHeader( private[spark] object MessageChunkHeader { - val HEADER_SIZE = 44 + val HEADER_SIZE = 45 def create(buffer: ByteBuffer): MessageChunkHeader = { if (buffer.remaining != HEADER_SIZE) { @@ -67,13 +69,14 @@ private[spark] object MessageChunkHeader { val totalSize = buffer.getInt() val chunkSize = buffer.getInt() val other = buffer.getInt() + val hasError = buffer.get() != 0 val securityNeg = buffer.getInt() val ipSize = buffer.getInt() val ipBytes = new Array[Byte](ipSize) buffer.get(ipBytes) val ip = InetAddress.getByAddress(ipBytes) val port = buffer.getInt() - new MessageChunkHeader(typ, id, totalSize, chunkSize, other, securityNeg, + new MessageChunkHeader(typ, id, totalSize, chunkSize, other, hasError, securityNeg, new InetSocketAddress(ip, port)) } } diff --git a/core/src/main/scala/org/apache/spark/network/SenderTest.scala b/core/src/main/scala/org/apache/spark/network/SenderTest.scala index b8ea7c2cff9a2..ea2ad104ecae1 100644 --- a/core/src/main/scala/org/apache/spark/network/SenderTest.scala +++ b/core/src/main/scala/org/apache/spark/network/SenderTest.scala @@ -20,6 +20,10 @@ package org.apache.spark.network import java.nio.ByteBuffer import org.apache.spark.{SecurityManager, SparkConf} +import scala.concurrent.Await +import scala.concurrent.duration.Duration +import scala.util.Try + private[spark] object SenderTest { def main(args: Array[String]) { @@ -51,7 +55,8 @@ private[spark] object SenderTest { val dataMessage = Message.createBufferMessage(buffer.duplicate) val startTime = System.currentTimeMillis /* println("Started timer at " + startTime) */ - val responseStr = manager.sendMessageReliablySync(targetConnectionManagerId, dataMessage) + val promise = manager.sendMessageReliably(targetConnectionManagerId, dataMessage) + val responseStr: String = Try(Await.result(promise, Duration.Inf)) .map { response => val buffer = response.asInstanceOf[BufferMessage].buffers(0) new String(buffer.array, "utf-8") diff --git a/core/src/main/scala/org/apache/spark/network/netty/FileClient.scala b/core/src/main/scala/org/apache/spark/network/netty/FileClient.scala new file mode 100644 index 0000000000000..c6d35f73db545 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/netty/FileClient.scala @@ -0,0 +1,85 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.network.netty + +import java.util.concurrent.TimeUnit + +import io.netty.bootstrap.Bootstrap +import io.netty.channel.{Channel, ChannelOption, EventLoopGroup} +import io.netty.channel.oio.OioEventLoopGroup +import io.netty.channel.socket.oio.OioSocketChannel + +import org.apache.spark.Logging + +class FileClient(handler: FileClientHandler, connectTimeout: Int) extends Logging { + + private var channel: Channel = _ + private var bootstrap: Bootstrap = _ + private var group: EventLoopGroup = _ + private val sendTimeout = 60 + + def init(): Unit = { + group = new OioEventLoopGroup + bootstrap = new Bootstrap + bootstrap.group(group) + .channel(classOf[OioSocketChannel]) + .option(ChannelOption.SO_KEEPALIVE, java.lang.Boolean.TRUE) + .option(ChannelOption.TCP_NODELAY, java.lang.Boolean.TRUE) + .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, Integer.valueOf(connectTimeout)) + .handler(new FileClientChannelInitializer(handler)) + } + + def connect(host: String, port: Int) { + try { + channel = bootstrap.connect(host, port).sync().channel() + } catch { + case e: InterruptedException => + logWarning("FileClient interrupted while trying to connect", e) + close() + } + } + + def waitForClose(): Unit = { + try { + channel.closeFuture.sync() + } catch { + case e: InterruptedException => + logWarning("FileClient interrupted", e) + } + } + + def sendRequest(file: String): Unit = { + try { + val bSent = channel.writeAndFlush(file + "\r\n").await(sendTimeout, TimeUnit.SECONDS) + if (!bSent) { + throw new RuntimeException("Failed to send") + } + } catch { + case e: InterruptedException => + logError("Error", e) + } + } + + def close(): Unit = { + if (group != null) { + group.shutdownGracefully() + group = null + bootstrap = null + } + } +} diff --git a/core/src/main/scala/org/apache/spark/network/netty/FileClientChannelInitializer.scala b/core/src/main/scala/org/apache/spark/network/netty/FileClientChannelInitializer.scala new file mode 100644 index 0000000000000..f4261c13f70a8 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/netty/FileClientChannelInitializer.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.network.netty + +import io.netty.channel.ChannelInitializer +import io.netty.channel.socket.SocketChannel +import io.netty.handler.codec.string.StringEncoder + + +class FileClientChannelInitializer(handler: FileClientHandler) + extends ChannelInitializer[SocketChannel] { + + def initChannel(channel: SocketChannel) { + channel.pipeline.addLast("encoder", new StringEncoder).addLast("handler", handler) + } +} diff --git a/core/src/main/java/org/apache/spark/network/netty/FileClientHandler.java b/core/src/main/scala/org/apache/spark/network/netty/FileClientHandler.scala similarity index 51% rename from core/src/main/java/org/apache/spark/network/netty/FileClientHandler.java rename to core/src/main/scala/org/apache/spark/network/netty/FileClientHandler.scala index 63d3d927255f9..017302ec7d33d 100644 --- a/core/src/main/java/org/apache/spark/network/netty/FileClientHandler.java +++ b/core/src/main/scala/org/apache/spark/network/netty/FileClientHandler.scala @@ -15,41 +15,36 @@ * limitations under the License. */ -package org.apache.spark.network.netty; +package org.apache.spark.network.netty -import io.netty.buffer.ByteBuf; -import io.netty.channel.ChannelHandlerContext; -import io.netty.channel.SimpleChannelInboundHandler; +import io.netty.buffer.ByteBuf +import io.netty.channel.{ChannelHandlerContext, SimpleChannelInboundHandler} -import org.apache.spark.storage.BlockId; +import org.apache.spark.storage.BlockId -abstract class FileClientHandler extends SimpleChannelInboundHandler { - private FileHeader currentHeader = null; +abstract class FileClientHandler extends SimpleChannelInboundHandler[ByteBuf] { - private volatile boolean handlerCalled = false; + private var currentHeader: FileHeader = null - public boolean isComplete() { - return handlerCalled; - } + @volatile + private var handlerCalled: Boolean = false + + def isComplete: Boolean = handlerCalled + + def handle(ctx: ChannelHandlerContext, in: ByteBuf, header: FileHeader) - public abstract void handle(ChannelHandlerContext ctx, ByteBuf in, FileHeader header); - public abstract void handleError(BlockId blockId); + def handleError(blockId: BlockId) - @Override - public void channelRead0(ChannelHandlerContext ctx, ByteBuf in) { - // get header - if (currentHeader == null && in.readableBytes() >= FileHeader.HEADER_SIZE()) { - currentHeader = FileHeader.create(in.readBytes(FileHeader.HEADER_SIZE())); + override def channelRead0(ctx: ChannelHandlerContext, in: ByteBuf) { + if (currentHeader == null && in.readableBytes >= FileHeader.HEADER_SIZE) { + currentHeader = FileHeader.create(in.readBytes(FileHeader.HEADER_SIZE)) } - // get file - if(in.readableBytes() >= currentHeader.fileLen()) { - handle(ctx, in, currentHeader); - handlerCalled = true; - currentHeader = null; - ctx.close(); + if (in.readableBytes >= currentHeader.fileLen) { + handle(ctx, in, currentHeader) + handlerCalled = true + currentHeader = null + ctx.close() } } - } - diff --git a/core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala b/core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala index 136c1912045aa..607e560ff277f 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala @@ -26,7 +26,7 @@ private[spark] class FileHeader ( val fileLen: Int, val blockId: BlockId) extends Logging { - lazy val buffer = { + lazy val buffer: ByteBuf = { val buf = Unpooled.buffer() buf.capacity(FileHeader.HEADER_SIZE) buf.writeInt(fileLen) @@ -62,11 +62,10 @@ private[spark] object FileHeader { new FileHeader(length, blockId) } - def main (args:Array[String]) { + def main(args:Array[String]) { val header = new FileHeader(25, TestBlockId("my_block")) val buf = header.buffer val newHeader = FileHeader.create(buf) System.out.println("id=" + newHeader.blockId + ",size=" + newHeader.fileLen) } } - diff --git a/core/src/main/scala/org/apache/spark/network/netty/FileServer.scala b/core/src/main/scala/org/apache/spark/network/netty/FileServer.scala new file mode 100644 index 0000000000000..dff77950659af --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/netty/FileServer.scala @@ -0,0 +1,91 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.network.netty + +import java.net.InetSocketAddress + +import io.netty.bootstrap.ServerBootstrap +import io.netty.channel.{ChannelFuture, ChannelOption, EventLoopGroup} +import io.netty.channel.oio.OioEventLoopGroup +import io.netty.channel.socket.oio.OioServerSocketChannel + +import org.apache.spark.Logging + +/** + * Server that accept the path of a file an echo back its content. + */ +class FileServer(pResolver: PathResolver, private var port: Int) extends Logging { + + private val addr: InetSocketAddress = new InetSocketAddress(port) + private var bossGroup: EventLoopGroup = new OioEventLoopGroup + private var workerGroup: EventLoopGroup = new OioEventLoopGroup + + private var channelFuture: ChannelFuture = { + val bootstrap = new ServerBootstrap + bootstrap.group(bossGroup, workerGroup) + .channel(classOf[OioServerSocketChannel]) + .option(ChannelOption.SO_BACKLOG, java.lang.Integer.valueOf(100)) + .option(ChannelOption.SO_RCVBUF, java.lang.Integer.valueOf(1500)) + .childHandler(new FileServerChannelInitializer(pResolver)) + bootstrap.bind(addr) + } + + try { + val boundAddress = channelFuture.sync.channel.localAddress.asInstanceOf[InetSocketAddress] + port = boundAddress.getPort + } catch { + case ie: InterruptedException => + port = 0 + } + + /** Start the file server asynchronously in a new thread. */ + def start(): Unit = { + val blockingThread: Thread = new Thread { + override def run(): Unit = { + try { + channelFuture.channel.closeFuture.sync + logInfo("FileServer exiting") + } catch { + case e: InterruptedException => + logError("File server start got interrupted", e) + } + // NOTE: bootstrap is shutdown in stop() + } + } + blockingThread.setDaemon(true) + blockingThread.start() + } + + def getPort: Int = port + + def stop(): Unit = { + if (channelFuture != null) { + channelFuture.channel().close().awaitUninterruptibly() + channelFuture = null + } + if (bossGroup != null) { + bossGroup.shutdownGracefully() + bossGroup = null + } + if (workerGroup != null) { + workerGroup.shutdownGracefully() + workerGroup = null + } + } +} + diff --git a/core/src/main/scala/org/apache/spark/network/netty/FileServerChannelInitializer.scala b/core/src/main/scala/org/apache/spark/network/netty/FileServerChannelInitializer.scala new file mode 100644 index 0000000000000..aaa2f913d0269 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/netty/FileServerChannelInitializer.scala @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.network.netty + +import io.netty.channel.ChannelInitializer +import io.netty.channel.socket.SocketChannel +import io.netty.handler.codec.{DelimiterBasedFrameDecoder, Delimiters} +import io.netty.handler.codec.string.StringDecoder + +class FileServerChannelInitializer(pResolver: PathResolver) + extends ChannelInitializer[SocketChannel] { + + override def initChannel(channel: SocketChannel): Unit = { + channel.pipeline + .addLast("framer", new DelimiterBasedFrameDecoder(8192, Delimiters.lineDelimiter : _*)) + .addLast("stringDecoder", new StringDecoder) + .addLast("handler", new FileServerHandler(pResolver)) + } +} diff --git a/core/src/main/scala/org/apache/spark/network/netty/FileServerHandler.scala b/core/src/main/scala/org/apache/spark/network/netty/FileServerHandler.scala new file mode 100644 index 0000000000000..96f60b2883ad9 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/netty/FileServerHandler.scala @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.network.netty + +import java.io.FileInputStream + +import io.netty.channel.{DefaultFileRegion, ChannelHandlerContext, SimpleChannelInboundHandler} + +import org.apache.spark.Logging +import org.apache.spark.storage.{BlockId, FileSegment} + + +class FileServerHandler(pResolver: PathResolver) + extends SimpleChannelInboundHandler[String] with Logging { + + override def channelRead0(ctx: ChannelHandlerContext, blockIdString: String): Unit = { + val blockId: BlockId = BlockId(blockIdString) + val fileSegment: FileSegment = pResolver.getBlockLocation(blockId) + if (fileSegment == null) { + return + } + val file = fileSegment.file + if (file.exists) { + if (!file.isFile) { + ctx.write(new FileHeader(0, blockId).buffer) + ctx.flush() + return + } + val length: Long = fileSegment.length + if (length > Integer.MAX_VALUE || length <= 0) { + ctx.write(new FileHeader(0, blockId).buffer) + ctx.flush() + return + } + ctx.write(new FileHeader(length.toInt, blockId).buffer) + try { + val channel = new FileInputStream(file).getChannel + ctx.write(new DefaultFileRegion(channel, fileSegment.offset, fileSegment.length)) + } catch { + case e: Exception => + logError("Exception: ", e) + } + } else { + ctx.write(new FileHeader(0, blockId).buffer) + } + ctx.flush() + } + + override def exceptionCaught(ctx: ChannelHandlerContext, cause: Throwable): Unit = { + logError("Exception: ", cause) + ctx.close() + } +} diff --git a/core/src/main/java/org/apache/spark/network/netty/PathResolver.java b/core/src/main/scala/org/apache/spark/network/netty/PathResolver.scala old mode 100755 new mode 100644 similarity index 80% rename from core/src/main/java/org/apache/spark/network/netty/PathResolver.java rename to core/src/main/scala/org/apache/spark/network/netty/PathResolver.scala index 7ad8d03efbadc..0d7695072a7b1 --- a/core/src/main/java/org/apache/spark/network/netty/PathResolver.java +++ b/core/src/main/scala/org/apache/spark/network/netty/PathResolver.scala @@ -15,12 +15,11 @@ * limitations under the License. */ -package org.apache.spark.network.netty; +package org.apache.spark.network.netty -import org.apache.spark.storage.BlockId; -import org.apache.spark.storage.FileSegment; +import org.apache.spark.storage.{BlockId, FileSegment} -public interface PathResolver { +trait PathResolver { /** Get the file segment in which the given block resides. */ - FileSegment getBlockLocation(BlockId blockId); + def getBlockLocation(blockId: BlockId): FileSegment } diff --git a/core/src/main/scala/org/apache/spark/network/netty/ShuffleSender.scala b/core/src/main/scala/org/apache/spark/network/netty/ShuffleSender.scala index 7ef7aecc6a9fb..95958e30f7eeb 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/ShuffleSender.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/ShuffleSender.scala @@ -32,7 +32,7 @@ private[spark] class ShuffleSender(portIn: Int, val pResolver: PathResolver) ext server.stop() } - def port: Int = server.getPort() + def port: Int = server.getPort } diff --git a/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala index 34c51b833025e..20938781ac694 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala @@ -141,7 +141,7 @@ private[spark] object CheckpointRDD extends Logging { val deserializeStream = serializer.deserializeStream(fileInputStream) // Register an on-task-completion callback to close the input stream. - context.addOnCompleteCallback(() => deserializeStream.close()) + context.addTaskCompletionListener(context => deserializeStream.close()) deserializeStream.asIterator.asInstanceOf[Iterator[T]] } 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 5951865e56c9d..fabb882cdd4b3 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -17,15 +17,16 @@ package org.apache.spark.rdd +import scala.language.existentials + import java.io.{IOException, ObjectOutputStream} import scala.collection.mutable.ArrayBuffer -import scala.language.existentials import org.apache.spark.{InterruptibleIterator, Partition, Partitioner, SparkEnv, TaskContext} 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.util.collection.{ExternalAppendOnlyMap, AppendOnlyMap, CompactBuffer} import org.apache.spark.serializer.Serializer import org.apache.spark.shuffle.ShuffleHandle @@ -66,14 +67,14 @@ private[spark] class CoGroupPartition(idx: Int, val deps: Array[CoGroupSplitDep] */ @DeveloperApi class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: Partitioner) - extends RDD[(K, Seq[Seq[_]])](rdds.head.context, Nil) { + extends RDD[(K, Array[Iterable[_]])](rdds.head.context, Nil) { // For example, `(k, a) cogroup (k, b)` produces k -> Seq(ArrayBuffer as, ArrayBuffer bs). // Each ArrayBuffer is represented as a CoGroup, and the resulting Seq as a CoGroupCombiner. // CoGroupValue is the intermediate state of each value before being merged in compute. - private type CoGroup = ArrayBuffer[Any] + private type CoGroup = CompactBuffer[Any] private type CoGroupValue = (Any, Int) // Int is dependency number - private type CoGroupCombiner = Seq[CoGroup] + private type CoGroupCombiner = Array[CoGroup] private var serializer: Option[Serializer] = None @@ -114,7 +115,7 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: override val partitioner: Some[Partitioner] = Some(part) - override def compute(s: Partition, context: TaskContext): Iterator[(K, CoGroupCombiner)] = { + override def compute(s: Partition, context: TaskContext): Iterator[(K, Array[Iterable[_]])] = { val sparkConf = SparkEnv.get.conf val externalSorting = sparkConf.getBoolean("spark.shuffle.spill", true) val split = s.asInstanceOf[CoGroupPartition] @@ -150,18 +151,17 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: getCombiner(kv._1)(depNum) += kv._2 } } - new InterruptibleIterator(context, map.iterator) + new InterruptibleIterator(context, + map.iterator.asInstanceOf[Iterator[(K, Array[Iterable[_]])]]) } else { val map = createExternalMap(numRdds) - rddIterators.foreach { case (it, depNum) => - while (it.hasNext) { - val kv = it.next() - map.insert(kv._1, new CoGroupValue(kv._2, depNum)) - } + for ((it, depNum) <- rddIterators) { + map.insertAll(it.map(pair => (pair._1, new CoGroupValue(pair._2, depNum)))) } - context.taskMetrics.memoryBytesSpilled = map.memoryBytesSpilled - context.taskMetrics.diskBytesSpilled = map.diskBytesSpilled - new InterruptibleIterator(context, map.iterator) + context.taskMetrics.memoryBytesSpilled += map.memoryBytesSpilled + context.taskMetrics.diskBytesSpilled += map.diskBytesSpilled + new InterruptibleIterator(context, + map.iterator.asInstanceOf[Iterator[(K, Array[Iterable[_]])]]) } } @@ -170,17 +170,22 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: val createCombiner: (CoGroupValue => CoGroupCombiner) = value => { val newCombiner = Array.fill(numRdds)(new CoGroup) - value match { case (v, depNum) => newCombiner(depNum) += v } + newCombiner(value._2) += value._1 newCombiner } val mergeValue: (CoGroupCombiner, CoGroupValue) => CoGroupCombiner = (combiner, value) => { - value match { case (v, depNum) => combiner(depNum) += v } + combiner(value._2) += value._1 combiner } val mergeCombiners: (CoGroupCombiner, CoGroupCombiner) => CoGroupCombiner = (combiner1, combiner2) => { - combiner1.zip(combiner2).map { case (v1, v2) => v1 ++ v2 } + var depNum = 0 + while (depNum < numRdds) { + combiner1(depNum) ++= combiner2(depNum) + depNum += 1 + } + combiner1 } new ExternalAppendOnlyMap[K, CoGroupValue, CoGroupCombiner]( createCombiner, mergeValue, mergeCombiners) diff --git a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala index c45b759f007cc..11ebafbf6d457 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala @@ -49,8 +49,8 @@ private[spark] case class CoalescedRDDPartition( } /** - * Computes how many of the parents partitions have getPreferredLocation - * as one of their preferredLocations + * Computes the fraction of the parents' partitions containing preferredLocation within + * their getPreferredLocs. * @return locality of this coalesced partition between 0 and 1 */ def localFraction: Double = { @@ -258,7 +258,7 @@ private[spark] class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanc val pgroup = PartitionGroup(nxt_replica) groupArr += pgroup addPartToPGroup(nxt_part, pgroup) - groupHash += (nxt_replica -> (ArrayBuffer(pgroup))) // list in case we have multiple + groupHash.put(nxt_replica, ArrayBuffer(pgroup)) // list in case we have multiple numCreated += 1 } } @@ -267,7 +267,7 @@ private[spark] class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanc var (nxt_replica, nxt_part) = rotIt.next() val pgroup = PartitionGroup(nxt_replica) groupArr += pgroup - groupHash.get(nxt_replica).get += pgroup + groupHash.getOrElseUpdate(nxt_replica, ArrayBuffer()) += pgroup var tries = 0 while (!addPartToPGroup(nxt_part, pgroup) && tries < targetLen) { // ensure at least one part nxt_part = rotIt.next()._2 diff --git a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala index 9ca971c8a4c27..f233544d128f5 100644 --- a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala @@ -119,11 +119,11 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable { /** * Compute a histogram using the provided buckets. The buckets are all open - * to the left except for the last which is closed + * to the right except for the last which is closed * e.g. for the array * [1, 10, 20, 50] the buckets are [1, 10) [10, 20) [20, 50] - * e.g 1<=x<10 , 10<=x<20, 20<=x<50 - * And on the input of 1 and 50 we would have a histogram of 1, 0, 0 + * e.g 1<=x<10 , 10<=x<20, 20<=x<=50 + * And on the input of 1 and 50 we would have a histogram of 1, 0, 1 * * Note: if your histogram is evenly spaced (e.g. [0, 10, 20, 30]) this can be switched * from an O(log n) inseration to O(1) per element. (where n = # buckets) if you set evenBuckets 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 041028514399b..c8623314c98eb 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -20,7 +20,9 @@ package org.apache.spark.rdd import java.text.SimpleDateFormat import java.util.Date import java.io.EOFException + import scala.collection.immutable.Map +import scala.reflect.ClassTag import org.apache.hadoop.conf.{Configurable, Configuration} import org.apache.hadoop.mapred.FileSplit @@ -39,6 +41,7 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.broadcast.Broadcast import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.executor.{DataReadMethod, InputMetrics} +import org.apache.spark.rdd.HadoopRDD.HadoopMapPartitionsWithSplitRDD import org.apache.spark.util.NextIterator /** @@ -140,8 +143,8 @@ 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. - // synchronize to prevent ConcurrentModificationException (Spark-1097, Hadoop-10456) - conf.synchronized { + // Synchronize to prevent ConcurrentModificationException (Spark-1097, Hadoop-10456). + HadoopRDD.CONFIGURATION_INSTANTIATION_LOCK.synchronized { val newJobConf = new JobConf(conf) initLocalJobConfFuncOpt.map(f => f(newJobConf)) HadoopRDD.putCachedMetadata(jobConfCacheKey, newJobConf) @@ -194,7 +197,7 @@ class HadoopRDD[K, V]( reader = inputFormat.getRecordReader(split.inputSplit.value, jobConf, Reporter.NULL) // Register an on-task-completion callback to close the input stream. - context.addOnCompleteCallback{ () => closeIfNeeded() } + context.addTaskCompletionListener{ context => closeIfNeeded() } val key: K = reader.createKey() val value: V = reader.createValue() @@ -232,6 +235,14 @@ class HadoopRDD[K, V]( new InterruptibleIterator[(K, V)](context, iter) } + /** Maps over a partition, providing the InputSplit that was used as the base of the partition. */ + @DeveloperApi + def mapPartitionsWithInputSplit[U: ClassTag]( + f: (InputSplit, Iterator[(K, V)]) => Iterator[U], + preservesPartitioning: Boolean = false): RDD[U] = { + new HadoopMapPartitionsWithSplitRDD(this, f, preservesPartitioning) + } + override def getPreferredLocations(split: Partition): Seq[String] = { // TODO: Filtering out "localhost" in case of file:// URLs val hadoopSplit = split.asInstanceOf[HadoopPartition] @@ -246,6 +257,9 @@ class HadoopRDD[K, V]( } private[spark] object HadoopRDD { + /** Constructing Configuration objects is not threadsafe, use this lock to serialize. */ + val CONFIGURATION_INSTANTIATION_LOCK = new Object() + /** * The three methods below are helpers for accessing the local map, a property of the SparkEnv of * the local process. @@ -269,4 +283,25 @@ private[spark] object HadoopRDD { conf.setInt("mapred.task.partition", splitId) conf.set("mapred.job.id", jobID.toString) } + + /** + * Analogous to [[org.apache.spark.rdd.MapPartitionsRDD]], but passes in an InputSplit to + * the given function rather than the index of the partition. + */ + private[spark] class HadoopMapPartitionsWithSplitRDD[U: ClassTag, T: ClassTag]( + prev: RDD[T], + f: (InputSplit, Iterator[T]) => Iterator[U], + preservesPartitioning: Boolean = false) + extends RDD[U](prev) { + + override val partitioner = if (preservesPartitioning) firstParent[T].partitioner else None + + override def getPartitions: Array[Partition] = firstParent[T].partitions + + override def compute(split: Partition, context: TaskContext) = { + val partition = split.asInstanceOf[HadoopPartition] + val inputSplit = partition.inputSplit.value + f(inputSplit, firstParent[T].iterator(split, context)) + } + } } diff --git a/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala index a76a070b5b863..0e38f224ac81d 100644 --- a/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala @@ -68,7 +68,7 @@ class JdbcRDD[T: ClassTag]( } override def compute(thePart: Partition, context: TaskContext) = new NextIterator[T] { - context.addOnCompleteCallback{ () => closeIfNeeded() } + context.addTaskCompletionListener{ context => closeIfNeeded() } val part = thePart.asInstanceOf[JdbcPartition] val conn = getConnection() val stmt = conn.prepareStatement(sql, ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY) @@ -96,17 +96,23 @@ class JdbcRDD[T: ClassTag]( override def close() { try { - if (null != rs && ! rs.isClosed()) rs.close() + if (null != rs && ! rs.isClosed()) { + rs.close() + } } catch { case e: Exception => logWarning("Exception closing resultset", e) } try { - if (null != stmt && ! stmt.isClosed()) stmt.close() + if (null != stmt && ! stmt.isClosed()) { + stmt.close() + } } catch { case e: Exception => logWarning("Exception closing statement", e) } try { - if (null != conn && ! stmt.isClosed()) conn.close() + if (null != conn && ! conn.isClosed()) { + conn.close() + } logInfo("closed connection") } catch { case e: Exception => logWarning("Exception closing connection", e) @@ -120,3 +126,4 @@ object JdbcRDD { Array.tabulate[Object](rs.getMetaData.getColumnCount)(i => rs.getObject(i + 1)) } } + diff --git a/core/src/main/scala/org/apache/spark/rdd/MappedValuesRDD.scala b/core/src/main/scala/org/apache/spark/rdd/MappedValuesRDD.scala index 2bc47eb9fcd74..a60952eee5901 100644 --- a/core/src/main/scala/org/apache/spark/rdd/MappedValuesRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/MappedValuesRDD.scala @@ -28,6 +28,6 @@ class MappedValuesRDD[K, V, U](prev: RDD[_ <: Product2[K, V]], f: V => U) override val partitioner = firstParent[Product2[K, U]].partitioner override def compute(split: Partition, context: TaskContext): Iterator[(K, U)] = { - firstParent[Product2[K, V]].iterator(split, context).map { case Product2(k ,v) => (k, f(v)) } + firstParent[Product2[K, V]].iterator(split, context).map { pair => (pair._1, f(pair._2)) } } } diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala index f2b3a64bf1345..58f707b9b4634 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -20,6 +20,8 @@ package org.apache.spark.rdd import java.text.SimpleDateFormat import java.util.Date +import scala.reflect.ClassTag + import org.apache.hadoop.conf.{Configurable, Configuration} import org.apache.hadoop.io.Writable import org.apache.hadoop.mapreduce._ @@ -32,6 +34,7 @@ import org.apache.spark.Partition import org.apache.spark.SerializableWritable import org.apache.spark.{SparkContext, TaskContext} import org.apache.spark.executor.{DataReadMethod, InputMetrics} +import org.apache.spark.rdd.NewHadoopRDD.NewHadoopMapPartitionsWithSplitRDD private[spark] class NewHadoopPartition( rddId: Int, @@ -126,7 +129,7 @@ class NewHadoopRDD[K, V]( context.taskMetrics.inputMetrics = Some(inputMetrics) // Register an on-task-completion callback to close the input stream. - context.addOnCompleteCallback(() => close()) + context.addTaskCompletionListener(context => close()) var havePair = false var finished = false @@ -157,6 +160,14 @@ class NewHadoopRDD[K, V]( new InterruptibleIterator(context, iter) } + /** Maps over a partition, providing the InputSplit that was used as the base of the partition. */ + @DeveloperApi + def mapPartitionsWithInputSplit[U: ClassTag]( + f: (InputSplit, Iterator[(K, V)]) => Iterator[U], + preservesPartitioning: Boolean = false): RDD[U] = { + new NewHadoopMapPartitionsWithSplitRDD(this, f, preservesPartitioning) + } + override def getPreferredLocations(split: Partition): Seq[String] = { val theSplit = split.asInstanceOf[NewHadoopPartition] theSplit.serializableHadoopSplit.value.getLocations.filter(_ != "localhost") @@ -165,6 +176,29 @@ class NewHadoopRDD[K, V]( def getConf: Configuration = confBroadcast.value.value } +private[spark] object NewHadoopRDD { + /** + * Analogous to [[org.apache.spark.rdd.MapPartitionsRDD]], but passes in an InputSplit to + * the given function rather than the index of the partition. + */ + private[spark] class NewHadoopMapPartitionsWithSplitRDD[U: ClassTag, T: ClassTag]( + prev: RDD[T], + f: (InputSplit, Iterator[T]) => Iterator[U], + preservesPartitioning: Boolean = false) + extends RDD[U](prev) { + + override val partitioner = if (preservesPartitioning) firstParent[T].partitioner else None + + override def getPartitions: Array[Partition] = firstParent[T].partitions + + override def compute(split: Partition, context: TaskContext) = { + val partition = split.asInstanceOf[NewHadoopPartition] + val inputSplit = partition.serializableHadoopSplit.value + f(inputSplit, firstParent[T].iterator(split, context)) + } + } +} + private[spark] class WholeTextFileRDD( sc : SparkContext, inputFormatClass: Class[_ <: WholeTextFileInputFormat], diff --git a/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala index f1f4b4324edfd..e98bad2026e32 100644 --- a/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala @@ -20,6 +20,7 @@ package org.apache.spark.rdd import scala.reflect.ClassTag import org.apache.spark.{Logging, RangePartitioner} +import org.apache.spark.annotation.DeveloperApi /** * Extra functions available on RDDs of (key, value) pairs where the key is sortable through @@ -43,10 +44,10 @@ import org.apache.spark.{Logging, RangePartitioner} */ class OrderedRDDFunctions[K : Ordering : ClassTag, V: ClassTag, - P <: Product2[K, V] : ClassTag]( + P <: Product2[K, V] : ClassTag] @DeveloperApi() ( self: RDD[P]) - extends Logging with Serializable { - + extends Logging with Serializable +{ private val ordering = implicitly[Ordering[K]] /** @@ -55,16 +56,12 @@ class OrderedRDDFunctions[K : Ordering : ClassTag, * (in the `save` case, they will be written to multiple `part-X` files in the filesystem, in * order of the keys). */ - def sortByKey(ascending: Boolean = true, numPartitions: Int = self.partitions.size): RDD[P] = { + // TODO: this currently doesn't work on P other than Tuple2! + def sortByKey(ascending: Boolean = true, numPartitions: Int = self.partitions.size) + : RDD[(K, V)] = + { val part = new RangePartitioner(numPartitions, self, ascending) - val shuffled = new ShuffledRDD[K, V, V, P](self, part).setKeyOrdering(ordering) - shuffled.mapPartitions(iter => { - val buf = iter.toArray - if (ascending) { - buf.sortWith((x, y) => ordering.lt(x._1, y._1)).iterator - } else { - buf.sortWith((x, y) => ordering.gt(x._1, y._1)).iterator - } - }, preservesPartitioning = true) + new ShuffledRDD[K, V, V](self, part) + .setKeyOrdering(if (ascending) ordering else ordering.reverse) } } 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 fc9beb166befe..f6d9d12fe9006 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -19,12 +19,10 @@ package org.apache.spark.rdd import java.nio.ByteBuffer import java.text.SimpleDateFormat -import java.util.Date -import java.util.{HashMap => JHashMap} +import java.util.{Date, HashMap => JHashMap} +import scala.collection.{Map, mutable} import scala.collection.JavaConversions._ -import scala.collection.Map -import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import scala.reflect.ClassTag @@ -34,18 +32,19 @@ import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.io.SequenceFile.CompressionType import org.apache.hadoop.io.compress.CompressionCodec import org.apache.hadoop.mapred.{FileOutputCommitter, FileOutputFormat, JobConf, OutputFormat} -import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat, Job => NewAPIHadoopJob, +import org.apache.hadoop.mapreduce.{Job => NewAPIHadoopJob, OutputFormat => NewOutputFormat, RecordWriter => NewRecordWriter, SparkHadoopMapReduceUtil} -import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat => NewFileOutputFormat} import org.apache.spark._ -import org.apache.spark.annotation.Experimental -import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.SparkHadoopWriter import org.apache.spark.Partitioner.defaultPartitioner import org.apache.spark.SparkContext._ +import org.apache.spark.annotation.Experimental +import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.serializer.Serializer +import org.apache.spark.util.Utils +import org.apache.spark.util.collection.CompactBuffer +import org.apache.spark.util.random.StratifiedSamplingUtils /** * Extra functions available on RDDs of (key, value) pairs through an implicit conversion. @@ -91,7 +90,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) new InterruptibleIterator(context, aggregator.combineValuesByKey(iter, context)) }, preservesPartitioning = true) } else { - new ShuffledRDD[K, V, C, (K, C)](self, partitioner) + new ShuffledRDD[K, V, C](self, partitioner) .setSerializer(serializer) .setAggregator(aggregator) .setMapSideCombine(mapSideCombine) @@ -125,7 +124,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) zeroBuffer.get(zeroArray) lazy val cachedSerializer = SparkEnv.get.closureSerializer.newInstance() - def createZero() = cachedSerializer.deserialize[U](ByteBuffer.wrap(zeroArray)) + val createZero = () => cachedSerializer.deserialize[U](ByteBuffer.wrap(zeroArray)) combineByKey[U]((v: V) => seqOp(createZero(), v), seqOp, combOp, partitioner) } @@ -171,7 +170,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) // When deserializing, use a lazy val to create just one instance of the serializer per task lazy val cachedSerializer = SparkEnv.get.closureSerializer.newInstance() - def createZero() = cachedSerializer.deserialize[V](ByteBuffer.wrap(zeroArray)) + val createZero = () => cachedSerializer.deserialize[V](ByteBuffer.wrap(zeroArray)) combineByKey[V]((v: V) => func(createZero(), v), func, func, partitioner) } @@ -194,6 +193,64 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) foldByKey(zeroValue, defaultPartitioner(self))(func) } + /** + * Return a subset of this RDD sampled by key (via stratified sampling). + * + * Create a sample of this RDD using variable sampling rates for different keys as specified by + * `fractions`, a key to sampling rate map, via simple random sampling with one pass over the + * RDD, to produce a sample of size that's approximately equal to the sum of + * math.ceil(numItems * samplingRate) over all key values. + * + * @param withReplacement whether to sample with or without replacement + * @param fractions map of specific keys to sampling rates + * @param seed seed for the random number generator + * @return RDD containing the sampled subset + */ + def sampleByKey(withReplacement: Boolean, + fractions: Map[K, Double], + seed: Long = Utils.random.nextLong): RDD[(K, V)] = { + + require(fractions.values.forall(v => v >= 0.0), "Negative sampling rates.") + + val samplingFunc = if (withReplacement) { + StratifiedSamplingUtils.getPoissonSamplingFunction(self, fractions, false, seed) + } else { + StratifiedSamplingUtils.getBernoulliSamplingFunction(self, fractions, false, seed) + } + self.mapPartitionsWithIndex(samplingFunc, preservesPartitioning = true) + } + + /** + * ::Experimental:: + * Return a subset of this RDD sampled by key (via stratified sampling) containing exactly + * math.ceil(numItems * samplingRate) for each stratum (group of pairs with the same key). + * + * This method differs from [[sampleByKey]] in that we make additional passes over the RDD to + * create a sample size that's exactly equal to the sum of math.ceil(numItems * samplingRate) + * over all key values with a 99.99% confidence. When sampling without replacement, we need one + * additional pass over the RDD to guarantee sample size; when sampling with replacement, we need + * two additional passes. + * + * @param withReplacement whether to sample with or without replacement + * @param fractions map of specific keys to sampling rates + * @param seed seed for the random number generator + * @return RDD containing the sampled subset + */ + @Experimental + def sampleByKeyExact(withReplacement: Boolean, + fractions: Map[K, Double], + seed: Long = Utils.random.nextLong): RDD[(K, V)] = { + + require(fractions.values.forall(v => v >= 0.0), "Negative sampling rates.") + + val samplingFunc = if (withReplacement) { + StratifiedSamplingUtils.getPoissonSamplingFunction(self, fractions, true, seed) + } else { + StratifiedSamplingUtils.getBernoulliSamplingFunction(self, fractions, true, seed) + } + self.mapPartitionsWithIndex(samplingFunc, preservesPartitioning = true) + } + /** * Merge the values for each key using an associative reduce function. This will also perform * the merging locally on each mapper before sending results to a reducer, similarly to a @@ -203,6 +260,25 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) combineByKey[V]((v: V) => v, func, func, partitioner) } + /** + * Merge the values for each key using an associative reduce function. This will also perform + * the merging locally on each mapper before sending results to a reducer, similarly to a + * "combiner" in MapReduce. Output will be hash-partitioned with numPartitions partitions. + */ + def reduceByKey(func: (V, V) => V, numPartitions: Int): RDD[(K, V)] = { + reduceByKey(new HashPartitioner(numPartitions), func) + } + + /** + * Merge the values for each key using an associative reduce function. This will also perform + * the merging locally on each mapper before sending results to a reducer, similarly to a + * "combiner" in MapReduce. Output will be hash-partitioned with the existing partitioner/ + * parallelism level. + */ + def reduceByKey(func: (V, V) => V): RDD[(K, V)] = { + reduceByKey(defaultPartitioner(self), func) + } + /** * Merge the values for each key using an associative reduce function, but return the results * immediately to the master as a Map. This will also perform the merging locally on each mapper @@ -214,22 +290,22 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) throw new SparkException("reduceByKeyLocally() does not support array keys") } - def reducePartition(iter: Iterator[(K, V)]): Iterator[JHashMap[K, V]] = { + val reducePartition = (iter: Iterator[(K, V)]) => { val map = new JHashMap[K, V] - iter.foreach { case (k, v) => - val old = map.get(k) - map.put(k, if (old == null) v else func(old, v)) + iter.foreach { pair => + val old = map.get(pair._1) + map.put(pair._1, if (old == null) pair._2 else func(old, pair._2)) } Iterator(map) - } + } : Iterator[JHashMap[K, V]] - def mergeMaps(m1: JHashMap[K, V], m2: JHashMap[K, V]): JHashMap[K, V] = { - m2.foreach { case (k, v) => - val old = m1.get(k) - m1.put(k, if (old == null) v else func(old, v)) + val mergeMaps = (m1: JHashMap[K, V], m2: JHashMap[K, V]) => { + m2.foreach { pair => + val old = m1.get(pair._1) + m1.put(pair._1, if (old == null) pair._2 else func(old, pair._2)) } m1 - } + } : JHashMap[K, V] self.mapPartitions(reducePartition).reduce(mergeMaps) } @@ -340,42 +416,33 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) countApproxDistinctByKey(relativeSD, defaultPartitioner(self)) } - /** - * Merge the values for each key using an associative reduce function. This will also perform - * the merging locally on each mapper before sending results to a reducer, similarly to a - * "combiner" in MapReduce. Output will be hash-partitioned with numPartitions partitions. - */ - def reduceByKey(func: (V, V) => V, numPartitions: Int): RDD[(K, V)] = { - reduceByKey(new HashPartitioner(numPartitions), func) - } - /** * Group the values for each key in the RDD into a single sequence. Allows controlling the * partitioning of the resulting key-value pair RDD by passing a Partitioner. * - * Note: If you are grouping in order to perform an aggregation (such as a sum or average) over - * each key, using [[PairRDDFunctions.reduceByKey]] or [[PairRDDFunctions.combineByKey]] - * will provide much better performance. + * Note: This operation may be very expensive. If you are grouping in order to perform an + * aggregation (such as a sum or average) over each key, using [[PairRDDFunctions.aggregateByKey]] + * or [[PairRDDFunctions.reduceByKey]] will provide much better performance. */ def groupByKey(partitioner: Partitioner): RDD[(K, Iterable[V])] = { // groupByKey shouldn't use map side combine because map side combine does not // reduce the amount of data shuffled and requires all map side data be inserted // into a hash table, leading to more objects in the old gen. - def createCombiner(v: V) = ArrayBuffer(v) - def mergeValue(buf: ArrayBuffer[V], v: V) = buf += v - def mergeCombiners(c1: ArrayBuffer[V], c2: ArrayBuffer[V]) = c1 ++ c2 - val bufs = combineByKey[ArrayBuffer[V]]( - createCombiner _, mergeValue _, mergeCombiners _, partitioner, mapSideCombine=false) - bufs.mapValues(_.toIterable) + val createCombiner = (v: V) => CompactBuffer(v) + val mergeValue = (buf: CompactBuffer[V], v: V) => buf += v + val mergeCombiners = (c1: CompactBuffer[V], c2: CompactBuffer[V]) => c1 ++= c2 + val bufs = combineByKey[CompactBuffer[V]]( + createCombiner, mergeValue, mergeCombiners, partitioner, mapSideCombine=false) + bufs.asInstanceOf[RDD[(K, Iterable[V])]] } /** * Group the values for each key in the RDD into a single sequence. Hash-partitions the * resulting RDD with into `numPartitions` partitions. * - * Note: If you are grouping in order to perform an aggregation (such as a sum or average) over - * each key, using [[PairRDDFunctions.reduceByKey]] or [[PairRDDFunctions.combineByKey]] - * will provide much better performance. + * Note: This operation may be very expensive. If you are grouping in order to perform an + * aggregation (such as a sum or average) over each key, using [[PairRDDFunctions.aggregateByKey]] + * or [[PairRDDFunctions.reduceByKey]] will provide much better performance. */ def groupByKey(numPartitions: Int): RDD[(K, Iterable[V])] = { groupByKey(new HashPartitioner(numPartitions)) @@ -391,7 +458,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) if (self.partitioner == Some(partitioner)) { self } else { - new ShuffledRDD[K, V, V, (K, V)](self, partitioner) + new ShuffledRDD[K, V, V](self, partitioner) } } @@ -401,9 +468,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * (k, v2) is in `other`. Uses the given Partitioner to partition the output RDD. */ def join[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, W))] = { - this.cogroup(other, partitioner).flatMapValues { case (vs, ws) => - for (v <- vs; w <- ws) yield (v, w) - } + this.cogroup(other, partitioner).flatMapValues( pair => + for (v <- pair._1; w <- pair._2) yield (v, w) + ) } /** @@ -413,11 +480,11 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * partition the output RDD. */ def leftOuterJoin[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, Option[W]))] = { - this.cogroup(other, partitioner).flatMapValues { case (vs, ws) => - if (ws.isEmpty) { - vs.map(v => (v, None)) + this.cogroup(other, partitioner).flatMapValues { pair => + if (pair._2.isEmpty) { + pair._1.map(v => (v, None)) } else { - for (v <- vs; w <- ws) yield (v, Some(w)) + for (v <- pair._1; w <- pair._2) yield (v, Some(w)) } } } @@ -430,11 +497,11 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) */ def rightOuterJoin[W](other: RDD[(K, W)], partitioner: Partitioner) : RDD[(K, (Option[V], W))] = { - this.cogroup(other, partitioner).flatMapValues { case (vs, ws) => - if (vs.isEmpty) { - ws.map(w => (None, w)) + this.cogroup(other, partitioner).flatMapValues { pair => + if (pair._1.isEmpty) { + pair._2.map(w => (None, w)) } else { - for (v <- vs; w <- ws) yield (Some(v), w) + for (v <- pair._1; w <- pair._2) yield (Some(v), w) } } } @@ -448,23 +515,13 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) combineByKey(createCombiner, mergeValue, mergeCombiners, defaultPartitioner(self)) } - /** - * Merge the values for each key using an associative reduce function. This will also perform - * the merging locally on each mapper before sending results to a reducer, similarly to a - * "combiner" in MapReduce. Output will be hash-partitioned with the existing partitioner/ - * parallelism level. - */ - def reduceByKey(func: (V, V) => V): RDD[(K, V)] = { - reduceByKey(defaultPartitioner(self), func) - } - /** * Group the values for each key in the RDD into a single sequence. Hash-partitions the * resulting RDD with the existing partitioner/parallelism level. * - * Note: If you are grouping in order to perform an aggregation (such as a sum or average) over - * each key, using [[PairRDDFunctions.reduceByKey]] or [[PairRDDFunctions.combineByKey]] - * will provide much better performance, + * Note: This operation may be very expensive. If you are grouping in order to perform an + * aggregation (such as a sum or average) over each key, using [[PairRDDFunctions.aggregateByKey]] + * or [[PairRDDFunctions.reduceByKey]] will provide much better performance. */ def groupByKey(): RDD[(K, Iterable[V])] = { groupByKey(defaultPartitioner(self)) @@ -530,12 +587,15 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) /** * Return the key-value pairs in this RDD to the master as a Map. + * + * Warning: this doesn't return a multimap (so if you have multiple values to the same key, only + * one value per key is preserved in the map returned) */ def collectAsMap(): Map[K, V] = { val data = self.collect() val map = new mutable.HashMap[K, V] map.sizeHint(data.length) - data.foreach { case (k, v) => map.put(k, v) } + data.foreach { pair => map.put(pair._1, pair._2) } map } @@ -571,11 +631,11 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) throw new SparkException("Default partitioner cannot partition array keys.") } val cg = new CoGroupedRDD[K](Seq(self, other1, other2, other3), partitioner) - cg.mapValues { case Seq(vs, w1s, w2s, w3s) => - (vs.asInstanceOf[Seq[V]], - w1s.asInstanceOf[Seq[W1]], - w2s.asInstanceOf[Seq[W2]], - w3s.asInstanceOf[Seq[W3]]) + cg.mapValues { case Array(vs, w1s, w2s, w3s) => + (vs.asInstanceOf[Iterable[V]], + w1s.asInstanceOf[Iterable[W1]], + w2s.asInstanceOf[Iterable[W2]], + w3s.asInstanceOf[Iterable[W3]]) } } @@ -589,8 +649,8 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) throw new SparkException("Default partitioner cannot partition array keys.") } val cg = new CoGroupedRDD[K](Seq(self, other), partitioner) - cg.mapValues { case Seq(vs, ws) => - (vs.asInstanceOf[Seq[V]], ws.asInstanceOf[Seq[W]]) + cg.mapValues { case Array(vs, w1s) => + (vs.asInstanceOf[Iterable[V]], w1s.asInstanceOf[Iterable[W]]) } } @@ -604,10 +664,10 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) throw new SparkException("Default partitioner cannot partition array keys.") } val cg = new CoGroupedRDD[K](Seq(self, other1, other2), partitioner) - cg.mapValues { case Seq(vs, w1s, w2s) => - (vs.asInstanceOf[Seq[V]], - w1s.asInstanceOf[Seq[W1]], - w2s.asInstanceOf[Seq[W2]]) + cg.mapValues { case Array(vs, w1s, w2s) => + (vs.asInstanceOf[Iterable[V]], + w1s.asInstanceOf[Iterable[W1]], + w2s.asInstanceOf[Iterable[W2]]) } } @@ -710,14 +770,14 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) self.partitioner match { case Some(p) => val index = p.getPartition(key) - def process(it: Iterator[(K, V)]): Seq[V] = { + val process = (it: Iterator[(K, V)]) => { val buf = new ArrayBuffer[V] - for ((k, v) <- it if k == key) { - buf += v + for (pair <- it if pair._1 == key) { + buf += pair._2 } buf - } - val res = self.context.runJob(self, process _, Array(index), false) + } : Seq[V] + val res = self.context.runJob(self, process, Array(index), false) res(0) case None => self.filter(_._1 == key).map(_._2).collect() @@ -840,7 +900,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) jobFormat.checkOutputSpecs(job) } - def writeShard(context: TaskContext, iter: Iterator[(K,V)]): Int = { + val writeShard = (context: TaskContext, iter: Iterator[(K,V)]) => { // Hadoop wants a 32-bit task attempt ID, so if ours is bigger than Int.MaxValue, roll it // around by taking a mod. We expect that no task will be attempted 2 billion times. val attemptNumber = (context.attemptId % Int.MaxValue).toInt @@ -858,22 +918,21 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) val writer = format.getRecordWriter(hadoopContext).asInstanceOf[NewRecordWriter[K,V]] try { while (iter.hasNext) { - val (k, v) = iter.next() - writer.write(k, v) + val pair = iter.next() + writer.write(pair._1, pair._2) } - } - finally { + } finally { writer.close(hadoopContext) } committer.commitTask(hadoopContext) - return 1 - } + 1 + } : Int val jobAttemptId = newTaskAttemptID(jobtrackerID, stageId, isMap = true, 0, 0) val jobTaskContext = newTaskAttemptContext(wrappedConf.value, jobAttemptId) val jobCommitter = jobFormat.getOutputCommitter(jobTaskContext) jobCommitter.setupJob(jobTaskContext) - self.context.runJob(self, writeShard _) + self.context.runJob(self, writeShard) jobCommitter.commitJob(jobTaskContext) } @@ -912,7 +971,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) val writer = new SparkHadoopWriter(hadoopConf) writer.preSetup() - def writeToFile(context: TaskContext, iter: Iterator[(K, V)]) { + val writeToFile = (context: TaskContext, iter: Iterator[(K, V)]) => { // Hadoop wants a 32-bit task attempt ID, so if ours is bigger than Int.MaxValue, roll it // around by taking a mod. We expect that no task will be attempted 2 billion times. val attemptNumber = (context.attemptId % Int.MaxValue).toInt @@ -921,19 +980,18 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) writer.open() try { var count = 0 - while(iter.hasNext) { + while (iter.hasNext) { val record = iter.next() count += 1 writer.write(record._1.asInstanceOf[AnyRef], record._2.asInstanceOf[AnyRef]) } - } - finally { + } finally { writer.close() } writer.commit() } - self.context.runJob(self, writeToFile _) + self.context.runJob(self, writeToFile) writer.commitJob() } diff --git a/core/src/main/scala/org/apache/spark/rdd/PartitionwiseSampledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PartitionwiseSampledRDD.scala index b5b8a5706deb3..a637d6f15b7e5 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PartitionwiseSampledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PartitionwiseSampledRDD.scala @@ -39,6 +39,7 @@ class PartitionwiseSampledRDDPartition(val prev: Partition, val seed: Long) * * @param prev RDD to be sampled * @param sampler a random sampler + * @param preservesPartitioning whether the sampler preserves the partitioner of the parent RDD * @param seed random seed * @tparam T input RDD item type * @tparam U sampled RDD item type @@ -46,9 +47,12 @@ class PartitionwiseSampledRDDPartition(val prev: Partition, val seed: Long) private[spark] class PartitionwiseSampledRDD[T: ClassTag, U: ClassTag]( prev: RDD[T], sampler: RandomSampler[T, U], + @transient preservesPartitioning: Boolean, @transient seed: Long = Utils.random.nextLong) extends RDD[U](prev) { + @transient override val partitioner = if (preservesPartitioning) prev.partitioner else None + override def getPartitions: Array[Partition] = { val random = new Random(seed) firstParent[T].partitions.map(x => new PartitionwiseSampledRDDPartition(x, random.nextLong())) 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 4e841bc992bff..daea2617e62ea 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -35,12 +35,13 @@ import org.apache.spark.Partitioner._ import org.apache.spark.SparkContext._ import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.api.java.JavaRDD +import org.apache.spark.broadcast.Broadcast import org.apache.spark.partial.BoundedDouble 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, CallSite, Utils} +import org.apache.spark.util.{BoundedPriorityQueue, Utils} import org.apache.spark.util.collection.OpenHashMap import org.apache.spark.util.random.{BernoulliSampler, PoissonSampler, SamplingUtils} @@ -328,19 +329,19 @@ abstract class RDD[T: ClassTag]( : RDD[T] = { if (shuffle) { /** Distributes elements evenly across output partitions, starting from a random partition. */ - def distributePartition(index: Int, items: Iterator[T]): Iterator[(Int, T)] = { + val distributePartition = (index: Int, items: Iterator[T]) => { var position = (new Random(index)).nextInt(numPartitions) items.map { t => - // Note that the hash code of the key will just be the key itself. The HashPartitioner + // Note that the hash code of the key will just be the key itself. The HashPartitioner // will mod it with the number of total partitions. position = position + 1 (position, t) } - } + } : Iterator[(Int, T)] // include a shuffle step so that our upstream tasks are still distributed new CoalescedRDD( - new ShuffledRDD[Int, T, T, (Int, T)](mapPartitionsWithIndex(distributePartition), + new ShuffledRDD[Int, T, T](mapPartitionsWithIndex(distributePartition), new HashPartitioner(numPartitions)), numPartitions).values } else { @@ -351,14 +352,14 @@ abstract class RDD[T: ClassTag]( /** * Return a sampled subset of this RDD. */ - def sample(withReplacement: Boolean, - fraction: Double, + def sample(withReplacement: Boolean, + fraction: Double, seed: Long = Utils.random.nextLong): RDD[T] = { - require(fraction >= 0.0, "Invalid fraction value: " + fraction) + require(fraction >= 0.0, "Negative fraction value: " + fraction) if (withReplacement) { - new PartitionwiseSampledRDD[T, T](this, new PoissonSampler[T](fraction), seed) + new PartitionwiseSampledRDD[T, T](this, new PoissonSampler[T](fraction), true, seed) } else { - new PartitionwiseSampledRDD[T, T](this, new BernoulliSampler[T](fraction), seed) + new PartitionwiseSampledRDD[T, T](this, new BernoulliSampler[T](fraction), true, seed) } } @@ -374,7 +375,7 @@ abstract class RDD[T: ClassTag]( val sum = weights.sum val normalizedCumWeights = weights.map(_ / sum).scanLeft(0.0d)(_ + _) normalizedCumWeights.sliding(2).map { x => - new PartitionwiseSampledRDD[T, T](this, new BernoulliSampler[T](x(0), x(1)), seed) + new PartitionwiseSampledRDD[T, T](this, new BernoulliSampler[T](x(0), x(1)), true, seed) }.toArray } @@ -509,6 +510,10 @@ abstract class RDD[T: ClassTag]( /** * Return an RDD of grouped items. Each group consists of a key and a sequence of elements * mapping to that key. + * + * Note: This operation may be very expensive. If you are grouping in order to perform an + * aggregation (such as a sum or average) over each key, using [[PairRDDFunctions.aggregateByKey]] + * or [[PairRDDFunctions.reduceByKey]] will provide much better performance. */ def groupBy[K](f: T => K)(implicit kt: ClassTag[K]): RDD[(K, Iterable[T])] = groupBy[K](f, defaultPartitioner(this)) @@ -516,6 +521,10 @@ abstract class RDD[T: ClassTag]( /** * Return an RDD of grouped elements. Each group consists of a key and a sequence of elements * mapping to that key. + * + * Note: This operation may be very expensive. If you are grouping in order to perform an + * aggregation (such as a sum or average) over each key, using [[PairRDDFunctions.aggregateByKey]] + * or [[PairRDDFunctions.reduceByKey]] will provide much better performance. */ def groupBy[K](f: T => K, numPartitions: Int)(implicit kt: ClassTag[K]): RDD[(K, Iterable[T])] = groupBy(f, new HashPartitioner(numPartitions)) @@ -523,6 +532,10 @@ abstract class RDD[T: ClassTag]( /** * Return an RDD of grouped items. Each group consists of a key and a sequence of elements * mapping to that key. + * + * Note: This operation may be very expensive. If you are grouping in order to perform an + * aggregation (such as a sum or average) over each key, using [[PairRDDFunctions.aggregateByKey]] + * or [[PairRDDFunctions.reduceByKey]] will provide much better performance. */ def groupBy[K](f: T => K, p: Partitioner)(implicit kt: ClassTag[K], ord: Ordering[K] = null) : RDD[(K, Iterable[T])] = { @@ -574,6 +587,9 @@ abstract class RDD[T: ClassTag]( /** * Return a new RDD by applying a function to each partition of this RDD. + * + * `preservesPartitioning` indicates whether the input function preserves the partitioner, which + * should be `false` unless this is a pair RDD and the input function doesn't modify the keys. */ def mapPartitions[U: ClassTag]( f: Iterator[T] => Iterator[U], preservesPartitioning: Boolean = false): RDD[U] = { @@ -584,6 +600,9 @@ abstract class RDD[T: ClassTag]( /** * Return a new RDD by applying a function to each partition of this RDD, while tracking the index * of the original partition. + * + * `preservesPartitioning` indicates whether the input function preserves the partitioner, which + * should be `false` unless this is a pair RDD and the input function doesn't modify the keys. */ def mapPartitionsWithIndex[U: ClassTag]( f: (Int, Iterator[T]) => Iterator[U], preservesPartitioning: Boolean = false): RDD[U] = { @@ -595,6 +614,9 @@ abstract class RDD[T: ClassTag]( * :: DeveloperApi :: * Return a new RDD by applying a function to each partition of this RDD. This is a variant of * mapPartitions that also passes the TaskContext into the closure. + * + * `preservesPartitioning` indicates whether the input function preserves the partitioner, which + * should be `false` unless this is a pair RDD and the input function doesn't modify the keys. */ @DeveloperApi def mapPartitionsWithContext[U: ClassTag]( @@ -677,7 +699,7 @@ abstract class RDD[T: ClassTag]( * a map on the other). */ def zip[U: ClassTag](other: RDD[U]): RDD[(T, U)] = { - zipPartitions(other, true) { (thisIter, otherIter) => + zipPartitions(other, preservesPartitioning = false) { (thisIter, otherIter) => new Iterator[(T, U)] { def hasNext = (thisIter.hasNext, otherIter.hasNext) match { case (true, true) => true @@ -733,14 +755,16 @@ abstract class RDD[T: ClassTag]( * Applies a function f to all elements of this RDD. */ def foreach(f: T => Unit) { - sc.runJob(this, (iter: Iterator[T]) => iter.foreach(f)) + val cleanF = sc.clean(f) + sc.runJob(this, (iter: Iterator[T]) => iter.foreach(cleanF)) } /** * Applies a function f to each partition of this RDD. */ def foreachPartition(f: Iterator[T] => Unit) { - sc.runJob(this, (iter: Iterator[T]) => f(iter)) + val cleanF = sc.clean(f) + sc.runJob(this, (iter: Iterator[T]) => cleanF(iter)) } /** @@ -907,19 +931,19 @@ abstract class RDD[T: ClassTag]( throw new SparkException("countByValue() does not support arrays") } // TODO: This should perhaps be distributed by default. - def countPartition(iter: Iterator[T]): Iterator[OpenHashMap[T,Long]] = { + val countPartition = (iter: Iterator[T]) => { val map = new OpenHashMap[T,Long] iter.foreach { t => map.changeValue(t, 1L, _ + 1L) } Iterator(map) - } - def mergeMaps(m1: OpenHashMap[T,Long], m2: OpenHashMap[T,Long]): OpenHashMap[T,Long] = { + }: Iterator[OpenHashMap[T,Long]] + val mergeMaps = (m1: OpenHashMap[T,Long], m2: OpenHashMap[T,Long]) => { m2.foreach { case (key, value) => m1.changeValue(key, value, _ + value) } m1 - } + }: OpenHashMap[T,Long] val myResult = mapPartitions(countPartition).reduce(mergeMaps) // Convert to a Scala mutable map val mutableResult = scala.collection.mutable.Map[T,Long]() @@ -980,7 +1004,7 @@ abstract class RDD[T: ClassTag]( }, (h1: HyperLogLogPlus, h2: HyperLogLogPlus) => { h1.addAll(h2) - h2 + h1 }).cardinality() } @@ -1183,16 +1207,12 @@ abstract class RDD[T: ClassTag]( /** * Return whether this RDD has been checkpointed or not */ - def isCheckpointed: Boolean = { - checkpointData.map(_.isCheckpointed).getOrElse(false) - } + def isCheckpointed: Boolean = checkpointData.exists(_.isCheckpointed) /** * Gets the name of the file to which this RDD was checkpointed */ - def getCheckpointFile: Option[String] = { - checkpointData.flatMap(_.getCheckpointFile) - } + def getCheckpointFile: Option[String] = checkpointData.flatMap(_.getCheckpointFile) // ======================================================================= // Other internal methods and fields @@ -1202,7 +1222,7 @@ abstract class RDD[T: ClassTag]( /** User code that created this RDD (e.g. `textFile`, `parallelize`). */ @transient private[spark] val creationSite = Utils.getCallSite - private[spark] def getCreationSite: String = Option(creationSite).map(_.short).getOrElse("") + private[spark] def getCreationSite: String = Option(creationSite).map(_.shortForm).getOrElse("") private[spark] def elementClassTag: ClassTag[T] = classTag[T] @@ -1213,9 +1233,31 @@ abstract class RDD[T: ClassTag]( dependencies.head.rdd.asInstanceOf[RDD[U]] } + /** Returns the jth parent RDD: e.g. rdd.parent[T](0) is equivalent to rdd.firstParent[T] */ + protected[spark] def parent[U: ClassTag](j: Int) = { + dependencies(j).rdd.asInstanceOf[RDD[U]] + } + /** The [[org.apache.spark.SparkContext]] that this RDD was created on. */ def context = sc + /** + * Private API for changing an RDD's ClassTag. + * Used for internal Java <-> Scala API compatibility. + */ + private[spark] def retag(cls: Class[T]): RDD[T] = { + val classTag: ClassTag[T] = ClassTag.apply(cls) + this.retag(classTag) + } + + /** + * Private API for changing an RDD's ClassTag. + * Used for internal Java <-> Scala API compatibility. + */ + private[spark] def retag(implicit classTag: ClassTag[T]): RDD[T] = { + this.mapPartitions(identity, preservesPartitioning = true)(classTag) + } + // Avoid handling doCheckpoint multiple times to prevent excessive recursion @transient private var doCheckpointCalled = false @@ -1257,11 +1299,75 @@ abstract class RDD[T: ClassTag]( /** A description of this RDD and its recursive dependencies for debugging. */ def toDebugString: String = { - def debugString(rdd: RDD[_], prefix: String = ""): Seq[String] = { - Seq(prefix + rdd + " (" + rdd.partitions.size + " partitions)") ++ - rdd.dependencies.flatMap(d => debugString(d.rdd, prefix + " ")) + // Get a debug description of an rdd without its children + def debugSelf (rdd: RDD[_]): Seq[String] = { + import Utils.bytesToString + + val persistence = storageLevel.description + val storageInfo = rdd.context.getRDDStorageInfo.filter(_.id == rdd.id).map(info => + " CachedPartitions: %d; MemorySize: %s; TachyonSize: %s; DiskSize: %s".format( + info.numCachedPartitions, bytesToString(info.memSize), + bytesToString(info.tachyonSize), bytesToString(info.diskSize))) + + s"$rdd [$persistence]" +: storageInfo + } + + // Apply a different rule to the last child + def debugChildren(rdd: RDD[_], prefix: String): Seq[String] = { + val len = rdd.dependencies.length + len match { + case 0 => Seq.empty + case 1 => + val d = rdd.dependencies.head + debugString(d.rdd, prefix, d.isInstanceOf[ShuffleDependency[_,_,_]], true) + case _ => + val frontDeps = rdd.dependencies.take(len - 1) + val frontDepStrings = frontDeps.flatMap( + d => debugString(d.rdd, prefix, d.isInstanceOf[ShuffleDependency[_,_,_]])) + + val lastDep = rdd.dependencies.last + val lastDepStrings = + debugString(lastDep.rdd, prefix, lastDep.isInstanceOf[ShuffleDependency[_,_,_]], true) + + (frontDepStrings ++ lastDepStrings) + } + } + // The first RDD in the dependency stack has no parents, so no need for a +- + def firstDebugString(rdd: RDD[_]): Seq[String] = { + val partitionStr = "(" + rdd.partitions.size + ")" + val leftOffset = (partitionStr.length - 1) / 2 + val nextPrefix = (" " * leftOffset) + "|" + (" " * (partitionStr.length - leftOffset)) + + debugSelf(rdd).zipWithIndex.map{ + case (desc: String, 0) => s"$partitionStr $desc" + case (desc: String, _) => s"$nextPrefix $desc" + } ++ debugChildren(rdd, nextPrefix) + } + def shuffleDebugString(rdd: RDD[_], prefix: String = "", isLastChild: Boolean): Seq[String] = { + val partitionStr = "(" + rdd.partitions.size + ")" + val leftOffset = (partitionStr.length - 1) / 2 + val thisPrefix = prefix.replaceAll("\\|\\s+$", "") + val nextPrefix = ( + thisPrefix + + (if (isLastChild) " " else "| ") + + (" " * leftOffset) + "|" + (" " * (partitionStr.length - leftOffset))) + + debugSelf(rdd).zipWithIndex.map{ + case (desc: String, 0) => s"$thisPrefix+-$partitionStr $desc" + case (desc: String, _) => s"$nextPrefix$desc" + } ++ debugChildren(rdd, nextPrefix) + } + def debugString(rdd: RDD[_], + prefix: String = "", + isShuffle: Boolean = true, + isLastChild: Boolean = false): Seq[String] = { + if (isShuffle) { + shuffleDebugString(rdd, prefix, isLastChild) + } else { + debugSelf(rdd).map(prefix + _) ++ debugChildren(rdd, prefix) + } } - debugString(this).mkString("\n") + firstDebugString(this).mkString("\n") } override def toString: String = "%s%s[%d] at %s".format( diff --git a/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala b/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala index c3b2a33fb54d0..f67e5f1857979 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala @@ -106,7 +106,6 @@ private[spark] class RDDCheckpointData[T: ClassTag](@transient rdd: RDD[T]) cpRDD = Some(newRDD) rdd.markCheckpointed(newRDD) // Update the RDD's dependencies and partitions cpState = Checkpointed - RDDCheckpointData.clearTaskCaches() } logInfo("Done checkpointing RDD " + rdd.id + " to " + path + ", new parent is RDD " + newRDD.id) } @@ -131,9 +130,5 @@ private[spark] class RDDCheckpointData[T: ClassTag](@transient rdd: RDD[T]) } } -private[spark] object RDDCheckpointData { - def clearTaskCaches() { - ShuffleMapTask.clearCache() - ResultTask.clearCache() - } -} +// Used for synchronization +private[spark] object RDDCheckpointData 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 bf02f68d0d3d3..d9fe6847254fa 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala @@ -37,11 +37,12 @@ private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { * @tparam V the value class. * @tparam C the combiner class. */ +// TODO: Make this return RDD[Product2[K, C]] or have some way to configure mutable pairs @DeveloperApi -class ShuffledRDD[K, V, C, P <: Product2[K, C] : ClassTag]( +class ShuffledRDD[K, V, C]( @transient var prev: RDD[_ <: Product2[K, V]], part: Partitioner) - extends RDD[P](prev.context, Nil) { + extends RDD[(K, C)](prev.context, Nil) { private var serializer: Option[Serializer] = None @@ -52,25 +53,25 @@ class ShuffledRDD[K, V, C, P <: Product2[K, C] : ClassTag]( private var mapSideCombine: Boolean = false /** Set a serializer for this RDD's shuffle, or null to use the default (spark.serializer) */ - def setSerializer(serializer: Serializer): ShuffledRDD[K, V, C, P] = { + def setSerializer(serializer: Serializer): ShuffledRDD[K, V, C] = { this.serializer = Option(serializer) this } /** Set key ordering for RDD's shuffle. */ - def setKeyOrdering(keyOrdering: Ordering[K]): ShuffledRDD[K, V, C, P] = { + def setKeyOrdering(keyOrdering: Ordering[K]): ShuffledRDD[K, V, C] = { this.keyOrdering = Option(keyOrdering) this } /** Set aggregator for RDD's shuffle. */ - def setAggregator(aggregator: Aggregator[K, V, C]): ShuffledRDD[K, V, C, P] = { + def setAggregator(aggregator: Aggregator[K, V, C]): ShuffledRDD[K, V, C] = { this.aggregator = Option(aggregator) this } /** Set mapSideCombine flag for RDD's shuffle. */ - def setMapSideCombine(mapSideCombine: Boolean): ShuffledRDD[K, V, C, P] = { + def setMapSideCombine(mapSideCombine: Boolean): ShuffledRDD[K, V, C] = { this.mapSideCombine = mapSideCombine this } @@ -85,11 +86,11 @@ class ShuffledRDD[K, V, C, P <: Product2[K, C] : ClassTag]( Array.tabulate[Partition](part.numPartitions)(i => new ShuffledRDDPartition(i)) } - override def compute(split: Partition, context: TaskContext): Iterator[P] = { + override def compute(split: Partition, context: TaskContext): Iterator[(K, C)] = { val dep = dependencies.head.asInstanceOf[ShuffleDependency[K, V, C]] SparkEnv.get.shuffleManager.getReader(dep.shuffleHandle, split.index, split.index + 1, context) .read() - .asInstanceOf[Iterator[P]] + .asInstanceOf[Iterator[(K, C)]] } override def clearDependencies() { diff --git a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala index 21c6e07d69f90..0c97eb0aaa51f 100644 --- a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala @@ -25,21 +25,32 @@ import scala.reflect.ClassTag import org.apache.spark.{Dependency, Partition, RangeDependency, SparkContext, TaskContext} import org.apache.spark.annotation.DeveloperApi -private[spark] class UnionPartition[T: ClassTag](idx: Int, rdd: RDD[T], splitIndex: Int) +/** + * Partition for UnionRDD. + * + * @param idx index of the partition + * @param rdd the parent RDD this partition refers to + * @param parentRddIndex index of the parent RDD this partition refers to + * @param parentRddPartitionIndex index of the partition within the parent RDD + * this partition refers to + */ +private[spark] class UnionPartition[T: ClassTag]( + idx: Int, + @transient rdd: RDD[T], + val parentRddIndex: Int, + @transient parentRddPartitionIndex: Int) extends Partition { - var split: Partition = rdd.partitions(splitIndex) - - def iterator(context: TaskContext) = rdd.iterator(split, context) + var parentPartition: Partition = rdd.partitions(parentRddPartitionIndex) - def preferredLocations() = rdd.preferredLocations(split) + def preferredLocations() = rdd.preferredLocations(parentPartition) override val index: Int = idx @throws(classOf[IOException]) private def writeObject(oos: ObjectOutputStream) { // Update the reference to parent split at the time of task serialization - split = rdd.partitions(splitIndex) + parentPartition = rdd.partitions(parentRddPartitionIndex) oos.defaultWriteObject() } } @@ -47,14 +58,14 @@ private[spark] class UnionPartition[T: ClassTag](idx: Int, rdd: RDD[T], splitInd @DeveloperApi class UnionRDD[T: ClassTag]( sc: SparkContext, - @transient var rdds: Seq[RDD[T]]) + var rdds: Seq[RDD[T]]) extends RDD[T](sc, Nil) { // Nil since we implement getDependencies override def getPartitions: Array[Partition] = { val array = new Array[Partition](rdds.map(_.partitions.size).sum) var pos = 0 - for (rdd <- rdds; split <- rdd.partitions) { - array(pos) = new UnionPartition(pos, rdd, split.index) + for ((rdd, rddIndex) <- rdds.zipWithIndex; split <- rdd.partitions) { + array(pos) = new UnionPartition(pos, rdd, rddIndex, split.index) pos += 1 } array @@ -70,9 +81,16 @@ class UnionRDD[T: ClassTag]( deps } - override def compute(s: Partition, context: TaskContext): Iterator[T] = - s.asInstanceOf[UnionPartition[T]].iterator(context) + override def compute(s: Partition, context: TaskContext): Iterator[T] = { + val part = s.asInstanceOf[UnionPartition[T]] + parent[T](part.parentRddIndex).iterator(part.parentPartition, context) + } override def getPreferredLocations(s: Partition): Seq[String] = s.asInstanceOf[UnionPartition[T]].preferredLocations() + + override def clearDependencies() { + super.clearDependencies() + rdds = null + } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/AccumulableInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/AccumulableInfo.scala new file mode 100644 index 0000000000000..fa83372bb4d11 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/AccumulableInfo.scala @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.scheduler + +import org.apache.spark.annotation.DeveloperApi + +/** + * :: DeveloperApi :: + * Information about an [[org.apache.spark.Accumulable]] modified during a task or stage. + */ +@DeveloperApi +class AccumulableInfo ( + val id: Long, + val name: String, + val update: Option[String], // represents a partial update within a task + val value: String) { + + override def equals(other: Any): Boolean = other match { + case acc: AccumulableInfo => + this.id == acc.id && this.name == acc.name && + this.update == acc.update && this.value == acc.value + case _ => false + } +} + +object AccumulableInfo { + def apply(id: Long, name: String, update: Option[String], value: String) = + new AccumulableInfo(id, name, update, value) + + def apply(id: Long, name: String, value: String) = new AccumulableInfo(id, name, None, value) +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala b/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala index cd5d44ad4a7e6..162158babc35b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala @@ -29,7 +29,7 @@ private[spark] class ApplicationEventListener extends SparkListener { var startTime = -1L var endTime = -1L var viewAcls = "" - var enableViewAcls = false + var adminAcls = "" def applicationStarted = startTime != -1 @@ -55,7 +55,7 @@ private[spark] class ApplicationEventListener extends SparkListener { val environmentDetails = environmentUpdate.environmentDetails val allProperties = environmentDetails("Spark Properties").toMap viewAcls = allProperties.getOrElse("spark.ui.view.acls", "") - enableViewAcls = allProperties.getOrElse("spark.ui.acls.enable", "false").toBoolean + adminAcls = allProperties.getOrElse("spark.admin.acls", "") } } } 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 f72bfde572c96..b86cfbfa48fbe 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -17,11 +17,11 @@ package org.apache.spark.scheduler -import java.io.{NotSerializableException, PrintWriter, StringWriter} +import java.io.NotSerializableException import java.util.Properties import java.util.concurrent.atomic.AtomicInteger -import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map} +import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map, Stack} import scala.concurrent.Await import scala.concurrent.duration._ import scala.language.postfixOps @@ -29,17 +29,18 @@ import scala.reflect.ClassTag import scala.util.control.NonFatal import akka.actor._ -import akka.actor.OneForOneStrategy import akka.actor.SupervisorStrategy.Stop import akka.pattern.ask import akka.util.Timeout import org.apache.spark._ +import org.apache.spark.broadcast.Broadcast 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.storage._ import org.apache.spark.util.{CallSite, SystemClock, Clock, Utils} +import org.apache.spark.storage.BlockManagerMessages.BlockManagerHeartbeat /** * The high-level scheduling layer that implements stage-oriented scheduling. It computes a DAG of @@ -85,12 +86,9 @@ class DAGScheduler( private val nextStageId = new AtomicInteger(0) private[scheduler] val jobIdToStageIds = new HashMap[Int, HashSet[Int]] - private[scheduler] val stageIdToJobIds = new HashMap[Int, HashSet[Int]] private[scheduler] val stageIdToStage = new HashMap[Int, Stage] private[scheduler] val shuffleToMapStage = new HashMap[Int, Stage] private[scheduler] val jobIdToActiveJob = new HashMap[Int, ActiveJob] - private[scheduler] val resultStageToJob = new HashMap[Stage, ActiveJob] - private[scheduler] val stageToInfos = new HashMap[Stage, StageInfo] // Stages we need to run whose parents aren't done private[scheduler] val waitingStages = new HashSet[Stage] @@ -101,9 +99,6 @@ class DAGScheduler( // Stages that must be resubmitted due to fetch failures private[scheduler] val failedStages = new HashSet[Stage] - // Missing tasks from each stage - private[scheduler] val pendingTasks = new HashMap[Stage, HashSet[Task[_]]] - private[scheduler] val activeJobs = new HashSet[ActiveJob] // Contains the locations that each RDD's partitions are cached on @@ -120,8 +115,15 @@ class DAGScheduler( private val dagSchedulerActorSupervisor = env.actorSystem.actorOf(Props(new DAGSchedulerActorSupervisor(this))) + // A closure serializer that we reuse. + // This is only safe because DAGScheduler runs in a single thread. + private val closureSerializer = SparkEnv.get.closureSerializer.newInstance() + private[scheduler] var eventProcessActor: ActorRef = _ + /** If enabled, we may run certain actions like take() and first() locally. */ + private val localExecutionEnabled = sc.getConf.getBoolean("spark.localExecution.enabled", false) + private def initializeEventProcessActor() { // blocking the thread until supervisor is started, which ensures eventProcessActor is // not null before any job is submitted @@ -155,6 +157,23 @@ class DAGScheduler( eventProcessActor ! CompletionEvent(task, reason, result, accumUpdates, taskInfo, taskMetrics) } + /** + * Update metrics for in-progress tasks and let the master know that the BlockManager is still + * alive. Return true if the driver knows about the given block manager. Otherwise, return false, + * indicating that the block manager should re-register. + */ + def executorHeartbeatReceived( + execId: String, + taskMetrics: Array[(Long, Int, TaskMetrics)], // (taskId, stageId, metrics) + blockManagerId: BlockManagerId): Boolean = { + listenerBus.post(SparkListenerExecutorMetricsUpdate(execId, taskMetrics)) + implicit val timeout = Timeout(600 seconds) + + Await.result( + blockManagerMaster.driverActor ? BlockManagerHeartbeat(blockManagerId), + timeout.duration).asInstanceOf[Boolean] + } + // Called by TaskScheduler when an executor fails. def executorLost(execId: String) { eventProcessActor ! ExecutorLost(execId) @@ -195,11 +214,15 @@ class DAGScheduler( shuffleToMapStage.get(shuffleDep.shuffleId) match { case Some(stage) => stage case None => + // We are going to register ancestor shuffle dependencies + registerShuffleDependencies(shuffleDep, jobId) + // Then register current shuffleDep val stage = newOrUsedStage( shuffleDep.rdd, shuffleDep.rdd.partitions.size, shuffleDep, jobId, shuffleDep.rdd.creationSite) shuffleToMapStage(shuffleDep.shuffleId) = stage + stage } } @@ -223,7 +246,6 @@ class DAGScheduler( new Stage(id, rdd, numTasks, shuffleDep, getParentStages(rdd, jobId), jobId, callSite) stageIdToStage(id) = stage updateJobIdStageIdMaps(jobId, stage) - stageToInfos(stage) = StageInfo.fromStage(stage) stage } @@ -265,6 +287,9 @@ class DAGScheduler( private def getParentStages(rdd: RDD[_], jobId: Int): List[Stage] = { val parents = new HashSet[Stage] val visited = new HashSet[RDD[_]] + // We are manually maintaining a stack here to prevent StackOverflowError + // caused by recursively visiting + val waitingForVisit = new Stack[RDD[_]] def visit(r: RDD[_]) { if (!visited(r)) { visited += r @@ -275,18 +300,69 @@ class DAGScheduler( case shufDep: ShuffleDependency[_, _, _] => parents += getShuffleMapStage(shufDep, jobId) case _ => - visit(dep.rdd) + waitingForVisit.push(dep.rdd) } } } } - visit(rdd) + waitingForVisit.push(rdd) + while (!waitingForVisit.isEmpty) { + visit(waitingForVisit.pop()) + } parents.toList } + // Find ancestor missing shuffle dependencies and register into shuffleToMapStage + private def registerShuffleDependencies(shuffleDep: ShuffleDependency[_, _, _], jobId: Int) = { + val parentsWithNoMapStage = getAncestorShuffleDependencies(shuffleDep.rdd) + while (!parentsWithNoMapStage.isEmpty) { + val currentShufDep = parentsWithNoMapStage.pop() + val stage = + newOrUsedStage( + currentShufDep.rdd, currentShufDep.rdd.partitions.size, currentShufDep, jobId, + currentShufDep.rdd.creationSite) + shuffleToMapStage(currentShufDep.shuffleId) = stage + } + } + + // Find ancestor shuffle dependencies that are not registered in shuffleToMapStage yet + private def getAncestorShuffleDependencies(rdd: RDD[_]): Stack[ShuffleDependency[_, _, _]] = { + val parents = new Stack[ShuffleDependency[_, _, _]] + val visited = new HashSet[RDD[_]] + // We are manually maintaining a stack here to prevent StackOverflowError + // caused by recursively visiting + val waitingForVisit = new Stack[RDD[_]] + def visit(r: RDD[_]) { + if (!visited(r)) { + visited += r + for (dep <- r.dependencies) { + dep match { + case shufDep: ShuffleDependency[_, _, _] => + if (!shuffleToMapStage.contains(shufDep.shuffleId)) { + parents.push(shufDep) + } + + waitingForVisit.push(shufDep.rdd) + case _ => + waitingForVisit.push(dep.rdd) + } + } + } + } + + waitingForVisit.push(rdd) + while (!waitingForVisit.isEmpty) { + visit(waitingForVisit.pop()) + } + parents + } + private def getMissingParentStages(stage: Stage): List[Stage] = { val missing = new HashSet[Stage] val visited = new HashSet[RDD[_]] + // We are manually maintaining a stack here to prevent StackOverflowError + // caused by recursively visiting + val waitingForVisit = new Stack[RDD[_]] def visit(rdd: RDD[_]) { if (!visited(rdd)) { visited += rdd @@ -299,13 +375,16 @@ class DAGScheduler( missing += mapStage } case narrowDep: NarrowDependency[_] => - visit(narrowDep.rdd) + waitingForVisit.push(narrowDep.rdd) } } } } } - visit(stage.rdd) + waitingForVisit.push(stage.rdd) + while (!waitingForVisit.isEmpty) { + visit(waitingForVisit.pop()) + } missing.toList } @@ -315,13 +394,12 @@ class DAGScheduler( */ private def updateJobIdStageIdMaps(jobId: Int, stage: Stage) { def updateJobIdStageIdMapsList(stages: List[Stage]) { - if (!stages.isEmpty) { + if (stages.nonEmpty) { val s = stages.head - stageIdToJobIds.getOrElseUpdate(s.id, new HashSet[Int]()) += jobId + s.jobIds += jobId jobIdToStageIds.getOrElseUpdate(jobId, new HashSet[Int]()) += s.id - val parents = getParentStages(s.rdd, jobId) - val parentsWithoutThisJobId = parents.filter(p => - !stageIdToJobIds.get(p.id).exists(_.contains(jobId))) + val parents: List[Stage] = getParentStages(s.rdd, jobId) + val parentsWithoutThisJobId = parents.filter { ! _.jobIds.contains(jobId) } updateJobIdStageIdMapsList(parentsWithoutThisJobId ++ stages.tail) } } @@ -333,16 +411,15 @@ class DAGScheduler( * handle cancelling tasks or notifying the SparkListener about finished jobs/stages/tasks. * * @param job The job whose state to cleanup. - * @param resultStage Specifies the result stage for the job; if set to None, this method - * searches resultStagesToJob to find and cleanup the appropriate result stage. */ - private def cleanupStateForJobAndIndependentStages(job: ActiveJob, resultStage: Option[Stage]) { + private def cleanupStateForJobAndIndependentStages(job: ActiveJob) { val registeredStages = jobIdToStageIds.get(job.jobId) if (registeredStages.isEmpty || registeredStages.get.isEmpty) { logError("No stages registered for job " + job.jobId) } else { - stageIdToJobIds.filterKeys(stageId => registeredStages.get.contains(stageId)).foreach { - case (stageId, jobSet) => + stageIdToStage.filterKeys(stageId => registeredStages.get.contains(stageId)).foreach { + case (stageId, stage) => + val jobSet = stage.jobIds if (!jobSet.contains(job.jobId)) { logError( "Job %d not registered for stage %d even though that stage was registered for the job" @@ -355,14 +432,9 @@ class DAGScheduler( logDebug("Removing running stage %d".format(stageId)) runningStages -= stage } - stageToInfos -= stage for ((k, v) <- shuffleToMapStage.find(_._2 == stage)) { shuffleToMapStage.remove(k) } - if (pendingTasks.contains(stage) && !pendingTasks(stage).isEmpty) { - logDebug("Removing pending status for stage %d".format(stageId)) - } - pendingTasks -= stage if (waitingStages.contains(stage)) { logDebug("Removing stage %d from waiting set.".format(stageId)) waitingStages -= stage @@ -374,10 +446,6 @@ class DAGScheduler( } // data structures based on StageId stageIdToStage -= stageId - stageIdToJobIds -= stageId - - ShuffleMapTask.removeStage(stageId) - ResultTask.removeStage(stageId) logDebug("After removal of stage %d, remaining stages = %d" .format(stageId, stageIdToStage.size)) @@ -393,19 +461,7 @@ class DAGScheduler( jobIdToStageIds -= job.jobId jobIdToActiveJob -= job.jobId activeJobs -= job - - if (resultStage.isEmpty) { - // Clean up result stages. - val resultStagesForJob = resultStageToJob.keySet.filter( - stage => resultStageToJob(stage).jobId == job.jobId) - if (resultStagesForJob.size != 1) { - logWarning( - s"${resultStagesForJob.size} result stages for job ${job.jobId} (expect exactly 1)") - } - resultStageToJob --= resultStagesForJob - } else { - resultStageToJob -= resultStage.get - } + job.finalStage.resultOfJob = None } /** @@ -455,7 +511,7 @@ class DAGScheduler( waiter.awaitResult() match { case JobSucceeded => {} case JobFailed(exception: Exception) => - logInfo("Failed to run " + callSite.short) + logInfo("Failed to run " + callSite.shortForm) throw exception } } @@ -578,7 +634,7 @@ class DAGScheduler( val result = job.func(taskContext, rdd.iterator(split, taskContext)) job.listener.taskSucceeded(0, result) } finally { - taskContext.executeOnCompleteCallbacks() + taskContext.markTaskCompleted() } } catch { case e: Exception => @@ -591,9 +647,10 @@ class DAGScheduler( job.listener.jobFailed(exception) } finally { val s = job.finalStage - stageIdToJobIds -= s.id // clean up data structures that were populated for a local job, - stageIdToStage -= s.id // but that won't get cleaned up via the normal paths through - stageToInfos -= s // completion events or stage abort + // clean up data structures that were populated for a local job, + // but that won't get cleaned up via the normal paths through + // completion events or stage abort + stageIdToStage -= s.id jobIdToStageIds -= job.jobId listenerBus.post(SparkListenerJobEnd(job.jobId, jobResult)) } @@ -605,12 +662,8 @@ class DAGScheduler( // That should take care of at least part of the priority inversion problem with // cross-job dependencies. private def activeJobForStage(stage: Stage): Option[Int] = { - if (stageIdToJobIds.contains(stage.id)) { - val jobsThatUseStage: Array[Int] = stageIdToJobIds(stage.id).toArray.sorted - jobsThatUseStage.find(jobIdToActiveJob.contains) - } else { - None - } + val jobsThatUseStage: Array[Int] = stage.jobIds.toArray.sorted + jobsThatUseStage.find(jobIdToActiveJob.contains) } private[scheduler] def handleJobGroupCancelled(groupId: String) { @@ -642,9 +695,8 @@ class DAGScheduler( // is in the process of getting stopped. val stageFailedMessage = "Stage cancelled because SparkContext was shut down" runningStages.foreach { stage => - val info = stageToInfos(stage) - info.stageFailed(stageFailedMessage) - listenerBus.post(SparkListenerStageCompleted(info)) + stage.info.stageFailed(stageFailedMessage) + listenerBus.post(SparkListenerStageCompleted(stage.info)) } listenerBus.post(SparkListenerJobEnd(job.jobId, JobFailed(error))) } @@ -679,18 +731,20 @@ 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.short, partitions.length, allowLocal)) + job.jobId, callSite.shortForm, partitions.length, allowLocal)) logInfo("Final stage: " + finalStage + "(" + finalStage.name + ")") logInfo("Parents of final stage: " + finalStage.parents) logInfo("Missing parents: " + getMissingParentStages(finalStage)) - if (allowLocal && finalStage.parents.size == 0 && partitions.length == 1) { + val shouldRunLocally = + localExecutionEnabled && allowLocal && finalStage.parents.isEmpty && partitions.length == 1 + if (shouldRunLocally) { // Compute very short actions like first() or take() with no parent stages locally. listenerBus.post(SparkListenerJobStart(job.jobId, Array[Int](), properties)) runLocally(job) } else { jobIdToActiveJob(jobId) = job activeJobs += job - resultStageToJob(finalStage) = job + finalStage.resultOfJob = Some(job) listenerBus.post(SparkListenerJobStart(job.jobId, jobIdToStageIds(jobId).toArray, properties)) submitStage(finalStage) @@ -710,7 +764,6 @@ class DAGScheduler( if (missing == Nil) { logInfo("Submitting " + stage + " (" + stage.rdd + "), which has no missing parents") submitMissingTasks(stage, jobId.get) - runningStages += stage } else { for (parent <- missing) { submitStage(parent) @@ -723,47 +776,83 @@ class DAGScheduler( } } - /** Called when stage's parents are available and we can now do its task. */ private def submitMissingTasks(stage: Stage, jobId: Int) { logDebug("submitMissingTasks(" + stage + ")") // Get our pending tasks and remember them in our pendingTasks entry - val myPending = pendingTasks.getOrElseUpdate(stage, new HashSet) - myPending.clear() + stage.pendingTasks.clear() var tasks = ArrayBuffer[Task[_]]() + + val properties = if (jobIdToActiveJob.contains(jobId)) { + jobIdToActiveJob(stage.jobId).properties + } else { + // this stage will be assigned to "default" pool + null + } + + runningStages += stage + // SparkListenerStageSubmitted should be posted before testing whether tasks are + // serializable. If tasks are not serializable, a SparkListenerStageCompleted event + // will be posted, which should always come after a corresponding SparkListenerStageSubmitted + // event. + listenerBus.post(SparkListenerStageSubmitted(stage.info, properties)) + + // TODO: Maybe we can keep the taskBinary in Stage to avoid serializing it multiple times. + // Broadcasted binary for the task, used to dispatch tasks to executors. Note that we broadcast + // the serialized copy of the RDD and for each task we will deserialize it, which means each + // task gets a different copy of the RDD. This provides stronger isolation between tasks that + // might modify state of objects referenced in their closures. This is necessary in Hadoop + // where the JobConf/Configuration object is not thread-safe. + var taskBinary: Broadcast[Array[Byte]] = null + try { + // For ShuffleMapTask, serialize and broadcast (rdd, shuffleDep). + // For ResultTask, serialize and broadcast (rdd, func). + val taskBinaryBytes: Array[Byte] = + if (stage.isShuffleMap) { + closureSerializer.serialize((stage.rdd, stage.shuffleDep.get) : AnyRef).array() + } else { + closureSerializer.serialize((stage.rdd, stage.resultOfJob.get.func) : AnyRef).array() + } + taskBinary = sc.broadcast(taskBinaryBytes) + } catch { + // In the case of a failure during serialization, abort the stage. + case e: NotSerializableException => + abortStage(stage, "Task not serializable: " + e.toString) + runningStages -= stage + return + case NonFatal(e) => + abortStage(stage, s"Task serialization failed: $e\n${e.getStackTraceString}") + runningStages -= stage + return + } + if (stage.isShuffleMap) { for (p <- 0 until stage.numPartitions if stage.outputLocs(p) == Nil) { val locs = getPreferredLocs(stage.rdd, p) - tasks += new ShuffleMapTask(stage.id, stage.rdd, stage.shuffleDep.get, p, locs) + val part = stage.rdd.partitions(p) + tasks += new ShuffleMapTask(stage.id, taskBinary, part, locs) } } else { // This is a final stage; figure out its job's missing partitions - val job = resultStageToJob(stage) + val job = stage.resultOfJob.get for (id <- 0 until job.numPartitions if !job.finished(id)) { - val partition = job.partitions(id) - val locs = getPreferredLocs(stage.rdd, partition) - tasks += new ResultTask(stage.id, stage.rdd, job.func, partition, locs, id) + val p: Int = job.partitions(id) + val part = stage.rdd.partitions(p) + val locs = getPreferredLocs(stage.rdd, p) + tasks += new ResultTask(stage.id, taskBinary, part, locs, id) } } - val properties = if (jobIdToActiveJob.contains(jobId)) { - jobIdToActiveJob(stage.jobId).properties - } else { - // this stage will be assigned to "default" pool - null - } - - // must be run listener before possible NotSerializableException - // should be "StageSubmitted" first and then "JobEnded" - listenerBus.post(SparkListenerStageSubmitted(stageToInfos(stage), properties)) - if (tasks.size > 0) { // Preemptively serialize a task to make sure it can be serialized. We are catching this // exception here because it would be fairly hard to catch the non-serializable exception // down the road, where we have several different implementations for local scheduler and // cluster schedulers. + // + // We've already serialized RDDs and closures in taskBinary, but here we check for all other + // objects such as Partition. try { - SparkEnv.get.closureSerializer.newInstance().serialize(tasks.head) + closureSerializer.serialize(tasks.head) } catch { case e: NotSerializableException => abortStage(stage, "Task not serializable: " + e.toString) @@ -776,12 +865,15 @@ class DAGScheduler( } logInfo("Submitting " + tasks.size + " missing tasks from " + stage + " (" + stage.rdd + ")") - myPending ++= tasks - logDebug("New pending tasks: " + myPending) + stage.pendingTasks ++= tasks + logDebug("New pending tasks: " + stage.pendingTasks) taskScheduler.submitTasks( new TaskSet(tasks.toArray, stage.id, stage.newAttemptId(), stage.jobId, properties)) - stageToInfos(stage).submissionTime = Some(clock.getTime()) + stage.info.submissionTime = Some(clock.getTime()) } else { + // Because we posted SparkListenerStageSubmitted earlier, we should post + // SparkListenerStageCompleted here in case there are no tasks to run. + listenerBus.post(SparkListenerStageCompleted(stage.info)) logDebug("Stage " + stage + " is actually done; %b %d %d".format( stage.isAvailable, stage.numAvailableOutputs, stage.numPartitions)) runningStages -= stage @@ -796,8 +888,14 @@ class DAGScheduler( val task = event.task val stageId = task.stageId val taskType = Utils.getFormattedClassName(task) - listenerBus.post(SparkListenerTaskEnd(stageId, taskType, event.reason, event.taskInfo, - event.taskMetrics)) + + // The success case is dealt with separately below, since we need to compute accumulator + // updates before posting. + if (event.reason != Success) { + listenerBus.post(SparkListenerTaskEnd(stageId, taskType, event.reason, event.taskInfo, + event.taskMetrics)) + } + if (!stageIdToStage.contains(task.stageId)) { // Skip all the actions if the stage has been cancelled. return @@ -805,26 +903,44 @@ class DAGScheduler( val stage = stageIdToStage(task.stageId) def markStageAsFinished(stage: Stage) = { - val serviceTime = stageToInfos(stage).submissionTime match { + val serviceTime = stage.info.submissionTime match { 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(clock.getTime()) - listenerBus.post(SparkListenerStageCompleted(stageToInfos(stage))) + stage.info.completionTime = Some(clock.getTime()) + listenerBus.post(SparkListenerStageCompleted(stage.info)) runningStages -= stage } event.reason match { case Success => - logInfo("Completed " + task) if (event.accumUpdates != null) { - // TODO: fail the stage if the accumulator update fails... - Accumulators.add(event.accumUpdates) // TODO: do this only if task wasn't resubmitted + try { + Accumulators.add(event.accumUpdates) + event.accumUpdates.foreach { case (id, partialValue) => + val acc = Accumulators.originals(id).asInstanceOf[Accumulable[Any, Any]] + // To avoid UI cruft, ignore cases where value wasn't updated + if (acc.name.isDefined && partialValue != acc.zero) { + val name = acc.name.get + val stringPartialValue = Accumulators.stringifyPartialValue(partialValue) + val stringValue = Accumulators.stringifyValue(acc.value) + stage.info.accumulables(id) = AccumulableInfo(id, name, stringValue) + event.taskInfo.accumulables += + AccumulableInfo(id, name, Some(stringPartialValue), stringValue) + } + } + } catch { + // If we see an exception during accumulator update, just log the error and move on. + case e: Exception => + logError(s"Failed to update accumulators for $task", e) + } } - pendingTasks(stage) -= task + listenerBus.post(SparkListenerTaskEnd(stageId, taskType, event.reason, event.taskInfo, + event.taskMetrics)) + stage.pendingTasks -= task task match { case rt: ResultTask[_, _] => - resultStageToJob.get(stage) match { + stage.resultOfJob match { case Some(job) => if (!job.finished(rt.outputId)) { job.finished(rt.outputId) = true @@ -832,7 +948,7 @@ class DAGScheduler( // If the whole job has finished, remove it if (job.numFinished == job.numPartitions) { markStageAsFinished(stage) - cleanupStateForJobAndIndependentStages(job, Some(stage)) + cleanupStateForJobAndIndependentStages(job) listenerBus.post(SparkListenerJobEnd(job.jobId, JobSucceeded)) } @@ -859,7 +975,7 @@ class DAGScheduler( } else { stage.addOutputLoc(smt.partitionId, status) } - if (runningStages.contains(stage) && pendingTasks(stage).isEmpty) { + if (runningStages.contains(stage) && stage.pendingTasks.isEmpty) { markStageAsFinished(stage) logInfo("looking for newly runnable stages") logInfo("running: " + runningStages) @@ -908,7 +1024,7 @@ class DAGScheduler( case Resubmitted => logInfo("Resubmitted " + task + ", so marking it as still running") - pendingTasks(stage) += task + stage.pendingTasks += task case FetchFailed(bmAddress, shuffleId, mapId, reduceId) => // Mark the stage that the reducer was in as unrunnable @@ -993,13 +1109,14 @@ class DAGScheduler( } private[scheduler] def handleStageCancellation(stageId: Int) { - if (stageIdToJobIds.contains(stageId)) { - val jobsThatUseStage: Array[Int] = stageIdToJobIds(stageId).toArray - jobsThatUseStage.foreach(jobId => { - handleJobCancellation(jobId, "because Stage %s was cancelled".format(stageId)) - }) - } else { - logInfo("No active jobs to kill for Stage " + stageId) + stageIdToStage.get(stageId) match { + case Some(stage) => + val jobsThatUseStage: Array[Int] = stage.jobIds.toArray + jobsThatUseStage.foreach { jobId => + handleJobCancellation(jobId, s"because Stage $stageId was cancelled") + } + case None => + logInfo("No active jobs to kill for Stage " + stageId) } submitWaitingStages() } @@ -1008,8 +1125,8 @@ class DAGScheduler( if (!jobIdToStageIds.contains(jobId)) { logDebug("Trying to cancel unregistered job " + jobId) } else { - failJobAndIndependentStages(jobIdToActiveJob(jobId), - "Job %d cancelled %s".format(jobId, reason), None) + failJobAndIndependentStages( + jobIdToActiveJob(jobId), "Job %d cancelled %s".format(jobId, reason)) } submitWaitingStages() } @@ -1023,26 +1140,21 @@ class DAGScheduler( // Skip all the actions if the stage has been removed. return } - val dependentStages = resultStageToJob.keys.filter(x => stageDependsOn(x, failedStage)).toSeq - stageToInfos(failedStage).completionTime = Some(clock.getTime()) - for (resultStage <- dependentStages) { - val job = resultStageToJob(resultStage) - failJobAndIndependentStages(job, s"Job aborted due to stage failure: $reason", - Some(resultStage)) + val dependentJobs: Seq[ActiveJob] = + activeJobs.filter(job => stageDependsOn(job.finalStage, failedStage)).toSeq + failedStage.info.completionTime = Some(clock.getTime()) + for (job <- dependentJobs) { + failJobAndIndependentStages(job, s"Job aborted due to stage failure: $reason") } - if (dependentStages.isEmpty) { + if (dependentJobs.isEmpty) { logInfo("Ignoring failure of " + failedStage + " because all jobs depending on it are done") } } /** * Fails a job and all stages that are only used by that job, and cleans up relevant state. - * - * @param resultStage The result stage for the job, if known. Used to cleanup state for the job - * slightly more efficiently than when not specified. */ - private def failJobAndIndependentStages(job: ActiveJob, failureReason: String, - resultStage: Option[Stage]) { + private def failJobAndIndependentStages(job: ActiveJob, failureReason: String) { val error = new SparkException(failureReason) var ableToCancelStages = true @@ -1056,7 +1168,7 @@ class DAGScheduler( logError("No stages registered for job " + job.jobId) } stages.foreach { stageId => - val jobsForStage = stageIdToJobIds.get(stageId) + val jobsForStage: Option[HashSet[Int]] = stageIdToStage.get(stageId).map(_.jobIds) if (jobsForStage.isEmpty || !jobsForStage.get.contains(job.jobId)) { logError( "Job %d not registered for stage %d even though that stage was registered for the job" @@ -1070,9 +1182,8 @@ class DAGScheduler( if (runningStages.contains(stage)) { try { // cancelTasks will fail if a SchedulerBackend does not implement killTask taskScheduler.cancelTasks(stageId, shouldInterruptThread) - val stageInfo = stageToInfos(stage) - stageInfo.stageFailed(failureReason) - listenerBus.post(SparkListenerStageCompleted(stageToInfos(stage))) + stage.info.stageFailed(failureReason) + listenerBus.post(SparkListenerStageCompleted(stage.info)) } catch { case e: UnsupportedOperationException => logInfo(s"Could not cancel tasks for stage $stageId", e) @@ -1085,7 +1196,7 @@ class DAGScheduler( if (ableToCancelStages) { job.listener.jobFailed(error) - cleanupStateForJobAndIndependentStages(job, resultStage) + cleanupStateForJobAndIndependentStages(job) listenerBus.post(SparkListenerJobEnd(job.jobId, JobFailed(error))) } } @@ -1099,6 +1210,9 @@ class DAGScheduler( } val visitedRdds = new HashSet[RDD[_]] val visitedStages = new HashSet[Stage] + // We are manually maintaining a stack here to prevent StackOverflowError + // caused by recursively visiting + val waitingForVisit = new Stack[RDD[_]] def visit(rdd: RDD[_]) { if (!visitedRdds(rdd)) { visitedRdds += rdd @@ -1108,15 +1222,18 @@ class DAGScheduler( val mapStage = getShuffleMapStage(shufDep, stage.jobId) if (!mapStage.isAvailable) { visitedStages += mapStage - visit(mapStage.rdd) + waitingForVisit.push(mapStage.rdd) } // Otherwise there's no need to follow the dependency back case narrowDep: NarrowDependency[_] => - visit(narrowDep.rdd) + waitingForVisit.push(narrowDep.rdd) } } } } - visit(stage.rdd) + waitingForVisit.push(stage.rdd) + while (!waitingForVisit.isEmpty) { + visit(waitingForVisit.pop()) + } visitedRdds.contains(target.rdd) } @@ -1128,6 +1245,22 @@ class DAGScheduler( */ private[spark] def getPreferredLocs(rdd: RDD[_], partition: Int): Seq[TaskLocation] = synchronized { + getPreferredLocsInternal(rdd, partition, new HashSet) + } + + /** Recursive implementation for getPreferredLocs. */ + private def getPreferredLocsInternal( + rdd: RDD[_], + partition: Int, + visited: HashSet[(RDD[_],Int)]) + : Seq[TaskLocation] = + { + // If the partition has already been visited, no need to re-visit. + // This avoids exponential path exploration. SPARK-695 + if (!visited.add((rdd,partition))) { + // Nil has already been returned for previously visited partitions. + return Nil + } // If the partition is cached, return the cache locations val cached = getCacheLocs(rdd)(partition) if (!cached.isEmpty) { @@ -1144,7 +1277,7 @@ class DAGScheduler( rdd.dependencies.foreach { case n: NarrowDependency[_] => for (inPart <- n.getParents(partition)) { - val locs = getPreferredLocs(n.rdd, inPart) + val locs = getPreferredLocsInternal(n.rdd, inPart, visited) if (locs != Nil) { return locs } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala index 5878e733908f5..94944399b134a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala @@ -24,8 +24,8 @@ import org.apache.spark.metrics.source.Source private[spark] class DAGSchedulerSource(val dagScheduler: DAGScheduler, sc: SparkContext) extends Source { - val metricRegistry = new MetricRegistry() - val sourceName = "%s.DAGScheduler".format(sc.appName) + override val metricRegistry = new MetricRegistry() + override val sourceName = "%s.DAGScheduler".format(sc.appName) metricRegistry.register(MetricRegistry.name("stage", "failedStages"), new Gauge[Int] { override def getValue: Int = dagScheduler.failedStages.size diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index a90b0d475c04e..7378ce923f0ae 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -29,7 +29,7 @@ import org.json4s.jackson.JsonMethods._ import org.apache.spark.{Logging, SparkConf, SparkContext} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.io.CompressionCodec -import org.apache.spark.util.{FileLogger, JsonProtocol} +import org.apache.spark.util.{FileLogger, JsonProtocol, Utils} /** * A SparkListener that logs events to persistent storage. @@ -55,7 +55,7 @@ private[spark] class EventLoggingListener( private val outputBufferSize = sparkConf.getInt("spark.eventLog.buffer.kb", 100) * 1024 private val logBaseDir = sparkConf.get("spark.eventLog.dir", DEFAULT_LOG_DIR).stripSuffix("/") private val name = appName.replaceAll("[ :/]", "-").toLowerCase + "-" + System.currentTimeMillis - val logDir = logBaseDir + "/" + name + val logDir = Utils.resolveURI(logBaseDir) + "/" + name.stripSuffix("/") protected val logger = new FileLogger(logDir, sparkConf, hadoopConf, outputBufferSize, shouldCompress, shouldOverwrite, Some(LOG_FILE_PERMISSIONS)) @@ -63,6 +63,13 @@ private[spark] class EventLoggingListener( // For testing. Keep track of all JSON serialized events that have been logged. private[scheduler] val loggedEvents = new ArrayBuffer[JValue] + /** + * Return only the unique application directory without the base directory. + */ + def getApplicationLogDir(): String = { + name + } + /** * Begin logging events. * If compression is used, log a file that indicates which compression library is used. @@ -120,6 +127,8 @@ private[spark] class EventLoggingListener( logEvent(event, flushLogger = true) override def onApplicationEnd(event: SparkListenerApplicationEnd) = logEvent(event, flushLogger = true) + // No-op because logging every update would be overkill + override def onExecutorMetricsUpdate(event: SparkListenerExecutorMetricsUpdate) { } /** * Stop logging events. @@ -208,7 +217,7 @@ private[spark] object EventLoggingListener extends Logging { } catch { case e: Exception => logError("Exception in parsing logging info from directory %s".format(logDir), e) - EventLoggingInfo.empty + EventLoggingInfo.empty } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala index 47dd112f68325..4d6b5c81883b6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala @@ -162,6 +162,7 @@ class JobLogger(val user: String, val logDirName: String) extends SparkListener " START_TIME=" + taskInfo.launchTime + " FINISH_TIME=" + taskInfo.finishTime + " EXECUTOR_ID=" + taskInfo.executorId + " HOST=" + taskMetrics.hostname val executorRunTime = " EXECUTOR_RUN_TIME=" + taskMetrics.executorRunTime + val gcTime = " GC_TIME=" + taskMetrics.jvmGCTime val inputMetrics = taskMetrics.inputMetrics match { case Some(metrics) => " READ_METHOD=" + metrics.readMethod.toString + @@ -179,11 +180,13 @@ class JobLogger(val user: String, val logDirName: String) extends SparkListener case None => "" } val writeMetrics = taskMetrics.shuffleWriteMetrics match { - case Some(metrics) => " SHUFFLE_BYTES_WRITTEN=" + metrics.shuffleBytesWritten + case Some(metrics) => + " SHUFFLE_BYTES_WRITTEN=" + metrics.shuffleBytesWritten + + " SHUFFLE_WRITE_TIME=" + metrics.shuffleWriteTime case None => "" } - stageLogInfo(stageId, status + info + executorRunTime + inputMetrics + shuffleReadMetrics + - writeMetrics) + stageLogInfo(stageId, status + info + executorRunTime + gcTime + inputMetrics + + shuffleReadMetrics + writeMetrics) } /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala index bbf9f7388b074..2ccbd8edeb028 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala @@ -17,134 +17,56 @@ package org.apache.spark.scheduler -import scala.language.existentials +import java.nio.ByteBuffer import java.io._ -import java.util.zip.{GZIPInputStream, GZIPOutputStream} - -import scala.collection.mutable.HashMap import org.apache.spark._ -import org.apache.spark.rdd.{RDD, RDDCheckpointData} - -private[spark] object ResultTask { - - // A simple map between the stage id to the serialized byte array of a task. - // Served as a cache for task serialization because serialization can be - // 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[_], func: (TaskContext, Iterator[_]) => _): Array[Byte] = - { - synchronized { - val old = serializedInfoCache.get(stageId).orNull - if (old != null) { - old - } else { - val out = new ByteArrayOutputStream - val ser = SparkEnv.get.closureSerializer.newInstance() - val objOut = ser.serializeStream(new GZIPOutputStream(out)) - objOut.writeObject(rdd) - objOut.writeObject(func) - objOut.close() - val bytes = out.toByteArray - serializedInfoCache.put(stageId, bytes) - bytes - } - } - } - - def deserializeInfo(stageId: Int, bytes: Array[Byte]): (RDD[_], (TaskContext, Iterator[_]) => _) = - { - 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 func = objIn.readObject().asInstanceOf[(TaskContext, Iterator[_]) => _] - (rdd, func) - } - - def removeStage(stageId: Int) { - serializedInfoCache.remove(stageId) - } - - def clearCache() { - synchronized { - serializedInfoCache.clear() - } - } -} - +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.rdd.RDD /** * A task that sends back the output to the driver application. * - * See [[org.apache.spark.scheduler.Task]] for more information. + * See [[Task]] for more information. * * @param stageId id of the stage this task belongs to - * @param rdd input to func - * @param func a function to apply on a partition of the RDD - * @param _partitionId index of the number in the RDD + * @param taskBinary broadcasted version of the serialized RDD and the function to apply on each + * partition of the given RDD. Once deserialized, the type should be + * (RDD[T], (TaskContext, Iterator[T]) => U). + * @param partition partition of the RDD this task is associated with * @param locs preferred task execution locations for locality scheduling * @param outputId index of the task in this job (a job can launch tasks on only a subset of the * input RDD's partitions). */ private[spark] class ResultTask[T, U]( stageId: Int, - var rdd: RDD[T], - var func: (TaskContext, Iterator[T]) => U, - _partitionId: Int, + taskBinary: Broadcast[Array[Byte]], + partition: Partition, @transient locs: Seq[TaskLocation], - var outputId: Int) - extends Task[U](stageId, _partitionId) with Externalizable { + val outputId: Int) + extends Task[U](stageId, partition.index) with Serializable { - def this() = this(0, null, null, 0, null, 0) - - var split = if (rdd == null) null else rdd.partitions(partitionId) - - @transient private val preferredLocs: Seq[TaskLocation] = { + @transient private[this] val preferredLocs: Seq[TaskLocation] = { if (locs == null) Nil else locs.toSet.toSeq } override def runTask(context: TaskContext): U = { + // Deserialize the RDD and the func using the broadcast variables. + val ser = SparkEnv.get.closureSerializer.newInstance() + val (rdd, func) = ser.deserialize[(RDD[T], (TaskContext, Iterator[T]) => U)]( + ByteBuffer.wrap(taskBinary.value), Thread.currentThread.getContextClassLoader) + metrics = Some(context.taskMetrics) try { - func(context, rdd.iterator(split, context)) + func(context, rdd.iterator(partition, context)) } finally { - context.executeOnCompleteCallbacks() + context.markTaskCompleted() } } + // This is only callable on the driver side. override def preferredLocations: Seq[TaskLocation] = preferredLocs override def toString = "ResultTask(" + stageId + ", " + partitionId + ")" - - override def writeExternal(out: ObjectOutput) { - RDDCheckpointData.synchronized { - split = rdd.partitions(partitionId) - out.writeInt(stageId) - val bytes = ResultTask.serializeInfo( - stageId, rdd, func.asInstanceOf[(TaskContext, Iterator[_]) => _]) - out.writeInt(bytes.length) - out.write(bytes) - out.writeInt(partitionId) - out.writeInt(outputId) - out.writeLong(epoch) - out.writeObject(split) - } - } - - override def readExternal(in: ObjectInput) { - val stageId = in.readInt() - val numBytes = in.readInt() - val bytes = new Array[Byte](numBytes) - in.readFully(bytes) - val (rdd_, func_) = ResultTask.deserializeInfo(stageId, bytes) - rdd = rdd_.asInstanceOf[RDD[T]] - func = func_.asInstanceOf[(TaskContext, Iterator[T]) => U] - partitionId = in.readInt() - outputId = in.readInt() - epoch = in.readLong() - split = in.readObject().asInstanceOf[Partition] - } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala index 6a6d8e609bc39..e41e0a9841691 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala @@ -30,4 +30,5 @@ private[spark] trait SchedulerBackend { def killTask(taskId: Long, executorId: String, interruptThread: Boolean): Unit = throw new UnsupportedOperationException + def isReady(): Boolean = true } 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 fdaf1de83f051..381eff2147e95 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala @@ -17,134 +17,55 @@ package org.apache.spark.scheduler -import scala.language.existentials - -import java.io._ -import java.util.zip.{GZIPInputStream, GZIPOutputStream} +import java.nio.ByteBuffer -import scala.collection.mutable.HashMap +import scala.language.existentials import org.apache.spark._ -import org.apache.spark.rdd.{RDD, RDDCheckpointData} +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.rdd.RDD import org.apache.spark.shuffle.ShuffleWriter -private[spark] object ShuffleMapTask { - - // A simple map between the stage id to the serialized byte array of a task. - // Served as a cache for task serialization because serialization can be - // 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] = { - synchronized { - val old = serializedInfoCache.get(stageId).orNull - if (old != null) { - return old - } else { - val out = new ByteArrayOutputStream - val ser = SparkEnv.get.closureSerializer.newInstance() - val objOut = ser.serializeStream(new GZIPOutputStream(out)) - objOut.writeObject(rdd) - objOut.writeObject(dep) - objOut.close() - val bytes = out.toByteArray - serializedInfoCache.put(stageId, bytes) - bytes - } - } - } - - 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[_, _, _]] - (rdd, dep) - } - - // Since both the JarSet and FileSet have the same format this is used for both. - def deserializeFileSet(bytes: Array[Byte]): HashMap[String, Long] = { - val in = new GZIPInputStream(new ByteArrayInputStream(bytes)) - val objIn = new ObjectInputStream(in) - val set = objIn.readObject().asInstanceOf[Array[(String, Long)]].toMap - HashMap(set.toSeq: _*) - } - - def removeStage(stageId: Int) { - serializedInfoCache.remove(stageId) - } - - def clearCache() { - synchronized { - serializedInfoCache.clear() - } - } -} - /** - * A ShuffleMapTask divides the elements of an RDD into multiple buckets (based on a partitioner - * specified in the ShuffleDependency). - * - * See [[org.apache.spark.scheduler.Task]] for more information. - * +* A ShuffleMapTask divides the elements of an RDD into multiple buckets (based on a partitioner +* specified in the ShuffleDependency). +* +* See [[org.apache.spark.scheduler.Task]] for more information. +* * @param stageId id of the stage this task belongs to - * @param rdd the final RDD in this stage - * @param dep the ShuffleDependency - * @param _partitionId index of the number in the RDD + * @param taskBinary broadcast version of of the RDD and the ShuffleDependency. Once deserialized, + * the type should be (RDD[_], ShuffleDependency[_, _, _]). + * @param partition partition of the RDD this task is associated with * @param locs preferred task execution locations for locality scheduling */ private[spark] class ShuffleMapTask( stageId: Int, - var rdd: RDD[_], - var dep: ShuffleDependency[_, _, _], - _partitionId: Int, + taskBinary: Broadcast[Array[Byte]], + partition: Partition, @transient private var locs: Seq[TaskLocation]) - extends Task[MapStatus](stageId, _partitionId) - with Externalizable - with Logging { + extends Task[MapStatus](stageId, partition.index) with Logging { - protected def this() = this(0, null, null, 0, null) + /** A constructor used only in test suites. This does not require passing in an RDD. */ + def this(partitionId: Int) { + this(0, null, new Partition { override def index = 0 }, null) + } @transient private val preferredLocs: Seq[TaskLocation] = { if (locs == null) Nil else locs.toSet.toSeq } - var split = if (rdd == null) null else rdd.partitions(partitionId) - - override def writeExternal(out: ObjectOutput) { - RDDCheckpointData.synchronized { - split = rdd.partitions(partitionId) - out.writeInt(stageId) - val bytes = ShuffleMapTask.serializeInfo(stageId, rdd, dep) - out.writeInt(bytes.length) - out.write(bytes) - out.writeInt(partitionId) - out.writeLong(epoch) - out.writeObject(split) - } - } - - override def readExternal(in: ObjectInput) { - val stageId = in.readInt() - val numBytes = in.readInt() - val bytes = new Array[Byte](numBytes) - in.readFully(bytes) - val (rdd_, dep_) = ShuffleMapTask.deserializeInfo(stageId, bytes) - rdd = rdd_ - dep = dep_ - partitionId = in.readInt() - epoch = in.readLong() - split = in.readObject().asInstanceOf[Partition] - } - override def runTask(context: TaskContext): MapStatus = { + // Deserialize the RDD using the broadcast variable. + val ser = SparkEnv.get.closureSerializer.newInstance() + val (rdd, dep) = ser.deserialize[(RDD[_], ShuffleDependency[_, _, _])]( + ByteBuffer.wrap(taskBinary.value), Thread.currentThread.getContextClassLoader) + metrics = Some(context.taskMetrics) var writer: ShuffleWriter[Any, Any] = null try { val manager = SparkEnv.get.shuffleManager writer = manager.getWriter[Any, Any](dep.shuffleHandle, partitionId, context) - writer.write(rdd.iterator(split, context).asInstanceOf[Iterator[_ <: Product2[Any, Any]]]) + writer.write(rdd.iterator(partition, context).asInstanceOf[Iterator[_ <: Product2[Any, Any]]]) return writer.stop(success = true).get } catch { case e: Exception => @@ -153,7 +74,7 @@ private[spark] class ShuffleMapTask( } throw e } finally { - context.executeOnCompleteCallbacks() + context.markTaskCompleted() } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index 82163eadd56e9..d01d318633877 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -75,6 +75,12 @@ case class SparkListenerBlockManagerRemoved(blockManagerId: BlockManagerId) @DeveloperApi case class SparkListenerUnpersistRDD(rddId: Int) extends SparkListenerEvent +@DeveloperApi +case class SparkListenerExecutorMetricsUpdate( + execId: String, + taskMetrics: Seq[(Long, Int, TaskMetrics)]) + extends SparkListenerEvent + @DeveloperApi case class SparkListenerApplicationStart(appName: String, time: Long, sparkUser: String) extends SparkListenerEvent @@ -158,6 +164,11 @@ trait SparkListener { * Called when the application ends */ def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd) { } + + /** + * Called when the driver receives task metrics from an executor in a heartbeat. + */ + def onExecutorMetricsUpdate(executorMetricsUpdate: SparkListenerExecutorMetricsUpdate) { } } /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala index ed9fb24bc8ce8..e79ffd7a3587d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala @@ -68,6 +68,8 @@ private[spark] trait SparkListenerBus extends Logging { foreachListener(_.onApplicationStart(applicationStart)) case applicationEnd: SparkListenerApplicationEnd => foreachListener(_.onApplicationEnd(applicationEnd)) + case metricsUpdate: SparkListenerExecutorMetricsUpdate => + foreachListener(_.onExecutorMetricsUpdate(metricsUpdate)) case SparkListenerShutdown => } } 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 8ec482a6f6d9c..800905413d145 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala @@ -17,6 +17,8 @@ package org.apache.spark.scheduler +import scala.collection.mutable.HashSet + import org.apache.spark._ import org.apache.spark.rdd.RDD import org.apache.spark.storage.BlockManagerId @@ -56,8 +58,22 @@ private[spark] class Stage( val numPartitions = rdd.partitions.size val outputLocs = Array.fill[List[MapStatus]](numPartitions)(Nil) var numAvailableOutputs = 0 + + /** Set of jobs that this stage belongs to. */ + val jobIds = new HashSet[Int] + + /** For stages that are the final (consists of only ResultTasks), link to the ActiveJob. */ + var resultOfJob: Option[ActiveJob] = None + var pendingTasks = new HashSet[Task[_]] + private var nextAttemptId = 0 + val name = callSite.shortForm + val details = callSite.longForm + + /** Pointer to the [StageInfo] object, set by DAGScheduler. */ + var info: StageInfo = StageInfo.fromStage(this) + def isAvailable: Boolean = { if (!isShuffleMap) { true @@ -108,9 +124,6 @@ private[spark] class Stage( def attemptId: Int = nextAttemptId - val name = callSite.short - val details = callSite.long - override def toString = "Stage " + id override def hashCode(): Int = 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 480891550eb60..2a407e47a05bd 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala @@ -17,6 +17,8 @@ package org.apache.spark.scheduler +import scala.collection.mutable.HashMap + import org.apache.spark.annotation.DeveloperApi import org.apache.spark.storage.RDDInfo @@ -37,6 +39,8 @@ class StageInfo( var completionTime: Option[Long] = None /** If the stage failed, the reason why. */ var failureReason: Option[String] = None + /** Terminal values of accumulables updated during this stage. */ + val accumulables = HashMap[Long, AccumulableInfo]() def stageFailed(reason: String) { failureReason = Some(reason) diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index 5871edeb856ad..6aa0cca06878d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -26,6 +26,8 @@ import org.apache.spark.TaskContext import org.apache.spark.executor.TaskMetrics import org.apache.spark.serializer.SerializerInstance import org.apache.spark.util.ByteBufferInputStream +import org.apache.spark.util.Utils + /** * A unit of execution. We have two kinds of Task's in Spark: @@ -44,6 +46,7 @@ private[spark] abstract class Task[T](val stageId: Int, var partitionId: Int) ex final def run(attemptId: Long): T = { context = new TaskContext(stageId, partitionId, attemptId, runningLocally = false) + context.taskMetrics.hostname = Utils.localHostName() taskThread = Thread.currentThread() if (_killed) { kill(interruptThread = false) @@ -84,7 +87,7 @@ private[spark] abstract class Task[T](val stageId: Int, var partitionId: Int) ex def kill(interruptThread: Boolean) { _killed = true if (context != null) { - context.interrupted = true + context.markInterrupted() } if (interruptThread && taskThread != null) { taskThread.interrupt() diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala index 29de0453ac19a..6fa1f2c880f7a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala @@ -17,6 +17,8 @@ package org.apache.spark.scheduler +import scala.collection.mutable.ListBuffer + import org.apache.spark.annotation.DeveloperApi /** @@ -41,6 +43,13 @@ class TaskInfo( */ var gettingResultTime: Long = 0 + /** + * Intermediate updates to accumulables during this task. Note that it is valid for the same + * accumulable to be updated multiple times in a single task or for two accumulables with the + * same name but different IDs to exist in a task. + */ + val accumulables = ListBuffer[AccumulableInfo]() + /** * The time when the task has completed successfully (including the time to remotely fetch * results, if necessary). @@ -84,6 +93,8 @@ class TaskInfo( } } + def id: String = s"$index.$attempt" + def duration: Long = { if (!finished) { throw new UnsupportedOperationException("duration() called on unfinished task") diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala index eb920ab0c0b67..f176d09816f5e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala @@ -22,7 +22,7 @@ import org.apache.spark.annotation.DeveloperApi @DeveloperApi object TaskLocality extends Enumeration { // Process local is expected to be used ONLY within TaskSetManager for now. - val PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL, ANY = Value + val PROCESS_LOCAL, NODE_LOCAL, NO_PREF, RACK_LOCAL, ANY = Value type TaskLocality = Value diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala index 819c35257b5a7..1a0b877c8a5e1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala @@ -18,6 +18,8 @@ package org.apache.spark.scheduler import org.apache.spark.scheduler.SchedulingMode.SchedulingMode +import org.apache.spark.executor.TaskMetrics +import org.apache.spark.storage.BlockManagerId /** * Low-level task scheduler interface, currently implemented exclusively by TaskSchedulerImpl. @@ -54,4 +56,12 @@ private[spark] trait TaskScheduler { // Get the default level of parallelism to use in the cluster, as a hint for sizing jobs. def defaultParallelism(): Int + + /** + * Update metrics for in-progress tasks and let the master know that the BlockManager is still + * alive. Return true if the driver knows about the given block manager. Otherwise, return false, + * indicating that the block manager should re-register. + */ + def executorHeartbeatReceived(execId: String, taskMetrics: Array[(Long, TaskMetrics)], + blockManagerId: BlockManagerId): Boolean } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 5ed2803d76afc..6c0d1b2752a81 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -32,6 +32,9 @@ import org.apache.spark._ import org.apache.spark.TaskState.TaskState import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.util.Utils +import org.apache.spark.executor.TaskMetrics +import org.apache.spark.storage.BlockManagerId +import akka.actor.Props /** * Schedules tasks for multiple types of clusters by acting through a SchedulerBackend. @@ -86,9 +89,11 @@ private[spark] class TaskSchedulerImpl( // The set of executors we have on each host; this is used to compute hostsAlive, which // in turn is used to decide when we can attain data locality on a given host - private val executorsByHost = new HashMap[String, HashSet[String]] + protected val executorsByHost = new HashMap[String, HashSet[String]] - private val executorIdToHost = new HashMap[String, String] + protected val hostsByRack = new HashMap[String, HashSet[String]] + + protected val executorIdToHost = new HashMap[String, String] // Listener object to pass upcalls into var dagScheduler: DAGScheduler = null @@ -145,6 +150,10 @@ private[spark] class TaskSchedulerImpl( } } + override def postStartHook() { + waitBackendReady() + } + override def submitTasks(taskSet: TaskSet) { val tasks = taskSet.tasks logInfo("Adding task set " + taskSet.id + " with " + tasks.length + " tasks") @@ -219,6 +228,9 @@ private[spark] class TaskSchedulerImpl( executorAdded(o.executorId, o.host) newExecAvail = true } + for (rack <- getRackForHost(o.host)) { + hostsByRack.getOrElseUpdate(rack, new HashSet[String]()) += o.host + } } // Randomly shuffle offers to avoid always placing tasks on the same set of workers. @@ -237,6 +249,7 @@ private[spark] class TaskSchedulerImpl( // Take each TaskSet in our scheduling order, and then offer it each node in increasing order // of locality levels so that it gets a chance to launch local tasks on all of them. + // NOTE: the preferredLocality order: PROCESS_LOCAL, NODE_LOCAL, NO_PREF, RACK_LOCAL, ANY var launchedTask = false for (taskSet <- sortedTaskSets; maxLocality <- taskSet.myLocalityLevels) { do { @@ -253,7 +266,7 @@ private[spark] class TaskSchedulerImpl( activeExecutorIds += execId executorsByHost(host) += execId availableCpus(i) -= CPUS_PER_TASK - assert (availableCpus(i) >= 0) + assert(availableCpus(i) >= 0) launchedTask = true } } @@ -311,6 +324,26 @@ private[spark] class TaskSchedulerImpl( } } + /** + * Update metrics for in-progress tasks and let the master know that the BlockManager is still + * alive. Return true if the driver knows about the given block manager. Otherwise, return false, + * indicating that the block manager should re-register. + */ + override def executorHeartbeatReceived( + execId: String, + taskMetrics: Array[(Long, TaskMetrics)], // taskId -> TaskMetrics + blockManagerId: BlockManagerId): Boolean = { + val metricsWithStageIds = taskMetrics.flatMap { + case (id, metrics) => { + taskIdToTaskSetId.get(id) + .flatMap(activeTaskSets.get) + .map(_.stageId) + .map(x => (id, x, metrics)) + } + } + dagScheduler.executorHeartbeatReceived(execId, metricsWithStageIds, blockManagerId) + } + def handleTaskGettingResult(taskSetManager: TaskSetManager, tid: Long) { taskSetManager.handleTaskGettingResult(tid) } @@ -414,6 +447,12 @@ private[spark] class TaskSchedulerImpl( execs -= executorId if (execs.isEmpty) { executorsByHost -= host + for (rack <- getRackForHost(host); hosts <- hostsByRack.get(rack)) { + hosts -= host + if (hosts.isEmpty) { + hostsByRack -= rack + } + } } executorIdToHost -= executorId rootPool.executorLost(executorId, host) @@ -431,12 +470,27 @@ private[spark] class TaskSchedulerImpl( executorsByHost.contains(host) } + def hasHostAliveOnRack(rack: String): Boolean = synchronized { + hostsByRack.contains(rack) + } + def isExecutorAlive(execId: String): Boolean = synchronized { activeExecutorIds.contains(execId) } // By default, rack is unknown def getRackForHost(value: String): Option[String] = None + + private def waitBackendReady(): Unit = { + if (backend.isReady) { + return + } + while (!backend.isReady) { + synchronized { + this.wait(100) + } + } + } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 059cc9085a2e7..d9d53faf843ff 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -26,8 +26,7 @@ import scala.collection.mutable.HashSet import scala.math.max import scala.math.min -import org.apache.spark.{ExceptionFailure, ExecutorLostFailure, FetchFailed, Logging, Resubmitted, - SparkEnv, Success, TaskEndReason, TaskKilled, TaskResultLost, TaskState} +import org.apache.spark._ import org.apache.spark.TaskState.TaskState import org.apache.spark.executor.TaskMetrics import org.apache.spark.util.{Clock, SystemClock} @@ -52,8 +51,8 @@ private[spark] class TaskSetManager( val taskSet: TaskSet, val maxTaskFailures: Int, clock: Clock = SystemClock) - extends Schedulable with Logging -{ + extends Schedulable with Logging { + val conf = sched.sc.conf /* @@ -80,6 +79,7 @@ private[spark] class TaskSetManager( private val numFailures = new Array[Int](numTasks) // key is taskId, value is a Map of executor id to when it failed private val failedExecutors = new HashMap[Int, HashMap[String, Long]]() + val taskAttempts = Array.fill[List[TaskInfo]](numTasks)(Nil) var tasksSuccessful = 0 @@ -180,24 +180,17 @@ private[spark] class TaskSetManager( } } - var hadAliveLocations = false for (loc <- tasks(index).preferredLocations) { for (execId <- loc.executorId) { addTo(pendingTasksForExecutor.getOrElseUpdate(execId, new ArrayBuffer)) } - if (sched.hasExecutorsAliveOnHost(loc.host)) { - hadAliveLocations = true - } addTo(pendingTasksForHost.getOrElseUpdate(loc.host, new ArrayBuffer)) for (rack <- sched.getRackForHost(loc.host)) { addTo(pendingTasksForRack.getOrElseUpdate(rack, new ArrayBuffer)) - hadAliveLocations = true } } - if (!hadAliveLocations) { - // Even though the task might've had preferred locations, all of those hosts or executors - // are dead; put it in the no-prefs list so we can schedule it elsewhere right away. + if (tasks(index).preferredLocations == Nil) { addTo(pendingTasksWithNoPrefs) } @@ -238,7 +231,6 @@ private[spark] class TaskSetManager( */ private def findTaskFromList(execId: String, list: ArrayBuffer[Int]): Option[Int] = { var indexOffset = list.size - while (indexOffset > 0) { indexOffset -= 1 val index = list(indexOffset) @@ -287,12 +279,12 @@ private[spark] class TaskSetManager( !hasAttemptOnHost(index, host) && !executorIsBlacklisted(execId, index) if (!speculatableTasks.isEmpty) { - // Check for process-local or preference-less tasks; note that tasks can be process-local + // Check for process-local tasks; note that tasks can be process-local // on multiple nodes when we replicate cached blocks, as in Spark Streaming for (index <- speculatableTasks if canRunOnHost(index)) { val prefs = tasks(index).preferredLocations val executors = prefs.flatMap(_.executorId) - if (prefs.size == 0 || executors.contains(execId)) { + if (executors.contains(execId)) { speculatableTasks -= index return Some((index, TaskLocality.PROCESS_LOCAL)) } @@ -309,6 +301,17 @@ private[spark] class TaskSetManager( } } + // Check for no-preference tasks + if (TaskLocality.isAllowed(locality, TaskLocality.NO_PREF)) { + for (index <- speculatableTasks if canRunOnHost(index)) { + val locations = tasks(index).preferredLocations + if (locations.size == 0) { + speculatableTasks -= index + return Some((index, TaskLocality.PROCESS_LOCAL)) + } + } + } + // Check for rack-local tasks if (TaskLocality.isAllowed(locality, TaskLocality.RACK_LOCAL)) { for (rack <- sched.getRackForHost(host)) { @@ -340,20 +343,27 @@ private[spark] class TaskSetManager( * * @return An option containing (task index within the task set, locality, is speculative?) */ - private def findTask(execId: String, host: String, locality: TaskLocality.Value) + private def findTask(execId: String, host: String, maxLocality: TaskLocality.Value) : Option[(Int, TaskLocality.Value, Boolean)] = { for (index <- findTaskFromList(execId, getPendingTasksForExecutor(execId))) { return Some((index, TaskLocality.PROCESS_LOCAL, false)) } - if (TaskLocality.isAllowed(locality, TaskLocality.NODE_LOCAL)) { + if (TaskLocality.isAllowed(maxLocality, TaskLocality.NODE_LOCAL)) { for (index <- findTaskFromList(execId, getPendingTasksForHost(host))) { return Some((index, TaskLocality.NODE_LOCAL, false)) } } - if (TaskLocality.isAllowed(locality, TaskLocality.RACK_LOCAL)) { + if (TaskLocality.isAllowed(maxLocality, TaskLocality.NO_PREF)) { + // Look for noPref tasks after NODE_LOCAL for minimize cross-rack traffic + for (index <- findTaskFromList(execId, pendingTasksWithNoPrefs)) { + return Some((index, TaskLocality.PROCESS_LOCAL, false)) + } + } + + if (TaskLocality.isAllowed(maxLocality, TaskLocality.RACK_LOCAL)) { for { rack <- sched.getRackForHost(host) index <- findTaskFromList(execId, getPendingTasksForRack(rack)) @@ -362,25 +372,27 @@ private[spark] class TaskSetManager( } } - // Look for no-pref tasks after rack-local tasks since they can run anywhere. - for (index <- findTaskFromList(execId, pendingTasksWithNoPrefs)) { - return Some((index, TaskLocality.PROCESS_LOCAL, false)) - } - - if (TaskLocality.isAllowed(locality, TaskLocality.ANY)) { + if (TaskLocality.isAllowed(maxLocality, TaskLocality.ANY)) { for (index <- findTaskFromList(execId, allPendingTasks)) { return Some((index, TaskLocality.ANY, false)) } } - // Finally, if all else has failed, find a speculative task - findSpeculativeTask(execId, host, locality).map { case (taskIndex, allowedLocality) => - (taskIndex, allowedLocality, true) - } + // find a speculative task if all others tasks have been scheduled + findSpeculativeTask(execId, host, maxLocality).map { + case (taskIndex, allowedLocality) => (taskIndex, allowedLocality, true)} } /** * Respond to an offer of a single executor from the scheduler by finding a task + * + * NOTE: this function is either called with a maxLocality which + * would be adjusted by delay scheduling algorithm or it will be with a special + * NO_PREF locality which will be not modified + * + * @param execId the executor Id of the offered resource + * @param host the host Id of the offered resource + * @param maxLocality the maximum locality we want to schedule the tasks at */ def resourceOffer( execId: String, @@ -391,9 +403,14 @@ private[spark] class TaskSetManager( if (!isZombie) { val curTime = clock.getTime() - var allowedLocality = getAllowedLocalityLevel(curTime) - if (allowedLocality > maxLocality) { - allowedLocality = maxLocality // We're not allowed to search for farther-away tasks + var allowedLocality = maxLocality + + if (maxLocality != TaskLocality.NO_PREF) { + allowedLocality = getAllowedLocalityLevel(curTime) + if (allowedLocality > maxLocality) { + // We're not allowed to search for farther-away tasks + allowedLocality = maxLocality + } } findTask(execId, host, allowedLocality) match { @@ -401,19 +418,19 @@ private[spark] class TaskSetManager( // Found a task; do some bookkeeping and return a task description val task = tasks(index) val taskId = sched.newTaskId() - // Figure out whether this should count as a preferred launch - logInfo("Starting task %s:%d as TID %s on executor %s: %s (%s)".format( - taskSet.id, index, taskId, execId, host, taskLocality)) // Do various bookkeeping copiesRunning(index) += 1 val attemptNum = taskAttempts(index).size - val info = new TaskInfo( - taskId, index, attemptNum + 1, curTime, execId, host, taskLocality, speculative) + val info = new TaskInfo(taskId, index, attemptNum, curTime, + execId, host, taskLocality, speculative) taskInfos(taskId) = info taskAttempts(index) = info :: taskAttempts(index) // Update our locality level for delay scheduling - currentLocalityIndex = getLocalityIndex(taskLocality) - lastLaunchTime = curTime + // NO_PREF will not affect the variables related to delay scheduling + if (maxLocality != TaskLocality.NO_PREF) { + currentLocalityIndex = getLocalityIndex(taskLocality) + lastLaunchTime = curTime + } // Serialize and return the task val startTime = clock.getTime() // We rely on the DAGScheduler to catch non-serializable closures and RDDs, so in here @@ -427,11 +444,15 @@ private[spark] class TaskSetManager( s"(${serializedTask.limit / 1024} KB). The maximum recommended task size is " + s"${TaskSetManager.TASK_SIZE_TO_WARN_KB} KB.") } - val timeTaken = clock.getTime() - startTime addRunningTask(taskId) - logInfo("Serialized task %s:%d as %d bytes in %d ms".format( - taskSet.id, index, serializedTask.limit, timeTaken)) - val taskName = "task %s:%d".format(taskSet.id, index) + + // We used to log the time it takes to serialize the task, but task size is already + // a good proxy to task serialization time. + // val timeTaken = clock.getTime() - startTime + val taskName = s"task ${info.id} in stage ${taskSet.id}" + logInfo("Starting %s (TID %d, %s, %s, %d bytes)".format( + taskName, taskId, host, taskLocality, serializedTask.limit)) + sched.dagScheduler.taskStarted(task, info) return Some(new TaskDescription(taskId, execId, taskName, index, serializedTask)) } @@ -490,19 +511,19 @@ private[spark] class TaskSetManager( info.markSuccessful() removeRunningTask(tid) sched.dagScheduler.taskEnded( - tasks(index), Success, result.value, result.accumUpdates, info, result.metrics) + tasks(index), Success, result.value(), result.accumUpdates, info, result.metrics) if (!successful(index)) { tasksSuccessful += 1 - logInfo("Finished TID %s in %d ms on %s (progress: %d/%d)".format( - tid, info.duration, info.host, tasksSuccessful, numTasks)) + logInfo("Finished task %s in stage %s (TID %d) in %d ms on %s (%d/%d)".format( + info.id, taskSet.id, info.taskId, info.duration, info.host, tasksSuccessful, numTasks)) // Mark successful and stop if all the tasks have succeeded. successful(index) = true if (tasksSuccessful == numTasks) { isZombie = true } } else { - logInfo("Ignorning task-finished event for TID " + tid + " because task " + - index + " has already completed successfully") + logInfo("Ignoring task-finished event for " + info.id + " in stage " + taskSet.id + + " because task " + index + " has already completed successfully") } failedExecutors.remove(index) maybeFinishTaskSet() @@ -521,14 +542,13 @@ private[spark] class TaskSetManager( info.markFailed() val index = info.index copiesRunning(index) -= 1 - if (!isZombie) { - logWarning("Lost TID %s (task %s:%d)".format(tid, taskSet.id, index)) - } var taskMetrics : TaskMetrics = null - var failureReason: String = null + + val failureReason = s"Lost task ${info.id} in stage ${taskSet.id} (TID $tid, ${info.host}): " + + reason.asInstanceOf[TaskFailedReason].toErrorString reason match { case fetchFailed: FetchFailed => - logWarning("Loss was due to fetch failure from " + fetchFailed.bmAddress) + logWarning(failureReason) if (!successful(index)) { successful(index) = true tasksSuccessful += 1 @@ -536,23 +556,17 @@ private[spark] class TaskSetManager( // Not adding to failed executors for FetchFailed. isZombie = true - case TaskKilled => - // Not adding to failed executors for TaskKilled. - logWarning("Task %d was killed.".format(tid)) - case ef: ExceptionFailure => - taskMetrics = ef.metrics.getOrElse(null) - if (ef.className == classOf[NotSerializableException].getName()) { + taskMetrics = ef.metrics.orNull + if (ef.className == classOf[NotSerializableException].getName) { // If the task result wasn't serializable, there's no point in trying to re-execute it. - logError("Task %s:%s had a not serializable result: %s; not retrying".format( - taskSet.id, index, ef.description)) - abort("Task %s:%s had a not serializable result: %s".format( - taskSet.id, index, ef.description)) + logError("Task %s in stage %s (TID %d) had a not serializable result: %s; not retrying" + .format(info.id, taskSet.id, tid, ef.description)) + abort("Task %s in stage %s (TID %d) had a not serializable result: %s".format( + info.id, taskSet.id, tid, ef.description)) return } val key = ef.description - failureReason = "Exception failure in TID %s on host %s: %s\n%s".format( - tid, info.host, ef.description, ef.stackTrace.map(" " + _).mkString("\n")) val now = clock.getTime() val (printFull, dupCount) = { if (recentExceptions.contains(key)) { @@ -570,19 +584,18 @@ private[spark] class TaskSetManager( } } if (printFull) { - val locs = ef.stackTrace.map(loc => "\tat %s".format(loc.toString)) - logWarning("Loss was due to %s\n%s\n%s".format( - ef.className, ef.description, locs.mkString("\n"))) + logWarning(failureReason) } else { - logInfo("Loss was due to %s [duplicate %d]".format(ef.description, dupCount)) + logInfo( + s"Lost task ${info.id} in stage ${taskSet.id} (TID $tid) on executor ${info.host}: " + + s"${ef.className} (${ef.description}) [duplicate $dupCount]") } - case TaskResultLost => - failureReason = "Lost result for TID %s on host %s".format(tid, info.host) + case e: TaskFailedReason => // TaskResultLost, TaskKilled, and others logWarning(failureReason) - case _ => - failureReason = "TID %s on host %s failed for unknown reason".format(tid, info.host) + case e: TaskEndReason => + logError("Unknown TaskEndReason: " + e) } // always add to failed executors failedExecutors.getOrElseUpdate(index, new HashMap[String, Long]()). @@ -593,10 +606,10 @@ private[spark] class TaskSetManager( assert (null != failureReason) numFailures(index) += 1 if (numFailures(index) >= maxTaskFailures) { - logError("Task %s:%d failed %d times; aborting job".format( - taskSet.id, index, maxTaskFailures)) - abort("Task %s:%d failed %d times, most recent failure: %s\nDriver stacktrace:".format( - taskSet.id, index, maxTaskFailures, failureReason)) + logError("Task %d in stage %s failed %d times; aborting job".format( + index, taskSet.id, maxTaskFailures)) + abort("Task %d in stage %s failed %d times, most recent failure: %s\nDriver stacktrace:" + .format(index, taskSet.id, maxTaskFailures, failureReason)) return } } @@ -645,8 +658,7 @@ private[spark] class TaskSetManager( override def executorLost(execId: String, host: String) { logInfo("Re-queueing tasks for " + execId + " from TaskSet " + taskSet.id) - // Re-enqueue pending tasks for this host based on the status of the cluster -- for example, a - // task that used to have locations on only this host might now go to the no-prefs list. Note + // Re-enqueue pending tasks for this host based on the status of the cluster. Note // that it's okay if we add a task to the same queue twice (if it had multiple preferred // locations), because findTaskFromList will skip already-running tasks. for (index <- getPendingTasksForExecutor(execId)) { @@ -677,6 +689,8 @@ private[spark] class TaskSetManager( for ((tid, info) <- taskInfos if info.running && info.executorId == execId) { handleFailedTask(tid, TaskState.FAILED, ExecutorLostFailure) } + // recalculate valid locality levels and waits when executor is lost + recomputeLocality() } /** @@ -709,8 +723,8 @@ private[spark] class TaskSetManager( if (!successful(index) && copiesRunning(index) == 1 && info.timeRunning(time) > threshold && !speculatableTasks.contains(index)) { logInfo( - "Marking task %s:%d (on %s) as speculatable because it ran more than %.0f ms".format( - taskSet.id, index, info.host, threshold)) + "Marking task %d in stage %s (on %s) as speculatable because it ran more than %.0f ms" + .format(index, taskSet.id, info.host, threshold)) speculatableTasks += index foundTasks = true } @@ -728,17 +742,17 @@ private[spark] class TaskSetManager( conf.get("spark.locality.wait.node", defaultWait).toLong case TaskLocality.RACK_LOCAL => conf.get("spark.locality.wait.rack", defaultWait).toLong - case TaskLocality.ANY => - 0L + case _ => 0L } } /** * Compute the locality levels used in this TaskSet. Assumes that all tasks have already been * added to queues using addPendingTask. + * */ private def computeValidLocalityLevels(): Array[TaskLocality.TaskLocality] = { - import TaskLocality.{PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL, ANY} + import TaskLocality.{PROCESS_LOCAL, NODE_LOCAL, NO_PREF, RACK_LOCAL, ANY} val levels = new ArrayBuffer[TaskLocality.TaskLocality] if (!pendingTasksForExecutor.isEmpty && getLocalityWait(PROCESS_LOCAL) != 0 && pendingTasksForExecutor.keySet.exists(sched.isExecutorAlive(_))) { @@ -748,7 +762,11 @@ private[spark] class TaskSetManager( pendingTasksForHost.keySet.exists(sched.hasExecutorsAliveOnHost(_))) { levels += NODE_LOCAL } - if (!pendingTasksForRack.isEmpty && getLocalityWait(RACK_LOCAL) != 0) { + if (!pendingTasksWithNoPrefs.isEmpty) { + levels += NO_PREF + } + if (!pendingTasksForRack.isEmpty && getLocalityWait(RACK_LOCAL) != 0 && + pendingTasksForRack.keySet.exists(sched.hasHostAliveOnRack(_))) { levels += RACK_LOCAL } levels += ANY @@ -756,21 +774,15 @@ private[spark] class TaskSetManager( levels.toArray } - // Re-compute pendingTasksWithNoPrefs since new preferred locations may become available - def executorAdded() { - def newLocAvail(index: Int): Boolean = { - for (loc <- tasks(index).preferredLocations) { - if (sched.hasExecutorsAliveOnHost(loc.host) || - sched.getRackForHost(loc.host).isDefined) { - return true - } - } - false - } - logInfo("Re-computing pending task lists.") - pendingTasksWithNoPrefs = pendingTasksWithNoPrefs.filter(!newLocAvail(_)) + def recomputeLocality() { + val previousLocalityLevel = myLocalityLevels(currentLocalityIndex) myLocalityLevels = computeValidLocalityLevels() localityWaits = myLocalityLevels.map(getLocalityWait) + currentLocalityIndex = getLocalityIndex(previousLocalityLevel) + } + + def executorAdded() { + recomputeLocality() } } 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 318e16552201c..6abf6d930c155 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 @@ -66,4 +66,7 @@ private[spark] object CoarseGrainedClusterMessages { case class RemoveExecutor(executorId: String, reason: String) extends CoarseGrainedClusterMessage + case class AddWebUIFilter(filterName:String, filterParams: String, proxyBase :String) + extends CoarseGrainedClusterMessage + } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 05d01b0c821f9..2a3711ae2a78c 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 @@ -30,7 +30,8 @@ import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} 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.{SerializableBuffer, AkkaUtils, Utils} +import org.apache.spark.util.{ActorLogReceive, SerializableBuffer, AkkaUtils, Utils} +import org.apache.spark.ui.JettyUtils /** * A scheduler backend that waits for coarse grained executors to connect to it through Akka. @@ -46,11 +47,24 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A { // Use an atomic variable to track total number of cores in the cluster for simplicity and speed var totalCoreCount = new AtomicInteger(0) + var totalRegisteredExecutors = new AtomicInteger(0) val conf = scheduler.sc.conf private val timeout = AkkaUtils.askTimeout(conf) private val akkaFrameSize = AkkaUtils.maxFrameSizeBytes(conf) + // Submit tasks only after (registered resources / total expected resources) + // is equal to at least this value, that is double between 0 and 1. + var minRegisteredRatio = + math.min(1, conf.getDouble("spark.scheduler.minRegisteredResourcesRatio", 0)) + // Submit tasks after maxRegisteredWaitingTime milliseconds + // if minRegisteredRatio has not yet been reached + val maxRegisteredWaitingTime = + conf.getInt("spark.scheduler.maxRegisteredResourcesWaitingTime", 30000) + val createTime = System.currentTimeMillis() + + class DriverActor(sparkProperties: Seq[(String, String)]) extends Actor with ActorLogReceive { + + override protected def log = CoarseGrainedSchedulerBackend.this.log - class DriverActor(sparkProperties: Seq[(String, String)]) extends Actor { private val executorActor = new HashMap[String, ActorRef] private val executorAddress = new HashMap[String, Address] private val executorHost = new HashMap[String, String] @@ -68,7 +82,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A context.system.scheduler.schedule(0.millis, reviveInterval.millis, self, ReviveOffers) } - def receive = { + def receiveWithLogging = { case RegisterExecutor(executorId, hostPort, cores) => Utils.checkHostPort(hostPort, "Host port expected " + hostPort) if (executorActor.contains(executorId)) { @@ -83,6 +97,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A executorAddress(executorId) = sender.path.address addressToExecutorId(sender.path.address) = executorId totalCoreCount.addAndGet(cores) + totalRegisteredExecutors.addAndGet(1) makeOffers() } @@ -120,6 +135,9 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A removeExecutor(executorId, reason) sender ! true + case AddWebUIFilter(filterName, filterParams, proxyBase) => + addWebUIFilter(filterName, filterParams, proxyBase) + sender ! true case DisassociatedEvent(_, address, _) => addressToExecutorId.get(address).foreach(removeExecutor(_, "remote Akka client disassociated")) @@ -247,6 +265,36 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A throw new SparkException("Error notifying standalone scheduler's driver actor", e) } } + + def sufficientResourcesRegistered(): Boolean = true + + override def isReady(): Boolean = { + if (sufficientResourcesRegistered) { + logInfo("SchedulerBackend is ready for scheduling beginning after " + + s"reached minRegisteredResourcesRatio: $minRegisteredRatio") + return true + } + if ((System.currentTimeMillis() - createTime) >= maxRegisteredWaitingTime) { + logInfo("SchedulerBackend is ready for scheduling beginning after waiting " + + s"maxRegisteredResourcesWaitingTime: $maxRegisteredWaitingTime(ms)") + return true + } + false + } + + // Add filters to the SparkUI + def addWebUIFilter(filterName: String, filterParams: String, proxyBase: String) { + if (proxyBase != null && proxyBase.nonEmpty) { + System.setProperty("spark.ui.proxyBase", proxyBase) + } + + if (Seq(filterName, filterParams).forall(t => t != null && t.nonEmpty)) { + logInfo(s"Add WebUI Filter. $filterName, $filterParams, $proxyBase") + conf.set("spark.ui.filters", filterName) + conf.set(s"spark.$filterName.params", filterParams) + JettyUtils.addFilters(scheduler.sc.ui.getHandlers, conf) + } + } } private[spark] object CoarseGrainedSchedulerBackend { 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 9c07b3f7b695a..589dba2e40d20 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 @@ -17,7 +17,7 @@ package org.apache.spark.scheduler.cluster -import org.apache.spark.{Logging, SparkContext} +import org.apache.spark.{Logging, SparkConf, SparkContext} import org.apache.spark.deploy.{ApplicationDescription, Command} import org.apache.spark.deploy.client.{AppClient, AppClientListener} import org.apache.spark.scheduler.{ExecutorExited, ExecutorLossReason, SlaveLost, TaskSchedulerImpl} @@ -36,6 +36,7 @@ private[spark] class SparkDeploySchedulerBackend( var shutdownCallback : (SparkDeploySchedulerBackend) => Unit = _ val maxCores = conf.getOption("spark.cores.max").map(_.toInt) + val totalExpectedCores = maxCores.getOrElse(0) override def start() { super.start() @@ -46,6 +47,7 @@ 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") + .map(Utils.splitCommandString).getOrElse(Seq.empty) val classPathEntries = sc.conf.getOption("spark.executor.extraClassPath").toSeq.flatMap { cp => cp.split(java.io.File.pathSeparator) } @@ -54,12 +56,13 @@ private[spark] class SparkDeploySchedulerBackend( cp.split(java.io.File.pathSeparator) } - val command = Command( - "org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.executorEnvs, - classPathEntries, libraryPathEntries, extraJavaOpts) - val sparkHome = sc.getSparkHome() + // Start executors with a few necessary configs for registering with the scheduler + val sparkJavaOpts = Utils.sparkJavaOpts(conf, SparkConf.isExecutorStartupConf) + val javaOpts = sparkJavaOpts ++ extraJavaOpts + val command = Command("org.apache.spark.executor.CoarseGrainedExecutorBackend", + args, sc.executorEnvs, classPathEntries, libraryPathEntries, javaOpts) val appDesc = new ApplicationDescription(sc.appName, maxCores, sc.executorMemory, command, - sparkHome, sc.ui.appUIAddress, sc.eventLogger.map(_.logDir)) + sc.ui.appUIAddress, sc.eventLogger.map(_.logDir)) client = new AppClient(sc.env.actorSystem, masters, appDesc, this, conf) client.start() @@ -107,4 +110,8 @@ private[spark] class SparkDeploySchedulerBackend( logInfo("Executor %s removed: %s".format(fullId, message)) removeExecutor(fullId.split("/")(1), reason.toString) } + + override def sufficientResourcesRegistered(): Boolean = { + totalCoreCount.get() >= totalExpectedCores * minRegisteredRatio + } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala index 9b95ccca0443e..bec9502f20466 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala @@ -25,6 +25,7 @@ import org.apache.spark.{Logging, SparkEnv, TaskState} import org.apache.spark.TaskState.TaskState import org.apache.spark.executor.{Executor, ExecutorBackend} import org.apache.spark.scheduler.{SchedulerBackend, TaskSchedulerImpl, WorkerOffer} +import org.apache.spark.util.ActorLogReceive private case class ReviveOffers() @@ -32,6 +33,8 @@ private case class StatusUpdate(taskId: Long, state: TaskState, serializedData: private case class KillTask(taskId: Long, interruptThread: Boolean) +private case class StopExecutor() + /** * Calls to LocalBackend are all serialized through LocalActor. Using an actor makes the calls on * LocalBackend asynchronous, which is necessary to prevent deadlock between LocalBackend @@ -40,7 +43,7 @@ private case class KillTask(taskId: Long, interruptThread: Boolean) private[spark] class LocalActor( scheduler: TaskSchedulerImpl, executorBackend: LocalBackend, - private val totalCores: Int) extends Actor with Logging { + private val totalCores: Int) extends Actor with ActorLogReceive with Logging { private var freeCores = totalCores @@ -50,26 +53,29 @@ private[spark] class LocalActor( val executor = new Executor( localExecutorId, localExecutorHostname, scheduler.conf.getAll, isLocal = true) - def receive = { + override def receiveWithLogging = { case ReviveOffers => reviveOffers() case StatusUpdate(taskId, state, serializedData) => scheduler.statusUpdate(taskId, state, serializedData) if (TaskState.isFinished(state)) { - freeCores += 1 + freeCores += scheduler.CPUS_PER_TASK reviveOffers() } case KillTask(taskId, interruptThread) => executor.killTask(taskId, interruptThread) + + case StopExecutor => + executor.stop() } def reviveOffers() { val offers = Seq(new WorkerOffer(localExecutorId, localExecutorHostname, freeCores)) for (task <- scheduler.resourceOffers(offers).flatten) { - freeCores -= 1 - executor.launchTask(executorBackend, task.taskId, task.serializedTask) + freeCores -= scheduler.CPUS_PER_TASK + executor.launchTask(executorBackend, task.taskId, task.name, task.serializedTask) } } } @@ -91,6 +97,7 @@ private[spark] class LocalBackend(scheduler: TaskSchedulerImpl, val totalCores: } override def stop() { + localActor ! StopExecutor } override def reviveOffers() { diff --git a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala index 0a7e1ec539679..554a33ce7f1a6 100644 --- a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala @@ -35,16 +35,15 @@ private[spark] class JavaSerializationStream(out: OutputStream, counterReset: In /** * Calling reset to avoid memory leak: * http://stackoverflow.com/questions/1281549/memory-leak-traps-in-the-java-standard-api - * But only call it every 10,000th time to avoid bloated serialization streams (when + * But only call it every 100th time to avoid bloated serialization streams (when * the stream 'resets' object class descriptions have to be re-written) */ def writeObject[T: ClassTag](t: T): SerializationStream = { objOut.writeObject(t) + counter += 1 if (counterReset > 0 && counter >= counterReset) { objOut.reset() counter = 0 - } else { - counter += 1 } this } @@ -64,8 +63,11 @@ extends DeserializationStream { def close() { objIn.close() } } -private[spark] class JavaSerializerInstance(counterReset: Int) extends SerializerInstance { - def serialize[T: ClassTag](t: T): ByteBuffer = { + +private[spark] class JavaSerializerInstance(counterReset: Int, defaultClassLoader: ClassLoader) + extends SerializerInstance { + + override def serialize[T: ClassTag](t: T): ByteBuffer = { val bos = new ByteArrayOutputStream() val out = serializeStream(bos) out.writeObject(t) @@ -73,23 +75,23 @@ private[spark] class JavaSerializerInstance(counterReset: Int) extends Serialize ByteBuffer.wrap(bos.toByteArray) } - def deserialize[T: ClassTag](bytes: ByteBuffer): T = { + override def deserialize[T: ClassTag](bytes: ByteBuffer): T = { val bis = new ByteBufferInputStream(bytes) val in = deserializeStream(bis) - in.readObject().asInstanceOf[T] + in.readObject() } - def deserialize[T: ClassTag](bytes: ByteBuffer, loader: ClassLoader): T = { + override def deserialize[T: ClassTag](bytes: ByteBuffer, loader: ClassLoader): T = { val bis = new ByteBufferInputStream(bytes) val in = deserializeStream(bis, loader) - in.readObject().asInstanceOf[T] + in.readObject() } - def serializeStream(s: OutputStream): SerializationStream = { + override def serializeStream(s: OutputStream): SerializationStream = { new JavaSerializationStream(s, counterReset) } - def deserializeStream(s: InputStream): DeserializationStream = { + override def deserializeStream(s: InputStream): DeserializationStream = { new JavaDeserializationStream(s, Utils.getContextOrSparkClassLoader) } @@ -108,9 +110,12 @@ private[spark] class JavaSerializerInstance(counterReset: Int) extends Serialize */ @DeveloperApi class JavaSerializer(conf: SparkConf) extends Serializer with Externalizable { - private var counterReset = conf.getInt("spark.serializer.objectStreamReset", 10000) + private var counterReset = conf.getInt("spark.serializer.objectStreamReset", 100) - def newInstance(): SerializerInstance = new JavaSerializerInstance(counterReset) + override def newInstance(): SerializerInstance = { + val classLoader = defaultClassLoader.getOrElse(Thread.currentThread.getContextClassLoader) + new JavaSerializerInstance(counterReset, classLoader) + } override def writeExternal(out: ObjectOutput) { out.writeInt(counterReset) diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index 1ce4243194798..87ef9bb0b43c6 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -31,6 +31,7 @@ import org.apache.spark.scheduler.MapStatus import org.apache.spark.storage._ import org.apache.spark.storage.{GetBlock, GotBlock, PutBlock} import org.apache.spark.util.BoundedPriorityQueue +import org.apache.spark.util.collection.CompactBuffer import scala.reflect.ClassTag @@ -46,16 +47,23 @@ class KryoSerializer(conf: SparkConf) with Logging with Serializable { - private val bufferSize = conf.getInt("spark.kryoserializer.buffer.mb", 2) * 1024 * 1024 + private val bufferSize = + (conf.getDouble("spark.kryoserializer.buffer.mb", 0.064) * 1024 * 1024).toInt + + private val maxBufferSize = conf.getInt("spark.kryoserializer.buffer.max.mb", 64) * 1024 * 1024 private val referenceTracking = conf.getBoolean("spark.kryo.referenceTracking", true) + private val registrationRequired = conf.getBoolean("spark.kryo.registrationRequired", false) private val registrator = conf.getOption("spark.kryo.registrator") - def newKryoOutput() = new KryoOutput(bufferSize) + def newKryoOutput() = new KryoOutput(bufferSize, math.max(bufferSize, maxBufferSize)) def newKryo(): Kryo = { val instantiator = new EmptyScalaKryoInstantiator val kryo = instantiator.newKryo() - val classLoader = Thread.currentThread.getContextClassLoader + kryo.setRegistrationRequired(registrationRequired) + + val oldClassLoader = Thread.currentThread.getContextClassLoader + val classLoader = defaultClassLoader.getOrElse(Thread.currentThread.getContextClassLoader) // Allow disabling Kryo reference tracking if user knows their object graphs don't have loops. // Do this before we invoke the user registrator so the user registrator can override this. @@ -73,15 +81,21 @@ class KryoSerializer(conf: SparkConf) kryo.register(classOf[HttpBroadcast[_]], new KryoJavaSerializer()) // Allow the user to register their own classes by setting spark.kryo.registrator - try { - for (regCls <- registrator) { - logDebug("Running user registrator: " + regCls) + for (regCls <- registrator) { + logDebug("Running user registrator: " + regCls) + try { val reg = Class.forName(regCls, true, classLoader).newInstance() .asInstanceOf[KryoRegistrator] + + // Use the default classloader when calling the user registrator. + Thread.currentThread.setContextClassLoader(classLoader) reg.registerClasses(kryo) + } catch { + case e: Exception => + throw new SparkException(s"Failed to invoke $regCls", e) + } finally { + Thread.currentThread.setContextClassLoader(oldClassLoader) } - } catch { - case e: Exception => logError("Failed to run spark.kryo.registrator", e) } // Register Chill's classes; we do this after our ranges and the user's own classes to let @@ -92,7 +106,7 @@ class KryoSerializer(conf: SparkConf) kryo } - def newInstance(): SerializerInstance = { + override def newInstance(): SerializerInstance = { new KryoSerializerInstance(this) } } @@ -101,20 +115,20 @@ private[spark] class KryoSerializationStream(kryo: Kryo, outStream: OutputStream) extends SerializationStream { val output = new KryoOutput(outStream) - def writeObject[T: ClassTag](t: T): SerializationStream = { + override def writeObject[T: ClassTag](t: T): SerializationStream = { kryo.writeClassAndObject(output, t) this } - def flush() { output.flush() } - def close() { output.close() } + override def flush() { output.flush() } + override def close() { output.close() } } private[spark] class KryoDeserializationStream(kryo: Kryo, inStream: InputStream) extends DeserializationStream { - val input = new KryoInput(inStream) + private val input = new KryoInput(inStream) - def readObject[T: ClassTag](): T = { + override def readObject[T: ClassTag](): T = { try { kryo.readClassAndObject(input).asInstanceOf[T] } catch { @@ -124,31 +138,31 @@ class KryoDeserializationStream(kryo: Kryo, inStream: InputStream) extends Deser } } - def close() { + override def close() { // Kryo's Input automatically closes the input stream it is using. input.close() } } private[spark] class KryoSerializerInstance(ks: KryoSerializer) extends SerializerInstance { - val kryo = ks.newKryo() + private val kryo = ks.newKryo() // Make these lazy vals to avoid creating a buffer unless we use them - lazy val output = ks.newKryoOutput() - lazy val input = new KryoInput() + private lazy val output = ks.newKryoOutput() + private lazy val input = new KryoInput() - def serialize[T: ClassTag](t: T): ByteBuffer = { + override def serialize[T: ClassTag](t: T): ByteBuffer = { output.clear() kryo.writeClassAndObject(output, t) ByteBuffer.wrap(output.toBytes) } - def deserialize[T: ClassTag](bytes: ByteBuffer): T = { + override def deserialize[T: ClassTag](bytes: ByteBuffer): T = { input.setBuffer(bytes.array) kryo.readClassAndObject(input).asInstanceOf[T] } - def deserialize[T: ClassTag](bytes: ByteBuffer, loader: ClassLoader): T = { + override def deserialize[T: ClassTag](bytes: ByteBuffer, loader: ClassLoader): T = { val oldClassLoader = kryo.getClassLoader kryo.setClassLoader(loader) input.setBuffer(bytes.array) @@ -157,11 +171,11 @@ private[spark] class KryoSerializerInstance(ks: KryoSerializer) extends Serializ obj } - def serializeStream(s: OutputStream): SerializationStream = { + override def serializeStream(s: OutputStream): SerializationStream = { new KryoSerializationStream(kryo, s) } - def deserializeStream(s: InputStream): DeserializationStream = { + override def deserializeStream(s: InputStream): DeserializationStream = { new KryoDeserializationStream(kryo, s) } } @@ -183,9 +197,11 @@ private[serializer] object KryoSerializer { classOf[GotBlock], classOf[GetBlock], classOf[MapStatus], + classOf[CompactBuffer[_]], classOf[BlockManagerId], classOf[Array[Byte]], - classOf[BoundedPriorityQueue[_]] + classOf[BoundedPriorityQueue[_]], + classOf[SparkConf] ) } 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 f2f5cea469c61..a9144cdd97b8c 100644 --- a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala @@ -43,11 +43,30 @@ import org.apache.spark.util.{ByteBufferInputStream, NextIterator} * They are intended to be used to serialize/de-serialize data within a single Spark application. */ @DeveloperApi -trait Serializer { +abstract class Serializer { + + /** + * Default ClassLoader to use in deserialization. Implementations of [[Serializer]] should + * make sure it is using this when set. + */ + @volatile protected var defaultClassLoader: Option[ClassLoader] = None + + /** + * Sets a class loader for the serializer to use in deserialization. + * + * @return this Serializer object + */ + def setDefaultClassLoader(classLoader: ClassLoader): Serializer = { + defaultClassLoader = Some(classLoader) + this + } + + /** Creates a new [[SerializerInstance]]. */ def newInstance(): SerializerInstance } +@DeveloperApi object Serializer { def getSerializer(serializer: Serializer): Serializer = { if (serializer == null) SparkEnv.get.serializer else serializer @@ -64,7 +83,7 @@ object Serializer { * An instance of a serializer, for use by one thread at a time. */ @DeveloperApi -trait SerializerInstance { +abstract class SerializerInstance { def serialize[T: ClassTag](t: T): ByteBuffer def deserialize[T: ClassTag](bytes: ByteBuffer): T @@ -74,21 +93,6 @@ trait SerializerInstance { def serializeStream(s: OutputStream): SerializationStream def deserializeStream(s: InputStream): DeserializationStream - - def serializeMany[T: ClassTag](iterator: Iterator[T]): ByteBuffer = { - // Default implementation uses serializeStream - val stream = new ByteArrayOutputStream() - serializeStream(stream).writeAll(iterator) - val buffer = ByteBuffer.wrap(stream.toByteArray) - buffer.flip() - buffer - } - - def deserializeMany(buffer: ByteBuffer): Iterator[Any] = { - // Default implementation uses deserializeStream - buffer.rewind() - deserializeStream(new ByteBufferInputStream(buffer)).asIterator - } } /** @@ -96,7 +100,7 @@ trait SerializerInstance { * A stream for writing serialized objects. */ @DeveloperApi -trait SerializationStream { +abstract class SerializationStream { def writeObject[T: ClassTag](t: T): SerializationStream def flush(): Unit def close(): Unit @@ -115,7 +119,7 @@ trait SerializationStream { * A stream for reading serialized objects. */ @DeveloperApi -trait DeserializationStream { +abstract class DeserializationStream { def readObject[T: ClassTag](): T def close(): Unit diff --git a/core/src/main/scala/org/apache/spark/serializer/package-info.java b/core/src/main/scala/org/apache/spark/serializer/package-info.java index 4c0b73ab36a00..207c6e02e4293 100644 --- a/core/src/main/scala/org/apache/spark/serializer/package-info.java +++ b/core/src/main/scala/org/apache/spark/serializer/package-info.java @@ -18,4 +18,4 @@ /** * Pluggable serializers for RDD and shuffle data. */ -package org.apache.spark.serializer; \ No newline at end of file +package org.apache.spark.serializer; diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleMemoryManager.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleMemoryManager.scala new file mode 100644 index 0000000000000..ee91a368b76ea --- /dev/null +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleMemoryManager.scala @@ -0,0 +1,125 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.shuffle + +import scala.collection.mutable + +import org.apache.spark.{Logging, SparkException, SparkConf} + +/** + * Allocates a pool of memory to task threads for use in shuffle operations. Each disk-spilling + * collection (ExternalAppendOnlyMap or ExternalSorter) used by these tasks can acquire memory + * from this pool and release it as it spills data out. When a task ends, all its memory will be + * released by the Executor. + * + * This class tries to ensure that each thread gets a reasonable share of memory, instead of some + * thread ramping up to a large amount first and then causing others to spill to disk repeatedly. + * If there are N threads, it ensures that each thread can acquire at least 1 / 2N of the memory + * before it has to spill, and at most 1 / N. Because N varies dynamically, we keep track of the + * set of active threads and redo the calculations of 1 / 2N and 1 / N in waiting threads whenever + * this set changes. This is all done by synchronizing access on "this" to mutate state and using + * wait() and notifyAll() to signal changes. + */ +private[spark] class ShuffleMemoryManager(maxMemory: Long) extends Logging { + private val threadMemory = new mutable.HashMap[Long, Long]() // threadId -> memory bytes + + def this(conf: SparkConf) = this(ShuffleMemoryManager.getMaxMemory(conf)) + + /** + * Try to acquire up to numBytes memory for the current thread, and return the number of bytes + * obtained, or 0 if none can be allocated. This call may block until there is enough free memory + * in some situations, to make sure each thread has a chance to ramp up to at least 1 / 2N of the + * total memory pool (where N is the # of active threads) before it is forced to spill. This can + * happen if the number of threads increases but an older thread had a lot of memory already. + */ + def tryToAcquire(numBytes: Long): Long = synchronized { + val threadId = Thread.currentThread().getId + assert(numBytes > 0, "invalid number of bytes requested: " + numBytes) + + // Add this thread to the threadMemory map just so we can keep an accurate count of the number + // of active threads, to let other threads ramp down their memory in calls to tryToAcquire + if (!threadMemory.contains(threadId)) { + threadMemory(threadId) = 0L + notifyAll() // Will later cause waiting threads to wake up and check numThreads again + } + + // Keep looping until we're either sure that we don't want to grant this request (because this + // thread would have more than 1 / numActiveThreads of the memory) or we have enough free + // memory to give it (we always let each thread get at least 1 / (2 * numActiveThreads)). + while (true) { + val numActiveThreads = threadMemory.keys.size + val curMem = threadMemory(threadId) + val freeMemory = maxMemory - threadMemory.values.sum + + // How much we can grant this thread; don't let it grow to more than 1 / numActiveThreads + val maxToGrant = math.min(numBytes, (maxMemory / numActiveThreads) - curMem) + + if (curMem < maxMemory / (2 * numActiveThreads)) { + // We want to let each thread get at least 1 / (2 * numActiveThreads) before blocking; + // if we can't give it this much now, wait for other threads to free up memory + // (this happens if older threads allocated lots of memory before N grew) + if (freeMemory >= math.min(maxToGrant, maxMemory / (2 * numActiveThreads) - curMem)) { + val toGrant = math.min(maxToGrant, freeMemory) + threadMemory(threadId) += toGrant + return toGrant + } else { + logInfo(s"Thread $threadId waiting for at least 1/2N of shuffle memory pool to be free") + wait() + } + } else { + // Only give it as much memory as is free, which might be none if it reached 1 / numThreads + val toGrant = math.min(maxToGrant, freeMemory) + threadMemory(threadId) += toGrant + return toGrant + } + } + 0L // Never reached + } + + /** Release numBytes bytes for the current thread. */ + def release(numBytes: Long): Unit = synchronized { + val threadId = Thread.currentThread().getId + val curMem = threadMemory.getOrElse(threadId, 0L) + if (curMem < numBytes) { + throw new SparkException( + s"Internal error: release called on ${numBytes} bytes but thread only has ${curMem}") + } + threadMemory(threadId) -= numBytes + notifyAll() // Notify waiters who locked "this" in tryToAcquire that memory has been freed + } + + /** Release all memory for the current thread and mark it as inactive (e.g. when a task ends). */ + def releaseMemoryForThisThread(): Unit = synchronized { + val threadId = Thread.currentThread().getId + threadMemory.remove(threadId) + notifyAll() // Notify waiters who locked "this" in tryToAcquire that memory has been freed + } +} + +private object ShuffleMemoryManager { + /** + * Figure out the shuffle memory limit from a SparkConf. We currently have both a fraction + * of the memory pool and a safety factor since collections can sometimes grow bigger than + * the size we target before we estimate their sizes again. + */ + def getMaxMemory(conf: SparkConf): Long = { + val memoryFraction = conf.getDouble("spark.shuffle.memoryFraction", 0.2) + val safetyFraction = conf.getDouble("spark.shuffle.safetyFraction", 0.8) + (Runtime.getRuntime.maxMemory * memoryFraction * safetyFraction).toLong + } +} diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala index a932455776e34..12b475658e29d 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala @@ -32,7 +32,8 @@ private[hash] object BlockStoreShuffleFetcher extends Logging { shuffleId: Int, reduceId: Int, context: TaskContext, - serializer: Serializer) + serializer: Serializer, + shuffleMetrics: ShuffleReadMetrics) : Iterator[T] = { logDebug("Fetching outputs for shuffle %d, reduce %d".format(shuffleId, reduceId)) @@ -73,18 +74,11 @@ private[hash] object BlockStoreShuffleFetcher extends Logging { } } - val blockFetcherItr = blockManager.getMultiple(blocksByAddress, serializer) + val blockFetcherItr = blockManager.getMultiple(blocksByAddress, serializer, shuffleMetrics) val itr = blockFetcherItr.flatMap(unpackBlock) val completionIter = CompletionIterator[T, Iterator[T]](itr, { - val shuffleMetrics = new ShuffleReadMetrics - shuffleMetrics.shuffleFinishTime = System.currentTimeMillis - shuffleMetrics.fetchWaitTime = blockFetcherItr.fetchWaitTime - shuffleMetrics.remoteBytesRead = blockFetcherItr.remoteBytesRead - shuffleMetrics.totalBlocksFetched = blockFetcherItr.totalBlocks - shuffleMetrics.localBlocksFetched = blockFetcherItr.numLocalBlocks - shuffleMetrics.remoteBlocksFetched = blockFetcherItr.numRemoteBlocks - context.taskMetrics.shuffleReadMetrics = Some(shuffleMetrics) + context.taskMetrics.updateShuffleReadMetrics() }) new InterruptibleIterator[T](context, completionIter) 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 index 5b0940ecce29d..df98d18fa8193 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleManager.scala @@ -24,7 +24,7 @@ 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 { +private[spark] 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, 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 index d45258c0a492b..7bed97a63f0f6 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala @@ -20,8 +20,9 @@ package org.apache.spark.shuffle.hash import org.apache.spark.{InterruptibleIterator, TaskContext} import org.apache.spark.serializer.Serializer import org.apache.spark.shuffle.{BaseShuffleHandle, ShuffleReader} +import org.apache.spark.util.collection.ExternalSorter -class HashShuffleReader[K, C]( +private[spark] class HashShuffleReader[K, C]( handle: BaseShuffleHandle[K, _, C], startPartition: Int, endPartition: Int, @@ -35,10 +36,12 @@ class HashShuffleReader[K, C]( /** Read the combined key-values for this reduce task */ override def read(): Iterator[Product2[K, C]] = { - val iter = BlockStoreShuffleFetcher.fetch(handle.shuffleId, startPartition, context, - Serializer.getSerializer(dep.serializer)) + val readMetrics = context.taskMetrics.createShuffleReadMetricsForDependency() + val ser = Serializer.getSerializer(dep.serializer) + val iter = BlockStoreShuffleFetcher.fetch(handle.shuffleId, startPartition, context, ser, + readMetrics) - if (dep.aggregator.isDefined) { + val aggregatedIter: Iterator[Product2[K, C]] = if (dep.aggregator.isDefined) { if (dep.mapSideCombine) { new InterruptibleIterator(context, dep.aggregator.get.combineCombinersByKey(iter, context)) } else { @@ -47,7 +50,22 @@ class HashShuffleReader[K, C]( } else if (dep.aggregator.isEmpty && dep.mapSideCombine) { throw new IllegalStateException("Aggregator is empty for map-side combine") } else { - iter + // Convert the Product2s to pairs since this is what downstream RDDs currently expect + iter.asInstanceOf[Iterator[Product2[K, C]]].map(pair => (pair._1, pair._2)) + } + + // Sort the output if there is a sort ordering defined. + dep.keyOrdering match { + case Some(keyOrd: Ordering[K]) => + // Create an ExternalSorter to sort the data. Note that if spark.shuffle.spill is disabled, + // the ExternalSorter won't spill to disk. + val sorter = new ExternalSorter[K, C, C](ordering = Some(keyOrd), serializer = Some(ser)) + sorter.insertAll(aggregatedIter) + context.taskMetrics.memoryBytesSpilled += sorter.memoryBytesSpilled + context.taskMetrics.diskBytesSpilled += sorter.diskBytesSpilled + sorter.iterator + case None => + aggregatedIter } } 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 index 9b78228519da4..51e454d9313c9 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala @@ -24,7 +24,7 @@ import org.apache.spark.serializer.Serializer import org.apache.spark.executor.ShuffleWriteMetrics import org.apache.spark.scheduler.MapStatus -class HashShuffleWriter[K, V]( +private[spark] class HashShuffleWriter[K, V]( handle: BaseShuffleHandle[K, V, _], mapId: Int, context: TaskContext) @@ -33,12 +33,20 @@ class HashShuffleWriter[K, V]( private val dep = handle.dependency private val numOutputSplits = dep.partitioner.numPartitions private val metrics = context.taskMetrics + + // Are we in the process of stopping? Because map tasks can call stop() with success = true + // and then call stop() with success = false if they get an exception, we want to make sure + // we don't try deleting files, etc twice. private var stopping = false + private val writeMetrics = new ShuffleWriteMetrics() + metrics.shuffleWriteMetrics = Some(writeMetrics) + 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) + private val shuffle = shuffleBlockManager.forMapTask(dep.shuffleId, mapId, numOutputSplits, ser, + writeMetrics) /** Write a bunch of records to this task's output */ override def write(records: Iterator[_ <: Product2[K, V]]): Unit = { @@ -61,7 +69,8 @@ class HashShuffleWriter[K, V]( } /** Close this writer, passing along whether the map completed */ - override def stop(success: Boolean): Option[MapStatus] = { + override def stop(initiallySuccess: Boolean): Option[MapStatus] = { + var success = initiallySuccess try { if (stopping) { return None @@ -69,15 +78,16 @@ class HashShuffleWriter[K, V]( stopping = true if (success) { try { - return Some(commitWritesAndBuildStatus()) + Some(commitWritesAndBuildStatus()) } catch { case e: Exception => + success = false revertWrites() throw e } } else { revertWrites() - return None + None } } finally { // Release the writers back to the shuffle block manager. @@ -93,31 +103,19 @@ class HashShuffleWriter[K, V]( 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() + writer.commitAndClose() 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() + writer.revertPartialWritesAndClose() } } } diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala new file mode 100644 index 0000000000000..6dcca47ea7c0c --- /dev/null +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.sort + +import java.io.{DataInputStream, FileInputStream} + +import org.apache.spark.shuffle._ +import org.apache.spark.{TaskContext, ShuffleDependency} +import org.apache.spark.shuffle.hash.HashShuffleReader +import org.apache.spark.storage.{DiskBlockManager, FileSegment, ShuffleBlockId} + +private[spark] class SortShuffleManager 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] = { + // We currently use the same block store shuffle fetcher as the hash-based shuffle. + 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 SortShuffleWriter(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 = {} + + /** Get the location of a block in a map output file. Uses the index file we create for it. */ + def getBlockLocation(blockId: ShuffleBlockId, diskManager: DiskBlockManager): FileSegment = { + // The block is actually going to be a range of a single map output file for this map, so + // figure out the ID of the consolidated file, then the offset within that from our index + val consolidatedId = blockId.copy(reduceId = 0) + val indexFile = diskManager.getFile(consolidatedId.name + ".index") + val in = new DataInputStream(new FileInputStream(indexFile)) + try { + in.skip(blockId.reduceId * 8) + val offset = in.readLong() + val nextOffset = in.readLong() + new FileSegment(diskManager.getFile(consolidatedId), offset, nextOffset - offset) + } finally { + in.close() + } + } +} diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala new file mode 100644 index 0000000000000..22f656fa371ea --- /dev/null +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala @@ -0,0 +1,120 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.sort + +import java.io.{BufferedOutputStream, File, FileOutputStream, DataOutputStream} + +import org.apache.spark.{MapOutputTracker, SparkEnv, Logging, TaskContext} +import org.apache.spark.executor.ShuffleWriteMetrics +import org.apache.spark.scheduler.MapStatus +import org.apache.spark.serializer.Serializer +import org.apache.spark.shuffle.{ShuffleWriter, BaseShuffleHandle} +import org.apache.spark.storage.ShuffleBlockId +import org.apache.spark.util.collection.ExternalSorter + +private[spark] class SortShuffleWriter[K, V, C]( + handle: BaseShuffleHandle[K, V, C], + mapId: Int, + context: TaskContext) + extends ShuffleWriter[K, V] with Logging { + + private val dep = handle.dependency + private val numPartitions = dep.partitioner.numPartitions + + private val blockManager = SparkEnv.get.blockManager + private val ser = Serializer.getSerializer(dep.serializer.orNull) + + private val conf = SparkEnv.get.conf + private val fileBufferSize = conf.getInt("spark.shuffle.file.buffer.kb", 32) * 1024 + + private var sorter: ExternalSorter[K, V, _] = null + private var outputFile: File = null + private var indexFile: File = null + + // Are we in the process of stopping? Because map tasks can call stop() with success = true + // and then call stop() with success = false if they get an exception, we want to make sure + // we don't try deleting files, etc twice. + private var stopping = false + + private var mapStatus: MapStatus = null + + private val writeMetrics = new ShuffleWriteMetrics() + context.taskMetrics.shuffleWriteMetrics = Some(writeMetrics) + + /** Write a bunch of records to this task's output */ + override def write(records: Iterator[_ <: Product2[K, V]]): Unit = { + if (dep.mapSideCombine) { + if (!dep.aggregator.isDefined) { + throw new IllegalStateException("Aggregator is empty for map-side combine") + } + sorter = new ExternalSorter[K, V, C]( + dep.aggregator, Some(dep.partitioner), dep.keyOrdering, dep.serializer) + sorter.insertAll(records) + } else { + // In this case we pass neither an aggregator nor an ordering to the sorter, because we don't + // care whether the keys get sorted in each partition; that will be done on the reduce side + // if the operation being run is sortByKey. + sorter = new ExternalSorter[K, V, V]( + None, Some(dep.partitioner), None, dep.serializer) + sorter.insertAll(records) + } + + // Create a single shuffle file with reduce ID 0 that we'll write all results to. We'll later + // serve different ranges of this file using an index file that we create at the end. + val blockId = ShuffleBlockId(dep.shuffleId, mapId, 0) + + outputFile = blockManager.diskBlockManager.getFile(blockId) + indexFile = blockManager.diskBlockManager.getFile(blockId.name + ".index") + + val partitionLengths = sorter.writePartitionedFile(blockId, context) + + // Register our map output with the ShuffleBlockManager, which handles cleaning it over time + blockManager.shuffleBlockManager.addCompletedMap(dep.shuffleId, mapId, numPartitions) + + mapStatus = new MapStatus(blockManager.blockManagerId, + partitionLengths.map(MapOutputTracker.compressSize)) + } + + /** 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) { + return Option(mapStatus) + } else { + // The map task failed, so delete our output file if we created one + if (outputFile != null) { + outputFile.delete() + } + if (indexFile != null) { + indexFile.delete() + } + return None + } + } finally { + // Clean up our sorter, which may have its own intermediate files + if (sorter != null) { + sorter.stop() + sorter = null + } + } + } +} diff --git a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala index 408a797088059..5f44f5f3197fd 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala @@ -22,10 +22,12 @@ import java.util.concurrent.LinkedBlockingQueue import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashSet import scala.collection.mutable.Queue +import scala.util.{Failure, Success} import io.netty.buffer.ByteBuf import org.apache.spark.{Logging, SparkException} +import org.apache.spark.executor.ShuffleReadMetrics import org.apache.spark.network.BufferMessage import org.apache.spark.network.ConnectionManagerId import org.apache.spark.network.netty.ShuffleCopier @@ -46,11 +48,6 @@ import org.apache.spark.util.Utils private[storage] trait BlockFetcherIterator extends Iterator[(BlockId, Option[Iterator[Any]])] with Logging { def initialize() - def totalBlocks: Int - def numLocalBlocks: Int - def numRemoteBlocks: Int - def fetchWaitTime: Long - def remoteBytesRead: Long } @@ -72,14 +69,12 @@ object BlockFetcherIterator { class BasicBlockFetcherIterator( private val blockManager: BlockManager, val blocksByAddress: Seq[(BlockManagerId, Seq[(BlockId, Long)])], - serializer: Serializer) + serializer: Serializer, + readMetrics: ShuffleReadMetrics) extends BlockFetcherIterator { import blockManager._ - private var _remoteBytesRead = 0L - private var _fetchWaitTime = 0L - if (blocksByAddress == null) { throw new IllegalArgumentException("BlocksByAddress is null") } @@ -89,13 +84,9 @@ object BlockFetcherIterator { protected var startTime = System.currentTimeMillis - // This represents the number of local blocks, also counting zero-sized blocks - private var numLocal = 0 // BlockIds for local blocks that need to be fetched. Excludes zero-sized blocks protected val localBlocksToFetch = new ArrayBuffer[BlockId]() - // This represents the number of remote blocks, also counting zero-sized blocks - private var numRemote = 0 // BlockIds for remote blocks that need to be fetched. Excludes zero-sized blocks protected val remoteBlocksToFetch = new HashSet[BlockId]() @@ -119,8 +110,8 @@ object BlockFetcherIterator { bytesInFlight += req.size val sizeMap = req.blocks.toMap // so we can look up the size of each blockID val future = connectionManager.sendMessageReliably(cmId, blockMessageArray.toBufferMessage) - future.onSuccess { - case Some(message) => { + future.onComplete { + case Success(message) => { val bufferMessage = message.asInstanceOf[BufferMessage] val blockMessageArray = BlockMessageArray.fromBufferMessage(bufferMessage) for (blockMessage <- blockMessageArray) { @@ -132,12 +123,15 @@ object BlockFetcherIterator { val networkSize = blockMessage.getData.limit() results.put(new FetchResult(blockId, sizeMap(blockId), () => dataDeserialize(blockId, blockMessage.getData, serializer))) - _remoteBytesRead += networkSize + // TODO: NettyBlockFetcherIterator has some race conditions where multiple threads can + // be incrementing bytes read at the same time (SPARK-2625). + readMetrics.remoteBytesRead += networkSize + readMetrics.remoteBlocksFetched += 1 logDebug("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime)) } } - case None => { - logError("Could not get block(s) from " + cmId) + case Failure(exception) => { + logError("Could not get block(s) from " + cmId, exception) for ((blockId, size) <- req.blocks) { results.put(new FetchResult(blockId, -1, null)) } @@ -155,14 +149,14 @@ object BlockFetcherIterator { // Split local and remote blocks. Remote blocks are further split into FetchRequests of size // at most maxBytesInFlight in order to limit the amount of data in flight. val remoteRequests = new ArrayBuffer[FetchRequest] + var totalBlocks = 0 for ((address, blockInfos) <- blocksByAddress) { + totalBlocks += blockInfos.size if (address == blockManagerId) { - numLocal = blockInfos.size // Filter out zero-sized blocks localBlocksToFetch ++= blockInfos.filter(_._2 != 0).map(_._1) _numBlocksToFetch += localBlocksToFetch.size } else { - numRemote += blockInfos.size val iterator = blockInfos.iterator var curRequestSize = 0L var curBlocks = new ArrayBuffer[(BlockId, Long)] @@ -180,9 +174,9 @@ object BlockFetcherIterator { if (curRequestSize >= targetRequestSize) { // Add this FetchRequest remoteRequests += new FetchRequest(address, curBlocks) - curRequestSize = 0 curBlocks = new ArrayBuffer[(BlockId, Long)] logDebug(s"Creating fetch request of $curRequestSize at $address") + curRequestSize = 0 } } // Add in the final request @@ -201,14 +195,18 @@ object BlockFetcherIterator { // these all at once because they will just memory-map some files, so they won't consume // any memory that might exceed our maxBytesInFlight for (id <- localBlocksToFetch) { - getLocalFromDisk(id, serializer) match { - case Some(iter) => { - // Pass 0 as size since it's not in flight - results.put(new FetchResult(id, 0, () => iter)) - logDebug("Got local block " + id) - } - case None => { - throw new BlockException(id, "Could not get block " + id + " from local machine") + try { + // getLocalFromDisk never return None but throws BlockException + val iter = getLocalFromDisk(id, serializer).get + // Pass 0 as size since it's not in flight + readMetrics.localBlocksFetched += 1 + results.put(new FetchResult(id, 0, () => iter)) + logDebug("Got local block " + id) + } catch { + case e: Exception => { + logError(s"Error occurred while fetching local blocks", e) + results.put(new FetchResult(id, -1, null)) + return } } } @@ -235,13 +233,6 @@ object BlockFetcherIterator { logDebug("Got local blocks in " + Utils.getUsedTimeMs(startTime) + " ms") } - override def totalBlocks: Int = numLocal + numRemote - override def numLocalBlocks: Int = numLocal - override def numRemoteBlocks: Int = numRemote - override def fetchWaitTime: Long = _fetchWaitTime - override def remoteBytesRead: Long = _remoteBytesRead - - // Implementing the Iterator methods with an iterator that reads fetched blocks off the queue // as they arrive. @volatile protected var resultsGotten = 0 @@ -253,7 +244,7 @@ object BlockFetcherIterator { val startFetchWait = System.currentTimeMillis() val result = results.take() val stopFetchWait = System.currentTimeMillis() - _fetchWaitTime += (stopFetchWait - startFetchWait) + readMetrics.fetchWaitTime += (stopFetchWait - startFetchWait) if (! result.failed) bytesInFlight -= result.size while (!fetchRequests.isEmpty && (bytesInFlight == 0 || bytesInFlight + fetchRequests.front.size <= maxBytesInFlight)) { @@ -267,8 +258,9 @@ object BlockFetcherIterator { class NettyBlockFetcherIterator( blockManager: BlockManager, blocksByAddress: Seq[(BlockManagerId, Seq[(BlockId, Long)])], - serializer: Serializer) - extends BasicBlockFetcherIterator(blockManager, blocksByAddress, serializer) { + serializer: Serializer, + readMetrics: ShuffleReadMetrics) + extends BasicBlockFetcherIterator(blockManager, blocksByAddress, serializer, readMetrics) { import blockManager._ diff --git a/core/src/main/scala/org/apache/spark/storage/BlockId.scala b/core/src/main/scala/org/apache/spark/storage/BlockId.scala index 42ec181b00bb3..c1756ac905417 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockId.scala @@ -54,11 +54,15 @@ case class RDDBlockId(rddId: Int, splitIndex: Int) extends BlockId { } @DeveloperApi -case class ShuffleBlockId(shuffleId: Int, mapId: Int, reduceId: Int) - extends BlockId { +case class ShuffleBlockId(shuffleId: Int, mapId: Int, reduceId: Int) extends BlockId { def name = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId } +@DeveloperApi +case class ShuffleIndexBlockId(shuffleId: Int, mapId: Int, reduceId: Int) extends BlockId { + def name = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId + ".index" +} + @DeveloperApi case class BroadcastBlockId(broadcastId: Long, field: String = "") extends BlockId { def name = "broadcast_" + broadcastId + (if (field == "") "" else "_" + field) @@ -88,6 +92,7 @@ private[spark] case class TestBlockId(id: String) extends BlockId { object BlockId { val RDD = "rdd_([0-9]+)_([0-9]+)".r val SHUFFLE = "shuffle_([0-9]+)_([0-9]+)_([0-9]+)".r + val SHUFFLE_INDEX = "shuffle_([0-9]+)_([0-9]+)_([0-9]+).index".r val BROADCAST = "broadcast_([0-9]+)([_A-Za-z0-9]*)".r val TASKRESULT = "taskresult_([0-9]+)".r val STREAM = "input-([0-9]+)-([0-9]+)".r @@ -99,6 +104,8 @@ object BlockId { RDDBlockId(rddId.toInt, splitIndex.toInt) case SHUFFLE(shuffleId, mapId, reduceId) => ShuffleBlockId(shuffleId.toInt, mapId.toInt, reduceId.toInt) + case SHUFFLE_INDEX(shuffleId, mapId, reduceId) => + ShuffleIndexBlockId(shuffleId.toInt, mapId.toInt, reduceId.toInt) case BROADCAST(broadcastId, field) => BroadcastBlockId(broadcastId.toLong, field.stripPrefix("_")) case TASKRESULT(taskId) => 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 0db0a5bc7341b..e4c3d58905e7f 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -29,16 +29,17 @@ import akka.actor.{ActorSystem, Cancellable, Props} import sun.nio.ch.DirectBuffer import org.apache.spark._ -import org.apache.spark.executor.{DataReadMethod, InputMetrics} +import org.apache.spark.executor._ import org.apache.spark.io.CompressionCodec import org.apache.spark.network._ import org.apache.spark.serializer.Serializer +import org.apache.spark.shuffle.ShuffleManager import org.apache.spark.util._ 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] case class ArrayValues(buffer: Array[Any]) extends BlockValues /* Class for returning a fetched block and associated metrics. */ private[spark] class BlockResult( @@ -57,13 +58,16 @@ private[spark] class BlockManager( maxMemory: Long, val conf: SparkConf, securityManager: SecurityManager, - mapOutputTracker: MapOutputTracker) + mapOutputTracker: MapOutputTracker, + shuffleManager: ShuffleManager) extends Logging { - val shuffleBlockManager = new ShuffleBlockManager(this) + private val port = conf.getInt("spark.blockManager.port", 0) + val shuffleBlockManager = new ShuffleBlockManager(this, shuffleManager) val diskBlockManager = new DiskBlockManager(shuffleBlockManager, conf.get("spark.local.dir", System.getProperty("java.io.tmpdir"))) - val connectionManager = new ConnectionManager(0, conf, securityManager) + val connectionManager = + new ConnectionManager(port, conf, securityManager, "Connection manager for block manager") implicit val futureExecContext = connectionManager.futureExecContext @@ -71,9 +75,9 @@ private[spark] class BlockManager( // 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) - private[storage] lazy val tachyonStore: TachyonStore = { + private[spark] val memoryStore = new MemoryStore(this, maxMemory) + private[spark] val diskStore = new DiskStore(this, diskBlockManager) + private[spark] 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}" @@ -116,15 +120,6 @@ private[spark] class BlockManager( private var asyncReregisterTask: Future[Unit] = null private val asyncReregisterLock = new Object - private def heartBeat(): Unit = { - if (!master.sendHeartBeat(blockManagerId)) { - reregister() - } - } - - private val heartBeatFrequency = BlockManager.getHeartBeatFrequency(conf) - private var heartBeatTask: Cancellable = null - private val metadataCleaner = new MetadataCleaner( MetadataCleanerType.BLOCK_MANAGER, this.dropOldNonBroadcastBlocks, conf) private val broadcastCleaner = new MetadataCleaner( @@ -149,9 +144,10 @@ private[spark] class BlockManager( serializer: Serializer, conf: SparkConf, securityManager: SecurityManager, - mapOutputTracker: MapOutputTracker) = { + mapOutputTracker: MapOutputTracker, + shuffleManager: ShuffleManager) = { this(execId, actorSystem, master, serializer, BlockManager.getMaxMemory(conf), - conf, securityManager, mapOutputTracker) + conf, securityManager, mapOutputTracker, shuffleManager) } /** @@ -161,11 +157,6 @@ private[spark] class BlockManager( private def initialize(): Unit = { master.registerBlockManager(blockManagerId, maxMemory, slaveActor) BlockManagerWorker.startBlockManagerWorker(this) - if (!BlockManager.getDisableHeartBeatsForTesting(conf)) { - heartBeatTask = actorSystem.scheduler.schedule(0.seconds, heartBeatFrequency.milliseconds) { - Utils.tryOrExit { heartBeat() } - } - } } /** @@ -195,7 +186,7 @@ private[spark] class BlockManager( * * Note that this method must be called without any BlockInfo locks held. */ - private def reregister(): Unit = { + def reregister(): Unit = { // TODO: We might need to rate limit re-registering. logInfo("BlockManager re-registering with master") master.registerBlockManager(blockManagerId, maxMemory, slaveActor) @@ -463,16 +454,17 @@ private[spark] class BlockManager( val values = dataDeserialize(blockId, bytes) if (level.deserialized) { // 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, returnValues = true).data - match { - case Left(values2) => - return Some(new BlockResult(values2, DataReadMethod.Disk, info.size)) - case _ => - throw new SparkException("Memory store did not return back an iterator") - } + val putResult = memoryStore.putIterator( + blockId, values, level, returnValues = true, allowPersistToDisk = false) + // The put may or may not have succeeded, depending on whether there was enough + // space to unroll the block. Either way, the put here should return an iterator. + putResult.data match { + case Left(it) => + return Some(new BlockResult(it, DataReadMethod.Disk, info.size)) + case _ => + // This only happens if we dropped the values back to disk (which is never) + throw new SparkException("Memory store did not return an iterator!") + } } else { return Some(new BlockResult(values, DataReadMethod.Disk, info.size)) } @@ -550,52 +542,59 @@ private[spark] class BlockManager( */ def getMultiple( blocksByAddress: Seq[(BlockManagerId, Seq[(BlockId, Long)])], - serializer: Serializer): BlockFetcherIterator = { + serializer: Serializer, + readMetrics: ShuffleReadMetrics): BlockFetcherIterator = { val iter = if (conf.getBoolean("spark.shuffle.use.netty", false)) { - new BlockFetcherIterator.NettyBlockFetcherIterator(this, blocksByAddress, serializer) + new BlockFetcherIterator.NettyBlockFetcherIterator(this, blocksByAddress, serializer, + readMetrics) } else { - new BlockFetcherIterator.BasicBlockFetcherIterator(this, blocksByAddress, serializer) + new BlockFetcherIterator.BasicBlockFetcherIterator(this, blocksByAddress, serializer, + readMetrics) } iter.initialize() iter } - def put( + def putIterator( blockId: BlockId, values: Iterator[Any], level: StorageLevel, - tellMaster: Boolean): Seq[(BlockId, BlockStatus)] = { + tellMaster: Boolean = true, + effectiveStorageLevel: Option[StorageLevel] = None): Seq[(BlockId, BlockStatus)] = { require(values != null, "Values is null") - doPut(blockId, IteratorValues(values), level, tellMaster) + doPut(blockId, IteratorValues(values), level, tellMaster, effectiveStorageLevel) } /** * A short circuited method to get a block writer that can write data directly to disk. - * The Block will be appended to the File specified by filename. This is currently used for - * writing shuffle files out. Callers should handle error cases. + * The Block will be appended to the File specified by filename. Callers should handle error + * cases. */ def getDiskWriter( blockId: BlockId, file: File, serializer: Serializer, - bufferSize: Int): BlockObjectWriter = { + bufferSize: Int, + writeMetrics: ShuffleWriteMetrics): BlockObjectWriter = { val compressStream: OutputStream => OutputStream = wrapForCompression(blockId, _) val syncWrites = conf.getBoolean("spark.shuffle.sync", false) - new DiskBlockObjectWriter(blockId, file, serializer, bufferSize, compressStream, syncWrites) + new DiskBlockObjectWriter(blockId, file, serializer, bufferSize, compressStream, syncWrites, + writeMetrics) } /** * Put a new block of values to the block manager. * Return a list of blocks updated as a result of this put. */ - def put( + def putArray( blockId: BlockId, - values: ArrayBuffer[Any], + values: Array[Any], level: StorageLevel, - tellMaster: Boolean = true): Seq[(BlockId, BlockStatus)] = { + tellMaster: Boolean = true, + effectiveStorageLevel: Option[StorageLevel] = None): Seq[(BlockId, BlockStatus)] = { require(values != null, "Values is null") - doPut(blockId, ArrayBufferValues(values), level, tellMaster) + doPut(blockId, ArrayValues(values), level, tellMaster, effectiveStorageLevel) } /** @@ -606,19 +605,33 @@ private[spark] class BlockManager( blockId: BlockId, bytes: ByteBuffer, level: StorageLevel, - tellMaster: Boolean = true): Seq[(BlockId, BlockStatus)] = { + tellMaster: Boolean = true, + effectiveStorageLevel: Option[StorageLevel] = None): Seq[(BlockId, BlockStatus)] = { require(bytes != null, "Bytes is null") - doPut(blockId, ByteBufferValues(bytes), level, tellMaster) + doPut(blockId, ByteBufferValues(bytes), level, tellMaster, effectiveStorageLevel) } + /** + * Put the given block according to the given level in one of the block stores, replicating + * the values if necessary. + * + * The effective storage level refers to the level according to which the block will actually be + * handled. This allows the caller to specify an alternate behavior of doPut while preserving + * the original level specified by the user. + */ private def doPut( blockId: BlockId, data: BlockValues, level: StorageLevel, - tellMaster: Boolean = true): Seq[(BlockId, BlockStatus)] = { + tellMaster: Boolean = true, + effectiveStorageLevel: Option[StorageLevel] = None) + : Seq[(BlockId, BlockStatus)] = { require(blockId != null, "BlockId is null") require(level != null && level.isValid, "StorageLevel is null or invalid") + effectiveStorageLevel.foreach { level => + require(level != null && level.isValid, "Effective StorageLevel is null or invalid") + } // Return value val updatedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] @@ -657,13 +670,16 @@ private[spark] class BlockManager( // Size of the block in bytes var size = 0L + // The level we actually use to put the block + val putLevel = effectiveStorageLevel.getOrElse(level) + // 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 = data match { - case b: ByteBufferValues if level.replication > 1 => + case b: ByteBufferValues if putLevel.replication > 1 => // Duplicate doesn't copy the bytes, but just creates a wrapper val bufferView = b.buffer.duplicate() - Future { replicate(blockId, bufferView, level) } + Future { replicate(blockId, bufferView, putLevel) } case _ => null } @@ -676,18 +692,18 @@ private[spark] class BlockManager( // returnValues - Whether to return the values put // blockStore - The type of storage to put these values into val (returnValues, blockStore: BlockStore) = { - if (level.useMemory) { + if (putLevel.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) { + } else if (putLevel.useOffHeap) { // Use tachyon for off-heap storage (false, tachyonStore) - } else if (level.useDisk) { + } else if (putLevel.useDisk) { // Don't get back the bytes from put unless we replicate them - (level.replication > 1, diskStore) + (putLevel.replication > 1, diskStore) } else { - assert(level == StorageLevel.NONE) + assert(putLevel == StorageLevel.NONE) throw new BlockException( blockId, s"Attempted to put block $blockId without specifying storage level!") } @@ -696,22 +712,22 @@ private[spark] class BlockManager( // 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) + blockStore.putIterator(blockId, iterator, putLevel, returnValues) + case ArrayValues(array) => + blockStore.putArray(blockId, array, putLevel, returnValues) case ByteBufferValues(bytes) => bytes.rewind() - blockStore.putBytes(blockId, bytes, level) + blockStore.putBytes(blockId, bytes, putLevel) } size = result.size result.data match { - case Left (newIterator) if level.useMemory => valuesAfterPut = newIterator + case Left (newIterator) if putLevel.useMemory => valuesAfterPut = newIterator case Right (newBytes) => bytesAfterPut = newBytes case _ => } // Keep track of which blocks are dropped from memory - if (level.useMemory) { + if (putLevel.useMemory) { result.droppedBlocks.foreach { updatedBlocks += _ } } @@ -742,7 +758,7 @@ private[spark] class BlockManager( // 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) { + if (putLevel.replication > 1) { data match { case ByteBufferValues(bytes) => if (replicationFuture != null) { @@ -758,7 +774,7 @@ private[spark] class BlockManager( } bytesAfterPut = dataSerialize(blockId, valuesAfterPut) } - replicate(blockId, bytesAfterPut, level) + replicate(blockId, bytesAfterPut, putLevel) logDebug("Put block %s remotely took %s" .format(blockId, Utils.getUsedTimeMs(remoteStartTime))) } @@ -766,7 +782,7 @@ private[spark] class BlockManager( BlockManager.dispose(bytesAfterPut) - if (level.replication > 1) { + if (putLevel.replication > 1) { logDebug("Putting block %s with replication took %s" .format(blockId, Utils.getUsedTimeMs(startTimeMs))) } else { @@ -818,7 +834,7 @@ private[spark] class BlockManager( value: Any, level: StorageLevel, tellMaster: Boolean = true): Seq[(BlockId, BlockStatus)] = { - put(blockId, Iterator(value), level, tellMaster) + putIterator(blockId, Iterator(value), level, tellMaster) } /** @@ -829,7 +845,7 @@ private[spark] class BlockManager( */ def dropFromMemory( blockId: BlockId, - data: Either[ArrayBuffer[Any], ByteBuffer]): Option[BlockStatus] = { + data: Either[Array[Any], ByteBuffer]): Option[BlockStatus] = { logInfo(s"Dropping block $blockId from memory") val info = blockInfo.get(blockId).orNull @@ -853,7 +869,7 @@ private[spark] class BlockManager( logInfo(s"Writing block $blockId to disk") data match { case Left(elements) => - diskStore.putValues(blockId, elements, level, returnValues = false) + diskStore.putArray(blockId, elements, level, returnValues = false) case Right(bytes) => diskStore.putBytes(blockId, bytes, level) } @@ -1045,9 +1061,6 @@ private[spark] class BlockManager( } def stop(): Unit = { - if (heartBeatTask != null) { - heartBeatTask.cancel() - } connectionManager.stop() shuffleBlockManager.stop() diskBlockManager.stop() @@ -1068,17 +1081,13 @@ private[spark] class BlockManager( private[spark] object BlockManager extends Logging { private val ID_GENERATOR = new IdGenerator + /** Return the total amount of storage memory available. */ private def getMaxMemory(conf: SparkConf): Long = { val memoryFraction = conf.getDouble("spark.storage.memoryFraction", 0.6) - (Runtime.getRuntime.maxMemory * memoryFraction).toLong + val safetyFraction = conf.getDouble("spark.storage.safetyFraction", 0.9) + (Runtime.getRuntime.maxMemory * memoryFraction * safetyFraction).toLong } - def getHeartBeatFrequency(conf: SparkConf): Long = - conf.getLong("spark.storage.blockManagerTimeoutIntervalMs", 60000) / 4 - - def getDisableHeartBeatsForTesting(conf: SparkConf): Boolean = - conf.getBoolean("spark.test.disableBlockManagerHeartBeat", false) - /** * Attempt to clean up a ByteBuffer if it is memory-mapped. This uses an *unsafe* Sun API that * might cause errors if one attempts to read from the unmapped buffer, but it's better than diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala index 7897fade2df2b..669307765d1fa 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -21,7 +21,6 @@ import scala.concurrent.{Await, Future} import scala.concurrent.ExecutionContext.Implicits.global import akka.actor._ -import akka.pattern.ask import org.apache.spark.{Logging, SparkConf, SparkException} import org.apache.spark.storage.BlockManagerMessages._ @@ -29,8 +28,8 @@ import org.apache.spark.util.AkkaUtils private[spark] class BlockManagerMaster(var driverActor: ActorRef, conf: SparkConf) extends Logging { - val AKKA_RETRY_ATTEMPTS: Int = conf.getInt("spark.akka.num.retries", 3) - val AKKA_RETRY_INTERVAL_MS: Int = conf.getInt("spark.akka.retry.wait", 3000) + private val AKKA_RETRY_ATTEMPTS: Int = AkkaUtils.numRetries(conf) + private val AKKA_RETRY_INTERVAL_MS: Int = AkkaUtils.retryWaitMs(conf) val DRIVER_AKKA_ACTOR_NAME = "BlockManagerMaster" @@ -42,15 +41,6 @@ class BlockManagerMaster(var driverActor: ActorRef, conf: SparkConf) extends Log logInfo("Removed " + execId + " successfully in removeExecutor") } - /** - * Send the driver actor a heart beat from the slave. Returns true if everything works out, - * false if the driver does not know about the given block manager, which means the block - * manager should re-register. - */ - def sendHeartBeat(blockManagerId: BlockManagerId): Boolean = { - askDriverWithReply[Boolean](HeartBeat(blockManagerId)) - } - /** Register the BlockManager's id with the driver. */ def registerBlockManager(blockManagerId: BlockManagerId, maxMemSize: Long, slaveActor: ActorRef) { logInfo("Trying to register BlockManager") @@ -223,33 +213,8 @@ class BlockManagerMaster(var driverActor: ActorRef, conf: SparkConf) extends Log * throw a SparkException if this fails. */ private def askDriverWithReply[T](message: Any): T = { - // TODO: Consider removing multiple attempts - if (driverActor == null) { - throw new SparkException("Error sending message to BlockManager as driverActor is null " + - "[message = " + message + "]") - } - var attempts = 0 - var lastException: Exception = null - while (attempts < AKKA_RETRY_ATTEMPTS) { - attempts += 1 - try { - val future = driverActor.ask(message)(timeout) - val result = Await.result(future, timeout) - if (result == null) { - throw new SparkException("BlockManagerMaster returned null") - } - return result.asInstanceOf[T] - } catch { - case ie: InterruptedException => throw ie - case e: Exception => - lastException = e - logWarning("Error sending message to BlockManagerMaster in " + attempts + " attempts", e) - } - Thread.sleep(AKKA_RETRY_INTERVAL_MS) - } - - throw new SparkException( - "Error sending message to BlockManagerMaster [message = " + message + "]", lastException) + AkkaUtils.askWithReply(message, driverActor, AKKA_RETRY_ATTEMPTS, AKKA_RETRY_INTERVAL_MS, + timeout) } } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala index 6aed322eeb185..3ab07703b6f85 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -31,7 +31,7 @@ import org.apache.spark.{Logging, SparkConf, SparkException} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.scheduler._ import org.apache.spark.storage.BlockManagerMessages._ -import org.apache.spark.util.{AkkaUtils, Utils} +import org.apache.spark.util.{ActorLogReceive, AkkaUtils, Utils} /** * BlockManagerMasterActor is an actor on the master node to track statuses of @@ -39,7 +39,7 @@ import org.apache.spark.util.{AkkaUtils, Utils} */ private[spark] class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus: LiveListenerBus) - extends Actor with Logging { + extends Actor with ActorLogReceive with Logging { // Mapping from block manager id to the block manager's information. private val blockManagerInfo = new mutable.HashMap[BlockManagerId, BlockManagerInfo] @@ -52,24 +52,21 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus private val akkaTimeout = AkkaUtils.askTimeout(conf) - val slaveTimeout = conf.get("spark.storage.blockManagerSlaveTimeoutMs", - "" + (BlockManager.getHeartBeatFrequency(conf) * 3)).toLong + val slaveTimeout = conf.getLong("spark.storage.blockManagerSlaveTimeoutMs", + math.max(conf.getInt("spark.executor.heartbeatInterval", 10000) * 3, 45000)) - val checkTimeoutInterval = conf.get("spark.storage.blockManagerTimeoutIntervalMs", - "60000").toLong + val checkTimeoutInterval = conf.getLong("spark.storage.blockManagerTimeoutIntervalMs", 60000) var timeoutCheckingTask: Cancellable = null override def preStart() { - if (!BlockManager.getDisableHeartBeatsForTesting(conf)) { - import context.dispatcher - timeoutCheckingTask = context.system.scheduler.schedule(0.seconds, - checkTimeoutInterval.milliseconds, self, ExpireDeadHosts) - } + import context.dispatcher + timeoutCheckingTask = context.system.scheduler.schedule(0.seconds, + checkTimeoutInterval.milliseconds, self, ExpireDeadHosts) super.preStart() } - def receive = { + override def receiveWithLogging = { case RegisterBlockManager(blockManagerId, maxMemSize, slaveActor) => register(blockManagerId, maxMemSize, slaveActor) sender ! true @@ -119,7 +116,6 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus sender ! true case StopBlockManagerMaster => - logInfo("Stopping BlockManagerMaster") sender ! true if (timeoutCheckingTask != null) { timeoutCheckingTask.cancel() @@ -129,8 +125,8 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus case ExpireDeadHosts => expireDeadHosts() - case HeartBeat(blockManagerId) => - sender ! heartBeat(blockManagerId) + case BlockManagerHeartbeat(blockManagerId) => + sender ! heartbeatReceived(blockManagerId) case other => logWarning("Got unknown message: " + other) @@ -216,7 +212,7 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus val minSeenTime = now - slaveTimeout val toRemove = new mutable.HashSet[BlockManagerId] for (info <- blockManagerInfo.values) { - if (info.lastSeenMs < minSeenTime) { + if (info.lastSeenMs < minSeenTime && info.blockManagerId.executorId != "") { logWarning("Removing BlockManager " + info.blockManagerId + " with no recent heart beats: " + (now - info.lastSeenMs) + "ms exceeds " + slaveTimeout + "ms") toRemove += info.blockManagerId @@ -230,7 +226,11 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus blockManagerIdByExecutor.get(execId).foreach(removeBlockManager) } - private def heartBeat(blockManagerId: BlockManagerId): Boolean = { + /** + * Return true if the driver knows about the given block manager. Otherwise, return false, + * indicating that the block manager should re-register. + */ + private def heartbeatReceived(blockManagerId: BlockManagerId): Boolean = { if (!blockManagerInfo.contains(blockManagerId)) { blockManagerId.executorId == "" && !isLocal } else { @@ -264,9 +264,8 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus } private def storageStatus: Array[StorageStatus] = { - blockManagerInfo.map { case(blockManagerId, info) => - val blockMap = mutable.Map[BlockId, BlockStatus](info.blocks.toSeq: _*) - new StorageStatus(blockManagerId, info.maxMem, blockMap) + blockManagerInfo.map { case (blockManagerId, info) => + new StorageStatus(blockManagerId, info.maxMem, info.blocks) }.toArray } @@ -336,6 +335,10 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus case None => blockManagerIdByExecutor(id.executorId) = id } + + logInfo("Registering block manager %s with %s RAM".format( + id.hostPort, Utils.bytesToString(maxMemSize))) + blockManagerInfo(id) = new BlockManagerInfo(id, System.currentTimeMillis(), maxMemSize, slaveActor) } @@ -417,7 +420,14 @@ case class BlockStatus( storageLevel: StorageLevel, memSize: Long, diskSize: Long, - tachyonSize: Long) + tachyonSize: Long) { + def isCached: Boolean = memSize + diskSize + tachyonSize > 0 +} + +@DeveloperApi +object BlockStatus { + def empty: BlockStatus = BlockStatus(StorageLevel.NONE, 0L, 0L, 0L) +} private[spark] class BlockManagerInfo( val blockManagerId: BlockManagerId, @@ -432,9 +442,6 @@ private[spark] class BlockManagerInfo( // Mapping from block id to its status. private val _blocks = new JHashMap[BlockId, BlockStatus] - logInfo("Registering block manager %s with %s RAM".format( - blockManagerId.hostPort, Utils.bytesToString(maxMem))) - def getStatus(blockId: BlockId) = Option(_blocks.get(blockId)) def updateLastSeenMs() { diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala index 2b53bf33b5fba..2ba16b8476600 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala @@ -21,7 +21,7 @@ import java.io.{Externalizable, ObjectInput, ObjectOutput} import akka.actor.ActorRef -private[storage] object BlockManagerMessages { +private[spark] object BlockManagerMessages { ////////////////////////////////////////////////////////////////////////////////// // Messages from the master to slaves. ////////////////////////////////////////////////////////////////////////////////// @@ -53,9 +53,7 @@ private[storage] object BlockManagerMessages { sender: ActorRef) extends ToBlockManagerMaster - case class HeartBeat(blockManagerId: BlockManagerId) extends ToBlockManagerMaster - - class UpdateBlockInfo( + case class UpdateBlockInfo( var blockManagerId: BlockManagerId, var blockId: BlockId, var storageLevel: StorageLevel, @@ -86,24 +84,6 @@ private[storage] object BlockManagerMessages { } } - object UpdateBlockInfo { - def apply( - blockManagerId: BlockManagerId, - blockId: BlockId, - storageLevel: StorageLevel, - memSize: Long, - diskSize: Long, - tachyonSize: Long): UpdateBlockInfo = { - new UpdateBlockInfo(blockManagerId, blockId, storageLevel, memSize, diskSize, tachyonSize) - } - - // For pattern-matching - def unapply(h: UpdateBlockInfo) - : Option[(BlockManagerId, BlockId, StorageLevel, Long, Long, Long)] = { - Some((h.blockManagerId, h.blockId, h.storageLevel, h.memSize, h.diskSize, h.tachyonSize)) - } - } - case class GetLocations(blockId: BlockId) extends ToBlockManagerMaster case class GetLocationsMultipleBlockIds(blockIds: Array[BlockId]) extends ToBlockManagerMaster @@ -124,5 +104,7 @@ private[storage] object BlockManagerMessages { case class GetMatchingBlockIds(filter: BlockId => Boolean, askSlaves: Boolean = true) extends ToBlockManagerMaster + case class BlockManagerHeartbeat(blockManagerId: BlockManagerId) extends ToBlockManagerMaster + case object ExpireDeadHosts extends ToBlockManagerMaster } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveActor.scala index 6d4db064dff58..c194e0fed3367 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveActor.scala @@ -23,6 +23,7 @@ import akka.actor.{ActorRef, Actor} import org.apache.spark.{Logging, MapOutputTracker} import org.apache.spark.storage.BlockManagerMessages._ +import org.apache.spark.util.ActorLogReceive /** * An actor to take commands from the master to execute options. For example, @@ -32,12 +33,12 @@ private[storage] class BlockManagerSlaveActor( blockManager: BlockManager, mapOutputTracker: MapOutputTracker) - extends Actor with Logging { + extends Actor with ActorLogReceive with Logging { import context.dispatcher // Operations that involve removing blocks may be slow and should be done asynchronously - override def receive = { + override def receiveWithLogging = { case RemoveBlock(blockId) => doAsync[Boolean]("removing block " + blockId, sender) { blockManager.removeBlock(blockId) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala index 687586490abfe..49fea6d9e2a76 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala @@ -24,13 +24,13 @@ import org.apache.spark.metrics.source.Source private[spark] class BlockManagerSource(val blockManager: BlockManager, sc: SparkContext) extends Source { - val metricRegistry = new MetricRegistry() - val sourceName = "%s.BlockManager".format(sc.appName) + override val metricRegistry = new MetricRegistry() + override val sourceName = "%s.BlockManager".format(sc.appName) metricRegistry.register(MetricRegistry.name("memory", "maxMem_MB"), new Gauge[Long] { override def getValue: Long = { val storageStatusList = blockManager.master.getStorageStatus - val maxMem = storageStatusList.map(_.maxMem).reduce(_ + _) + val maxMem = storageStatusList.map(_.maxMem).sum maxMem / 1024 / 1024 } }) @@ -38,7 +38,7 @@ private[spark] class BlockManagerSource(val blockManager: BlockManager, sc: Spar metricRegistry.register(MetricRegistry.name("memory", "remainingMem_MB"), new Gauge[Long] { override def getValue: Long = { val storageStatusList = blockManager.master.getStorageStatus - val remainingMem = storageStatusList.map(_.memRemaining).reduce(_ + _) + val remainingMem = storageStatusList.map(_.memRemaining).sum remainingMem / 1024 / 1024 } }) @@ -46,20 +46,15 @@ private[spark] class BlockManagerSource(val blockManager: BlockManager, sc: Spar metricRegistry.register(MetricRegistry.name("memory", "memUsed_MB"), new Gauge[Long] { override def getValue: Long = { val storageStatusList = blockManager.master.getStorageStatus - val maxMem = storageStatusList.map(_.maxMem).reduce(_ + _) - val remainingMem = storageStatusList.map(_.memRemaining).reduce(_ + _) - (maxMem - remainingMem) / 1024 / 1024 + val memUsed = storageStatusList.map(_.memUsed).sum + memUsed / 1024 / 1024 } }) metricRegistry.register(MetricRegistry.name("disk", "diskSpaceUsed_MB"), new Gauge[Long] { override def getValue: Long = { val storageStatusList = blockManager.master.getStorageStatus - val diskSpaceUsed = storageStatusList - .flatMap(_.blocks.values.map(_.diskSize)) - .reduceOption(_ + _) - .getOrElse(0L) - + val diskSpaceUsed = storageStatusList.map(_.diskUsed).sum diskSpaceUsed / 1024 / 1024 } }) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerWorker.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerWorker.scala index c7766a3a65671..bf002a42d5dc5 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerWorker.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerWorker.scala @@ -23,6 +23,10 @@ import org.apache.spark.Logging import org.apache.spark.network._ import org.apache.spark.util.Utils +import scala.concurrent.Await +import scala.concurrent.duration.Duration +import scala.util.{Try, Failure, Success} + /** * A network interface for BlockManager. Each slave should have one * BlockManagerWorker. @@ -44,13 +48,19 @@ private[spark] class BlockManagerWorker(val blockManager: BlockManager) extends val responseMessages = blockMessages.map(processBlockMessage).filter(_ != None).map(_.get) Some(new BlockMessageArray(responseMessages).toBufferMessage) } catch { - case e: Exception => logError("Exception handling buffer message", e) - None + case e: Exception => { + logError("Exception handling buffer message", e) + val errorMessage = Message.createBufferMessage(msg.id) + errorMessage.hasError = true + Some(errorMessage) + } } } case otherMessage: Any => { logError("Unknown type message received: " + otherMessage) - None + val errorMessage = Message.createBufferMessage(msg.id) + errorMessage.hasError = true + Some(errorMessage) } } } @@ -109,9 +119,9 @@ private[spark] object BlockManagerWorker extends Logging { val connectionManager = blockManager.connectionManager val blockMessage = BlockMessage.fromPutBlock(msg) val blockMessageArray = new BlockMessageArray(blockMessage) - val resultMessage = connectionManager.sendMessageReliablySync( - toConnManagerId, blockMessageArray.toBufferMessage) - resultMessage.isDefined + val resultMessage = Try(Await.result(connectionManager.sendMessageReliably( + toConnManagerId, blockMessageArray.toBufferMessage), Duration.Inf)) + resultMessage.isSuccess } def syncGetBlock(msg: GetBlock, toConnManagerId: ConnectionManagerId): ByteBuffer = { @@ -119,10 +129,10 @@ private[spark] object BlockManagerWorker extends Logging { val connectionManager = blockManager.connectionManager val blockMessage = BlockMessage.fromGetBlock(msg) val blockMessageArray = new BlockMessageArray(blockMessage) - val responseMessage = connectionManager.sendMessageReliablySync( - toConnManagerId, blockMessageArray.toBufferMessage) + val responseMessage = Try(Await.result(connectionManager.sendMessageReliably( + toConnManagerId, blockMessageArray.toBufferMessage), Duration.Inf)) responseMessage match { - case Some(message) => { + case Success(message) => { val bufferMessage = message.asInstanceOf[BufferMessage] logDebug("Response message received " + bufferMessage) BlockMessageArray.fromBufferMessage(bufferMessage).foreach(blockMessage => { @@ -130,7 +140,7 @@ private[spark] object BlockManagerWorker extends Logging { return blockMessage.getData }) } - case None => logDebug("No response message received") + case Failure(exception) => logDebug("No response message received") } null } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala index a2687e6be4e34..adda971fd7b47 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala @@ -22,6 +22,7 @@ import java.nio.channels.FileChannel import org.apache.spark.Logging import org.apache.spark.serializer.{SerializationStream, Serializer} +import org.apache.spark.executor.ShuffleWriteMetrics /** * An interface for writing JVM objects to some underlying storage. This interface allows @@ -39,16 +40,16 @@ private[spark] abstract class BlockObjectWriter(val blockId: BlockId) { def isOpen: Boolean /** - * Flush the partial writes and commit them as a single atomic block. Return the - * number of bytes written for this commit. + * Flush the partial writes and commit them as a single atomic block. */ - def commit(): Long + def commitAndClose(): Unit /** * Reverts writes that haven't been flushed yet. Callers should invoke this function - * when there are runtime exceptions. + * when there are runtime exceptions. This method will not throw, though it may be + * unsuccessful in truncating written data. */ - def revertPartialWrites() + def revertPartialWritesAndClose() /** * Writes an object. @@ -57,43 +58,29 @@ private[spark] abstract class BlockObjectWriter(val blockId: BlockId) { /** * Returns the file segment of committed data that this Writer has written. + * This is only valid after commitAndClose() has been called. */ def fileSegment(): FileSegment - - /** - * Cumulative time spent performing blocking writes, in ns. - */ - def timeWriting(): Long - - /** - * Number of bytes written so far - */ - def bytesWritten: Long } -/** BlockObjectWriter which writes directly to a file on disk. Appends to the given file. */ +/** + * BlockObjectWriter which writes directly to a file on disk. Appends to the given file. + * The given write metrics will be updated incrementally, but will not necessarily be current until + * commitAndClose is called. + */ private[spark] class DiskBlockObjectWriter( blockId: BlockId, file: File, serializer: Serializer, bufferSize: Int, compressStream: OutputStream => OutputStream, - syncWrites: Boolean) + syncWrites: Boolean, + writeMetrics: ShuffleWriteMetrics) extends BlockObjectWriter(blockId) with Logging { - /** Intercepts write calls and tracks total time spent writing. Not thread safe. */ private class TimeTrackingOutputStream(out: OutputStream) extends OutputStream { - def timeWriting = _timeWriting - private var _timeWriting = 0L - - private def callWithTiming(f: => Unit) = { - val start = System.nanoTime() - f - _timeWriting += (System.nanoTime() - start) - } - def write(i: Int): Unit = callWithTiming(out.write(i)) override def write(b: Array[Byte]) = callWithTiming(out.write(b)) override def write(b: Array[Byte], off: Int, len: Int) = callWithTiming(out.write(b, off, len)) @@ -108,15 +95,18 @@ private[spark] class DiskBlockObjectWriter( private var ts: TimeTrackingOutputStream = null private var objOut: SerializationStream = null private val initialPosition = file.length() - private var lastValidPosition = initialPosition + private var finalPosition: Long = -1 private var initialized = false - private var _timeWriting = 0L + + /** Calling channel.position() to update the write metrics can be a little bit expensive, so we + * only call it every N writes */ + private var writesSinceMetricsUpdate = 0 + private var lastPosition = initialPosition override def open(): BlockObjectWriter = { fos = new FileOutputStream(file, true) ts = new TimeTrackingOutputStream(fos) channel = fos.getChannel() - lastValidPosition = initialPosition bs = compressStream(new BufferedOutputStream(ts, bufferSize)) objOut = serializer.newInstance().serializeStream(bs) initialized = true @@ -128,14 +118,11 @@ private[spark] class DiskBlockObjectWriter( if (syncWrites) { // Force outstanding writes to disk and track how long it takes objOut.flush() - val start = System.nanoTime() - fos.getFD.sync() - _timeWriting += System.nanoTime() - start + def sync = fos.getFD.sync() + callWithTiming(sync) } objOut.close() - _timeWriting += ts.timeWriting - channel = null bs = null fos = null @@ -147,28 +134,39 @@ private[spark] class DiskBlockObjectWriter( override def isOpen: Boolean = objOut != null - override def commit(): Long = { + override def commitAndClose(): Unit = { if (initialized) { // NOTE: Because Kryo doesn't flush the underlying stream we explicitly flush both the // serializer stream and the lower level stream. objOut.flush() bs.flush() - val prevPos = lastValidPosition - lastValidPosition = channel.position() - lastValidPosition - prevPos - } else { - // lastValidPosition is zero if stream is uninitialized - lastValidPosition + updateBytesWritten() + close() } + finalPosition = file.length() } - override def revertPartialWrites() { - if (initialized) { - // Discard current writes. We do this by flushing the outstanding writes and - // truncate the file to the last valid position. - objOut.flush() - bs.flush() - channel.truncate(lastValidPosition) + // Discard current writes. We do this by flushing the outstanding writes and then + // truncating the file to its initial position. + override def revertPartialWritesAndClose() { + try { + writeMetrics.shuffleBytesWritten -= (lastPosition - initialPosition) + + if (initialized) { + objOut.flush() + bs.flush() + close() + } + + val truncateStream = new FileOutputStream(file, true) + try { + truncateStream.getChannel.truncate(initialPosition) + } finally { + truncateStream.close() + } + } catch { + case e: Exception => + logError("Uncaught exception while reverting partial writes to file " + file, e) } } @@ -176,18 +174,36 @@ private[spark] class DiskBlockObjectWriter( if (!initialized) { open() } + objOut.writeObject(value) + + if (writesSinceMetricsUpdate == 32) { + writesSinceMetricsUpdate = 0 + updateBytesWritten() + } else { + writesSinceMetricsUpdate += 1 + } } override def fileSegment(): FileSegment = { - new FileSegment(file, initialPosition, bytesWritten) + new FileSegment(file, initialPosition, finalPosition - initialPosition) } - // Only valid if called after close() - override def timeWriting() = _timeWriting + private def updateBytesWritten() { + val pos = channel.position() + writeMetrics.shuffleBytesWritten += (pos - lastPosition) + lastPosition = pos + } + + private def callWithTiming(f: => Unit) = { + val start = System.nanoTime() + f + writeMetrics.shuffleWriteTime += (System.nanoTime() - start) + } - // Only valid if called after commit() - override def bytesWritten: Long = { - lastValidPosition - initialPosition + // For testing + private[spark] def flush() { + objOut.flush() + bs.flush() } } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockStore.scala b/core/src/main/scala/org/apache/spark/storage/BlockStore.scala index b9b53b1a2f118..69985c9759e2d 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockStore.scala @@ -37,15 +37,15 @@ private[spark] abstract class BlockStore(val blockManager: BlockManager) extends * @return a PutResult that contains the size of the data, as well as the values put if * returnValues is true (if not, the result's data field can be null) */ - def putValues( + def putIterator( blockId: BlockId, values: Iterator[Any], level: StorageLevel, returnValues: Boolean): PutResult - def putValues( + def putArray( blockId: BlockId, - values: ArrayBuffer[Any], + values: Array[Any], level: StorageLevel, returnValues: Boolean): PutResult 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 673fc19c060a4..4d66ccea211fa 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -21,10 +21,11 @@ import java.io.File import java.text.SimpleDateFormat import java.util.{Date, Random, UUID} -import org.apache.spark.Logging +import org.apache.spark.{SparkEnv, Logging} import org.apache.spark.executor.ExecutorExitCode import org.apache.spark.network.netty.{PathResolver, ShuffleSender} import org.apache.spark.util.Utils +import org.apache.spark.shuffle.sort.SortShuffleManager /** * Creates and maintains the logical mapping between logical blocks and physical on-disk @@ -34,16 +35,18 @@ import org.apache.spark.util.Utils * * @param rootDirs The directories to use for storing block files. Data will be hashed among these. */ -private[spark] class DiskBlockManager(shuffleManager: ShuffleBlockManager, rootDirs: String) +private[spark] class DiskBlockManager(shuffleBlockManager: ShuffleBlockManager, rootDirs: String) extends PathResolver with Logging { private val MAX_DIR_CREATION_ATTEMPTS: Int = 10 - private val subDirsPerLocalDir = shuffleManager.conf.getInt("spark.diskStore.subDirectories", 64) + + private val subDirsPerLocalDir = + shuffleBlockManager.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. */ - private val localDirs: Array[File] = createLocalDirs() + val localDirs: Array[File] = createLocalDirs() if (localDirs.isEmpty) { logError("Failed to create any local dir.") System.exit(ExecutorExitCode.DISK_STORE_FAILED_TO_CREATE_DIR) @@ -54,13 +57,19 @@ private[spark] class DiskBlockManager(shuffleManager: ShuffleBlockManager, rootD addShutdownHook() /** - * Returns the physical file segment in which the given BlockId is located. - * If the BlockId has been mapped to a specific FileSegment, that will be returned. - * Otherwise, we assume the Block is mapped to a whole file identified by the BlockId directly. + * Returns the physical file segment in which the given BlockId is located. If the BlockId has + * been mapped to a specific FileSegment by the shuffle layer, that will be returned. + * Otherwise, we assume the Block is mapped to the whole file identified by the BlockId. */ def getBlockLocation(blockId: BlockId): FileSegment = { - if (blockId.isShuffle && shuffleManager.consolidateShuffleFiles) { - shuffleManager.getBlockLocation(blockId.asInstanceOf[ShuffleBlockId]) + val env = SparkEnv.get // NOTE: can be null in unit tests + if (blockId.isShuffle && env != null && env.shuffleManager.isInstanceOf[SortShuffleManager]) { + // For sort-based shuffle, let it figure out its blocks + val sortShuffleManager = env.shuffleManager.asInstanceOf[SortShuffleManager] + sortShuffleManager.getBlockLocation(blockId.asInstanceOf[ShuffleBlockId], this) + } else if (blockId.isShuffle && shuffleBlockManager.consolidateShuffleFiles) { + // For hash-based shuffle with consolidated files, ShuffleBlockManager takes care of this + shuffleBlockManager.getBlockLocation(blockId.asInstanceOf[ShuffleBlockId]) } else { val file = getFile(blockId.name) new FileSegment(file, 0, file.length()) @@ -99,13 +108,18 @@ private[spark] class DiskBlockManager(shuffleManager: ShuffleBlockManager, rootD getBlockLocation(blockId).file.exists() } - /** List all the blocks currently stored on disk by the disk manager. */ - def getAllBlocks(): Seq[BlockId] = { + /** List all the files currently stored on disk by the disk manager. */ + def getAllFiles(): Seq[File] = { // Get all the files inside the array of array of directories subDirs.flatten.filter(_ != null).flatMap { dir => - val files = dir.list() + val files = dir.listFiles() if (files != null) files else Seq.empty - }.map(BlockId.apply) + } + } + + /** List all the blocks currently stored on disk by the disk manager. */ + def getAllBlocks(): Seq[BlockId] = { + getAllFiles().map(f => BlockId(f.getName)) } /** Produces a unique block id and File suitable for intermediate results. */ 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 ebff0cb5ba153..c83261dd91b36 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala @@ -21,8 +21,6 @@ import java.io.{FileOutputStream, RandomAccessFile} import java.nio.ByteBuffer import java.nio.channels.FileChannel.MapMode -import scala.collection.mutable.ArrayBuffer - import org.apache.spark.Logging import org.apache.spark.serializer.Serializer import org.apache.spark.util.Utils @@ -30,7 +28,7 @@ import org.apache.spark.util.Utils /** * Stores BlockManager blocks on disk. */ -private class DiskStore(blockManager: BlockManager, diskManager: DiskBlockManager) +private[spark] class DiskStore(blockManager: BlockManager, diskManager: DiskBlockManager) extends BlockStore(blockManager) with Logging { val minMemoryMapBytes = blockManager.conf.getLong("spark.storage.memoryMapThreshold", 2 * 4096L) @@ -57,15 +55,15 @@ private class DiskStore(blockManager: BlockManager, diskManager: DiskBlockManage PutResult(bytes.limit(), Right(bytes.duplicate())) } - override def putValues( + override def putArray( blockId: BlockId, - values: ArrayBuffer[Any], + values: Array[Any], level: StorageLevel, returnValues: Boolean): PutResult = { - putValues(blockId, values.toIterator, level, returnValues) + putIterator(blockId, values.toIterator, level, returnValues) } - override def putValues( + override def putIterator( blockId: BlockId, values: Iterator[Any], level: StorageLevel, 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 71f66c826c5b3..0a09c24d61879 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -20,27 +20,45 @@ package org.apache.spark.storage import java.nio.ByteBuffer import java.util.LinkedHashMap +import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import org.apache.spark.util.{SizeEstimator, Utils} +import org.apache.spark.util.collection.SizeTrackingVector private case class MemoryEntry(value: Any, size: Long, deserialized: Boolean) /** - * Stores blocks in memory, either as ArrayBuffers of deserialized Java objects or as + * Stores blocks in memory, either as Arrays of deserialized Java objects or as * serialized ByteBuffers. */ -private class MemoryStore(blockManager: BlockManager, maxMemory: Long) +private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) extends BlockStore(blockManager) { + private val conf = blockManager.conf 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() + + // Ensure only one thread is putting, and if necessary, dropping blocks at any given time + private val accountingLock = new Object + + // A mapping from thread ID to amount of memory used for unrolling a block (in bytes) + // All accesses of this map are assumed to have manually synchronized on `accountingLock` + private val unrollMemoryMap = mutable.HashMap[Long, Long]() + + /** + * The amount of space ensured for unrolling values in memory, shared across all cores. + * This space is not reserved in advance, but allocated dynamically by dropping existing blocks. + */ + private val maxUnrollMemory: Long = { + val unrollFraction = conf.getDouble("spark.storage.unrollFraction", 0.2) + (maxMemory * unrollFraction).toLong + } logInfo("MemoryStore started with capacity %s".format(Utils.bytesToString(maxMemory))) + /** Free memory not occupied by existing blocks. Note that this does not include unroll memory. */ def freeMemory: Long = maxMemory - currentMemory override def getSize(blockId: BlockId): Long = { @@ -55,20 +73,16 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) bytes.rewind() if (level.deserialized) { val values = blockManager.dataDeserialize(blockId, bytes) - val elements = new ArrayBuffer[Any] - elements ++= values - val sizeEstimate = SizeEstimator.estimate(elements.asInstanceOf[AnyRef]) - val putAttempt = tryToPut(blockId, elements, sizeEstimate, deserialized = true) - PutResult(sizeEstimate, Left(values.toIterator), putAttempt.droppedBlocks) + putIterator(blockId, values, level, returnValues = true) } else { val putAttempt = tryToPut(blockId, bytes, bytes.limit, deserialized = false) PutResult(bytes.limit(), Right(bytes.duplicate()), putAttempt.droppedBlocks) } } - override def putValues( + override def putArray( blockId: BlockId, - values: ArrayBuffer[Any], + values: Array[Any], level: StorageLevel, returnValues: Boolean): PutResult = { if (level.deserialized) { @@ -82,14 +96,52 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) } } - override def putValues( + override def putIterator( blockId: BlockId, values: Iterator[Any], level: StorageLevel, returnValues: Boolean): PutResult = { - val valueEntries = new ArrayBuffer[Any]() - valueEntries ++= values - putValues(blockId, valueEntries, level, returnValues) + putIterator(blockId, values, level, returnValues, allowPersistToDisk = true) + } + + /** + * Attempt to put the given block in memory store. + * + * There may not be enough space to fully unroll the iterator in memory, in which case we + * optionally drop the values to disk if + * (1) the block's storage level specifies useDisk, and + * (2) `allowPersistToDisk` is true. + * + * One scenario in which `allowPersistToDisk` is false is when the BlockManager reads a block + * back from disk and attempts to cache it in memory. In this case, we should not persist the + * block back on disk again, as it is already in disk store. + */ + private[storage] def putIterator( + blockId: BlockId, + values: Iterator[Any], + level: StorageLevel, + returnValues: Boolean, + allowPersistToDisk: Boolean): PutResult = { + val droppedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] + val unrolledValues = unrollSafely(blockId, values, droppedBlocks) + unrolledValues match { + case Left(arrayValues) => + // Values are fully unrolled in memory, so store them as an array + val res = putArray(blockId, arrayValues, level, returnValues) + droppedBlocks ++= res.droppedBlocks + PutResult(res.size, res.data, droppedBlocks) + case Right(iteratorValues) => + // Not enough space to unroll this block; drop to disk if applicable + logWarning(s"Not enough space to store block $blockId in memory! " + + s"Free memory is $freeMemory bytes.") + if (level.useDisk && allowPersistToDisk) { + logWarning(s"Persisting block $blockId to disk instead.") + val res = blockManager.diskStore.putIterator(blockId, iteratorValues, level, returnValues) + PutResult(res.size, res.data, droppedBlocks) + } else { + PutResult(0, Left(iteratorValues), droppedBlocks) + } + } } override def getBytes(blockId: BlockId): Option[ByteBuffer] = { @@ -99,7 +151,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) if (entry == null) { None } else if (entry.deserialized) { - Some(blockManager.dataSerialize(blockId, entry.value.asInstanceOf[ArrayBuffer[Any]].iterator)) + Some(blockManager.dataSerialize(blockId, entry.value.asInstanceOf[Array[Any]].iterator)) } else { Some(entry.value.asInstanceOf[ByteBuffer].duplicate()) // Doesn't actually copy the data } @@ -112,7 +164,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) if (entry == null) { None } else if (entry.deserialized) { - Some(entry.value.asInstanceOf[ArrayBuffer[Any]].iterator) + Some(entry.value.asInstanceOf[Array[Any]].iterator) } else { val buffer = entry.value.asInstanceOf[ByteBuffer].duplicate() // Doesn't actually copy data Some(blockManager.dataDeserialize(blockId, buffer)) @@ -140,6 +192,93 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) logInfo("MemoryStore cleared") } + /** + * Unroll the given block in memory safely. + * + * The safety of this operation refers to avoiding potential OOM exceptions caused by + * unrolling the entirety of the block in memory at once. This is achieved by periodically + * checking whether the memory restrictions for unrolling blocks are still satisfied, + * stopping immediately if not. This check is a safeguard against the scenario in which + * there is not enough free memory to accommodate the entirety of a single block. + * + * This method returns either an array with the contents of the entire block or an iterator + * containing the values of the block (if the array would have exceeded available memory). + */ + def unrollSafely( + blockId: BlockId, + values: Iterator[Any], + droppedBlocks: ArrayBuffer[(BlockId, BlockStatus)]) + : Either[Array[Any], Iterator[Any]] = { + + // Number of elements unrolled so far + var elementsUnrolled = 0 + // Whether there is still enough memory for us to continue unrolling this block + var keepUnrolling = true + // Initial per-thread memory to request for unrolling blocks (bytes). Exposed for testing. + val initialMemoryThreshold = conf.getLong("spark.storage.unrollMemoryThreshold", 1024 * 1024) + // How often to check whether we need to request more memory + val memoryCheckPeriod = 16 + // Memory currently reserved by this thread for this particular unrolling operation + var memoryThreshold = initialMemoryThreshold + // Memory to request as a multiple of current vector size + val memoryGrowthFactor = 1.5 + // Previous unroll memory held by this thread, for releasing later (only at the very end) + val previousMemoryReserved = currentUnrollMemoryForThisThread + // Underlying vector for unrolling the block + var vector = new SizeTrackingVector[Any] + + // Request enough memory to begin unrolling + keepUnrolling = reserveUnrollMemoryForThisThread(initialMemoryThreshold) + + // Unroll this block safely, checking whether we have exceeded our threshold periodically + try { + while (values.hasNext && keepUnrolling) { + vector += values.next() + if (elementsUnrolled % memoryCheckPeriod == 0) { + // If our vector's size has exceeded the threshold, request more memory + val currentSize = vector.estimateSize() + if (currentSize >= memoryThreshold) { + val amountToRequest = (currentSize * memoryGrowthFactor - memoryThreshold).toLong + // Hold the accounting lock, in case another thread concurrently puts a block that + // takes up the unrolling space we just ensured here + accountingLock.synchronized { + if (!reserveUnrollMemoryForThisThread(amountToRequest)) { + // If the first request is not granted, try again after ensuring free space + // If there is still not enough space, give up and drop the partition + val spaceToEnsure = maxUnrollMemory - currentUnrollMemory + if (spaceToEnsure > 0) { + val result = ensureFreeSpace(blockId, spaceToEnsure) + droppedBlocks ++= result.droppedBlocks + } + keepUnrolling = reserveUnrollMemoryForThisThread(amountToRequest) + } + } + // New threshold is currentSize * memoryGrowthFactor + memoryThreshold += amountToRequest + } + } + elementsUnrolled += 1 + } + + if (keepUnrolling) { + // We successfully unrolled the entirety of this block + Left(vector.toArray) + } else { + // We ran out of space while unrolling the values for this block + Right(vector.iterator ++ values) + } + + } finally { + // If we return an array, the values returned do not depend on the underlying vector and + // we can immediately free up space for other threads. Otherwise, if we return an iterator, + // we release the memory claimed by this thread later on when the task finishes. + if (keepUnrolling) { + val amountToRelease = currentUnrollMemoryForThisThread - previousMemoryReserved + releaseUnrollMemoryForThisThread(amountToRelease) + } + } + } + /** * Return the RDD ID that a given block ID is from, or None if it is not an RDD block. */ @@ -149,10 +288,10 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) /** * Try to put in a set of values, if we can free up enough space. The value should either be - * an ArrayBuffer if deserialized is true or a ByteBuffer otherwise. Its (possibly estimated) - * size must also be passed by the caller. + * an Array if deserialized is true or a ByteBuffer otherwise. Its (possibly estimated) size + * must also be passed by the caller. * - * Lock on the object putLock to ensure that all the put requests and its associated block + * Synchronize on `accountingLock` to ensure that all the put requests and its associated block * dropping is done by only on thread at a time. Otherwise while one thread is dropping * blocks to free memory for one block, another thread may use up the freed space for * another block. @@ -174,7 +313,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) var putSuccess = false val droppedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] - putLock.synchronized { + accountingLock.synchronized { val freeSpaceResult = ensureFreeSpace(blockId, size) val enoughFreeSpace = freeSpaceResult.success droppedBlocks ++= freeSpaceResult.droppedBlocks @@ -193,7 +332,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) // Tell the block manager that we couldn't put it in memory so that it can drop it to // disk if the block allows disk storage. val data = if (deserialized) { - Left(value.asInstanceOf[ArrayBuffer[Any]]) + Left(value.asInstanceOf[Array[Any]]) } else { Right(value.asInstanceOf[ByteBuffer].duplicate()) } @@ -210,12 +349,14 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) * from the same RDD (which leads to a wasteful cyclic replacement pattern for RDDs that * don't fit into memory that we want to avoid). * - * Assume that a lock is held by the caller to ensure only one thread is dropping blocks. - * Otherwise, the freed space may fill up before the caller puts in their new value. + * Assume that `accountingLock` is held by the caller to ensure only one thread is dropping + * blocks. Otherwise, the freed space may fill up before the caller puts in their new value. * * Return whether there is enough free space, along with the blocks dropped in the process. */ - private def ensureFreeSpace(blockIdToAdd: BlockId, space: Long): ResultWithDroppedBlocks = { + private def ensureFreeSpace( + blockIdToAdd: BlockId, + space: Long): ResultWithDroppedBlocks = { logInfo(s"ensureFreeSpace($space) called with curMem=$currentMemory, maxMem=$maxMemory") val droppedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] @@ -225,9 +366,12 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) return ResultWithDroppedBlocks(success = false, droppedBlocks) } - if (maxMemory - currentMemory < space) { + // Take into account the amount of memory currently occupied by unrolling blocks + val actualFreeMemory = freeMemory - currentUnrollMemory + + if (actualFreeMemory < space) { val rddToAdd = getRddId(blockIdToAdd) - val selectedBlocks = new ArrayBuffer[BlockId]() + val selectedBlocks = new ArrayBuffer[BlockId] var selectedMemory = 0L // This is synchronized to ensure that the set of entries is not changed @@ -235,7 +379,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) // can lead to exceptions. entries.synchronized { val iterator = entries.entrySet().iterator() - while (maxMemory - (currentMemory - selectedMemory) < space && iterator.hasNext) { + while (actualFreeMemory + selectedMemory < space && iterator.hasNext) { val pair = iterator.next() val blockId = pair.getKey if (rddToAdd.isEmpty || rddToAdd != getRddId(blockId)) { @@ -245,7 +389,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) } } - if (maxMemory - (currentMemory - selectedMemory) >= space) { + if (actualFreeMemory + selectedMemory >= space) { logInfo(s"${selectedBlocks.size} blocks selected for dropping") for (blockId <- selectedBlocks) { val entry = entries.synchronized { entries.get(blockId) } @@ -254,7 +398,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) // future safety. if (entry != null) { val data = if (entry.deserialized) { - Left(entry.value.asInstanceOf[ArrayBuffer[Any]]) + Left(entry.value.asInstanceOf[Array[Any]]) } else { Right(entry.value.asInstanceOf[ByteBuffer].duplicate()) } @@ -275,8 +419,56 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) override def contains(blockId: BlockId): Boolean = { entries.synchronized { entries.containsKey(blockId) } } + + /** + * Reserve additional memory for unrolling blocks used by this thread. + * Return whether the request is granted. + */ + private[spark] def reserveUnrollMemoryForThisThread(memory: Long): Boolean = { + accountingLock.synchronized { + val granted = freeMemory > currentUnrollMemory + memory + if (granted) { + val threadId = Thread.currentThread().getId + unrollMemoryMap(threadId) = unrollMemoryMap.getOrElse(threadId, 0L) + memory + } + granted + } + } + + /** + * Release memory used by this thread for unrolling blocks. + * If the amount is not specified, remove the current thread's allocation altogether. + */ + private[spark] def releaseUnrollMemoryForThisThread(memory: Long = -1L): Unit = { + val threadId = Thread.currentThread().getId + accountingLock.synchronized { + if (memory < 0) { + unrollMemoryMap.remove(threadId) + } else { + unrollMemoryMap(threadId) = unrollMemoryMap.getOrElse(threadId, memory) - memory + // If this thread claims no more unroll memory, release it completely + if (unrollMemoryMap(threadId) <= 0) { + unrollMemoryMap.remove(threadId) + } + } + } + } + + /** + * Return the amount of memory currently occupied for unrolling blocks across all threads. + */ + private[spark] def currentUnrollMemory: Long = accountingLock.synchronized { + unrollMemoryMap.values.sum + } + + /** + * Return the amount of memory currently occupied for unrolling blocks by this thread. + */ + private[spark] def currentUnrollMemoryForThisThread: Long = accountingLock.synchronized { + unrollMemoryMap.getOrElse(Thread.currentThread().getId, 0L) + } } -private case class ResultWithDroppedBlocks( +private[spark] case class ResultWithDroppedBlocks( success: Boolean, droppedBlocks: Seq[(BlockId, BlockStatus)]) 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 5a72e216872a6..120c327a7e580 100644 --- a/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala +++ b/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala @@ -34,6 +34,8 @@ class RDDInfo( var diskSize = 0L var tachyonSize = 0L + def isCached: Boolean = (memSize + diskSize + tachyonSize > 0) && numCachedPartitions > 0 + override def toString = { import Utils.bytesToString ("RDD \"%s\" (%d) StorageLevel: %s; CachedPartitions: %d; TotalPartitions: %d; " + diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala index 35910e552fe86..b8f5d3a5b02aa 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala @@ -25,9 +25,12 @@ import scala.collection.JavaConversions._ import org.apache.spark.Logging import org.apache.spark.serializer.Serializer +import org.apache.spark.shuffle.ShuffleManager import org.apache.spark.storage.ShuffleBlockManager.ShuffleFileGroup import org.apache.spark.util.{MetadataCleaner, MetadataCleanerType, TimeStampedHashMap} import org.apache.spark.util.collection.{PrimitiveKeyOpenHashMap, PrimitiveVector} +import org.apache.spark.shuffle.sort.SortShuffleManager +import org.apache.spark.executor.ShuffleWriteMetrics /** A group of writers for a ShuffleMapTask, one writer per reducer. */ private[spark] trait ShuffleWriterGroup { @@ -58,8 +61,10 @@ private[spark] trait ShuffleWriterGroup { * each block stored in each file. In order to find the location of a shuffle block, we search the * files within a ShuffleFileGroups associated with the block's reducer. */ +// TODO: Factor this into a separate class for each ShuffleManager implementation private[spark] -class ShuffleBlockManager(blockManager: BlockManager) extends Logging { +class ShuffleBlockManager(blockManager: BlockManager, + shuffleManager: ShuffleManager) extends Logging { def conf = blockManager.conf // Turning off shuffle file consolidation causes all shuffle Blocks to get their own file. @@ -67,7 +72,10 @@ class ShuffleBlockManager(blockManager: BlockManager) extends Logging { val consolidateShuffleFiles = conf.getBoolean("spark.shuffle.consolidateFiles", false) - private val bufferSize = conf.getInt("spark.shuffle.file.buffer.kb", 100) * 1024 + // Are we using sort-based shuffle? + val sortBasedShuffle = shuffleManager.isInstanceOf[SortShuffleManager] + + private val bufferSize = conf.getInt("spark.shuffle.file.buffer.kb", 32) * 1024 /** * Contains all the state related to a particular shuffle. This includes a pool of unused @@ -91,7 +99,22 @@ class ShuffleBlockManager(blockManager: BlockManager) extends Logging { private val metadataCleaner = new MetadataCleaner(MetadataCleanerType.SHUFFLE_BLOCK_MANAGER, this.cleanup, conf) - def forMapTask(shuffleId: Int, mapId: Int, numBuckets: Int, serializer: Serializer) = { + /** + * Register a completed map without getting a ShuffleWriterGroup. Used by sort-based shuffle + * because it just writes a single file by itself. + */ + def addCompletedMap(shuffleId: Int, mapId: Int, numBuckets: Int): Unit = { + shuffleStates.putIfAbsent(shuffleId, new ShuffleState(numBuckets)) + val shuffleState = shuffleStates(shuffleId) + shuffleState.completedMapTasks.add(mapId) + } + + /** + * Get a ShuffleWriterGroup for the given map task, which will register it as complete + * when the writers are closed successfully + */ + def forMapTask(shuffleId: Int, mapId: Int, numBuckets: Int, serializer: Serializer, + writeMetrics: ShuffleWriteMetrics) = { new ShuffleWriterGroup { shuffleStates.putIfAbsent(shuffleId, new ShuffleState(numBuckets)) private val shuffleState = shuffleStates(shuffleId) @@ -101,7 +124,8 @@ class ShuffleBlockManager(blockManager: BlockManager) extends Logging { fileGroup = getUnusedFileGroup() Array.tabulate[BlockObjectWriter](numBuckets) { bucketId => val blockId = ShuffleBlockId(shuffleId, mapId, bucketId) - blockManager.getDiskWriter(blockId, fileGroup(bucketId), serializer, bufferSize) + blockManager.getDiskWriter(blockId, fileGroup(bucketId), serializer, bufferSize, + writeMetrics) } } else { Array.tabulate[BlockObjectWriter](numBuckets) { bucketId => @@ -116,7 +140,7 @@ class ShuffleBlockManager(blockManager: BlockManager) extends Logging { logWarning(s"Failed to remove existing shuffle file $blockFile") } } - blockManager.getDiskWriter(blockId, blockFile, serializer, bufferSize) + blockManager.getDiskWriter(blockId, blockFile, serializer, bufferSize, writeMetrics) } } @@ -124,7 +148,8 @@ class ShuffleBlockManager(blockManager: BlockManager) extends Logging { if (consolidateShuffleFiles) { if (success) { val offsets = writers.map(_.fileSegment().offset) - fileGroup.recordMapOutput(mapId, offsets) + val lengths = writers.map(_.fileSegment().length) + fileGroup.recordMapOutput(mapId, offsets, lengths) } recycleFileGroup(fileGroup) } else { @@ -182,7 +207,14 @@ class ShuffleBlockManager(blockManager: BlockManager) extends Logging { private def removeShuffleBlocks(shuffleId: ShuffleId): Boolean = { shuffleStates.get(shuffleId) match { case Some(state) => - if (consolidateShuffleFiles) { + if (sortBasedShuffle) { + // There's a single block ID for each map, plus an index file for it + for (mapId <- state.completedMapTasks) { + val blockId = new ShuffleBlockId(shuffleId, mapId, 0) + blockManager.diskBlockManager.getFile(blockId).delete() + blockManager.diskBlockManager.getFile(blockId.name + ".index").delete() + } + } else if (consolidateShuffleFiles) { for (fileGroup <- state.allFileGroups; file <- fileGroup.files) { file.delete() } @@ -220,6 +252,8 @@ object ShuffleBlockManager { * A particular mapper will be assigned a single ShuffleFileGroup to write its output to. */ private class ShuffleFileGroup(val shuffleId: Int, val fileId: Int, val files: Array[File]) { + private var numBlocks: Int = 0 + /** * Stores the absolute index of each mapId in the files of this group. For instance, * if mapId 5 is the first block in each file, mapIdToIndex(5) = 0. @@ -227,23 +261,27 @@ object ShuffleBlockManager { private val mapIdToIndex = new PrimitiveKeyOpenHashMap[Int, Int]() /** - * Stores consecutive offsets of blocks into each reducer file, ordered by position in the file. - * This ordering allows us to compute block lengths by examining the following block offset. + * Stores consecutive offsets and lengths of blocks into each reducer file, ordered by + * position in the file. * Note: mapIdToIndex(mapId) returns the index of the mapper into the vector for every * reducer. */ private val blockOffsetsByReducer = Array.fill[PrimitiveVector[Long]](files.length) { new PrimitiveVector[Long]() } - - def numBlocks = mapIdToIndex.size + private val blockLengthsByReducer = Array.fill[PrimitiveVector[Long]](files.length) { + new PrimitiveVector[Long]() + } def apply(bucketId: Int) = files(bucketId) - def recordMapOutput(mapId: Int, offsets: Array[Long]) { + def recordMapOutput(mapId: Int, offsets: Array[Long], lengths: Array[Long]) { + assert(offsets.length == lengths.length) mapIdToIndex(mapId) = numBlocks + numBlocks += 1 for (i <- 0 until offsets.length) { blockOffsetsByReducer(i) += offsets(i) + blockLengthsByReducer(i) += lengths(i) } } @@ -251,16 +289,11 @@ object ShuffleBlockManager { def getFileSegmentFor(mapId: Int, reducerId: Int): Option[FileSegment] = { val file = files(reducerId) val blockOffsets = blockOffsetsByReducer(reducerId) + val blockLengths = blockLengthsByReducer(reducerId) val index = mapIdToIndex.getOrElse(mapId, -1) if (index >= 0) { val offset = blockOffsets(index) - val length = - if (index + 1 < numBlocks) { - blockOffsets(index + 1) - offset - } else { - file.length() - offset - } - assert(length >= 0) + val length = blockLengths(index) Some(new FileSegment(file, offset, length)) } else { None 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 41c960c867e2e..d9066f766476e 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala @@ -35,13 +35,12 @@ class StorageStatusListener extends SparkListener { /** Update storage status list to reflect updated block statuses */ private def updateStorageStatus(execId: String, updatedBlocks: Seq[(BlockId, BlockStatus)]) { - val filteredStatus = executorIdToStorageStatus.get(execId) - filteredStatus.foreach { storageStatus => + executorIdToStorageStatus.get(execId).foreach { storageStatus => updatedBlocks.foreach { case (blockId, updatedStatus) => if (updatedStatus.storageLevel == StorageLevel.NONE) { - storageStatus.blocks.remove(blockId) + storageStatus.removeBlock(blockId) } else { - storageStatus.blocks(blockId) = updatedStatus + storageStatus.updateBlock(blockId, updatedStatus) } } } @@ -50,9 +49,8 @@ class StorageStatusListener extends SparkListener { /** Update storage status list to reflect the removal of an RDD from the cache */ private def updateStorageStatus(unpersistedRDDId: Int) { storageStatusList.foreach { storageStatus => - val unpersistedBlocksIds = storageStatus.rddBlocks.keys.filter(_.rddId == unpersistedRDDId) - unpersistedBlocksIds.foreach { blockId => - storageStatus.blocks.remove(blockId) + storageStatus.rddBlocksById(unpersistedRDDId).foreach { case (blockId, _) => + storageStatus.removeBlock(blockId) } } } 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 177281f663367..2bd6b749be261 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -20,122 +20,255 @@ package org.apache.spark.storage import scala.collection.Map import scala.collection.mutable -import org.apache.spark.SparkContext import org.apache.spark.annotation.DeveloperApi /** * :: DeveloperApi :: * Storage information for each BlockManager. + * + * This class assumes BlockId and BlockStatus are immutable, such that the consumers of this + * class cannot mutate the source of the information. Accesses are not thread-safe. */ @DeveloperApi -class StorageStatus( - val blockManagerId: BlockManagerId, - val maxMem: Long, - val blocks: mutable.Map[BlockId, BlockStatus] = mutable.Map.empty) { +class StorageStatus(val blockManagerId: BlockManagerId, val maxMem: Long) { - def memUsed = blocks.values.map(_.memSize).reduceOption(_ + _).getOrElse(0L) + /** + * Internal representation of the blocks stored in this block manager. + * + * We store RDD blocks and non-RDD blocks separately to allow quick retrievals of RDD blocks. + * These collections should only be mutated through the add/update/removeBlock methods. + */ + private val _rddBlocks = new mutable.HashMap[Int, mutable.Map[BlockId, BlockStatus]] + private val _nonRddBlocks = new mutable.HashMap[BlockId, BlockStatus] - def memUsedByRDD(rddId: Int) = - rddBlocks.filterKeys(_.rddId == rddId).values.map(_.memSize).reduceOption(_ + _).getOrElse(0L) + /** + * Storage information of the blocks that entails memory, disk, and off-heap memory usage. + * + * As with the block maps, we store the storage information separately for RDD blocks and + * non-RDD blocks for the same reason. In particular, RDD storage information is stored + * in a map indexed by the RDD ID to the following 4-tuple: + * + * (memory size, disk size, off-heap size, storage level) + * + * We assume that all the blocks that belong to the same RDD have the same storage level. + * This field is not relevant to non-RDD blocks, however, so the storage information for + * non-RDD blocks contains only the first 3 fields (in the same order). + */ + private val _rddStorageInfo = new mutable.HashMap[Int, (Long, Long, Long, StorageLevel)] + private var _nonRddStorageInfo: (Long, Long, Long) = (0L, 0L, 0L) - def diskUsed = blocks.values.map(_.diskSize).reduceOption(_ + _).getOrElse(0L) + /** Create a storage status with an initial set of blocks, leaving the source unmodified. */ + def this(bmid: BlockManagerId, maxMem: Long, initialBlocks: Map[BlockId, BlockStatus]) { + this(bmid, maxMem) + initialBlocks.foreach { case (bid, bstatus) => addBlock(bid, bstatus) } + } - def diskUsedByRDD(rddId: Int) = - rddBlocks.filterKeys(_.rddId == rddId).values.map(_.diskSize).reduceOption(_ + _).getOrElse(0L) + /** + * Return the blocks stored in this block manager. + * + * Note that this is somewhat expensive, as it involves cloning the underlying maps and then + * concatenating them together. Much faster alternatives exist for common operations such as + * contains, get, and size. + */ + def blocks: Map[BlockId, BlockStatus] = _nonRddBlocks ++ rddBlocks - def memRemaining: Long = maxMem - memUsed + /** + * Return the RDD blocks stored in this block manager. + * + * Note that this is somewhat expensive, as it involves cloning the underlying maps and then + * concatenating them together. Much faster alternatives exist for common operations such as + * getting the memory, disk, and off-heap memory sizes occupied by this RDD. + */ + def rddBlocks: Map[BlockId, BlockStatus] = _rddBlocks.flatMap { case (_, blocks) => blocks } - def rddBlocks = blocks.collect { case (rdd: RDDBlockId, status) => (rdd, status) } -} + /** Return the blocks that belong to the given RDD stored in this block manager. */ + def rddBlocksById(rddId: Int): Map[BlockId, BlockStatus] = { + _rddBlocks.get(rddId).getOrElse(Map.empty) + } -/** Helper methods for storage-related objects. */ -private[spark] object StorageUtils { + /** Add the given block to this storage status. If it already exists, overwrite it. */ + private[spark] def addBlock(blockId: BlockId, blockStatus: BlockStatus): Unit = { + updateStorageInfo(blockId, blockStatus) + blockId match { + case RDDBlockId(rddId, _) => + _rddBlocks.getOrElseUpdate(rddId, new mutable.HashMap)(blockId) = blockStatus + case _ => + _nonRddBlocks(blockId) = blockStatus + } + } + + /** Update the given block in this storage status. If it doesn't already exist, add it. */ + private[spark] def updateBlock(blockId: BlockId, blockStatus: BlockStatus): Unit = { + addBlock(blockId, blockStatus) + } + + /** Remove the given block from this storage status. */ + private[spark] def removeBlock(blockId: BlockId): Option[BlockStatus] = { + updateStorageInfo(blockId, BlockStatus.empty) + blockId match { + case RDDBlockId(rddId, _) => + // Actually remove the block, if it exists + if (_rddBlocks.contains(rddId)) { + val removed = _rddBlocks(rddId).remove(blockId) + // If the given RDD has no more blocks left, remove the RDD + if (_rddBlocks(rddId).isEmpty) { + _rddBlocks.remove(rddId) + } + removed + } else { + None + } + case _ => + _nonRddBlocks.remove(blockId) + } + } /** - * Returns basic information of all RDDs persisted in the given SparkContext. This does not - * include storage information. + * Return whether the given block is stored in this block manager in O(1) time. + * Note that this is much faster than `this.blocks.contains`, which is O(blocks) time. */ - def rddInfoFromSparkContext(sc: SparkContext): Array[RDDInfo] = { - sc.persistentRdds.values.map { rdd => - val rddName = Option(rdd.name).getOrElse(rdd.id.toString) - val rddNumPartitions = rdd.partitions.size - val rddStorageLevel = rdd.getStorageLevel - val rddInfo = new RDDInfo(rdd.id, rddName, rddNumPartitions, rddStorageLevel) - rddInfo - }.toArray + def containsBlock(blockId: BlockId): Boolean = { + blockId match { + case RDDBlockId(rddId, _) => + _rddBlocks.get(rddId).exists(_.contains(blockId)) + case _ => + _nonRddBlocks.contains(blockId) + } } - /** Returns storage information of all RDDs persisted in the given SparkContext. */ - def rddInfoFromStorageStatus( - storageStatuses: Seq[StorageStatus], - sc: SparkContext): Array[RDDInfo] = { - rddInfoFromStorageStatus(storageStatuses, rddInfoFromSparkContext(sc)) + /** + * Return the given block stored in this block manager in O(1) time. + * Note that this is much faster than `this.blocks.get`, which is O(blocks) time. + */ + def getBlock(blockId: BlockId): Option[BlockStatus] = { + blockId match { + case RDDBlockId(rddId, _) => + _rddBlocks.get(rddId).map(_.get(blockId)).flatten + case _ => + _nonRddBlocks.get(blockId) + } } - /** Returns storage information of all RDDs in the given list. */ - def rddInfoFromStorageStatus( - storageStatuses: Seq[StorageStatus], - rddInfos: Seq[RDDInfo], - updatedBlocks: Seq[(BlockId, BlockStatus)] = Seq.empty): Array[RDDInfo] = { - - // Mapping from a block ID -> its status - val blockMap = mutable.Map(storageStatuses.flatMap(_.rddBlocks): _*) - - // Record updated blocks, if any - updatedBlocks - .collect { case (id: RDDBlockId, status) => (id, status) } - .foreach { case (id, status) => blockMap(id) = status } - - // Mapping from RDD ID -> an array of associated BlockStatuses - val rddBlockMap = blockMap - .groupBy { case (k, _) => k.rddId } - .mapValues(_.values.toArray) - - // Mapping from RDD ID -> the associated RDDInfo (with potentially outdated storage information) - val rddInfoMap = rddInfos.map { info => (info.id, info) }.toMap - - val rddStorageInfos = rddBlockMap.flatMap { case (rddId, blocks) => - // 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 - rddInfo.tachyonSize = tachyonSize - rddInfo - } - }.toArray + /** + * Return the number of blocks stored in this block manager in O(RDDs) time. + * Note that this is much faster than `this.blocks.size`, which is O(blocks) time. + */ + def numBlocks: Int = _nonRddBlocks.size + numRddBlocks + + /** + * Return the number of RDD blocks stored in this block manager in O(RDDs) time. + * Note that this is much faster than `this.rddBlocks.size`, which is O(RDD blocks) time. + */ + def numRddBlocks: Int = _rddBlocks.values.map(_.size).sum - scala.util.Sorting.quickSort(rddStorageInfos) - rddStorageInfos + /** + * Return the number of blocks that belong to the given RDD in O(1) time. + * Note that this is much faster than `this.rddBlocksById(rddId).size`, which is + * O(blocks in this RDD) time. + */ + def numRddBlocksById(rddId: Int): Int = _rddBlocks.get(rddId).map(_.size).getOrElse(0) + + /** Return the memory remaining in this block manager. */ + def memRemaining: Long = maxMem - memUsed + + /** Return the memory used by this block manager. */ + def memUsed: Long = _nonRddStorageInfo._1 + _rddBlocks.keys.toSeq.map(memUsedByRdd).sum + + /** Return the disk space used by this block manager. */ + def diskUsed: Long = _nonRddStorageInfo._2 + _rddBlocks.keys.toSeq.map(diskUsedByRdd).sum + + /** Return the off-heap space used by this block manager. */ + def offHeapUsed: Long = _nonRddStorageInfo._3 + _rddBlocks.keys.toSeq.map(offHeapUsedByRdd).sum + + /** Return the memory used by the given RDD in this block manager in O(1) time. */ + def memUsedByRdd(rddId: Int): Long = _rddStorageInfo.get(rddId).map(_._1).getOrElse(0L) + + /** Return the disk space used by the given RDD in this block manager in O(1) time. */ + def diskUsedByRdd(rddId: Int): Long = _rddStorageInfo.get(rddId).map(_._2).getOrElse(0L) + + /** Return the off-heap space used by the given RDD in this block manager in O(1) time. */ + def offHeapUsedByRdd(rddId: Int): Long = _rddStorageInfo.get(rddId).map(_._3).getOrElse(0L) + + /** Return the storage level, if any, used by the given RDD in this block manager. */ + def rddStorageLevel(rddId: Int): Option[StorageLevel] = _rddStorageInfo.get(rddId).map(_._4) + + /** + * Update the relevant storage info, taking into account any existing status for this block. + */ + private def updateStorageInfo(blockId: BlockId, newBlockStatus: BlockStatus): Unit = { + val oldBlockStatus = getBlock(blockId).getOrElse(BlockStatus.empty) + val changeInMem = newBlockStatus.memSize - oldBlockStatus.memSize + val changeInDisk = newBlockStatus.diskSize - oldBlockStatus.diskSize + val changeInTachyon = newBlockStatus.tachyonSize - oldBlockStatus.tachyonSize + val level = newBlockStatus.storageLevel + + // Compute new info from old info + val (oldMem, oldDisk, oldTachyon) = blockId match { + case RDDBlockId(rddId, _) => + _rddStorageInfo.get(rddId) + .map { case (mem, disk, tachyon, _) => (mem, disk, tachyon) } + .getOrElse((0L, 0L, 0L)) + case _ => + _nonRddStorageInfo + } + val newMem = math.max(oldMem + changeInMem, 0L) + val newDisk = math.max(oldDisk + changeInDisk, 0L) + val newTachyon = math.max(oldTachyon + changeInTachyon, 0L) + + // Set the correct info + blockId match { + case RDDBlockId(rddId, _) => + // If this RDD is no longer persisted, remove it + if (newMem + newDisk + newTachyon == 0) { + _rddStorageInfo.remove(rddId) + } else { + _rddStorageInfo(rddId) = (newMem, newDisk, newTachyon, level) + } + case _ => + _nonRddStorageInfo = (newMem, newDisk, newTachyon) + } } - /** Returns a mapping from BlockId to the locations of the associated block. */ - def blockLocationsFromStorageStatus( - storageStatuses: Seq[StorageStatus]): Map[BlockId, Seq[String]] = { - val blockLocationPairs = storageStatuses.flatMap { storageStatus => - storageStatus.blocks.map { case (bid, _) => (bid, storageStatus.blockManagerId.hostPort) } +} + +/** Helper methods for storage-related objects. */ +private[spark] object StorageUtils { + + /** + * Update the given list of RDDInfo with the given list of storage statuses. + * This method overwrites the old values stored in the RDDInfo's. + */ + def updateRddInfo(rddInfos: Seq[RDDInfo], statuses: Seq[StorageStatus]): Unit = { + rddInfos.foreach { rddInfo => + val rddId = rddInfo.id + // Assume all blocks belonging to the same RDD have the same storage level + val storageLevel = statuses + .flatMap(_.rddStorageLevel(rddId)).headOption.getOrElse(StorageLevel.NONE) + val numCachedPartitions = statuses.map(_.numRddBlocksById(rddId)).sum + val memSize = statuses.map(_.memUsedByRdd(rddId)).sum + val diskSize = statuses.map(_.diskUsedByRdd(rddId)).sum + val tachyonSize = statuses.map(_.offHeapUsedByRdd(rddId)).sum + + rddInfo.storageLevel = storageLevel + rddInfo.numCachedPartitions = numCachedPartitions + rddInfo.memSize = memSize + rddInfo.diskSize = diskSize + rddInfo.tachyonSize = tachyonSize } - blockLocationPairs.toMap - .groupBy { case (blockId, _) => blockId } - .mapValues(_.values.toSeq) } - /** Filters the given list of StorageStatus by the given RDD ID. */ - def filterStorageStatusByRDD( - storageStatuses: Seq[StorageStatus], - rddId: Int): Array[StorageStatus] = { - storageStatuses.map { status => - val filteredBlocks = status.rddBlocks.filterKeys(_.rddId == rddId).toSeq - val filteredBlockMap = mutable.Map[BlockId, BlockStatus](filteredBlocks: _*) - new StorageStatus(status.blockManagerId, status.maxMem, filteredBlockMap) - }.toArray + /** + * Return a mapping from block ID to its locations for each block that belongs to the given RDD. + */ + def getRddBlockLocations(rddId: Int, statuses: Seq[StorageStatus]): Map[BlockId, Seq[String]] = { + val blockLocations = new mutable.HashMap[BlockId, mutable.ListBuffer[String]] + statuses.foreach { status => + status.rddBlocksById(rddId).foreach { case (bid, _) => + val location = status.blockManagerId.hostPort + blockLocations.getOrElseUpdate(bid, mutable.ListBuffer.empty) += location + } + } + blockLocations } + } 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 d8ff4ff6bd42c..932b5616043b4 100644 --- a/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala @@ -20,8 +20,6 @@ package org.apache.spark.storage import java.io.IOException import java.nio.ByteBuffer -import scala.collection.mutable.ArrayBuffer - import tachyon.client.{ReadType, WriteType} import org.apache.spark.Logging @@ -30,7 +28,7 @@ import org.apache.spark.util.Utils /** * Stores BlockManager blocks on Tachyon. */ -private class TachyonStore( +private[spark] class TachyonStore( blockManager: BlockManager, tachyonManager: TachyonBlockManager) extends BlockStore(blockManager: BlockManager) with Logging { @@ -45,15 +43,15 @@ private class TachyonStore( putIntoTachyonStore(blockId, bytes, returnValues = true) } - override def putValues( + override def putArray( blockId: BlockId, - values: ArrayBuffer[Any], + values: Array[Any], level: StorageLevel, returnValues: Boolean): PutResult = { - putValues(blockId, values.toIterator, level, returnValues) + putIterator(blockId, values.toIterator, level, returnValues) } - override def putValues( + override def putIterator( blockId: BlockId, values: Iterator[Any], level: StorageLevel, diff --git a/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala index 328be158db680..aa83ea90ee9ee 100644 --- a/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala +++ b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala @@ -20,6 +20,7 @@ package org.apache.spark.storage import java.util.concurrent.ArrayBlockingQueue import akka.actor._ +import org.apache.spark.shuffle.hash.HashShuffleManager import util.Random import org.apache.spark.{MapOutputTrackerMaster, SecurityManager, SparkConf} @@ -48,7 +49,7 @@ private[spark] object ThreadingTest { val block = (1 to blockSize).map(_ => Random.nextInt()) val level = randomLevel() val startTime = System.currentTimeMillis() - manager.put(blockId, block.iterator, level, tellMaster = true) + manager.putIterator(blockId, block.iterator, level, tellMaster = true) println("Pushed block " + blockId + " in " + (System.currentTimeMillis - startTime) + " ms") queue.add((blockId, block)) } @@ -101,7 +102,7 @@ private[spark] object ThreadingTest { conf) val blockManager = new BlockManager( "", actorSystem, blockManagerMaster, serializer, 1024 * 1024, conf, - new SecurityManager(conf), new MapOutputTrackerMaster(conf)) + new SecurityManager(conf), new MapOutputTrackerMaster(conf), new HashShuffleManager(conf)) val producers = (1 to numProducers).map(i => new ProducerThread(blockManager, i)) val consumers = producers.map(p => new ConsumerThread(blockManager, p.queue)) producers.foreach(_.start) diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index a2535e3c1c41f..6b4689291097f 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -93,7 +93,7 @@ private[spark] object JettyUtils extends Logging { def createServletHandler( path: String, servlet: HttpServlet, - basePath: String = ""): ServletContextHandler = { + basePath: String): ServletContextHandler = { val prefixedPath = attachPrefix(basePath, path) val contextHandler = new ServletContextHandler val holder = new ServletHolder(servlet) @@ -174,40 +174,32 @@ private[spark] object JettyUtils extends Logging { hostName: String, port: Int, handlers: Seq[ServletContextHandler], - conf: SparkConf): ServerInfo = { + conf: SparkConf, + serverName: String = ""): ServerInfo = { val collection = new ContextHandlerCollection collection.setHandlers(handlers.toArray) addFilters(handlers, conf) - @tailrec + // Bind to the given port, or throw a java.net.BindException if the port is occupied def connect(currentPort: Int): (Server, Int) = { val server = new Server(new InetSocketAddress(hostName, currentPort)) val pool = new QueuedThreadPool pool.setDaemon(true) server.setThreadPool(pool) server.setHandler(collection) - - Try { + try { server.start() - } match { - case s: Success[_] => - (server, server.getConnectors.head.getLocalPort) - case f: Failure[_] => - val nextPort = (currentPort + 1) % 65536 + (server, server.getConnectors.head.getLocalPort) + } catch { + case e: Exception => server.stop() pool.stop() - val msg = s"Failed to create UI on port $currentPort. Trying again on port $nextPort." - if (f.toString.contains("Address already in use")) { - logWarning(s"$msg - $f") - } else { - logError(msg, f.exception) - } - connect(nextPort) + throw e } } - val (server, boundPort) = connect(port) + val (server, boundPort) = Utils.startServiceOnPort[Server](port, connect, serverName) ServerInfo(server, boundPort, collection) } diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index 097a1b81e1dd1..6c788a37dc70b 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -36,7 +36,7 @@ private[spark] class SparkUI( val listenerBus: SparkListenerBus, var appName: String, val basePath: String = "") - extends WebUI(securityManager, SparkUI.getUIPort(conf), conf, basePath) + extends WebUI(securityManager, SparkUI.getUIPort(conf), conf, basePath, "SparkUI") with Logging { def this(sc: SparkContext) = this(sc, sc.conf, sc.env.securityManager, sc.listenerBus, sc.appName) diff --git a/core/src/main/scala/org/apache/spark/ui/ToolTips.scala b/core/src/main/scala/org/apache/spark/ui/ToolTips.scala index 37708d75489c8..9ced9b8107ebf 100644 --- a/core/src/main/scala/org/apache/spark/ui/ToolTips.scala +++ b/core/src/main/scala/org/apache/spark/ui/ToolTips.scala @@ -20,9 +20,9 @@ package org.apache.spark.ui private[spark] object ToolTips { val SCHEDULER_DELAY = """Scheduler delay includes time to ship the task from the scheduler to - the executor, and time the time to send a message from the executor to the scheduler stating - that the task has completed. When the scheduler becomes overloaded, task completion messages - become queued up, and scheduler delay increases.""" + the executor, and time to send the task result from the executor to the scheduler. If + scheduler delay is large, consider decreasing the size of tasks or decreasing the size + of task results.""" val INPUT = "Bytes read from Hadoop or from Spark storage." 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 9cb50d9b83dda..715cc2f4df8dd 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -136,11 +136,20 @@ private[spark] object UIUtils extends Logging { } // Yarn has to go through a proxy so the base uri is provided and has to be on all links - val uiRoot : String = Option(System.getenv("APPLICATION_WEB_PROXY_BASE")).getOrElse("") + def uiRoot: String = { + if (System.getenv("APPLICATION_WEB_PROXY_BASE") != null) { + System.getenv("APPLICATION_WEB_PROXY_BASE") + } else if (System.getProperty("spark.ui.proxyBase") != null) { + System.getProperty("spark.ui.proxyBase") + } + else { + "" + } + } def prependBaseUri(basePath: String = "", resource: String = "") = uiRoot + basePath + resource - val commonHeaderNodes = { + def commonHeaderNodes = { diff --git a/core/src/main/scala/org/apache/spark/ui/WebUI.scala b/core/src/main/scala/org/apache/spark/ui/WebUI.scala index 856273e1d4e21..5f52f95088007 100644 --- a/core/src/main/scala/org/apache/spark/ui/WebUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/WebUI.scala @@ -39,7 +39,8 @@ private[spark] abstract class WebUI( securityManager: SecurityManager, port: Int, conf: SparkConf, - basePath: String = "") + basePath: String = "", + name: String = "") extends Logging { protected val tabs = ArrayBuffer[WebUITab]() @@ -97,7 +98,7 @@ private[spark] abstract class WebUI( def bind() { assert(!serverInfo.isDefined, "Attempted to bind %s more than once!".format(className)) try { - serverInfo = Some(startJettyServer("0.0.0.0", port, handlers, conf)) + serverInfo = Some(startJettyServer("0.0.0.0", port, handlers, conf, name)) logInfo("Started %s at http://%s:%d".format(className, publicHostName, boundPort)) } catch { case e: Exception => diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala index b358c855e1c88..b814b0e6b8509 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala @@ -49,9 +49,9 @@ private[ui] class ExecutorsPage(parent: ExecutorsTab) extends WebUIPage("") { def render(request: HttpServletRequest): Seq[Node] = { val storageStatusList = listener.storageStatusList - val maxMem = storageStatusList.map(_.maxMem).fold(0L)(_ + _) - val memUsed = storageStatusList.map(_.memUsed).fold(0L)(_ + _) - val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)).fold(0L)(_ + _) + val maxMem = storageStatusList.map(_.maxMem).sum + val memUsed = storageStatusList.map(_.memUsed).sum + val diskUsed = storageStatusList.map(_.diskUsed).sum val execInfo = for (statusId <- 0 until storageStatusList.size) yield getExecInfo(statusId) val execInfoSorted = execInfo.sortBy(_.id) @@ -80,7 +80,7 @@ private[ui] class ExecutorsPage(parent: ExecutorsTab) extends WebUIPage("") { - {execInfoSorted.map(execRow(_))} + {execInfoSorted.map(execRow)} @@ -91,7 +91,7 @@ private[ui] class ExecutorsPage(parent: ExecutorsTab) extends WebUIPage("") {
  • Memory: {Utils.bytesToString(memUsed)} Used ({Utils.bytesToString(maxMem)} Total)
  • -
  • Disk: {Utils.bytesToString(diskSpaceUsed)} Used
  • +
  • Disk: {Utils.bytesToString(diskUsed)} Used
  • @@ -145,7 +145,7 @@ private[ui] class ExecutorsPage(parent: ExecutorsTab) extends WebUIPage("") { val status = listener.storageStatusList(statusId) val execId = status.blockManagerId.executorId val hostPort = status.blockManagerId.hostPort - val rddBlocks = status.blocks.size + val rddBlocks = status.numBlocks val memUsed = status.memUsed val maxMem = status.maxMem val diskUsed = status.diskUsed diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala index 52020954ea57c..0cc51c873727d 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala @@ -21,6 +21,7 @@ import scala.collection.mutable import scala.xml.Node import org.apache.spark.ui.{ToolTips, UIUtils} +import org.apache.spark.ui.jobs.UIData.StageUIData import org.apache.spark.util.Utils /** Page showing executor summary */ @@ -64,11 +65,9 @@ private[ui] class ExecutorTable(stageId: Int, parent: JobProgressTab) { executorIdToAddress.put(executorId, address) } - val executorIdToSummary = listener.stageIdToExecutorSummaries.get(stageId) - executorIdToSummary match { - case Some(x) => - x.toSeq.sortBy(_._1).map { case (k, v) => { - // scalastyle:off + listener.stageIdToData.get(stageId) match { + case Some(stageData: StageUIData) => + stageData.executorSummary.toSeq.sortBy(_._1).map { case (k, v) => {k} {executorIdToAddress.getOrElse(k, "CANNOT FIND ADDRESS")} @@ -76,16 +75,20 @@ private[ui] class ExecutorTable(stageId: Int, parent: JobProgressTab) { {v.failedTasks + v.succeededTasks} {v.failedTasks} {v.succeededTasks} - {Utils.bytesToString(v.inputBytes)} - {Utils.bytesToString(v.shuffleRead)} - {Utils.bytesToString(v.shuffleWrite)} - {Utils.bytesToString(v.memoryBytesSpilled)} - {Utils.bytesToString(v.diskBytesSpilled)} + + {Utils.bytesToString(v.inputBytes)} + + {Utils.bytesToString(v.shuffleRead)} + + {Utils.bytesToString(v.shuffleWrite)} + + {Utils.bytesToString(v.memoryBytesSpilled)} + + {Utils.bytesToString(v.diskBytesSpilled)} - // scalastyle:on } - } - case _ => Seq[Node]() + case None => + Seq.empty[Node] } } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index 2286a7f952f28..74cd637d88155 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -17,7 +17,7 @@ package org.apache.spark.ui.jobs -import scala.collection.mutable.{HashMap, ListBuffer} +import scala.collection.mutable.{HashMap, ListBuffer, Map} import org.apache.spark._ import org.apache.spark.annotation.DeveloperApi @@ -25,6 +25,7 @@ import org.apache.spark.executor.TaskMetrics import org.apache.spark.scheduler._ import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.storage.BlockManagerId +import org.apache.spark.ui.jobs.UIData._ /** * :: DeveloperApi :: @@ -35,7 +36,7 @@ import org.apache.spark.storage.BlockManagerId * updating the internal data structures concurrently. */ @DeveloperApi -class JobProgressListener(conf: SparkConf) extends SparkListener { +class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { import JobProgressListener._ @@ -46,20 +47,8 @@ class JobProgressListener(conf: SparkConf) extends SparkListener { val completedStages = ListBuffer[StageInfo]() val failedStages = ListBuffer[StageInfo]() - // TODO: Should probably consolidate all following into a single hash map. - val stageIdToTime = HashMap[Int, Long]() - val stageIdToInputBytes = HashMap[Int, Long]() - val stageIdToShuffleRead = HashMap[Int, Long]() - val stageIdToShuffleWrite = HashMap[Int, Long]() - val stageIdToMemoryBytesSpilled = HashMap[Int, Long]() - val stageIdToDiskBytesSpilled = HashMap[Int, Long]() - val stageIdToTasksActive = HashMap[Int, HashMap[Long, TaskInfo]]() - val stageIdToTasksComplete = HashMap[Int, Int]() - val stageIdToTasksFailed = HashMap[Int, Int]() - val stageIdToTaskData = HashMap[Int, HashMap[Long, TaskUIData]]() - val stageIdToExecutorSummaries = HashMap[Int, HashMap[String, ExecutorSummary]]() - val stageIdToPool = HashMap[Int, String]() - val stageIdToDescription = HashMap[Int, String]() + val stageIdToData = new HashMap[Int, StageUIData] + val poolToActiveStages = HashMap[String, HashMap[Int, StageInfo]]() val executorIdToBlockManagerId = HashMap[String, BlockManagerId]() @@ -71,8 +60,16 @@ class JobProgressListener(conf: SparkConf) extends SparkListener { override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) = synchronized { val stage = stageCompleted.stageInfo val stageId = stage.stageId - // Remove by stageId, rather than by StageInfo, in case the StageInfo is from storage - poolToActiveStages(stageIdToPool(stageId)).remove(stageId) + val stageData = stageIdToData.getOrElseUpdate(stageId, { + logWarning("Stage completed for unknown stage " + stageId) + new StageUIData + }) + + for ((id, info) <- stageCompleted.stageInfo.accumulables) { + stageData.accumulables(id) = info + } + + poolToActiveStages.get(stageData.schedulingPool).foreach(_.remove(stageId)) activeStages.remove(stageId) if (stage.failureReason.isEmpty) { completedStages += stage @@ -87,21 +84,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener { private def trimIfNecessary(stages: ListBuffer[StageInfo]) = synchronized { if (stages.size > retainedStages) { val toRemove = math.max(retainedStages / 10, 1) - stages.take(toRemove).foreach { s => - stageIdToTime.remove(s.stageId) - stageIdToInputBytes.remove(s.stageId) - stageIdToShuffleRead.remove(s.stageId) - stageIdToShuffleWrite.remove(s.stageId) - stageIdToMemoryBytesSpilled.remove(s.stageId) - stageIdToDiskBytesSpilled.remove(s.stageId) - stageIdToTasksActive.remove(s.stageId) - stageIdToTasksComplete.remove(s.stageId) - stageIdToTasksFailed.remove(s.stageId) - stageIdToTaskData.remove(s.stageId) - stageIdToExecutorSummaries.remove(s.stageId) - stageIdToPool.remove(s.stageId) - stageIdToDescription.remove(s.stageId) - } + stages.take(toRemove).foreach { s => stageIdToData.remove(s.stageId) } stages.trimStart(toRemove) } } @@ -114,26 +97,27 @@ class JobProgressListener(conf: SparkConf) extends SparkListener { val poolName = Option(stageSubmitted.properties).map { p => p.getProperty("spark.scheduler.pool", DEFAULT_POOL_NAME) }.getOrElse(DEFAULT_POOL_NAME) - stageIdToPool(stage.stageId) = poolName - val description = Option(stageSubmitted.properties).flatMap { + val stageData = stageIdToData.getOrElseUpdate(stage.stageId, new StageUIData) + stageData.schedulingPool = poolName + + stageData.description = Option(stageSubmitted.properties).flatMap { p => Option(p.getProperty(SparkContext.SPARK_JOB_DESCRIPTION)) } - description.map(d => stageIdToDescription(stage.stageId) = d) val stages = poolToActiveStages.getOrElseUpdate(poolName, new HashMap[Int, StageInfo]()) stages(stage.stageId) = stage } override def onTaskStart(taskStart: SparkListenerTaskStart) = synchronized { - val sid = taskStart.stageId val taskInfo = taskStart.taskInfo if (taskInfo != null) { - val tasksActive = stageIdToTasksActive.getOrElseUpdate(sid, new HashMap[Long, TaskInfo]()) - tasksActive(taskInfo.taskId) = taskInfo - val taskMap = stageIdToTaskData.getOrElse(sid, HashMap[Long, TaskUIData]()) - taskMap(taskInfo.taskId) = new TaskUIData(taskInfo) - stageIdToTaskData(sid) = taskMap + val stageData = stageIdToData.getOrElseUpdate(taskStart.stageId, { + logWarning("Task start for unknown stage " + taskStart.stageId) + new StageUIData + }) + stageData.numActiveTasks += 1 + stageData.taskData.put(taskInfo.taskId, new TaskUIData(taskInfo)) } } @@ -143,86 +127,116 @@ class JobProgressListener(conf: SparkConf) extends SparkListener { } override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = synchronized { - val sid = taskEnd.stageId val info = taskEnd.taskInfo - if (info != null) { - // create executor summary map if necessary - val executorSummaryMap = stageIdToExecutorSummaries.getOrElseUpdate(key = sid, - op = new HashMap[String, ExecutorSummary]()) - executorSummaryMap.getOrElseUpdate(key = info.executorId, op = new ExecutorSummary) - - val executorSummary = executorSummaryMap.get(info.executorId) - executorSummary match { - case Some(y) => { - // first update failed-task, succeed-task - taskEnd.reason match { - case Success => - y.succeededTasks += 1 - case _ => - y.failedTasks += 1 - } - - // update duration - y.taskTime += info.duration - - val metrics = taskEnd.taskMetrics - if (metrics != null) { - metrics.inputMetrics.foreach { y.inputBytes += _.bytesRead } - metrics.shuffleReadMetrics.foreach { y.shuffleRead += _.remoteBytesRead } - metrics.shuffleWriteMetrics.foreach { y.shuffleWrite += _.shuffleBytesWritten } - y.memoryBytesSpilled += metrics.memoryBytesSpilled - y.diskBytesSpilled += metrics.diskBytesSpilled - } - } - case _ => {} + val stageData = stageIdToData.getOrElseUpdate(taskEnd.stageId, { + logWarning("Task end for unknown stage " + taskEnd.stageId) + new StageUIData + }) + + for (accumulableInfo <- info.accumulables) { + stageData.accumulables(accumulableInfo.id) = accumulableInfo } - val tasksActive = stageIdToTasksActive.getOrElseUpdate(sid, new HashMap[Long, TaskInfo]()) - // Remove by taskId, rather than by TaskInfo, in case the TaskInfo is from storage - tasksActive.remove(info.taskId) + val execSummaryMap = stageData.executorSummary + val execSummary = execSummaryMap.getOrElseUpdate(info.executorId, new ExecutorSummary) + + taskEnd.reason match { + case Success => + execSummary.succeededTasks += 1 + case _ => + execSummary.failedTasks += 1 + } + execSummary.taskTime += info.duration + stageData.numActiveTasks -= 1 val (errorMessage, metrics): (Option[String], Option[TaskMetrics]) = taskEnd.reason match { case org.apache.spark.Success => - stageIdToTasksComplete(sid) = stageIdToTasksComplete.getOrElse(sid, 0) + 1 + stageData.completedIndices.add(info.index) + stageData.numCompleteTasks += 1 (None, Option(taskEnd.taskMetrics)) case e: ExceptionFailure => // Handle ExceptionFailure because we might have metrics - stageIdToTasksFailed(sid) = stageIdToTasksFailed.getOrElse(sid, 0) + 1 + stageData.numFailedTasks += 1 (Some(e.toErrorString), e.metrics) case e: TaskFailedReason => // All other failure cases - stageIdToTasksFailed(sid) = stageIdToTasksFailed.getOrElse(sid, 0) + 1 + stageData.numFailedTasks += 1 (Some(e.toErrorString), None) } - stageIdToTime.getOrElseUpdate(sid, 0L) - val time = metrics.map(_.executorRunTime).getOrElse(0L) - stageIdToTime(sid) += time - - stageIdToInputBytes.getOrElseUpdate(sid, 0L) - val inputBytes = metrics.flatMap(_.inputMetrics).map(_.bytesRead).getOrElse(0L) - stageIdToInputBytes(sid) += inputBytes - - stageIdToShuffleRead.getOrElseUpdate(sid, 0L) - val shuffleRead = metrics.flatMap(_.shuffleReadMetrics).map(_.remoteBytesRead).getOrElse(0L) - stageIdToShuffleRead(sid) += shuffleRead - - stageIdToShuffleWrite.getOrElseUpdate(sid, 0L) - val shuffleWrite = - metrics.flatMap(_.shuffleWriteMetrics).map(_.shuffleBytesWritten).getOrElse(0L) - stageIdToShuffleWrite(sid) += shuffleWrite + if (!metrics.isEmpty) { + val oldMetrics = stageData.taskData.get(info.taskId).flatMap(_.taskMetrics) + updateAggregateMetrics(stageData, info.executorId, metrics.get, oldMetrics) + } - stageIdToMemoryBytesSpilled.getOrElseUpdate(sid, 0L) - val memoryBytesSpilled = metrics.map(_.memoryBytesSpilled).getOrElse(0L) - stageIdToMemoryBytesSpilled(sid) += memoryBytesSpilled + val taskData = stageData.taskData.getOrElseUpdate(info.taskId, new TaskUIData(info)) + taskData.taskInfo = info + taskData.taskMetrics = metrics + taskData.errorMessage = errorMessage + } + } - stageIdToDiskBytesSpilled.getOrElseUpdate(sid, 0L) - val diskBytesSpilled = metrics.map(_.diskBytesSpilled).getOrElse(0L) - stageIdToDiskBytesSpilled(sid) += diskBytesSpilled + /** + * Upon receiving new metrics for a task, updates the per-stage and per-executor-per-stage + * aggregate metrics by calculating deltas between the currently recorded metrics and the new + * metrics. + */ + def updateAggregateMetrics( + stageData: StageUIData, + execId: String, + taskMetrics: TaskMetrics, + oldMetrics: Option[TaskMetrics]) { + val execSummary = stageData.executorSummary.getOrElseUpdate(execId, new ExecutorSummary) + + val shuffleWriteDelta = + (taskMetrics.shuffleWriteMetrics.map(_.shuffleBytesWritten).getOrElse(0L) + - oldMetrics.flatMap(_.shuffleWriteMetrics).map(_.shuffleBytesWritten).getOrElse(0L)) + stageData.shuffleWriteBytes += shuffleWriteDelta + execSummary.shuffleWrite += shuffleWriteDelta + + val shuffleReadDelta = + (taskMetrics.shuffleReadMetrics.map(_.remoteBytesRead).getOrElse(0L) + - oldMetrics.flatMap(_.shuffleReadMetrics).map(_.remoteBytesRead).getOrElse(0L)) + stageData.shuffleReadBytes += shuffleReadDelta + execSummary.shuffleRead += shuffleReadDelta + + val inputBytesDelta = + (taskMetrics.inputMetrics.map(_.bytesRead).getOrElse(0L) + - oldMetrics.flatMap(_.inputMetrics).map(_.bytesRead).getOrElse(0L)) + stageData.inputBytes += inputBytesDelta + execSummary.inputBytes += inputBytesDelta + + val diskSpillDelta = + taskMetrics.diskBytesSpilled - oldMetrics.map(_.diskBytesSpilled).getOrElse(0L) + stageData.diskBytesSpilled += diskSpillDelta + execSummary.diskBytesSpilled += diskSpillDelta + + val memorySpillDelta = + taskMetrics.memoryBytesSpilled - oldMetrics.map(_.memoryBytesSpilled).getOrElse(0L) + stageData.memoryBytesSpilled += memorySpillDelta + execSummary.memoryBytesSpilled += memorySpillDelta + + val timeDelta = + taskMetrics.executorRunTime - oldMetrics.map(_.executorRunTime).getOrElse(0L) + stageData.executorRunTime += timeDelta + } - val taskMap = stageIdToTaskData.getOrElse(sid, HashMap[Long, TaskUIData]()) - taskMap(info.taskId) = new TaskUIData(info, metrics, errorMessage) - stageIdToTaskData(sid) = taskMap + override def onExecutorMetricsUpdate(executorMetricsUpdate: SparkListenerExecutorMetricsUpdate) { + for ((taskId, sid, taskMetrics) <- executorMetricsUpdate.taskMetrics) { + val stageData = stageIdToData.getOrElseUpdate(sid, { + logWarning("Metrics update for task in unknown stage " + sid) + new StageUIData + }) + val taskData = stageData.taskData.get(taskId) + taskData.map { t => + if (!t.taskInfo.finished) { + updateAggregateMetrics(stageData, executorMetricsUpdate.execId, taskMetrics, + t.taskMetrics) + + // Overwrite task metrics + t.taskMetrics = Some(taskMetrics) + } + } } } @@ -252,12 +266,6 @@ class JobProgressListener(conf: SparkConf) extends SparkListener { } -@DeveloperApi -case class TaskUIData( - taskInfo: TaskInfo, - taskMetrics: Option[TaskMetrics] = None, - errorMessage: Option[String] = None) - private object JobProgressListener { val DEFAULT_POOL_NAME = "default" val DEFAULT_RETAINED_STAGES = 1000 diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala index 3308c8c8a3d37..8a01ec80c9dd6 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala @@ -41,7 +41,7 @@ private[ui] class JobProgressTab(parent: SparkUI) extends WebUITab(parent, "stag def isFairScheduler = listener.schedulingMode.exists(_ == SchedulingMode.FAIR) def handleKillRequest(request: HttpServletRequest) = { - if (killEnabled) { + if ((killEnabled) && (parent.securityManager.checkModifyPermissions(request.getRemoteUser))) { val killFlag = Option(request.getParameter("terminate")).getOrElse("false").toBoolean val stageId = Option(request.getParameter("id")).getOrElse("-1").toInt if (stageId >= 0 && killFlag && listener.activeStages.contains(stageId)) { diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 8c3821bd7c3eb..8bc1ba758cf77 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -20,10 +20,12 @@ package org.apache.spark.ui.jobs import java.util.Date import javax.servlet.http.HttpServletRequest -import scala.xml.Node +import scala.xml.{Node, Unparsed} import org.apache.spark.ui.{ToolTips, WebUIPage, UIUtils} +import org.apache.spark.ui.jobs.UIData._ import org.apache.spark.util.{Utils, Distribution} +import org.apache.spark.scheduler.AccumulableInfo /** Page showing statistics and task list for a given stage */ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { @@ -34,8 +36,9 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { def render(request: HttpServletRequest): Seq[Node] = { listener.synchronized { val stageId = request.getParameter("id").toInt + val stageDataOption = listener.stageIdToData.get(stageId) - if (!listener.stageIdToTaskData.contains(stageId)) { + if (stageDataOption.isEmpty || stageDataOption.get.taskData.isEmpty) { val content =

    Summary Metrics

    No tasks have started yet @@ -45,23 +48,15 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { "Details for Stage %s".format(stageId), parent.headerTabs, parent) } - val tasks = listener.stageIdToTaskData(stageId).values.toSeq.sortBy(_.taskInfo.launchTime) + val stageData = stageDataOption.get + val tasks = stageData.taskData.values.toSeq.sortBy(_.taskInfo.launchTime) val numCompleted = tasks.count(_.taskInfo.finished) - val inputBytes = listener.stageIdToInputBytes.getOrElse(stageId, 0L) - val hasInput = inputBytes > 0 - val shuffleReadBytes = listener.stageIdToShuffleRead.getOrElse(stageId, 0L) - val hasShuffleRead = shuffleReadBytes > 0 - val shuffleWriteBytes = listener.stageIdToShuffleWrite.getOrElse(stageId, 0L) - val hasShuffleWrite = shuffleWriteBytes > 0 - val memoryBytesSpilled = listener.stageIdToMemoryBytesSpilled.getOrElse(stageId, 0L) - val diskBytesSpilled = listener.stageIdToDiskBytesSpilled.getOrElse(stageId, 0L) - val hasBytesSpilled = memoryBytesSpilled > 0 && diskBytesSpilled > 0 - - var activeTime = 0L - val now = System.currentTimeMillis - val tasksActive = listener.stageIdToTasksActive(stageId).values - tasksActive.foreach(activeTime += _.timeRunning(now)) + val accumulables = listener.stageIdToData(stageId).accumulables + val hasInput = stageData.inputBytes > 0 + val hasShuffleRead = stageData.shuffleReadBytes > 0 + val hasShuffleWrite = stageData.shuffleWriteBytes > 0 + val hasBytesSpilled = stageData.memoryBytesSpilled > 0 && stageData.diskBytesSpilled > 0 // scalastyle:off val summary = @@ -69,43 +64,48 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") {
    • Total task time across all tasks: - {UIUtils.formatDuration(listener.stageIdToTime.getOrElse(stageId, 0L) + activeTime)} + {UIUtils.formatDuration(stageData.executorRunTime)}
    • {if (hasInput)
    • Input: - {Utils.bytesToString(inputBytes)} + {Utils.bytesToString(stageData.inputBytes)}
    • } {if (hasShuffleRead)
    • Shuffle read: - {Utils.bytesToString(shuffleReadBytes)} + {Utils.bytesToString(stageData.shuffleReadBytes)}
    • } {if (hasShuffleWrite)
    • Shuffle write: - {Utils.bytesToString(shuffleWriteBytes)} + {Utils.bytesToString(stageData.shuffleWriteBytes)}
    • } {if (hasBytesSpilled)
    • Shuffle spill (memory): - {Utils.bytesToString(memoryBytesSpilled)} + {Utils.bytesToString(stageData.memoryBytesSpilled)}
    • Shuffle spill (disk): - {Utils.bytesToString(diskBytesSpilled)} + {Utils.bytesToString(stageData.diskBytesSpilled)}
    • }
    // scalastyle:on + val accumulableHeaders: Seq[String] = Seq("Accumulable", "Value") + def accumulableRow(acc: AccumulableInfo) = {acc.name}{acc.value} + val accumulableTable = UIUtils.listingTable(accumulableHeaders, accumulableRow, + accumulables.values.toSeq) + val taskHeaders: Seq[String] = Seq( "Index", "ID", "Attempt", "Status", "Locality Level", "Executor", - "Launch Time", "Duration", "GC Time") ++ + "Launch Time", "Duration", "GC Time", "Accumulators") ++ {if (hasInput) Seq("Input") else Nil} ++ {if (hasShuffleRead) Seq("Shuffle Read") else Nil} ++ {if (hasShuffleWrite) Seq("Write Time", "Shuffle Write") else Nil} ++ @@ -215,11 +215,16 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { Some(UIUtils.listingTable(quantileHeaders, quantileRow, listings, fixedWidth = true)) } val executorTable = new ExecutorTable(stageId, parent) + + val maybeAccumulableTable: Seq[Node] = + if (accumulables.size > 0) {

    Accumulators

    ++ accumulableTable } else Seq() + val content = summary ++

    Summary Metrics for {numCompleted} Completed Tasks

    ++
    {summaryTable.getOrElse("No tasks have reported metrics yet.")}
    ++

    Aggregated Metrics by Executor

    ++ executorTable.toNodeSeq ++ + maybeAccumulableTable ++

    Tasks

    ++ taskTable UIUtils.headerSparkPage(content, basePath, appName, "Details for Stage %d".format(stageId), @@ -286,6 +291,11 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { {if (gcTime > 0) UIUtils.formatDuration(gcTime) else ""} + + {Unparsed( + info.accumulables.map{acc => s"${acc.name}: ${acc.update.get}"}.mkString("
    ") + )} + " + rddSizeAfterCheckpoint + "]" ) - } /** @@ -300,8 +311,12 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { * This function should be called only those RDD whose partitions refer to parent RDD's * partitions (i.e., do not call it on simple RDD like MappedRDD). * + * @param op an operation to run on the RDD + * @param collectFunc a function for collecting the values in the RDD, in case there are + * non-comparable types like arrays that we want to convert to something that supports == */ - def testRDDPartitions[U: ClassTag](op: (RDD[Int]) => RDD[U]) { + def testRDDPartitions[U: ClassTag](op: (RDD[Int]) => RDD[U], + collectFunc: RDD[U] => Any = defaultCollectFunc[U] _) { // Generate the final RDD using given RDD operation val baseRDD = generateFatRDD() val operatedRDD = op(baseRDD) @@ -316,13 +331,13 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { logInfo("RDD after checkpoint: " + operatedRDD + "\n" + operatedRDD.toDebugString) val (rddSizeBeforeCheckpoint, partitionSizeBeforeCheckpoint) = getSerializedSizes(operatedRDD) parentRDDs.foreach(_.checkpoint()) // checkpoint the parent RDD, not the generated one - val result = operatedRDD.collect() // force checkpointing + val result = collectFunc(operatedRDD) // force checkpointing operatedRDD.collect() // force re-initialization of post-checkpoint lazy variables val (rddSizeAfterCheckpoint, partitionSizeAfterCheckpoint) = getSerializedSizes(operatedRDD) logInfo("RDD after checkpoint: " + operatedRDD + "\n" + operatedRDD.toDebugString) // Test whether the data in the checkpointed RDD is same as original - assert(operatedRDD.collect() === result) + assert(collectFunc(operatedRDD) === result) // Test whether serialized size of the partitions has reduced logInfo("Size of partitions of " + rddType + @@ -436,7 +451,7 @@ object CheckpointSuite { new CoGroupedRDD[K]( Seq(first.asInstanceOf[RDD[(K, _)]], second.asInstanceOf[RDD[(K, _)]]), part - ).asInstanceOf[RDD[(K, Seq[Seq[V]])]] + ).asInstanceOf[RDD[(K, Array[Iterable[V]])]] } } diff --git a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala index 13b415cccb647..4bc4346c0a288 100644 --- a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala @@ -31,15 +31,28 @@ import org.scalatest.time.SpanSugar._ import org.apache.spark.SparkContext._ import org.apache.spark.rdd.RDD -import org.apache.spark.storage.{BlockId, BroadcastBlockId, RDDBlockId, ShuffleBlockId} - -class ContextCleanerSuite extends FunSuite with BeforeAndAfter with LocalSparkContext { - +import org.apache.spark.storage._ +import org.apache.spark.shuffle.hash.HashShuffleManager +import org.apache.spark.shuffle.sort.SortShuffleManager +import org.apache.spark.storage.BroadcastBlockId +import org.apache.spark.storage.RDDBlockId +import org.apache.spark.storage.ShuffleBlockId +import org.apache.spark.storage.ShuffleIndexBlockId + +/** + * An abstract base class for context cleaner tests, which sets up a context with a config + * suitable for cleaner tests and provides some utility functions. Subclasses can use different + * config options, in particular, a different shuffle manager class + */ +abstract class ContextCleanerSuiteBase(val shuffleManager: Class[_] = classOf[HashShuffleManager]) + extends FunSuite with BeforeAndAfter with LocalSparkContext +{ implicit val defaultTimeout = timeout(10000 millis) val conf = new SparkConf() .setMaster("local[2]") .setAppName("ContextCleanerSuite") .set("spark.cleaner.referenceTracking.blocking", "true") + .set("spark.shuffle.manager", shuffleManager.getName) before { sc = new SparkContext(conf) @@ -52,9 +65,61 @@ class ContextCleanerSuite extends FunSuite with BeforeAndAfter with LocalSparkCo } } + //------ Helper functions ------ + + protected def newRDD() = sc.makeRDD(1 to 10) + protected def newPairRDD() = newRDD().map(_ -> 1) + protected def newShuffleRDD() = newPairRDD().reduceByKey(_ + _) + protected def newBroadcast() = sc.broadcast(1 to 100) + + protected def newRDDWithShuffleDependencies(): (RDD[_], Seq[ShuffleDependency[_, _, _]]) = { + def getAllDependencies(rdd: RDD[_]): Seq[Dependency[_]] = { + rdd.dependencies ++ rdd.dependencies.flatMap { dep => + getAllDependencies(dep.rdd) + } + } + val rdd = newShuffleRDD() + + // Get all the shuffle dependencies + val shuffleDeps = getAllDependencies(rdd) + .filter(_.isInstanceOf[ShuffleDependency[_, _, _]]) + .map(_.asInstanceOf[ShuffleDependency[_, _, _]]) + (rdd, shuffleDeps) + } + + protected def randomRdd() = { + val rdd: RDD[_] = Random.nextInt(3) match { + case 0 => newRDD() + case 1 => newShuffleRDD() + case 2 => newPairRDD.join(newPairRDD()) + } + if (Random.nextBoolean()) rdd.persist() + rdd.count() + rdd + } + + /** Run GC and make sure it actually has run */ + protected def runGC() { + val weakRef = new WeakReference(new Object()) + val startTime = System.currentTimeMillis + System.gc() // Make a best effort to run the garbage collection. It *usually* runs GC. + // Wait until a weak reference object has been GCed + while (System.currentTimeMillis - startTime < 10000 && weakRef.get != null) { + System.gc() + Thread.sleep(200) + } + } + + protected def cleaner = sc.cleaner.get +} + +/** + * Basic ContextCleanerSuite, which uses sort-based shuffle + */ +class ContextCleanerSuite extends ContextCleanerSuiteBase { test("cleanup RDD") { - val rdd = newRDD.persist() + val rdd = newRDD().persist() val collected = rdd.collect().toList val tester = new CleanerTester(sc, rddIds = Seq(rdd.id)) @@ -67,7 +132,7 @@ class ContextCleanerSuite extends FunSuite with BeforeAndAfter with LocalSparkCo } test("cleanup shuffle") { - val (rdd, shuffleDeps) = newRDDWithShuffleDependencies + val (rdd, shuffleDeps) = newRDDWithShuffleDependencies() val collected = rdd.collect().toList val tester = new CleanerTester(sc, shuffleIds = shuffleDeps.map(_.shuffleId)) @@ -80,7 +145,7 @@ class ContextCleanerSuite extends FunSuite with BeforeAndAfter with LocalSparkCo } test("cleanup broadcast") { - val broadcast = newBroadcast + val broadcast = newBroadcast() val tester = new CleanerTester(sc, broadcastIds = Seq(broadcast.id)) // Explicit cleanup @@ -89,7 +154,7 @@ class ContextCleanerSuite extends FunSuite with BeforeAndAfter with LocalSparkCo } test("automatically cleanup RDD") { - var rdd = newRDD.persist() + var rdd = newRDD().persist() rdd.count() // Test that GC does not cause RDD cleanup due to a strong reference @@ -107,7 +172,7 @@ class ContextCleanerSuite extends FunSuite with BeforeAndAfter with LocalSparkCo } test("automatically cleanup shuffle") { - var rdd = newShuffleRDD + var rdd = newShuffleRDD() rdd.count() // Test that GC does not cause shuffle cleanup due to a strong reference @@ -125,7 +190,7 @@ class ContextCleanerSuite extends FunSuite with BeforeAndAfter with LocalSparkCo } test("automatically cleanup broadcast") { - var broadcast = newBroadcast + var broadcast = newBroadcast() // Test that GC does not cause broadcast cleanup due to a strong reference val preGCTester = new CleanerTester(sc, broadcastIds = Seq(broadcast.id)) @@ -144,11 +209,11 @@ class ContextCleanerSuite extends FunSuite with BeforeAndAfter with LocalSparkCo test("automatically cleanup RDD + shuffle + broadcast") { val numRdds = 100 val numBroadcasts = 4 // Broadcasts are more costly - val rddBuffer = (1 to numRdds).map(i => randomRdd).toBuffer - val broadcastBuffer = (1 to numBroadcasts).map(i => randomBroadcast).toBuffer + val rddBuffer = (1 to numRdds).map(i => randomRdd()).toBuffer + val broadcastBuffer = (1 to numBroadcasts).map(i => newBroadcast()).toBuffer val rddIds = sc.persistentRdds.keys.toSeq val shuffleIds = 0 until sc.newShuffleId - val broadcastIds = 0L until numBroadcasts + val broadcastIds = broadcastBuffer.map(_.id) val preGCTester = new CleanerTester(sc, rddIds, shuffleIds, broadcastIds) runGC() @@ -162,6 +227,13 @@ class ContextCleanerSuite extends FunSuite with BeforeAndAfter with LocalSparkCo rddBuffer.clear() runGC() postGCTester.assertCleanup() + + // Make sure the broadcasted task closure no longer exists after GC. + val taskClosureBroadcastId = broadcastIds.max + 1 + assert(sc.env.blockManager.master.getMatchingBlockIds({ + case BroadcastBlockId(`taskClosureBroadcastId`, _) => true + case _ => false + }, askSlaves = true).isEmpty) } test("automatically cleanup RDD + shuffle + broadcast in distributed mode") { @@ -171,15 +243,16 @@ class ContextCleanerSuite extends FunSuite with BeforeAndAfter with LocalSparkCo .setMaster("local-cluster[2, 1, 512]") .setAppName("ContextCleanerSuite") .set("spark.cleaner.referenceTracking.blocking", "true") + .set("spark.shuffle.manager", shuffleManager.getName) sc = new SparkContext(conf2) val numRdds = 10 val numBroadcasts = 4 // Broadcasts are more costly - val rddBuffer = (1 to numRdds).map(i => randomRdd).toBuffer - val broadcastBuffer = (1 to numBroadcasts).map(i => randomBroadcast).toBuffer + val rddBuffer = (1 to numRdds).map(i => randomRdd()).toBuffer + val broadcastBuffer = (1 to numBroadcasts).map(i => newBroadcast()).toBuffer val rddIds = sc.persistentRdds.keys.toSeq val shuffleIds = 0 until sc.newShuffleId - val broadcastIds = 0L until numBroadcasts + val broadcastIds = broadcastBuffer.map(_.id) val preGCTester = new CleanerTester(sc, rddIds, shuffleIds, broadcastIds) runGC() @@ -193,57 +266,90 @@ class ContextCleanerSuite extends FunSuite with BeforeAndAfter with LocalSparkCo rddBuffer.clear() runGC() postGCTester.assertCleanup() + + // Make sure the broadcasted task closure no longer exists after GC. + val taskClosureBroadcastId = broadcastIds.max + 1 + assert(sc.env.blockManager.master.getMatchingBlockIds({ + case BroadcastBlockId(`taskClosureBroadcastId`, _) => true + case _ => false + }, askSlaves = true).isEmpty) } +} - //------ Helper functions ------ - def newRDD = sc.makeRDD(1 to 10) - def newPairRDD = newRDD.map(_ -> 1) - def newShuffleRDD = newPairRDD.reduceByKey(_ + _) - def newBroadcast = sc.broadcast(1 to 100) - def newRDDWithShuffleDependencies: (RDD[_], Seq[ShuffleDependency[_, _, _]]) = { - def getAllDependencies(rdd: RDD[_]): Seq[Dependency[_]] = { - rdd.dependencies ++ rdd.dependencies.flatMap { dep => - getAllDependencies(dep.rdd) - } - } - val rdd = newShuffleRDD +/** + * A copy of the shuffle tests for sort-based shuffle + */ +class SortShuffleContextCleanerSuite extends ContextCleanerSuiteBase(classOf[SortShuffleManager]) { + test("cleanup shuffle") { + val (rdd, shuffleDeps) = newRDDWithShuffleDependencies() + val collected = rdd.collect().toList + val tester = new CleanerTester(sc, shuffleIds = shuffleDeps.map(_.shuffleId)) - // Get all the shuffle dependencies - val shuffleDeps = getAllDependencies(rdd) - .filter(_.isInstanceOf[ShuffleDependency[_, _, _]]) - .map(_.asInstanceOf[ShuffleDependency[_, _, _]]) - (rdd, shuffleDeps) + // Explicit cleanup + shuffleDeps.foreach(s => cleaner.doCleanupShuffle(s.shuffleId, blocking = true)) + tester.assertCleanup() + + // Verify that shuffles can be re-executed after cleaning up + assert(rdd.collect().toList.equals(collected)) } - def randomRdd = { - val rdd: RDD[_] = Random.nextInt(3) match { - case 0 => newRDD - case 1 => newShuffleRDD - case 2 => newPairRDD.join(newPairRDD) - } - if (Random.nextBoolean()) rdd.persist() + test("automatically cleanup shuffle") { + var rdd = newShuffleRDD() rdd.count() - rdd - } - def randomBroadcast = { - sc.broadcast(Random.nextInt(Int.MaxValue)) + // Test that GC does not cause shuffle cleanup due to a strong reference + val preGCTester = new CleanerTester(sc, shuffleIds = Seq(0)) + runGC() + intercept[Exception] { + preGCTester.assertCleanup()(timeout(1000 millis)) + } + + // Test that GC causes shuffle cleanup after dereferencing the RDD + val postGCTester = new CleanerTester(sc, shuffleIds = Seq(0)) + rdd = null // Make RDD out of scope, so that corresponding shuffle goes out of scope + runGC() + postGCTester.assertCleanup() } - /** Run GC and make sure it actually has run */ - def runGC() { - val weakRef = new WeakReference(new Object()) - val startTime = System.currentTimeMillis - System.gc() // Make a best effort to run the garbage collection. It *usually* runs GC. - // Wait until a weak reference object has been GCed - while(System.currentTimeMillis - startTime < 10000 && weakRef.get != null) { - System.gc() - Thread.sleep(200) + test("automatically cleanup RDD + shuffle + broadcast in distributed mode") { + sc.stop() + + val conf2 = new SparkConf() + .setMaster("local-cluster[2, 1, 512]") + .setAppName("ContextCleanerSuite") + .set("spark.cleaner.referenceTracking.blocking", "true") + .set("spark.shuffle.manager", shuffleManager.getName) + sc = new SparkContext(conf2) + + val numRdds = 10 + val numBroadcasts = 4 // Broadcasts are more costly + val rddBuffer = (1 to numRdds).map(i => randomRdd).toBuffer + val broadcastBuffer = (1 to numBroadcasts).map(i => newBroadcast).toBuffer + val rddIds = sc.persistentRdds.keys.toSeq + val shuffleIds = 0 until sc.newShuffleId() + val broadcastIds = broadcastBuffer.map(_.id) + + val preGCTester = new CleanerTester(sc, rddIds, shuffleIds, broadcastIds) + runGC() + intercept[Exception] { + preGCTester.assertCleanup()(timeout(1000 millis)) } - } - def cleaner = sc.cleaner.get + // Test that GC triggers the cleanup of all variables after the dereferencing them + val postGCTester = new CleanerTester(sc, rddIds, shuffleIds, broadcastIds) + broadcastBuffer.clear() + rddBuffer.clear() + runGC() + postGCTester.assertCleanup() + + // Make sure the broadcasted task closure no longer exists after GC. + val taskClosureBroadcastId = broadcastIds.max + 1 + assert(sc.env.blockManager.master.getMatchingBlockIds({ + case BroadcastBlockId(`taskClosureBroadcastId`, _) => true + case _ => false + }, askSlaves = true).isEmpty) + } } @@ -401,6 +507,7 @@ class CleanerTester( private def getShuffleBlocks(shuffleId: Int): Seq[BlockId] = { blockManager.master.getMatchingBlockIds( _ match { case ShuffleBlockId(`shuffleId`, _, _) => true + case ShuffleIndexBlockId(`shuffleId`, _, _) => true case _ => false }, askSlaves = true) } diff --git a/core/src/test/scala/org/apache/spark/DriverSuite.scala b/core/src/test/scala/org/apache/spark/DriverSuite.scala index de4bd90c8f7e5..a73e1ef0288a5 100644 --- a/core/src/test/scala/org/apache/spark/DriverSuite.scala +++ b/core/src/test/scala/org/apache/spark/DriverSuite.scala @@ -34,7 +34,7 @@ import scala.language.postfixOps class DriverSuite extends FunSuite with Timeouts { test("driver should exit after finishing") { - val sparkHome = sys.env.get("SPARK_HOME").orElse(sys.props.get("spark.home")).get + val sparkHome = sys.props.getOrElse("spark.test.home", fail("spark.test.home is not set!")) // Regression test for SPARK-530: "Spark driver process doesn't exit after finishing" val masters = Table(("master"), ("local"), ("local-cluster[2,1,512]")) forAll(masters) { (master: String) => diff --git a/core/src/test/scala/org/apache/spark/FailureSuite.scala b/core/src/test/scala/org/apache/spark/FailureSuite.scala index e755d2e309398..2229e6acc425d 100644 --- a/core/src/test/scala/org/apache/spark/FailureSuite.scala +++ b/core/src/test/scala/org/apache/spark/FailureSuite.scala @@ -104,8 +104,9 @@ class FailureSuite extends FunSuite with LocalSparkContext { results.collect() } assert(thrown.getClass === classOf[SparkException]) - assert(thrown.getMessage.contains("NotSerializableException") || - thrown.getCause.getClass === classOf[NotSerializableException]) + assert(thrown.getMessage.contains("serializable") || + thrown.getCause.getClass === classOf[NotSerializableException], + "Exception does not contain \"serializable\": " + thrown.getMessage) FailureSuiteState.clear() } diff --git a/core/src/test/scala/org/apache/spark/FileSuite.scala b/core/src/test/scala/org/apache/spark/FileSuite.scala index 070e974657860..4a53d25012ad9 100644 --- a/core/src/test/scala/org/apache/spark/FileSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileSuite.scala @@ -24,12 +24,14 @@ import scala.io.Source import com.google.common.io.Files import org.apache.hadoop.io._ import org.apache.hadoop.io.compress.DefaultCodec -import org.apache.hadoop.mapred.{JobConf, FileAlreadyExistsException, TextOutputFormat} -import org.apache.hadoop.mapreduce.lib.output.{TextOutputFormat => NewTextOutputFormat} +import org.apache.hadoop.mapred.{JobConf, FileAlreadyExistsException, FileSplit, TextInputFormat, TextOutputFormat} import org.apache.hadoop.mapreduce.Job +import org.apache.hadoop.mapreduce.lib.input.{FileSplit => NewFileSplit, TextInputFormat => NewTextInputFormat} +import org.apache.hadoop.mapreduce.lib.output.{TextOutputFormat => NewTextOutputFormat} import org.scalatest.FunSuite import org.apache.spark.SparkContext._ +import org.apache.spark.rdd.{NewHadoopRDD, HadoopRDD} import org.apache.spark.util.Utils class FileSuite extends FunSuite with LocalSparkContext { @@ -177,6 +179,31 @@ class FileSuite extends FunSuite with LocalSparkContext { assert(output.collect().toList === List((1, "a"), (2, "aa"), (3, "aaa"))) } + test("object files of classes from a JAR") { + val original = Thread.currentThread().getContextClassLoader + val className = "FileSuiteObjectFileTest" + val jar = TestUtils.createJarWithClasses(Seq(className)) + val loader = new java.net.URLClassLoader(Array(jar), Utils.getContextOrSparkClassLoader) + Thread.currentThread().setContextClassLoader(loader) + try { + sc = new SparkContext("local", "test") + val objs = sc.makeRDD(1 to 3).map { x => + val loader = Thread.currentThread().getContextClassLoader + Class.forName(className, true, loader).newInstance() + } + val outputDir = new File(tempDir, "output").getAbsolutePath + objs.saveAsObjectFile(outputDir) + // Try reading the output back as an object file + val ct = reflect.ClassTag[Any](Class.forName(className, true, loader)) + val output = sc.objectFile[Any](outputDir) + assert(output.collect().size === 3) + assert(output.collect().head.getClass.getName === className) + } + finally { + Thread.currentThread().setContextClassLoader(original) + } + } + test("write SequenceFile using new Hadoop API") { import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat sc = new SparkContext("local", "test") @@ -293,4 +320,32 @@ class FileSuite extends FunSuite with LocalSparkContext { randomRDD.saveAsNewAPIHadoopDataset(job.getConfiguration) assert(new File(tempDir.getPath + "/outputDataset_new/part-r-00000").exists() === true) } + + test("Get input files via old Hadoop API") { + sc = new SparkContext("local", "test") + val outDir = new File(tempDir, "output").getAbsolutePath + sc.makeRDD(1 to 4, 2).saveAsTextFile(outDir) + + val inputPaths = + sc.hadoopFile(outDir, classOf[TextInputFormat], classOf[LongWritable], classOf[Text]) + .asInstanceOf[HadoopRDD[_, _]] + .mapPartitionsWithInputSplit { (split, part) => + Iterator(split.asInstanceOf[FileSplit].getPath.toUri.getPath) + }.collect() + assert(inputPaths.toSet === Set(s"$outDir/part-00000", s"$outDir/part-00001")) + } + + test("Get input files via new Hadoop API") { + sc = new SparkContext("local", "test") + val outDir = new File(tempDir, "output").getAbsolutePath + sc.makeRDD(1 to 4, 2).saveAsTextFile(outDir) + + val inputPaths = + sc.newAPIHadoopFile(outDir, classOf[NewTextInputFormat], classOf[LongWritable], classOf[Text]) + .asInstanceOf[NewHadoopRDD[_, _]] + .mapPartitionsWithInputSplit { (split, part) => + Iterator(split.asInstanceOf[NewFileSplit].getPath.toUri.getPath) + }.collect() + assert(inputPaths.toSet === Set(s"$outDir/part-00000", s"$outDir/part-00001")) + } } diff --git a/core/src/test/scala/org/apache/spark/PartitioningSuite.scala b/core/src/test/scala/org/apache/spark/PartitioningSuite.scala index 7c30626a0c421..fc0cee3e8749d 100644 --- a/core/src/test/scala/org/apache/spark/PartitioningSuite.scala +++ b/core/src/test/scala/org/apache/spark/PartitioningSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark +import scala.collection.mutable.ArrayBuffer import scala.math.abs import org.scalatest.{FunSuite, PrivateMethodTester} @@ -52,14 +53,12 @@ class PartitioningSuite extends FunSuite with SharedSparkContext with PrivateMet assert(p2 === p2) assert(p4 === p4) - assert(p2 != p4) - assert(p4 != p2) + assert(p2 === p4) assert(p4 === anotherP4) assert(anotherP4 === p4) assert(descendingP2 === descendingP2) assert(descendingP4 === descendingP4) - assert(descendingP2 != descendingP4) - assert(descendingP4 != descendingP2) + assert(descendingP2 === descendingP4) assert(p2 != descendingP2) assert(p4 != descendingP4) assert(descendingP2 != p2) @@ -91,6 +90,74 @@ class PartitioningSuite extends FunSuite with SharedSparkContext with PrivateMet } } + test("RangePartitioner for keys that are not Comparable (but with Ordering)") { + // Row does not extend Comparable, but has an implicit Ordering defined. + implicit object RowOrdering extends Ordering[Row] { + override def compare(x: Row, y: Row) = x.value - y.value + } + + val rdd = sc.parallelize(1 to 4500).map(x => (Row(x), Row(x))) + val partitioner = new RangePartitioner(1500, rdd) + partitioner.getPartition(Row(100)) + } + + test("RangPartitioner.sketch") { + val rdd = sc.makeRDD(0 until 20, 20).flatMap { i => + val random = new java.util.Random(i) + Iterator.fill(i)(random.nextDouble()) + }.cache() + val sampleSizePerPartition = 10 + val (count, sketched) = RangePartitioner.sketch(rdd, sampleSizePerPartition) + assert(count === rdd.count()) + sketched.foreach { case (idx, n, sample) => + assert(n === idx) + assert(sample.size === math.min(n, sampleSizePerPartition)) + } + } + + test("RangePartitioner.determineBounds") { + assert(RangePartitioner.determineBounds(ArrayBuffer.empty[(Int, Float)], 10).isEmpty, + "Bounds on an empty candidates set should be empty.") + val candidates = ArrayBuffer( + (0.7, 2.0f), (0.1, 1.0f), (0.4, 1.0f), (0.3, 1.0f), (0.2, 1.0f), (0.5, 1.0f), (1.0, 3.0f)) + assert(RangePartitioner.determineBounds(candidates, 3) === Array(0.4, 0.7)) + } + + test("RangePartitioner should run only one job if data is roughly balanced") { + val rdd = sc.makeRDD(0 until 20, 20).flatMap { i => + val random = new java.util.Random(i) + Iterator.fill(5000 * i)((random.nextDouble() + i, i)) + }.cache() + for (numPartitions <- Seq(10, 20, 40)) { + val partitioner = new RangePartitioner(numPartitions, rdd) + assert(partitioner.numPartitions === numPartitions) + val counts = rdd.keys.map(key => partitioner.getPartition(key)).countByValue().values + assert(counts.max < 3.0 * counts.min) + } + } + + test("RangePartitioner should work well on unbalanced data") { + val rdd = sc.makeRDD(0 until 20, 20).flatMap { i => + val random = new java.util.Random(i) + Iterator.fill(20 * i * i * i)((random.nextDouble() + i, i)) + }.cache() + for (numPartitions <- Seq(2, 4, 8)) { + val partitioner = new RangePartitioner(numPartitions, rdd) + assert(partitioner.numPartitions === numPartitions) + val counts = rdd.keys.map(key => partitioner.getPartition(key)).countByValue().values + assert(counts.max < 3.0 * counts.min) + } + } + + test("RangePartitioner should return a single partition for empty RDDs") { + val empty1 = sc.emptyRDD[(Int, Double)] + val partitioner1 = new RangePartitioner(0, empty1) + assert(partitioner1.numPartitions === 1) + val empty2 = sc.makeRDD(0 until 2, 2).flatMap(i => Seq.empty[(Int, Double)]) + val partitioner2 = new RangePartitioner(2, empty2) + assert(partitioner2.numPartitions === 1) + } + test("HashPartitioner not equal to RangePartitioner") { val rdd = sc.parallelize(1 to 10).map(x => (x, x)) val rangeP2 = new RangePartitioner(2, rdd) @@ -177,3 +244,6 @@ class PartitioningSuite extends FunSuite with SharedSparkContext with PrivateMet // Add other tests here for classes that should be able to handle empty partitions correctly } } + + +private sealed case class Row(value: Int) diff --git a/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala b/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala index e39093e24d68a..fcca0867b8072 100644 --- a/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala @@ -31,7 +31,7 @@ class SecurityManagerSuite extends FunSuite { conf.set("spark.ui.view.acls", "user1,user2") val securityManager = new SecurityManager(conf); assert(securityManager.isAuthenticationEnabled() === true) - assert(securityManager.uiAclsEnabled() === true) + assert(securityManager.aclsEnabled() === true) assert(securityManager.checkUIViewPermissions("user1") === true) assert(securityManager.checkUIViewPermissions("user2") === true) assert(securityManager.checkUIViewPermissions("user3") === false) @@ -41,17 +41,17 @@ class SecurityManagerSuite extends FunSuite { val conf = new SparkConf conf.set("spark.ui.view.acls", "user1,user2") val securityManager = new SecurityManager(conf); - securityManager.setUIAcls(true) - assert(securityManager.uiAclsEnabled() === true) - securityManager.setUIAcls(false) - assert(securityManager.uiAclsEnabled() === false) + securityManager.setAcls(true) + assert(securityManager.aclsEnabled() === true) + securityManager.setAcls(false) + assert(securityManager.aclsEnabled() === false) // acls are off so doesn't matter what view acls set to assert(securityManager.checkUIViewPermissions("user4") === true) - securityManager.setUIAcls(true) - assert(securityManager.uiAclsEnabled() === true) - securityManager.setViewAcls(ArrayBuffer[String]("user5"), "user6,user7") + securityManager.setAcls(true) + assert(securityManager.aclsEnabled() === true) + securityManager.setViewAcls(Set[String]("user5"), "user6,user7") assert(securityManager.checkUIViewPermissions("user1") === false) assert(securityManager.checkUIViewPermissions("user5") === true) assert(securityManager.checkUIViewPermissions("user6") === true) @@ -59,5 +59,72 @@ class SecurityManagerSuite extends FunSuite { assert(securityManager.checkUIViewPermissions("user8") === false) assert(securityManager.checkUIViewPermissions(null) === true) } + + test("set security modify acls") { + val conf = new SparkConf + conf.set("spark.modify.acls", "user1,user2") + + val securityManager = new SecurityManager(conf); + securityManager.setAcls(true) + assert(securityManager.aclsEnabled() === true) + securityManager.setAcls(false) + assert(securityManager.aclsEnabled() === false) + + // acls are off so doesn't matter what view acls set to + assert(securityManager.checkModifyPermissions("user4") === true) + + securityManager.setAcls(true) + assert(securityManager.aclsEnabled() === true) + securityManager.setModifyAcls(Set("user5"), "user6,user7") + assert(securityManager.checkModifyPermissions("user1") === false) + assert(securityManager.checkModifyPermissions("user5") === true) + assert(securityManager.checkModifyPermissions("user6") === true) + assert(securityManager.checkModifyPermissions("user7") === true) + assert(securityManager.checkModifyPermissions("user8") === false) + assert(securityManager.checkModifyPermissions(null) === true) + } + + test("set security admin acls") { + val conf = new SparkConf + conf.set("spark.admin.acls", "user1,user2") + conf.set("spark.ui.view.acls", "user3") + conf.set("spark.modify.acls", "user4") + + val securityManager = new SecurityManager(conf); + securityManager.setAcls(true) + assert(securityManager.aclsEnabled() === true) + + assert(securityManager.checkModifyPermissions("user1") === true) + assert(securityManager.checkModifyPermissions("user2") === true) + assert(securityManager.checkModifyPermissions("user4") === true) + assert(securityManager.checkModifyPermissions("user3") === false) + assert(securityManager.checkModifyPermissions("user5") === false) + assert(securityManager.checkModifyPermissions(null) === true) + assert(securityManager.checkUIViewPermissions("user1") === true) + assert(securityManager.checkUIViewPermissions("user2") === true) + assert(securityManager.checkUIViewPermissions("user3") === true) + assert(securityManager.checkUIViewPermissions("user4") === false) + assert(securityManager.checkUIViewPermissions("user5") === false) + assert(securityManager.checkUIViewPermissions(null) === true) + + securityManager.setAdminAcls("user6") + securityManager.setViewAcls(Set[String]("user8"), "user9") + securityManager.setModifyAcls(Set("user11"), "user9") + assert(securityManager.checkModifyPermissions("user6") === true) + assert(securityManager.checkModifyPermissions("user11") === true) + assert(securityManager.checkModifyPermissions("user9") === true) + assert(securityManager.checkModifyPermissions("user1") === false) + assert(securityManager.checkModifyPermissions("user4") === false) + assert(securityManager.checkModifyPermissions(null) === true) + assert(securityManager.checkUIViewPermissions("user6") === true) + assert(securityManager.checkUIViewPermissions("user8") === true) + assert(securityManager.checkUIViewPermissions("user9") === true) + assert(securityManager.checkUIViewPermissions("user1") === false) + assert(securityManager.checkUIViewPermissions("user3") === false) + assert(securityManager.checkUIViewPermissions(null) === true) + + } + + } diff --git a/core/src/test/scala/org/apache/spark/ShuffleNettySuite.scala b/core/src/test/scala/org/apache/spark/ShuffleNettySuite.scala index 47df00050c1e2..d7b2d2e1e330f 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleNettySuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleNettySuite.scala @@ -28,6 +28,6 @@ class ShuffleNettySuite extends ShuffleSuite with BeforeAndAfterAll { } override def afterAll() { - System.setProperty("spark.shuffle.use.netty", "false") + System.clearProperty("spark.shuffle.use.netty") } } diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index c4f2f7e34f4d5..b13ddf96bc77c 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -58,8 +58,7 @@ class ShuffleSuite extends FunSuite with Matchers with LocalSparkContext { // default Java serializer cannot handle the non serializable class. val c = new ShuffledRDD[Int, NonJavaSerializableClass, - NonJavaSerializableClass, - (Int, NonJavaSerializableClass)](b, new HashPartitioner(NUM_BLOCKS)) + NonJavaSerializableClass](b, new HashPartitioner(NUM_BLOCKS)) c.setSerializer(new KryoSerializer(conf)) val shuffleId = c.dependencies.head.asInstanceOf[ShuffleDependency[_, _, _]].shuffleId @@ -83,8 +82,7 @@ class ShuffleSuite extends FunSuite with Matchers with LocalSparkContext { // default Java serializer cannot handle the non serializable class. val c = new ShuffledRDD[Int, NonJavaSerializableClass, - NonJavaSerializableClass, - (Int, NonJavaSerializableClass)](b, new HashPartitioner(3)) + NonJavaSerializableClass](b, new HashPartitioner(3)) c.setSerializer(new KryoSerializer(conf)) assert(c.count === 10) } @@ -100,7 +98,7 @@ class ShuffleSuite extends FunSuite with Matchers with LocalSparkContext { // NOTE: The default Java serializer doesn't create zero-sized blocks. // So, use Kryo - val c = new ShuffledRDD[Int, Int, Int, (Int, Int)](b, new HashPartitioner(10)) + val c = new ShuffledRDD[Int, Int, Int](b, new HashPartitioner(10)) .setSerializer(new KryoSerializer(conf)) val shuffleId = c.dependencies.head.asInstanceOf[ShuffleDependency[_, _, _]].shuffleId @@ -126,7 +124,7 @@ class ShuffleSuite extends FunSuite with Matchers with LocalSparkContext { val b = a.map(x => (x, x*2)) // NOTE: The default Java serializer should create zero-sized blocks - val c = new ShuffledRDD[Int, Int, Int, (Int, Int)](b, new HashPartitioner(10)) + val c = new ShuffledRDD[Int, Int, Int](b, new HashPartitioner(10)) val shuffleId = c.dependencies.head.asInstanceOf[ShuffleDependency[_, _, _]].shuffleId assert(c.count === 4) @@ -141,19 +139,19 @@ class ShuffleSuite extends FunSuite with Matchers with LocalSparkContext { assert(nonEmptyBlocks.size <= 4) } - test("shuffle using mutable pairs") { + test("shuffle on mutable pairs") { // Use a local cluster with 2 processes to make sure there are both local and remote blocks sc = new SparkContext("local-cluster[2,1,512]", "test") def p[T1, T2](_1: T1, _2: T2) = MutablePair(_1, _2) val data = Array(p(1, 1), p(1, 2), p(1, 3), p(2, 1)) val pairs: RDD[MutablePair[Int, Int]] = sc.parallelize(data, 2) - val results = new ShuffledRDD[Int, Int, Int, MutablePair[Int, Int]](pairs, + val results = new ShuffledRDD[Int, Int, Int](pairs, new HashPartitioner(2)).collect() - data.foreach { pair => results should contain (pair) } + data.foreach { pair => results should contain ((pair._1, pair._2)) } } - test("sorting using mutable pairs") { + test("sorting on mutable pairs") { // This is not in SortingSuite because of the local cluster setup. // Use a local cluster with 2 processes to make sure there are both local and remote blocks sc = new SparkContext("local-cluster[2,1,512]", "test") @@ -162,10 +160,10 @@ class ShuffleSuite extends FunSuite with Matchers with LocalSparkContext { val pairs: RDD[MutablePair[Int, Int]] = sc.parallelize(data, 2) val results = new OrderedRDDFunctions[Int, Int, MutablePair[Int, Int]](pairs) .sortByKey().collect() - results(0) should be (p(1, 11)) - results(1) should be (p(2, 22)) - results(2) should be (p(3, 33)) - results(3) should be (p(100, 100)) + results(0) should be ((1, 11)) + results(1) should be ((2, 22)) + results(2) should be ((3, 33)) + results(3) should be ((100, 100)) } test("cogroup using mutable pairs") { @@ -176,7 +174,9 @@ class ShuffleSuite extends FunSuite with Matchers with LocalSparkContext { val data2 = Seq(p(1, "11"), p(1, "12"), p(2, "22"), p(3, "3")) val pairs1: RDD[MutablePair[Int, Int]] = sc.parallelize(data1, 2) val pairs2: RDD[MutablePair[Int, String]] = sc.parallelize(data2, 2) - val results = new CoGroupedRDD[Int](Seq(pairs1, pairs2), new HashPartitioner(2)).collectAsMap() + val results = new CoGroupedRDD[Int](Seq(pairs1, pairs2), new HashPartitioner(2)) + .map(p => (p._1, p._2.map(_.toArray))) + .collectAsMap() assert(results(1)(0).length === 3) assert(results(1)(0).contains(1)) @@ -240,7 +240,7 @@ class ShuffleSuite extends FunSuite with Matchers with LocalSparkContext { } assert(thrown.getClass === classOf[SparkException]) - assert(thrown.getMessage.contains("NotSerializableException")) + assert(thrown.getMessage.toLowerCase.contains("serializable")) } } diff --git a/core/src/test/scala/org/apache/spark/SortShuffleSuite.scala b/core/src/test/scala/org/apache/spark/SortShuffleSuite.scala new file mode 100644 index 0000000000000..5c02c00586ef4 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/SortShuffleSuite.scala @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark + +import org.scalatest.BeforeAndAfterAll + +class SortShuffleSuite extends ShuffleSuite with BeforeAndAfterAll { + + // This test suite should run all tests in ShuffleSuite with sort-based shuffle. + + override def beforeAll() { + System.setProperty("spark.shuffle.manager", + "org.apache.spark.shuffle.sort.SortShuffleManager") + } + + override def afterAll() { + System.clearProperty("spark.shuffle.manager") + } +} diff --git a/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala index 1fde4badda949..e6ab538d77bcc 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark import org.scalatest.{Assertions, FunSuite} +import org.apache.spark.storage.StorageLevel class SparkContextInfoSuite extends FunSuite with LocalSparkContext { test("getPersistentRDDs only returns RDDs that are marked as cached") { @@ -35,26 +36,33 @@ class SparkContextInfoSuite extends FunSuite with LocalSparkContext { test("getPersistentRDDs returns an immutable map") { sc = new SparkContext("local", "test") val rdd1 = sc.makeRDD(Array(1, 2, 3, 4), 2).cache() - val myRdds = sc.getPersistentRDDs assert(myRdds.size === 1) - assert(myRdds.values.head === rdd1) + assert(myRdds(0) === rdd1) + assert(myRdds(0).getStorageLevel === StorageLevel.MEMORY_ONLY) + // myRdds2 should have 2 RDDs, but myRdds should not change val rdd2 = sc.makeRDD(Array(5, 6, 7, 8), 1).cache() - - // getPersistentRDDs should have 2 RDDs, but myRdds should not change - assert(sc.getPersistentRDDs.size === 2) + val myRdds2 = sc.getPersistentRDDs + assert(myRdds2.size === 2) + assert(myRdds2(0) === rdd1) + assert(myRdds2(1) === rdd2) + assert(myRdds2(0).getStorageLevel === StorageLevel.MEMORY_ONLY) + assert(myRdds2(1).getStorageLevel === StorageLevel.MEMORY_ONLY) assert(myRdds.size === 1) + assert(myRdds(0) === rdd1) + assert(myRdds(0).getStorageLevel === StorageLevel.MEMORY_ONLY) } test("getRDDStorageInfo only reports on RDDs that actually persist data") { sc = new SparkContext("local", "test") val rdd = sc.makeRDD(Array(1, 2, 3, 4), 2).cache() - assert(sc.getRDDStorageInfo.size === 0) - rdd.collect() assert(sc.getRDDStorageInfo.size === 1) + assert(sc.getRDDStorageInfo.head.isCached) + assert(sc.getRDDStorageInfo.head.memSize > 0) + assert(sc.getRDDStorageInfo.head.storageLevel === StorageLevel.MEMORY_ONLY) } test("call sites report correct locations") { @@ -70,7 +78,7 @@ package object testPackage extends Assertions { def runCallSiteTest(sc: SparkContext) { val rdd = sc.makeRDD(Array(1, 2, 3, 4), 2) val rddCreationSite = rdd.getCreationSite - val curCallSite = sc.getCallSite().short // note: 2 lines after definition of "rdd" + val curCallSite = sc.getCallSite().shortForm // note: 2 lines after definition of "rdd" val rddCreationLine = rddCreationSite match { case CALL_SITE_REGEX(func, file, line) => { diff --git a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala index 67e3be21c3c93..495a0d48633a4 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark -import org.scalatest.{FunSuite, PrivateMethodTester} +import org.scalatest.{BeforeAndAfterEach, FunSuite, PrivateMethodTester} import org.apache.spark.scheduler.{TaskScheduler, TaskSchedulerImpl} import org.apache.spark.scheduler.cluster.{SimrSchedulerBackend, SparkDeploySchedulerBackend} @@ -25,12 +25,12 @@ import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, Me import org.apache.spark.scheduler.local.LocalBackend class SparkContextSchedulerCreationSuite - extends FunSuite with PrivateMethodTester with LocalSparkContext with Logging { + extends FunSuite with PrivateMethodTester with Logging with BeforeAndAfterEach { def createTaskScheduler(master: String): TaskSchedulerImpl = { // Create local SparkContext to setup a SparkEnv. We don't actually want to start() the // real schedulers, so we don't want to create a full SparkContext with the desired scheduler. - sc = new SparkContext("local", "test") + val sc = new SparkContext("local", "test") val createTaskSchedulerMethod = PrivateMethod[TaskScheduler]('createTaskScheduler) val sched = SparkContext invokePrivate createTaskSchedulerMethod(sc, master) sched.asInstanceOf[TaskSchedulerImpl] @@ -68,6 +68,15 @@ class SparkContextSchedulerCreationSuite } } + test("local-*-n-failures") { + val sched = createTaskScheduler("local[* ,2]") + assert(sched.maxTaskFailures === 2) + sched.backend match { + case s: LocalBackend => assert(s.totalCores === Runtime.getRuntime.availableProcessors()) + case _ => fail() + } + } + test("local-n-failures") { val sched = createTaskScheduler("local[4, 2]") assert(sched.maxTaskFailures === 2) @@ -77,6 +86,20 @@ class SparkContextSchedulerCreationSuite } } + test("bad-local-n") { + val e = intercept[SparkException] { + createTaskScheduler("local[2*]") + } + assert(e.getMessage.contains("Could not parse Master URL")) + } + + test("bad-local-n-failures") { + val e = intercept[SparkException] { + createTaskScheduler("local[2*,4]") + } + assert(e.getMessage.contains("Could not parse Master URL")) + } + test("local-default-parallelism") { val defaultParallelism = System.getProperty("spark.default.parallelism") System.setProperty("spark.default.parallelism", "16") diff --git a/core/src/test/scala/org/apache/spark/BroadcastSuite.scala b/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala similarity index 96% rename from core/src/test/scala/org/apache/spark/BroadcastSuite.scala rename to core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala index c9936256a5b95..17c64455b2429 100644 --- a/core/src/test/scala/org/apache/spark/BroadcastSuite.scala +++ b/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala @@ -15,14 +15,12 @@ * limitations under the License. */ -package org.apache.spark +package org.apache.spark.broadcast +import org.apache.spark.storage.{BroadcastBlockId, _} +import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkException} import org.scalatest.FunSuite -import org.apache.spark.storage._ -import org.apache.spark.broadcast.{Broadcast, HttpBroadcast} -import org.apache.spark.storage.BroadcastBlockId - class BroadcastSuite extends FunSuite with LocalSparkContext { private val httpConf = broadcastConf("HttpBroadcastFactory") @@ -46,7 +44,10 @@ class BroadcastSuite extends FunSuite with LocalSparkContext { test("Accessing HttpBroadcast variables in a local cluster") { val numSlaves = 4 - sc = new SparkContext("local-cluster[%d, 1, 512]".format(numSlaves), "test", httpConf) + val conf = httpConf.clone + conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + conf.set("spark.broadcast.compress", "true") + sc = new SparkContext("local-cluster[%d, 1, 512]".format(numSlaves), "test", conf) val list = List[Int](1, 2, 3, 4) val broadcast = sc.broadcast(list) val results = sc.parallelize(1 to numSlaves).map(x => (x, broadcast.value.sum)) @@ -71,7 +72,10 @@ class BroadcastSuite extends FunSuite with LocalSparkContext { test("Accessing TorrentBroadcast variables in a local cluster") { val numSlaves = 4 - sc = new SparkContext("local-cluster[%d, 1, 512]".format(numSlaves), "test", torrentConf) + val conf = torrentConf.clone + conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + conf.set("spark.broadcast.compress", "true") + sc = new SparkContext("local-cluster[%d, 1, 512]".format(numSlaves), "test", conf) val list = List[Int](1, 2, 3, 4) val broadcast = sc.broadcast(list) val results = sc.parallelize(1 to numSlaves).map(x => (x, broadcast.value.sum)) diff --git a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala index 01ab2d549325c..31aa7ec837f43 100644 --- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala @@ -88,8 +88,8 @@ class JsonProtocolSuite extends FunSuite { } def createAppDesc(): ApplicationDescription = { - val cmd = new Command("mainClass", List("arg1", "arg2"), Map(), Seq(), Seq()) - new ApplicationDescription("name", Some(4), 1234, cmd, Some("sparkHome"), "appUiUrl") + val cmd = new Command("mainClass", List("arg1", "arg2"), Map(), Seq(), Seq(), Seq()) + new ApplicationDescription("name", Some(4), 1234, cmd, "appUiUrl") } def createAppInfo() : ApplicationInfo = { @@ -101,7 +101,7 @@ class JsonProtocolSuite extends FunSuite { def createDriverCommand() = new Command( "org.apache.spark.FakeClass", Seq("some arg --and-some options -g foo"), - Map(("K1", "V1"), ("K2", "V2")), Seq("cp1", "cp2"), Seq("lp1", "lp2"), Some("-Dfoo") + Map(("K1", "V1"), ("K2", "V2")), Seq("cp1", "cp2"), Seq("lp1", "lp2"), Seq("-Dfoo") ) def createDriverDesc() = new DriverDescription("hdfs://some-dir/some.jar", 100, 3, @@ -169,8 +169,7 @@ object JsonConstants { val appDescJsonStr = """ |{"name":"name","cores":4,"memoryperslave":1234, - |"user":"%s","sparkhome":"sparkHome", - |"command":"Command(mainClass,List(arg1, arg2),Map(),List(),List(),None)"} + |"user":"%s","command":"Command(mainClass,List(arg1, arg2),Map(),List(),List(),List())"} """.format(System.getProperty("user.name", "")).stripMargin val executorRunnerJsonStr = diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index 565c53e9529ff..7e1ef80c84561 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -106,6 +106,18 @@ class SparkSubmitSuite extends FunSuite with Matchers { appArgs.childArgs should be (Seq("some", "--weird", "args")) } + test("handles arguments to user program with name collision") { + val clArgs = Seq( + "--name", "myApp", + "--class", "Foo", + "userjar.jar", + "--master", "local", + "some", + "--weird", "args") + val appArgs = new SparkSubmitArguments(clArgs) + appArgs.childArgs should be (Seq("--master", "local", "some", "--weird", "args")) + } + test("handles YARN cluster mode") { val clArgs = Seq( "--deploy-mode", "cluster", @@ -120,6 +132,7 @@ class SparkSubmitSuite extends FunSuite with Matchers { "--archives", "archive1.txt,archive2.txt", "--num-executors", "6", "--name", "beauty", + "--conf", "spark.shuffle.spill=false", "thejar.jar", "arg1", "arg2") val appArgs = new SparkSubmitArguments(clArgs) @@ -139,6 +152,7 @@ class SparkSubmitSuite extends FunSuite with Matchers { mainClass should be ("org.apache.spark.deploy.yarn.Client") classpath should have length (0) sysProps("spark.app.name") should be ("beauty") + sysProps("spark.shuffle.spill") should be ("false") sysProps("SPARK_SUBMIT") should be ("true") } @@ -156,6 +170,7 @@ class SparkSubmitSuite extends FunSuite with Matchers { "--archives", "archive1.txt,archive2.txt", "--num-executors", "6", "--name", "trill", + "--conf", "spark.shuffle.spill=false", "thejar.jar", "arg1", "arg2") val appArgs = new SparkSubmitArguments(clArgs) @@ -176,6 +191,7 @@ class SparkSubmitSuite extends FunSuite with Matchers { sysProps("spark.yarn.dist.archives") should include regex (".*archive1.txt,.*archive2.txt") sysProps("spark.jars") should include regex (".*one.jar,.*two.jar,.*three.jar,.*thejar.jar") sysProps("SPARK_SUBMIT") should be ("true") + sysProps("spark.shuffle.spill") should be ("false") } test("handles standalone cluster mode") { @@ -186,6 +202,7 @@ class SparkSubmitSuite extends FunSuite with Matchers { "--supervise", "--driver-memory", "4g", "--driver-cores", "5", + "--conf", "spark.shuffle.spill=false", "thejar.jar", "arg1", "arg2") val appArgs = new SparkSubmitArguments(clArgs) @@ -195,9 +212,13 @@ class SparkSubmitSuite extends FunSuite with Matchers { childArgsStr should include regex ("launch spark://h:p .*thejar.jar org.SomeClass arg1 arg2") mainClass should be ("org.apache.spark.deploy.Client") classpath should have size (0) - sysProps should have size (2) - sysProps.keys should contain ("spark.jars") + sysProps should have size (5) sysProps.keys should contain ("SPARK_SUBMIT") + sysProps.keys should contain ("spark.master") + sysProps.keys should contain ("spark.app.name") + sysProps.keys should contain ("spark.jars") + sysProps.keys should contain ("spark.shuffle.spill") + sysProps("spark.shuffle.spill") should be ("false") } test("handles standalone client mode") { @@ -208,6 +229,7 @@ class SparkSubmitSuite extends FunSuite with Matchers { "--total-executor-cores", "5", "--class", "org.SomeClass", "--driver-memory", "4g", + "--conf", "spark.shuffle.spill=false", "thejar.jar", "arg1", "arg2") val appArgs = new SparkSubmitArguments(clArgs) @@ -218,6 +240,7 @@ class SparkSubmitSuite extends FunSuite with Matchers { classpath(0) should endWith ("thejar.jar") sysProps("spark.executor.memory") should be ("5g") sysProps("spark.cores.max") should be ("5") + sysProps("spark.shuffle.spill") should be ("false") } test("handles mesos client mode") { @@ -228,6 +251,7 @@ class SparkSubmitSuite extends FunSuite with Matchers { "--total-executor-cores", "5", "--class", "org.SomeClass", "--driver-memory", "4g", + "--conf", "spark.shuffle.spill=false", "thejar.jar", "arg1", "arg2") val appArgs = new SparkSubmitArguments(clArgs) @@ -238,6 +262,23 @@ class SparkSubmitSuite extends FunSuite with Matchers { classpath(0) should endWith ("thejar.jar") sysProps("spark.executor.memory") should be ("5g") sysProps("spark.cores.max") should be ("5") + sysProps("spark.shuffle.spill") should be ("false") + } + + test("handles confs with flag equivalents") { + val clArgs = Seq( + "--deploy-mode", "cluster", + "--executor-memory", "5g", + "--class", "org.SomeClass", + "--conf", "spark.executor.memory=4g", + "--conf", "spark.master=yarn", + "thejar.jar", + "arg1", "arg2") + val appArgs = new SparkSubmitArguments(clArgs) + val (_, _, sysProps, mainClass) = createLaunchEnv(appArgs) + sysProps("spark.executor.memory") should be ("5g") + sysProps("spark.master") should be ("yarn-cluster") + mainClass should be ("org.apache.spark.deploy.yarn.Client") } test("launch simple application with spark-submit") { @@ -266,7 +307,7 @@ class SparkSubmitSuite extends FunSuite with Matchers { // NOTE: This is an expensive operation in terms of time (10 seconds+). Use sparingly. def runSparkSubmit(args: Seq[String]): String = { - val sparkHome = sys.env.get("SPARK_HOME").orElse(sys.props.get("spark.home")).get + val sparkHome = sys.props.getOrElse("spark.test.home", fail("spark.test.home is not set!")) Utils.executeAndGetOutput( Seq("./bin/spark-submit") ++ args, new File(sparkHome), diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala index 4633bc3f7f25e..c930839b47f11 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala @@ -29,7 +29,7 @@ import org.apache.spark.deploy.{Command, DriverDescription} class DriverRunnerTest extends FunSuite { private def createDriverRunner() = { - val command = new Command("mainClass", Seq(), Map(), Seq(), Seq()) + val command = new Command("mainClass", Seq(), Map(), Seq(), Seq(), Seq()) val driverDescription = new DriverDescription("jarUrl", 512, 1, true, command) new DriverRunner("driverId", new File("workDir"), new File("sparkHome"), driverDescription, null, "akka://1.2.3.4/worker/") diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala index e5f748d55500d..39ab53cf0b5b1 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala @@ -27,12 +27,11 @@ import org.apache.spark.SparkConf class ExecutorRunnerTest extends FunSuite { test("command includes appId") { def f(s:String) = new File(s) - val sparkHome = sys.env.get("SPARK_HOME").orElse(sys.props.get("spark.home")) + val sparkHome = sys.props.getOrElse("spark.test.home", fail("spark.test.home is not set!")) val appDesc = new ApplicationDescription("app name", Some(8), 500, - Command("foo", Seq(), Map(), Seq(), Seq()), - sparkHome, "appUiUrl") + Command("foo", Seq(), Map(), Seq(), Seq(), Seq()), "appUiUrl") val appId = "12345-worker321-9876" - val er = new ExecutorRunner(appId, 1, appDesc, 8, 500, null, "blah", "worker321", f(sparkHome.getOrElse(".")), + val er = new ExecutorRunner(appId, 1, appDesc, 8, 500, null, "blah", "worker321", f(sparkHome), f("ooga"), "blah", new SparkConf, ExecutorState.RUNNING) assert(er.getCommandSeq.last === appId) diff --git a/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala b/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala index 68a0ea36aa545..25be7f25c21bb 100644 --- a/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala +++ b/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala @@ -46,7 +46,19 @@ class CompressionCodecSuite extends FunSuite { test("default compression codec") { val codec = CompressionCodec.createCodec(conf) - assert(codec.getClass === classOf[LZFCompressionCodec]) + assert(codec.getClass === classOf[SnappyCompressionCodec]) + testCodec(codec) + } + + test("lz4 compression codec") { + val codec = CompressionCodec.createCodec(conf, classOf[LZ4CompressionCodec].getName) + assert(codec.getClass === classOf[LZ4CompressionCodec]) + testCodec(codec) + } + + test("lz4 compression codec short form") { + val codec = CompressionCodec.createCodec(conf, "lz4") + assert(codec.getClass === classOf[LZ4CompressionCodec]) testCodec(codec) } @@ -56,9 +68,21 @@ class CompressionCodecSuite extends FunSuite { testCodec(codec) } + test("lzf compression codec short form") { + val codec = CompressionCodec.createCodec(conf, "lzf") + assert(codec.getClass === classOf[LZFCompressionCodec]) + testCodec(codec) + } + test("snappy compression codec") { val codec = CompressionCodec.createCodec(conf, classOf[SnappyCompressionCodec].getName) assert(codec.getClass === classOf[SnappyCompressionCodec]) testCodec(codec) } + + test("snappy compression codec short form") { + val codec = CompressionCodec.createCodec(conf, "snappy") + assert(codec.getClass === classOf[SnappyCompressionCodec]) + testCodec(codec) + } } diff --git a/core/src/test/scala/org/apache/spark/ConnectionManagerSuite.scala b/core/src/test/scala/org/apache/spark/network/ConnectionManagerSuite.scala similarity index 70% rename from core/src/test/scala/org/apache/spark/ConnectionManagerSuite.scala rename to core/src/test/scala/org/apache/spark/network/ConnectionManagerSuite.scala index df6b2604c8d8a..e2f4d4c57cdb5 100644 --- a/core/src/test/scala/org/apache/spark/ConnectionManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/network/ConnectionManagerSuite.scala @@ -15,17 +15,23 @@ * limitations under the License. */ -package org.apache.spark +package org.apache.spark.network +import java.io.IOException +import java.nio._ +import java.util.concurrent.TimeoutException + +import org.apache.spark.{SecurityManager, SparkConf} import org.scalatest.FunSuite -import java.nio._ +import org.mockito.Mockito._ +import org.mockito.Matchers._ -import org.apache.spark.network.{ConnectionManager, Message, ConnectionManagerId} -import scala.concurrent.Await import scala.concurrent.TimeoutException +import scala.concurrent.{Await, TimeoutException} import scala.concurrent.duration._ import scala.language.postfixOps +import scala.util.{Failure, Success, Try} /** * Test the ConnectionManager with various security settings. @@ -47,7 +53,7 @@ class ConnectionManagerSuite extends FunSuite { buffer.flip val bufferMessage = Message.createBufferMessage(buffer.duplicate) - manager.sendMessageReliablySync(manager.id, bufferMessage) + Await.result(manager.sendMessageReliably(manager.id, bufferMessage), 10 seconds) assert(receivedMessage == true) @@ -80,7 +86,7 @@ class ConnectionManagerSuite extends FunSuite { (0 until count).map(i => { val bufferMessage = Message.createBufferMessage(buffer.duplicate) - manager.sendMessageReliablySync(managerServer.id, bufferMessage) + Await.result(manager.sendMessageReliably(managerServer.id, bufferMessage), 10 seconds) }) assert(numReceivedServerMessages == 10) @@ -119,7 +125,10 @@ class ConnectionManagerSuite extends FunSuite { val buffer = ByteBuffer.allocate(size).put(Array.tabulate[Byte](size)(x => x.toByte)) buffer.flip val bufferMessage = Message.createBufferMessage(buffer.duplicate) - manager.sendMessageReliablySync(managerServer.id, bufferMessage) + // Expect managerServer to close connection, which we'll report as an error: + intercept[IOException] { + Await.result(manager.sendMessageReliably(managerServer.id, bufferMessage), 10 seconds) + } assert(numReceivedServerMessages == 0) assert(numReceivedMessages == 0) @@ -164,6 +173,8 @@ class ConnectionManagerSuite extends FunSuite { val g = Await.result(f, 1 second) assert(false) } catch { + case i: IOException => + assert(true) case e: TimeoutException => { // we should timeout here since the client can't do the negotiation assert(true) @@ -210,7 +221,6 @@ class ConnectionManagerSuite extends FunSuite { }).foreach(f => { try { val g = Await.result(f, 1 second) - if (!g.isDefined) assert(false) else assert(true) } catch { case e: Exception => { assert(false) @@ -224,7 +234,68 @@ class ConnectionManagerSuite extends FunSuite { managerServer.stop() } + test("Ack error message") { + val conf = new SparkConf + conf.set("spark.authenticate", "false") + val securityManager = new SecurityManager(conf) + val manager = new ConnectionManager(0, conf, securityManager) + val managerServer = new ConnectionManager(0, conf, securityManager) + managerServer.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { + throw new Exception + }) + + val size = 10 * 1024 * 1024 + val buffer = ByteBuffer.allocate(size).put(Array.tabulate[Byte](size)(x => x.toByte)) + buffer.flip + val bufferMessage = Message.createBufferMessage(buffer) + + val future = manager.sendMessageReliably(managerServer.id, bufferMessage) + + intercept[IOException] { + Await.result(future, 1 second) + } + + manager.stop() + managerServer.stop() + + } + + test("sendMessageReliably timeout") { + val clientConf = new SparkConf + clientConf.set("spark.authenticate", "false") + val ackTimeout = 30 + clientConf.set("spark.core.connection.ack.wait.timeout", s"${ackTimeout}") + + val clientSecurityManager = new SecurityManager(clientConf) + val manager = new ConnectionManager(0, clientConf, clientSecurityManager) + val serverConf = new SparkConf + serverConf.set("spark.authenticate", "false") + val serverSecurityManager = new SecurityManager(serverConf) + val managerServer = new ConnectionManager(0, serverConf, serverSecurityManager) + managerServer.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { + // sleep 60 sec > ack timeout for simulating server slow down or hang up + Thread.sleep(ackTimeout * 3 * 1000) + None + }) + + val size = 10 * 1024 * 1024 + val buffer = ByteBuffer.allocate(size).put(Array.tabulate[Byte](size)(x => x.toByte)) + buffer.flip + val bufferMessage = Message.createBufferMessage(buffer.duplicate) + + val future = manager.sendMessageReliably(managerServer.id, bufferMessage) + + // Future should throw IOException in 30 sec. + // Otherwise TimeoutExcepton is thrown from Await.result. + // We expect TimeoutException is not thrown. + intercept[IOException] { + Await.result(future, (ackTimeout * 2) second) + } + + manager.stop() + managerServer.stop() + } } diff --git a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala index 447e38ec9dbd0..63d3ddb4af98a 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala @@ -83,6 +83,85 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { assert(valuesFor2.toList.sorted === List(1)) } + test("sampleByKey") { + + val defaultSeed = 1L + + // vary RDD size + for (n <- List(100, 1000, 1000000)) { + val data = sc.parallelize(1 to n, 2) + val fractionPositive = 0.3 + val stratifiedData = data.keyBy(StratifiedAuxiliary.stratifier(fractionPositive)) + val samplingRate = 0.1 + StratifiedAuxiliary.testSample(stratifiedData, samplingRate, defaultSeed, n) + } + + // vary fractionPositive + for (fractionPositive <- List(0.1, 0.3, 0.5, 0.7, 0.9)) { + val n = 100 + val data = sc.parallelize(1 to n, 2) + val stratifiedData = data.keyBy(StratifiedAuxiliary.stratifier(fractionPositive)) + val samplingRate = 0.1 + StratifiedAuxiliary.testSample(stratifiedData, samplingRate, defaultSeed, n) + } + + // Use the same data for the rest of the tests + val fractionPositive = 0.3 + val n = 100 + val data = sc.parallelize(1 to n, 2) + val stratifiedData = data.keyBy(StratifiedAuxiliary.stratifier(fractionPositive)) + + // vary seed + for (seed <- defaultSeed to defaultSeed + 5L) { + val samplingRate = 0.1 + StratifiedAuxiliary.testSample(stratifiedData, samplingRate, seed, n) + } + + // vary sampling rate + for (samplingRate <- List(0.01, 0.05, 0.1, 0.5)) { + StratifiedAuxiliary.testSample(stratifiedData, samplingRate, defaultSeed, n) + } + } + + test("sampleByKeyExact") { + val defaultSeed = 1L + + // vary RDD size + for (n <- List(100, 1000, 1000000)) { + val data = sc.parallelize(1 to n, 2) + val fractionPositive = 0.3 + val stratifiedData = data.keyBy(StratifiedAuxiliary.stratifier(fractionPositive)) + val samplingRate = 0.1 + StratifiedAuxiliary.testSampleExact(stratifiedData, samplingRate, defaultSeed, n) + } + + // vary fractionPositive + for (fractionPositive <- List(0.1, 0.3, 0.5, 0.7, 0.9)) { + val n = 100 + val data = sc.parallelize(1 to n, 2) + val stratifiedData = data.keyBy(StratifiedAuxiliary.stratifier(fractionPositive)) + val samplingRate = 0.1 + StratifiedAuxiliary.testSampleExact(stratifiedData, samplingRate, defaultSeed, n) + } + + // Use the same data for the rest of the tests + val fractionPositive = 0.3 + val n = 100 + val data = sc.parallelize(1 to n, 2) + val stratifiedData = data.keyBy(StratifiedAuxiliary.stratifier(fractionPositive)) + + // vary seed + for (seed <- defaultSeed to defaultSeed + 5L) { + val samplingRate = 0.1 + StratifiedAuxiliary.testSampleExact(stratifiedData, samplingRate, seed, n) + } + + // vary sampling rate + for (samplingRate <- List(0.01, 0.05, 0.1, 0.5)) { + StratifiedAuxiliary.testSampleExact(stratifiedData, samplingRate, defaultSeed, n) + } + } + test("reduceByKey") { val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) val sums = pairs.reduceByKey(_+_).collect() @@ -440,6 +519,98 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { intercept[IllegalArgumentException] {shuffled.lookup(-1)} } + private object StratifiedAuxiliary { + def stratifier (fractionPositive: Double) = { + (x: Int) => if (x % 10 < (10 * fractionPositive).toInt) "1" else "0" + } + + def checkSize(exact: Boolean, + withReplacement: Boolean, + expected: Long, + actual: Long, + p: Double): Boolean = { + if (exact) { + return expected == actual + } + val stdev = if (withReplacement) math.sqrt(expected) else math.sqrt(expected * p * (1 - p)) + // Very forgiving margin since we're dealing with very small sample sizes most of the time + math.abs(actual - expected) <= 6 * stdev + } + + def testSampleExact(stratifiedData: RDD[(String, Int)], + samplingRate: Double, + seed: Long, + n: Long) = { + testBernoulli(stratifiedData, true, samplingRate, seed, n) + testPoisson(stratifiedData, true, samplingRate, seed, n) + } + + def testSample(stratifiedData: RDD[(String, Int)], + samplingRate: Double, + seed: Long, + n: Long) = { + testBernoulli(stratifiedData, false, samplingRate, seed, n) + testPoisson(stratifiedData, false, samplingRate, seed, n) + } + + // Without replacement validation + def testBernoulli(stratifiedData: RDD[(String, Int)], + exact: Boolean, + samplingRate: Double, + seed: Long, + n: Long) = { + val expectedSampleSize = stratifiedData.countByKey() + .mapValues(count => math.ceil(count * samplingRate).toInt) + val fractions = Map("1" -> samplingRate, "0" -> samplingRate) + val sample = if (exact) { + stratifiedData.sampleByKeyExact(false, fractions, seed) + } else { + stratifiedData.sampleByKey(false, fractions, seed) + } + val sampleCounts = sample.countByKey() + val takeSample = sample.collect() + sampleCounts.foreach { case(k, v) => + assert(checkSize(exact, false, expectedSampleSize(k), v, samplingRate)) } + assert(takeSample.size === takeSample.toSet.size) + takeSample.foreach { x => assert(1 <= x._2 && x._2 <= n, s"elements not in [1, $n]") } + } + + // With replacement validation + def testPoisson(stratifiedData: RDD[(String, Int)], + exact: Boolean, + samplingRate: Double, + seed: Long, + n: Long) = { + val expectedSampleSize = stratifiedData.countByKey().mapValues(count => + math.ceil(count * samplingRate).toInt) + val fractions = Map("1" -> samplingRate, "0" -> samplingRate) + val sample = if (exact) { + stratifiedData.sampleByKeyExact(true, fractions, seed) + } else { + stratifiedData.sampleByKey(true, fractions, seed) + } + val sampleCounts = sample.countByKey() + val takeSample = sample.collect() + sampleCounts.foreach { case (k, v) => + assert(checkSize(exact, true, expectedSampleSize(k), v, samplingRate)) + } + val groupedByKey = takeSample.groupBy(_._1) + for ((key, v) <- groupedByKey) { + if (expectedSampleSize(key) >= 100 && samplingRate >= 0.1) { + // sample large enough for there to be repeats with high likelihood + assert(v.toSet.size < expectedSampleSize(key)) + } else { + if (exact) { + assert(v.toSet.size <= expectedSampleSize(key)) + } else { + assert(checkSize(false, true, expectedSampleSize(key), v.toSet.size, samplingRate)) + } + } + } + takeSample.foreach(x => assert(1 <= x._2 && x._2 <= n, s"elements not in [1, $n]")) + } + } + } /* diff --git a/core/src/test/scala/org/apache/spark/rdd/PartitionwiseSampledRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PartitionwiseSampledRDDSuite.scala index 5dd8de319a654..a0483886f8db3 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PartitionwiseSampledRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PartitionwiseSampledRDDSuite.scala @@ -43,7 +43,7 @@ class PartitionwiseSampledRDDSuite extends FunSuite with SharedSparkContext { test("seed distribution") { val rdd = sc.makeRDD(Array(1L, 2L, 3L, 4L), 2) val sampler = new MockSampler - val sample = new PartitionwiseSampledRDD[Long, Long](rdd, sampler, 0L) + val sample = new PartitionwiseSampledRDD[Long, Long](rdd, sampler, false, 0L) assert(sample.distinct().count == 2, "Seeds must be different.") } @@ -52,7 +52,7 @@ class PartitionwiseSampledRDDSuite extends FunSuite with SharedSparkContext { // We want to make sure there are no concurrency issues. val rdd = sc.parallelize(0 until 111, 10) for (sampler <- Seq(new BernoulliSampler[Int](0.5), new PoissonSampler[Int](0.5))) { - val sampled = new PartitionwiseSampledRDD[Int, Int](rdd, sampler) + val sampled = new PartitionwiseSampledRDD[Int, Int](rdd, sampler, true) sampled.zip(sampled).count() } } diff --git a/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala similarity index 95% rename from core/src/test/scala/org/apache/spark/PipedRDDSuite.scala rename to core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala index db56a4acdd6f5..be972c5e97a7e 100644 --- a/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala @@ -15,25 +15,21 @@ * limitations under the License. */ -package org.apache.spark +package org.apache.spark.rdd import java.io.File -import org.scalatest.FunSuite - -import org.apache.spark.rdd.{HadoopRDD, PipedRDD, HadoopPartition} -import org.apache.hadoop.mapred.{JobConf, TextInputFormat, FileSplit} import org.apache.hadoop.fs.Path +import org.apache.hadoop.io.{LongWritable, Text} +import org.apache.hadoop.mapred.{FileSplit, JobConf, TextInputFormat} +import org.apache.spark._ +import org.scalatest.FunSuite import scala.collection.Map import scala.language.postfixOps import scala.sys.process._ import scala.util.Try -import org.apache.hadoop.io.{Text, LongWritable} - -import org.apache.spark.executor.TaskMetrics - class PipedRDDSuite extends FunSuite with SharedSparkContext { test("basic pipe") { diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index 0f9cbe213ea17..926d4fecb5b91 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.rdd import scala.collection.mutable.{ArrayBuffer, HashMap} +import scala.collection.JavaConverters._ import scala.reflect.ClassTag import org.scalatest.FunSuite @@ -26,6 +27,7 @@ import org.apache.spark._ import org.apache.spark.SparkContext._ import org.apache.spark.util.Utils +import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} import org.apache.spark.rdd.RDDSuiteUtils._ class RDDSuite extends FunSuite with SharedSparkContext { @@ -79,11 +81,11 @@ class RDDSuite extends FunSuite with SharedSparkContext { def error(est: Long, size: Long) = math.abs(est - size) / size.toDouble - val size = 100 - val uniformDistro = for (i <- 1 to 100000) yield i % size - val simpleRdd = sc.makeRDD(uniformDistro) - assert(error(simpleRdd.countApproxDistinct(4, 0), size) < 0.4) - assert(error(simpleRdd.countApproxDistinct(8, 0), size) < 0.1) + val size = 1000 + val uniformDistro = for (i <- 1 to 5000) yield i % size + val simpleRdd = sc.makeRDD(uniformDistro, 10) + assert(error(simpleRdd.countApproxDistinct(8, 0), size) < 0.2) + assert(error(simpleRdd.countApproxDistinct(12, 0), size) < 0.1) } test("SparkContext.union") { @@ -121,6 +123,18 @@ class RDDSuite extends FunSuite with SharedSparkContext { assert(union.partitioner === nums1.partitioner) } + test("UnionRDD partition serialized size should be small") { + val largeVariable = new Array[Byte](1000 * 1000) + val rdd1 = sc.parallelize(1 to 10, 2).map(i => largeVariable.length) + val rdd2 = sc.parallelize(1 to 10, 3) + + val ser = SparkEnv.get.closureSerializer.newInstance() + val union = rdd1.union(rdd2) + // The UnionRDD itself should be large, but each individual partition should be small. + assert(ser.serialize(union).limit() > 2000) + assert(ser.serialize(union.partitions.head).limit() < 2000) + } + test("aggregate") { val pairs = sc.makeRDD(Array(("a", 1), ("b", 2), ("a", 2), ("c", 5), ("a", 3))) type StringMap = HashMap[String, Int] @@ -155,19 +169,13 @@ class RDDSuite extends FunSuite with SharedSparkContext { override def getPartitions: Array[Partition] = Array(onlySplit) override val getDependencies = List[Dependency[_]]() override def compute(split: Partition, context: TaskContext): Iterator[Int] = { - if (shouldFail) { - throw new Exception("injected failure") - } else { - Array(1, 2, 3, 4).iterator - } + throw new Exception("injected failure") } }.cache() val thrown = intercept[Exception]{ rdd.collect() } assert(thrown.getMessage.contains("injected failure")) - shouldFail = false - assert(rdd.collect().toList === List(1, 2, 3, 4)) } test("empty RDD") { @@ -276,7 +284,7 @@ class RDDSuite extends FunSuite with SharedSparkContext { // we can optionally shuffle to keep the upstream parallel val coalesced5 = data.coalesce(1, shuffle = true) val isEquals = coalesced5.dependencies.head.rdd.dependencies.head.rdd. - asInstanceOf[ShuffledRDD[_, _, _, _]] != null + asInstanceOf[ShuffledRDD[_, _, _]] != null assert(isEquals) // when shuffling, we can increase the number of partitions @@ -351,6 +359,20 @@ class RDDSuite extends FunSuite with SharedSparkContext { } } + // Test for SPARK-2412 -- ensure that the second pass of the algorithm does not throw an exception + test("coalesced RDDs with locality, fail first pass") { + val initialPartitions = 1000 + val targetLen = 50 + val couponCount = 2 * (math.log(targetLen)*targetLen + targetLen + 0.5).toInt // = 492 + + val blocks = (1 to initialPartitions).map { i => + (i, List(if (i > couponCount) "m2" else "m1")) + } + val data = sc.makeRDD(blocks) + val coalesced = data.coalesce(targetLen) + assert(coalesced.partitions.length == targetLen) + } + test("zipped RDDs") { val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) val zipped = nums.zip(nums.map(_ + 1.0)) @@ -379,6 +401,7 @@ class RDDSuite extends FunSuite with SharedSparkContext { test("mapWith") { import java.util.Random val ones = sc.makeRDD(Array(1, 1, 1, 1, 1, 1), 2) + @deprecated("suppress compile time deprecation warning", "1.0.0") val randoms = ones.mapWith( (index: Int) => new Random(index + 42)) {(t: Int, prng: Random) => prng.nextDouble * t}.collect() @@ -397,6 +420,7 @@ class RDDSuite extends FunSuite with SharedSparkContext { test("flatMapWith") { import java.util.Random val ones = sc.makeRDD(Array(1, 1, 1, 1, 1, 1), 2) + @deprecated("suppress compile time deprecation warning", "1.0.0") val randoms = ones.flatMapWith( (index: Int) => new Random(index + 42)) {(t: Int, prng: Random) => @@ -418,6 +442,7 @@ class RDDSuite extends FunSuite with SharedSparkContext { test("filterWith") { import java.util.Random val ints = sc.makeRDD(Array(1, 2, 3, 4, 5, 6), 2) + @deprecated("suppress compile time deprecation warning", "1.0.0") val sample = ints.filterWith( (index: Int) => new Random(index + 42)) {(t: Int, prng: Random) => prng.nextInt(3) == 0}. @@ -506,6 +531,15 @@ class RDDSuite extends FunSuite with SharedSparkContext { assert(sortedTopK === nums.sorted(ord).take(5)) } + test("sample preserves partitioner") { + val partitioner = new HashPartitioner(2) + val rdd = sc.parallelize(Seq((0, 1), (2, 3))).partitionBy(partitioner) + for (withReplacement <- Seq(true, false)) { + val sampled = rdd.sample(withReplacement, 1.0) + assert(sampled.partitioner === rdd.partitioner) + } + } + test("takeSample") { val n = 1000000 val data = sc.parallelize(1 to n, 2) @@ -587,6 +621,11 @@ class RDDSuite extends FunSuite with SharedSparkContext { } } + test("sort an empty RDD") { + val data = sc.emptyRDD[Int] + assert(data.sortBy(x => x).collect() === Array.empty) + } + test("sortByKey") { val data = sc.parallelize(Seq("5|50|A","4|60|C", "6|40|B")) @@ -681,6 +720,22 @@ class RDDSuite extends FunSuite with SharedSparkContext { assert(ids.length === n) } + test("retag with implicit ClassTag") { + val jsc: JavaSparkContext = new JavaSparkContext(sc) + val jrdd: JavaRDD[String] = jsc.parallelize(Seq("A", "B", "C").asJava) + jrdd.rdd.retag.collect() + } + + test("parent method") { + val rdd1 = sc.parallelize(1 to 10, 2) + val rdd2 = rdd1.filter(_ % 2 == 0) + val rdd3 = rdd2.map(_ + 1) + val rdd4 = new UnionRDD(sc, List(rdd1, rdd2, rdd3)) + assert(rdd4.parent(0).isInstanceOf[ParallelCollectionRDD[_]]) + assert(rdd4.parent(1).isInstanceOf[FilteredRDD[_]]) + assert(rdd4.parent(2).isInstanceOf[MappedRDD[_, _]]) + } + test("getNarrowAncestors") { val rdd1 = sc.parallelize(1 to 100, 4) val rdd2 = rdd1.filter(_ % 2 == 0).map(_ + 1) @@ -705,9 +760,9 @@ class RDDSuite extends FunSuite with SharedSparkContext { // Any ancestors before the shuffle are not considered assert(ancestors4.size === 0) - assert(ancestors4.count(_.isInstanceOf[ShuffledRDD[_, _, _, _]]) === 0) + assert(ancestors4.count(_.isInstanceOf[ShuffledRDD[_, _, _]]) === 0) assert(ancestors5.size === 3) - assert(ancestors5.count(_.isInstanceOf[ShuffledRDD[_, _, _, _]]) === 1) + assert(ancestors5.count(_.isInstanceOf[ShuffledRDD[_, _, _]]) === 1) assert(ancestors5.count(_.isInstanceOf[MapPartitionsRDD[_, _]]) === 0) assert(ancestors5.count(_.isInstanceOf[MappedValuesRDD[_, _, _]]) === 2) } diff --git a/core/src/test/scala/org/apache/spark/ZippedPartitionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/ZippedPartitionsSuite.scala similarity index 95% rename from core/src/test/scala/org/apache/spark/ZippedPartitionsSuite.scala rename to core/src/test/scala/org/apache/spark/rdd/ZippedPartitionsSuite.scala index 4f87fd8654c4a..72596e86865b2 100644 --- a/core/src/test/scala/org/apache/spark/ZippedPartitionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/ZippedPartitionsSuite.scala @@ -15,8 +15,9 @@ * limitations under the License. */ -package org.apache.spark +package org.apache.spark.rdd +import org.apache.spark.SharedSparkContext import org.scalatest.FunSuite object ZippedPartitionsSuite { diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 9f498d579a095..bd829752eb401 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -23,12 +23,15 @@ import scala.language.reflectiveCalls import akka.actor._ import akka.testkit.{ImplicitSender, TestKit, TestActorRef} import org.scalatest.{BeforeAndAfter, FunSuiteLike} +import org.scalatest.concurrent.Timeouts +import org.scalatest.time.SpanSugar._ import org.apache.spark._ import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.storage.{BlockId, BlockManagerId, BlockManagerMaster} import org.apache.spark.util.CallSite +import org.apache.spark.executor.TaskMetrics class BuggyDAGEventProcessActor extends Actor { val state = 0 @@ -37,10 +40,33 @@ class BuggyDAGEventProcessActor extends Actor { } } +/** + * An RDD for passing to DAGScheduler. These RDDs will use the dependencies and + * preferredLocations (if any) that are passed to them. They are deliberately not executable + * so we can test that DAGScheduler does not try to execute RDDs locally. + */ +class MyRDD( + sc: SparkContext, + numPartitions: Int, + dependencies: List[Dependency[_]], + locations: Seq[Seq[String]] = Nil) extends RDD[(Int, Int)](sc, dependencies) with Serializable { + override def compute(split: Partition, context: TaskContext): Iterator[(Int, Int)] = + throw new RuntimeException("should not be reached") + override def getPartitions = (0 until numPartitions).map(i => new Partition { + override def index = i + }).toArray + override def getPreferredLocations(split: Partition): Seq[String] = + if (locations.isDefinedAt(split.index)) + locations(split.index) + else + Nil + override def toString: String = "DAGSchedulerSuiteRDD " + id +} + class DAGSchedulerSuiteDummyException extends Exception class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with FunSuiteLike - with ImplicitSender with BeforeAndAfter with LocalSparkContext { + with ImplicitSender with BeforeAndAfter with LocalSparkContext with Timeouts { val conf = new SparkConf /** Set of TaskSets the DAGScheduler has requested executed. */ @@ -54,6 +80,8 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F override def schedulingMode: SchedulingMode = SchedulingMode.NONE override def start() = {} override def stop() = {} + override def executorHeartbeatReceived(execId: String, taskMetrics: Array[(Long, TaskMetrics)], + blockManagerId: BlockManagerId): Boolean = true override def submitTasks(taskSet: TaskSet) = { // normally done by TaskSetManager taskSet.tasks.foreach(_.epoch = mapOutputTracker.getEpoch) @@ -113,7 +141,9 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F } before { - sc = new SparkContext("local", "DAGSchedulerSuite") + // Enable local execution for this test + val conf = new SparkConf().set("spark.localExecution.enabled", "true") + sc = new SparkContext("local", "DAGSchedulerSuite", conf) sparkListener.successfulStages.clear() sparkListener.failedStages.clear() failure = null @@ -148,34 +178,7 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F * Type of RDD we use for testing. Note that we should never call the real RDD compute methods. * This is a pair RDD type so it can always be used in ShuffleDependencies. */ - type MyRDD = RDD[(Int, Int)] - - /** - * Create an RDD for passing to DAGScheduler. These RDDs will use the dependencies and - * preferredLocations (if any) that are passed to them. They are deliberately not executable - * so we can test that DAGScheduler does not try to execute RDDs locally. - */ - private def makeRdd( - numPartitions: Int, - dependencies: List[Dependency[_]], - locations: Seq[Seq[String]] = Nil - ): MyRDD = { - val maxPartition = numPartitions - 1 - val newRDD = new MyRDD(sc, dependencies) { - override def compute(split: Partition, context: TaskContext): Iterator[(Int, Int)] = - throw new RuntimeException("should not be reached") - override def getPartitions = (0 to maxPartition).map(i => new Partition { - override def index = i - }).toArray - override def getPreferredLocations(split: Partition): Seq[String] = - if (locations.isDefinedAt(split.index)) - locations(split.index) - else - Nil - override def toString: String = "DAGSchedulerSuiteRDD " + id - } - newRDD - } + type PairOfIntsRDD = RDD[(Int, Int)] /** * Process the supplied event as if it were the top of the DAGScheduler event queue, expecting @@ -234,19 +237,19 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F override def taskSucceeded(partition: Int, value: Any) = numResults += 1 override def jobFailed(exception: Exception) = throw exception } - submit(makeRdd(0, Nil), Array(), listener = fakeListener) + submit(new MyRDD(sc, 0, Nil), Array(), listener = fakeListener) assert(numResults === 0) } test("run trivial job") { - submit(makeRdd(1, Nil), Array(0)) + submit(new MyRDD(sc, 1, Nil), Array(0)) complete(taskSets(0), List((Success, 42))) assert(results === Map(0 -> 42)) assertDataStructuresEmpty } test("local job") { - val rdd = new MyRDD(sc, Nil) { + val rdd = new PairOfIntsRDD(sc, Nil) { override def compute(split: Partition, context: TaskContext): Iterator[(Int, Int)] = Array(42 -> 0).iterator override def getPartitions = Array( new Partition { override def index = 0 } ) @@ -260,7 +263,7 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F } test("local job oom") { - val rdd = new MyRDD(sc, Nil) { + val rdd = new PairOfIntsRDD(sc, Nil) { override def compute(split: Partition, context: TaskContext): Iterator[(Int, Int)] = throw new java.lang.OutOfMemoryError("test local job oom") override def getPartitions = Array( new Partition { override def index = 0 } ) @@ -274,8 +277,8 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F } test("run trivial job w/ dependency") { - val baseRdd = makeRdd(1, Nil) - val finalRdd = makeRdd(1, List(new OneToOneDependency(baseRdd))) + val baseRdd = new MyRDD(sc, 1, Nil) + val finalRdd = new MyRDD(sc, 1, List(new OneToOneDependency(baseRdd))) submit(finalRdd, Array(0)) complete(taskSets(0), Seq((Success, 42))) assert(results === Map(0 -> 42)) @@ -283,8 +286,8 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F } test("cache location preferences w/ dependency") { - val baseRdd = makeRdd(1, Nil) - val finalRdd = makeRdd(1, List(new OneToOneDependency(baseRdd))) + val baseRdd = new MyRDD(sc, 1, Nil) + val finalRdd = new MyRDD(sc, 1, List(new OneToOneDependency(baseRdd))) cacheLocations(baseRdd.id -> 0) = Seq(makeBlockManagerId("hostA"), makeBlockManagerId("hostB")) submit(finalRdd, Array(0)) @@ -295,8 +298,34 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F assertDataStructuresEmpty } + test("avoid exponential blowup when getting preferred locs list") { + // Build up a complex dependency graph with repeated zip operations, without preferred locations. + var rdd: RDD[_] = new MyRDD(sc, 1, Nil) + (1 to 30).foreach(_ => rdd = rdd.zip(rdd)) + // getPreferredLocs runs quickly, indicating that exponential graph traversal is avoided. + failAfter(10 seconds) { + val preferredLocs = scheduler.getPreferredLocs(rdd,0) + // No preferred locations are returned. + assert(preferredLocs.length === 0) + } + } + + test("unserializable task") { + val unserializableRdd = new MyRDD(sc, 1, Nil) { + class UnserializableClass + val unserializable = new UnserializableClass + } + submit(unserializableRdd, Array(0)) + assert(failure.getMessage.startsWith( + "Job aborted due to stage failure: Task not serializable:")) + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + assert(sparkListener.failedStages.contains(0)) + assert(sparkListener.failedStages.size === 1) + assertDataStructuresEmpty + } + test("trivial job failure") { - submit(makeRdd(1, Nil), Array(0)) + submit(new MyRDD(sc, 1, Nil), Array(0)) failed(taskSets(0), "some failure") assert(failure.getMessage === "Job aborted due to stage failure: some failure") assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) @@ -306,7 +335,7 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F } test("trivial job cancellation") { - val rdd = makeRdd(1, Nil) + val rdd = new MyRDD(sc, 1, Nil) val jobId = submit(rdd, Array(0)) cancel(jobId) assert(failure.getMessage === s"Job $jobId cancelled ") @@ -332,6 +361,8 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F } override def setDAGScheduler(dagScheduler: DAGScheduler) = {} override def defaultParallelism() = 2 + override def executorHeartbeatReceived(execId: String, taskMetrics: Array[(Long, TaskMetrics)], + blockManagerId: BlockManagerId): Boolean = true } val noKillScheduler = new DAGScheduler( sc, @@ -347,8 +378,7 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F } dagEventProcessTestActor = TestActorRef[DAGSchedulerEventProcessActor]( Props(classOf[DAGSchedulerEventProcessActor], noKillScheduler))(system) - val rdd = makeRdd(1, Nil) - val jobId = submit(rdd, Array(0)) + val jobId = submit(new MyRDD(sc, 1, Nil), Array(0)) cancel(jobId) // Because the job wasn't actually cancelled, we shouldn't have received a failure message. assert(failure === null) @@ -364,10 +394,10 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F } test("run trivial shuffle") { - val shuffleMapRdd = makeRdd(2, Nil) + val shuffleMapRdd = new MyRDD(sc, 2, Nil) val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) val shuffleId = shuffleDep.shuffleId - val reduceRdd = makeRdd(1, List(shuffleDep)) + val reduceRdd = new MyRDD(sc, 1, List(shuffleDep)) submit(reduceRdd, Array(0)) complete(taskSets(0), Seq( (Success, makeMapStatus("hostA", 1)), @@ -380,10 +410,10 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F } test("run trivial shuffle with fetch failure") { - val shuffleMapRdd = makeRdd(2, Nil) + val shuffleMapRdd = new MyRDD(sc, 2, Nil) val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) val shuffleId = shuffleDep.shuffleId - val reduceRdd = makeRdd(2, List(shuffleDep)) + val reduceRdd = new MyRDD(sc, 2, List(shuffleDep)) submit(reduceRdd, Array(0, 1)) complete(taskSets(0), Seq( (Success, makeMapStatus("hostA", 1)), @@ -406,10 +436,10 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F } test("ignore late map task completions") { - val shuffleMapRdd = makeRdd(2, Nil) + val shuffleMapRdd = new MyRDD(sc, 2, Nil) val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) val shuffleId = shuffleDep.shuffleId - val reduceRdd = makeRdd(2, List(shuffleDep)) + val reduceRdd = new MyRDD(sc, 2, List(shuffleDep)) submit(reduceRdd, Array(0, 1)) // pretend we were told hostA went away val oldEpoch = mapOutputTracker.getEpoch @@ -435,9 +465,9 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F } test("run shuffle with map stage failure") { - val shuffleMapRdd = makeRdd(2, Nil) + val shuffleMapRdd = new MyRDD(sc, 2, Nil) val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) - val reduceRdd = makeRdd(2, List(shuffleDep)) + val reduceRdd = new MyRDD(sc, 2, List(shuffleDep)) submit(reduceRdd, Array(0, 1)) // Fail the map stage. This should cause the entire job to fail. @@ -472,13 +502,13 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F * without shuffleMapRdd1. */ test("failure of stage used by two jobs") { - val shuffleMapRdd1 = makeRdd(2, Nil) + val shuffleMapRdd1 = new MyRDD(sc, 2, Nil) val shuffleDep1 = new ShuffleDependency(shuffleMapRdd1, null) - val shuffleMapRdd2 = makeRdd(2, Nil) + val shuffleMapRdd2 = new MyRDD(sc, 2, Nil) val shuffleDep2 = new ShuffleDependency(shuffleMapRdd2, null) - val reduceRdd1 = makeRdd(2, List(shuffleDep1)) - val reduceRdd2 = makeRdd(2, List(shuffleDep1, shuffleDep2)) + val reduceRdd1 = new MyRDD(sc, 2, List(shuffleDep1)) + val reduceRdd2 = new MyRDD(sc, 2, List(shuffleDep1, shuffleDep2)) // We need to make our own listeners for this test, since by default submit uses the same // listener for all jobs, and here we want to capture the failure for each job separately. @@ -511,10 +541,10 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F } test("run trivial shuffle with out-of-band failure and retry") { - val shuffleMapRdd = makeRdd(2, Nil) + val shuffleMapRdd = new MyRDD(sc, 2, Nil) val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) val shuffleId = shuffleDep.shuffleId - val reduceRdd = makeRdd(1, List(shuffleDep)) + val reduceRdd = new MyRDD(sc, 1, List(shuffleDep)) submit(reduceRdd, Array(0)) // blockManagerMaster.removeExecutor("exec-hostA") // pretend we were told hostA went away @@ -534,11 +564,11 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F } test("recursive shuffle failures") { - val shuffleOneRdd = makeRdd(2, Nil) + val shuffleOneRdd = new MyRDD(sc, 2, Nil) val shuffleDepOne = new ShuffleDependency(shuffleOneRdd, null) - val shuffleTwoRdd = makeRdd(2, List(shuffleDepOne)) + val shuffleTwoRdd = new MyRDD(sc, 2, List(shuffleDepOne)) val shuffleDepTwo = new ShuffleDependency(shuffleTwoRdd, null) - val finalRdd = makeRdd(1, List(shuffleDepTwo)) + val finalRdd = new MyRDD(sc, 1, List(shuffleDepTwo)) submit(finalRdd, Array(0)) // have the first stage complete normally complete(taskSets(0), Seq( @@ -563,11 +593,11 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F } test("cached post-shuffle") { - val shuffleOneRdd = makeRdd(2, Nil) + val shuffleOneRdd = new MyRDD(sc, 2, Nil) val shuffleDepOne = new ShuffleDependency(shuffleOneRdd, null) - val shuffleTwoRdd = makeRdd(2, List(shuffleDepOne)) + val shuffleTwoRdd = new MyRDD(sc, 2, List(shuffleDepOne)) val shuffleDepTwo = new ShuffleDependency(shuffleTwoRdd, null) - val finalRdd = makeRdd(1, List(shuffleDepTwo)) + val finalRdd = new MyRDD(sc, 1, List(shuffleDepTwo)) submit(finalRdd, Array(0)) cacheLocations(shuffleTwoRdd.id -> 0) = Seq(makeBlockManagerId("hostD")) cacheLocations(shuffleTwoRdd.id -> 1) = Seq(makeBlockManagerId("hostC")) @@ -594,8 +624,7 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F assertDataStructuresEmpty } - // TODO: Fix this and un-ignore the test. - ignore("misbehaved accumulator should not crash DAGScheduler and SparkContext") { + test("misbehaved accumulator should not crash DAGScheduler and SparkContext") { val acc = new Accumulator[Int](0, new AccumulatorParam[Int] { override def addAccumulator(t1: Int, t2: Int): Int = t1 + t2 override def zero(initialValue: Int): Int = 0 @@ -605,14 +634,10 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F }) // Run this on executors - intercept[SparkDriverExecutionException] { - sc.parallelize(1 to 10, 2).foreach { item => acc.add(1) } - } + sc.parallelize(1 to 10, 2).foreach { item => acc.add(1) } // Run this within a local thread - intercept[SparkDriverExecutionException] { - sc.parallelize(1 to 10, 2).map { item => acc.add(1) }.take(1) - } + sc.parallelize(1 to 10, 2).map { item => acc.add(1) }.take(1) // Make sure we can still run local commands as well as cluster commands. assert(sc.parallelize(1 to 10, 2).count() === 10) @@ -677,15 +702,11 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F BlockManagerId("exec-" + host, host, 12345, 0) private def assertDataStructuresEmpty = { - assert(scheduler.pendingTasks.isEmpty) assert(scheduler.activeJobs.isEmpty) assert(scheduler.failedStages.isEmpty) assert(scheduler.jobIdToActiveJob.isEmpty) assert(scheduler.jobIdToStageIds.isEmpty) - assert(scheduler.stageIdToJobIds.isEmpty) assert(scheduler.stageIdToStage.isEmpty) - assert(scheduler.stageToInfos.isEmpty) - assert(scheduler.resultStageToJob.isEmpty) assert(scheduler.runningStages.isEmpty) assert(scheduler.shuffleToMapStage.isEmpty) assert(scheduler.waitingStages.isEmpty) diff --git a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala index 21e3db34b8b7a..10d8b299317ea 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -259,7 +259,7 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter { assert(sc.eventLogger.isDefined) val eventLogger = sc.eventLogger.get val expectedLogDir = logDirPath.toString - assert(eventLogger.logDir.startsWith(expectedLogDir)) + assert(eventLogger.logDir.contains(expectedLogDir)) // Begin listening for events that trigger asserts val eventExistenceListener = new EventExistenceListener(eventLogger) diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala index 71f48e295ecca..3b0b8e2f68c97 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -258,8 +258,8 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers if (stageInfo.rddInfos.exists(_.name == d4.name)) { taskMetrics.shuffleReadMetrics should be ('defined) val sm = taskMetrics.shuffleReadMetrics.get - sm.totalBlocksFetched should be > (0) - sm.localBlocksFetched should be > (0) + sm.totalBlocksFetched should be (128) + sm.localBlocksFetched should be (128) sm.remoteBlocksFetched should be (0) sm.remoteBytesRead should be (0l) } diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala index 8bb5317cd2875..db2ad829a48f9 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala @@ -20,31 +20,35 @@ package org.apache.spark.scheduler import org.scalatest.FunSuite import org.scalatest.BeforeAndAfter -import org.apache.spark.LocalSparkContext -import org.apache.spark.Partition -import org.apache.spark.SparkContext -import org.apache.spark.TaskContext +import org.apache.spark._ import org.apache.spark.rdd.RDD +import org.apache.spark.util.Utils class TaskContextSuite extends FunSuite with BeforeAndAfter with LocalSparkContext { test("Calls executeOnCompleteCallbacks after failure") { - var completed = false + TaskContextSuite.completed = false sc = new SparkContext("local", "test") val rdd = new RDD[String](sc, List()) { override def getPartitions = Array[Partition](StubPartition(0)) override def compute(split: Partition, context: TaskContext) = { - context.addOnCompleteCallback(() => completed = true) + context.addTaskCompletionListener(context => TaskContextSuite.completed = true) sys.error("failed") } } - val func = (c: TaskContext, i: Iterator[String]) => i.next - val task = new ResultTask[String, String](0, rdd, func, 0, Seq(), 0) + val closureSerializer = SparkEnv.get.closureSerializer.newInstance() + val func = (c: TaskContext, i: Iterator[String]) => i.next() + val task = new ResultTask[String, String]( + 0, sc.broadcast(closureSerializer.serialize((rdd, func)).array), rdd.partitions(0), Seq(), 0) intercept[RuntimeException] { task.run(0) } - assert(completed === true) + assert(TaskContextSuite.completed === true) } +} - case class StubPartition(val index: Int) extends Partition +private object TaskContextSuite { + @volatile var completed = false } + +private case class StubPartition(index: Int) extends Partition diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 9ff2a487005c4..93e8ddacf8865 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -54,6 +54,23 @@ class FakeDAGScheduler(sc: SparkContext, taskScheduler: FakeTaskScheduler) } } +// Get the rack for a given host +object FakeRackUtil { + private val hostToRack = new mutable.HashMap[String, String]() + + def cleanUp() { + hostToRack.clear() + } + + def assignHostToRack(host: String, rack: String) { + hostToRack(host) = rack + } + + def getRackForHost(host: String) = { + hostToRack.get(host) + } +} + /** * A mock TaskSchedulerImpl implementation that just remembers information about tasks started and * feedback received from the TaskSetManagers. Note that it's important to initialize this with @@ -68,11 +85,31 @@ class FakeTaskScheduler(sc: SparkContext, liveExecutors: (String, String)* /* ex val finishedManagers = new ArrayBuffer[TaskSetManager] val taskSetsFailed = new ArrayBuffer[String] - val executors = new mutable.HashMap[String, String] ++ liveExecutors + val executors = new mutable.HashMap[String, String] + for ((execId, host) <- liveExecutors) { + addExecutor(execId, host) + } + + for ((execId, host) <- liveExecutors; rack <- getRackForHost(host)) { + hostsByRack.getOrElseUpdate(rack, new mutable.HashSet[String]()) += host + } dagScheduler = new FakeDAGScheduler(sc, this) - def removeExecutor(execId: String): Unit = executors -= execId + def removeExecutor(execId: String) { + executors -= execId + val host = executorIdToHost.get(execId) + assert(host != None) + val hostId = host.get + val executorsOnHost = executorsByHost(hostId) + executorsOnHost -= execId + for (rack <- getRackForHost(hostId); hosts <- hostsByRack.get(rack)) { + hosts -= hostId + if (hosts.isEmpty) { + hostsByRack -= rack + } + } + } override def taskSetFinished(manager: TaskSetManager): Unit = finishedManagers += manager @@ -80,9 +117,21 @@ class FakeTaskScheduler(sc: SparkContext, liveExecutors: (String, String)* /* ex override def hasExecutorsAliveOnHost(host: String): Boolean = executors.values.exists(_ == host) + override def hasHostAliveOnRack(rack: String): Boolean = { + hostsByRack.get(rack) != None + } + def addExecutor(execId: String, host: String) { executors.put(execId, host) + val executorsOnHost = executorsByHost.getOrElseUpdate(host, new mutable.HashSet[String]) + executorsOnHost += execId + executorIdToHost += execId -> host + for (rack <- getRackForHost(host)) { + hostsByRack.getOrElseUpdate(rack, new mutable.HashSet[String]()) += host + } } + + override def getRackForHost(value: String): Option[String] = FakeRackUtil.getRackForHost(value) } /** @@ -98,29 +147,29 @@ class LargeTask(stageId: Int) extends Task[Array[Byte]](stageId, 0) { } class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { - import TaskLocality.{ANY, PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL} + import TaskLocality.{ANY, PROCESS_LOCAL, NO_PREF, NODE_LOCAL, RACK_LOCAL} private val conf = new SparkConf val LOCALITY_WAIT = conf.getLong("spark.locality.wait", 3000) val MAX_TASK_FAILURES = 4 + override def beforeEach() { + super.beforeEach() + FakeRackUtil.cleanUp() + } + test("TaskSet with no preferences") { sc = new SparkContext("local", "test") val sched = new FakeTaskScheduler(sc, ("exec1", "host1")) val taskSet = FakeTask.createTaskSet(1) - val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES) + val clock = new FakeClock + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) - // Offer a host with process-local as the constraint; this should work because the TaskSet - // above won't have any locality preferences - val taskOption = manager.resourceOffer("exec1", "host1", TaskLocality.PROCESS_LOCAL) + // Offer a host with NO_PREF as the constraint, + // we should get a nopref task immediately since that's what we only have + var taskOption = manager.resourceOffer("exec1", "host1", NO_PREF) assert(taskOption.isDefined) - val task = taskOption.get - assert(task.executorId === "exec1") - assert(sched.startedTasks.contains(0)) - - // Re-offer the host -- now we should get no more tasks - assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL) === None) // Tell it the task has finished manager.handleSuccessfulTask(0, createTaskResult(0)) @@ -136,7 +185,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { // First three offers should all find tasks for (i <- 0 until 3) { - val taskOption = manager.resourceOffer("exec1", "host1", PROCESS_LOCAL) + var taskOption = manager.resourceOffer("exec1", "host1", NO_PREF) assert(taskOption.isDefined) val task = taskOption.get assert(task.executorId === "exec1") @@ -144,7 +193,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { assert(sched.startedTasks.toSet === Set(0, 1, 2)) // Re-offer the host -- now we should get no more tasks - assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL) === None) + assert(manager.resourceOffer("exec1", "host1", NO_PREF) === None) // Finish the first two tasks manager.handleSuccessfulTask(0, createTaskResult(0)) @@ -186,37 +235,40 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { ) val clock = new FakeClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) - // First offer host1, exec1: first task should be chosen assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0) - - // Offer host1, exec1 again: the last task, which has no prefs, should be chosen - assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 3) - - // Offer host1, exec1 again, at PROCESS_LOCAL level: nothing should get chosen - assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL) === None) + assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL) == None) clock.advance(LOCALITY_WAIT) - - // Offer host1, exec1 again, at PROCESS_LOCAL level: nothing should get chosen - assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL) === None) - - // Offer host1, exec1 again, at NODE_LOCAL level: we should choose task 2 + // Offer host1, exec1 again, at NODE_LOCAL level: the node local (task 2) should + // get chosen before the noPref task assert(manager.resourceOffer("exec1", "host1", NODE_LOCAL).get.index == 2) - // Offer host1, exec1 again, at NODE_LOCAL level: nothing should get chosen - assert(manager.resourceOffer("exec1", "host1", NODE_LOCAL) === None) - - // Offer host1, exec1 again, at ANY level: nothing should get chosen - assert(manager.resourceOffer("exec1", "host1", ANY) === None) + // Offer host2, exec3 again, at NODE_LOCAL level: we should choose task 2 + assert(manager.resourceOffer("exec2", "host2", NODE_LOCAL).get.index == 1) + // Offer host2, exec3 again, at NODE_LOCAL level: we should get noPref task + // after failing to find a node_Local task + assert(manager.resourceOffer("exec2", "host2", NODE_LOCAL) == None) clock.advance(LOCALITY_WAIT) + assert(manager.resourceOffer("exec2", "host2", NO_PREF).get.index == 3) + } - // Offer host1, exec1 again, at ANY level: task 1 should get chosen - assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 1) - - // Offer host1, exec1 again, at ANY level: nothing should be chosen as we've launched all tasks - assert(manager.resourceOffer("exec1", "host1", ANY) === None) + test("we do not need to delay scheduling when we only have noPref tasks in the queue") { + sc = new SparkContext("local", "test") + val sched = new FakeTaskScheduler(sc, ("exec1", "host1"), ("exec3", "host2")) + val taskSet = FakeTask.createTaskSet(3, + Seq(TaskLocation("host1", "exec1")), + Seq(TaskLocation("host2", "exec3")), + Seq() // Last task has no locality prefs + ) + val clock = new FakeClock + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) + // First offer host1, exec1: first task should be chosen + assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL).get.index === 0) + assert(manager.resourceOffer("exec3", "host2", PROCESS_LOCAL).get.index === 1) + assert(manager.resourceOffer("exec3", "host2", NODE_LOCAL) == None) + assert(manager.resourceOffer("exec3", "host2", NO_PREF).get.index === 2) } test("delay scheduling with fallback") { @@ -273,20 +325,24 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { // First offer host1: first task should be chosen assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0) - // Offer host1 again: third task should be chosen immediately because host3 is not up - assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 2) - - // After this, nothing should get chosen + // After this, nothing should get chosen, because we have separated tasks with unavailable preference + // from the noPrefPendingTasks assert(manager.resourceOffer("exec1", "host1", ANY) === None) // Now mark host2 as dead sched.removeExecutor("exec2") manager.executorLost("exec2", "host2") - // Task 1 should immediately be launched on host1 because its original host is gone + // nothing should be chosen + assert(manager.resourceOffer("exec1", "host1", ANY) === None) + + clock.advance(LOCALITY_WAIT * 2) + + // task 1 and 2 would be scheduled as nonLocal task assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 1) + assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 2) - // Now that all tasks have launched, nothing new should be launched anywhere else + // all finished assert(manager.resourceOffer("exec1", "host1", ANY) === None) assert(manager.resourceOffer("exec2", "host2", ANY) === None) } @@ -348,7 +404,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { val manager = new TaskSetManager(sched, taskSet, 4, clock) { - val offerResult = manager.resourceOffer("exec1", "host1", TaskLocality.PROCESS_LOCAL) + val offerResult = manager.resourceOffer("exec1", "host1", PROCESS_LOCAL) assert(offerResult.isDefined, "Expect resource offer to return a task") assert(offerResult.get.index === 0) @@ -359,15 +415,15 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { assert(!sched.taskSetsFailed.contains(taskSet.id)) // Ensure scheduling on exec1 fails after failure 1 due to blacklist - assert(manager.resourceOffer("exec1", "host1", TaskLocality.PROCESS_LOCAL).isEmpty) - assert(manager.resourceOffer("exec1", "host1", TaskLocality.NODE_LOCAL).isEmpty) - assert(manager.resourceOffer("exec1", "host1", TaskLocality.RACK_LOCAL).isEmpty) - assert(manager.resourceOffer("exec1", "host1", TaskLocality.ANY).isEmpty) + assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL).isEmpty) + assert(manager.resourceOffer("exec1", "host1", NODE_LOCAL).isEmpty) + assert(manager.resourceOffer("exec1", "host1", RACK_LOCAL).isEmpty) + assert(manager.resourceOffer("exec1", "host1", ANY).isEmpty) } // Run the task on exec1.1 - should work, and then fail it on exec1.1 { - val offerResult = manager.resourceOffer("exec1.1", "host1", TaskLocality.NODE_LOCAL) + val offerResult = manager.resourceOffer("exec1.1", "host1", NODE_LOCAL) assert(offerResult.isDefined, "Expect resource offer to return a task for exec1.1, offerResult = " + offerResult) @@ -379,12 +435,12 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { assert(!sched.taskSetsFailed.contains(taskSet.id)) // Ensure scheduling on exec1.1 fails after failure 2 due to blacklist - assert(manager.resourceOffer("exec1.1", "host1", TaskLocality.NODE_LOCAL).isEmpty) + assert(manager.resourceOffer("exec1.1", "host1", NODE_LOCAL).isEmpty) } // Run the task on exec2 - should work, and then fail it on exec2 { - val offerResult = manager.resourceOffer("exec2", "host2", TaskLocality.ANY) + val offerResult = manager.resourceOffer("exec2", "host2", ANY) assert(offerResult.isDefined, "Expect resource offer to return a task") assert(offerResult.get.index === 0) @@ -395,20 +451,20 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { assert(!sched.taskSetsFailed.contains(taskSet.id)) // Ensure scheduling on exec2 fails after failure 3 due to blacklist - assert(manager.resourceOffer("exec2", "host2", TaskLocality.ANY).isEmpty) + assert(manager.resourceOffer("exec2", "host2", ANY).isEmpty) } // After reschedule delay, scheduling on exec1 should be possible. clock.advance(rescheduleDelay) { - val offerResult = manager.resourceOffer("exec1", "host1", TaskLocality.PROCESS_LOCAL) + val offerResult = manager.resourceOffer("exec1", "host1", PROCESS_LOCAL) assert(offerResult.isDefined, "Expect resource offer to return a task") assert(offerResult.get.index === 0) assert(offerResult.get.executorId === "exec1") - assert(manager.resourceOffer("exec1", "host1", TaskLocality.PROCESS_LOCAL).isEmpty) + assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL).isEmpty) // Cause exec1 to fail : failure 4 manager.handleFailedTask(offerResult.get.taskId, TaskState.FINISHED, TaskResultLost) @@ -418,7 +474,9 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { assert(sched.taskSetsFailed.contains(taskSet.id)) } - test("new executors get added") { + test("new executors get added and lost") { + // Assign host2 to rack2 + FakeRackUtil.assignHostToRack("host2", "rack2") sc = new SparkContext("local", "test") val sched = new FakeTaskScheduler(sc) val taskSet = FakeTask.createTaskSet(4, @@ -428,24 +486,54 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { Seq()) val clock = new FakeClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) - // All tasks added to no-pref list since no preferred location is available - assert(manager.pendingTasksWithNoPrefs.size === 4) // Only ANY is valid - assert(manager.myLocalityLevels.sameElements(Array(ANY))) + assert(manager.myLocalityLevels.sameElements(Array(NO_PREF, ANY))) // Add a new executor sched.addExecutor("execD", "host1") manager.executorAdded() - // Task 0 and 1 should be removed from no-pref list - assert(manager.pendingTasksWithNoPrefs.size === 2) // Valid locality should contain NODE_LOCAL and ANY - assert(manager.myLocalityLevels.sameElements(Array(NODE_LOCAL, ANY))) + assert(manager.myLocalityLevels.sameElements(Array(NODE_LOCAL, NO_PREF, ANY))) // Add another executor sched.addExecutor("execC", "host2") manager.executorAdded() - // No-pref list now only contains task 3 - assert(manager.pendingTasksWithNoPrefs.size === 1) - // Valid locality should contain PROCESS_LOCAL, NODE_LOCAL and ANY - assert(manager.myLocalityLevels.sameElements(Array(PROCESS_LOCAL, NODE_LOCAL, ANY))) + // Valid locality should contain PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL and ANY + assert(manager.myLocalityLevels.sameElements(Array(PROCESS_LOCAL, NODE_LOCAL, NO_PREF, RACK_LOCAL, ANY))) + // test if the valid locality is recomputed when the executor is lost + sched.removeExecutor("execC") + manager.executorLost("execC", "host2") + assert(manager.myLocalityLevels.sameElements(Array(NODE_LOCAL, NO_PREF, ANY))) + sched.removeExecutor("execD") + manager.executorLost("execD", "host1") + assert(manager.myLocalityLevels.sameElements(Array(NO_PREF, ANY))) + } + + test("test RACK_LOCAL tasks") { + // Assign host1 to rack1 + FakeRackUtil.assignHostToRack("host1", "rack1") + // Assign host2 to rack1 + FakeRackUtil.assignHostToRack("host2", "rack1") + // Assign host3 to rack2 + FakeRackUtil.assignHostToRack("host3", "rack2") + sc = new SparkContext("local", "test") + val sched = new FakeTaskScheduler(sc, + ("execA", "host1"), ("execB", "host2"), ("execC", "host3")) + val taskSet = FakeTask.createTaskSet(2, + Seq(TaskLocation("host1", "execA")), + Seq(TaskLocation("host1", "execA"))) + val clock = new FakeClock + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) + + assert(manager.myLocalityLevels.sameElements(Array(PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL, ANY))) + // Set allowed locality to ANY + clock.advance(LOCALITY_WAIT * 3) + // Offer host3 + // No task is scheduled if we restrict locality to RACK_LOCAL + assert(manager.resourceOffer("execC", "host3", RACK_LOCAL) === None) + // Task 0 can be scheduled with ANY + assert(manager.resourceOffer("execC", "host3", ANY).get.index === 0) + // Offer host2 + // Task 1 can be scheduled with RACK_LOCAL + assert(manager.resourceOffer("execB", "host2", RACK_LOCAL).get.index === 1) } test("do not emit warning when serialized task is small") { @@ -475,6 +563,86 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { assert(manager.emittedTaskSizeWarning) } + test("speculative and noPref task should be scheduled after node-local") { + sc = new SparkContext("local", "test") + val sched = new FakeTaskScheduler(sc, ("execA", "host1"), ("execB", "host2"), ("execC", "host3")) + val taskSet = FakeTask.createTaskSet(4, + Seq(TaskLocation("host1", "execA")), + Seq(TaskLocation("host2"), TaskLocation("host1")), + Seq(), + Seq(TaskLocation("host3", "execC"))) + val clock = new FakeClock + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) + + assert(manager.resourceOffer("execA", "host1", PROCESS_LOCAL).get.index === 0) + assert(manager.resourceOffer("execA", "host1", NODE_LOCAL) == None) + assert(manager.resourceOffer("execA", "host1", NO_PREF).get.index == 1) + + manager.speculatableTasks += 1 + clock.advance(LOCALITY_WAIT) + // schedule the nonPref task + assert(manager.resourceOffer("execA", "host1", NO_PREF).get.index === 2) + // schedule the speculative task + assert(manager.resourceOffer("execB", "host2", NO_PREF).get.index === 1) + clock.advance(LOCALITY_WAIT * 3) + // schedule non-local tasks + assert(manager.resourceOffer("execB", "host2", ANY).get.index === 3) + } + + test("node-local tasks should be scheduled right away when there are only node-local and no-preference tasks") { + sc = new SparkContext("local", "test") + val sched = new FakeTaskScheduler(sc, ("execA", "host1"), ("execB", "host2"), ("execC", "host3")) + val taskSet = FakeTask.createTaskSet(4, + Seq(TaskLocation("host1")), + Seq(TaskLocation("host2")), + Seq(), + Seq(TaskLocation("host3"))) + val clock = new FakeClock + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) + + // node-local tasks are scheduled without delay + assert(manager.resourceOffer("execA", "host1", NODE_LOCAL).get.index === 0) + assert(manager.resourceOffer("execA", "host2", NODE_LOCAL).get.index === 1) + assert(manager.resourceOffer("execA", "host3", NODE_LOCAL).get.index === 3) + assert(manager.resourceOffer("execA", "host3", NODE_LOCAL) === None) + + // schedule no-preference after node local ones + assert(manager.resourceOffer("execA", "host3", NO_PREF).get.index === 2) + } + + test("Ensure TaskSetManager is usable after addition of levels") { + // Regression test for SPARK-2931 + sc = new SparkContext("local", "test") + val sched = new FakeTaskScheduler(sc) + val taskSet = FakeTask.createTaskSet(2, + Seq(TaskLocation("host1", "execA")), + Seq(TaskLocation("host2", "execB.1"))) + val clock = new FakeClock + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) + // Only ANY is valid + assert(manager.myLocalityLevels.sameElements(Array(ANY))) + // Add a new executor + sched.addExecutor("execA", "host1") + sched.addExecutor("execB.2", "host2") + manager.executorAdded() + assert(manager.pendingTasksWithNoPrefs.size === 0) + // Valid locality should contain PROCESS_LOCAL, NODE_LOCAL and ANY + assert(manager.myLocalityLevels.sameElements(Array(PROCESS_LOCAL, NODE_LOCAL, ANY))) + assert(manager.resourceOffer("execA", "host1", ANY) !== None) + clock.advance(LOCALITY_WAIT * 4) + assert(manager.resourceOffer("execB.2", "host2", ANY) !== None) + sched.removeExecutor("execA") + sched.removeExecutor("execB.2") + manager.executorLost("execA", "host1") + manager.executorLost("execB.2", "host2") + clock.advance(LOCALITY_WAIT * 4) + sched.addExecutor("execC", "host3") + manager.executorAdded() + // Prior to the fix, this line resulted in an ArrayIndexOutOfBoundsException: + assert(manager.resourceOffer("execC", "host3", ANY) !== None) + } + + def createTaskResult(id: Int): DirectTaskResult[Int] = { val valueSer = SparkEnv.get.serializer.newInstance() new DirectTaskResult[Int](valueSer.serialize(id), mutable.Map.empty, new TaskMetrics) diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala new file mode 100644 index 0000000000000..11e8c9c4cb37f --- /dev/null +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.serializer + +import org.apache.spark.util.Utils + +import com.esotericsoftware.kryo.Kryo +import org.scalatest.FunSuite + +import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkEnv, TestUtils} +import org.apache.spark.SparkContext._ +import org.apache.spark.serializer.KryoDistributedTest._ + +class KryoSerializerDistributedSuite extends FunSuite { + + test("kryo objects are serialised consistently in different processes") { + val conf = new SparkConf(false) + conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + conf.set("spark.kryo.registrator", classOf[AppJarRegistrator].getName) + conf.set("spark.task.maxFailures", "1") + + val jar = TestUtils.createJarWithClasses(List(AppJarRegistrator.customClassName)) + conf.setJars(List(jar.getPath)) + + val sc = new SparkContext("local-cluster[2,1,512]", "test", conf) + val original = Thread.currentThread.getContextClassLoader + val loader = new java.net.URLClassLoader(Array(jar), Utils.getContextOrSparkClassLoader) + SparkEnv.get.serializer.setDefaultClassLoader(loader) + + val cachedRDD = sc.parallelize((0 until 10).map((_, new MyCustomClass)), 3).cache() + + // Randomly mix the keys so that the join below will require a shuffle with each partition + // sending data to multiple other partitions. + val shuffledRDD = cachedRDD.map { case (i, o) => (i * i * i - 10 * i * i, o)} + + // Join the two RDDs, and force evaluation + assert(shuffledRDD.join(cachedRDD).collect().size == 1) + + LocalSparkContext.stop(sc) + } +} + +object KryoDistributedTest { + class MyCustomClass + + class AppJarRegistrator extends KryoRegistrator { + override def registerClasses(k: Kryo) { + val classLoader = Thread.currentThread.getContextClassLoader + k.register(Class.forName(AppJarRegistrator.customClassName, true, classLoader)) + } + } + + object AppJarRegistrator { + val customClassName = "KryoSerializerDistributedSuiteCustomClass" + } +} diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerResizableOutputSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerResizableOutputSuite.scala new file mode 100644 index 0000000000000..967c9e9899c9d --- /dev/null +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerResizableOutputSuite.scala @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.serializer + +import org.scalatest.FunSuite + +import org.apache.spark.SparkConf +import org.apache.spark.SparkContext +import org.apache.spark.LocalSparkContext +import org.apache.spark.SparkException + + +class KryoSerializerResizableOutputSuite extends FunSuite { + + // trial and error showed this will not serialize with 1mb buffer + val x = (1 to 400000).toArray + + test("kryo without resizable output buffer should fail on large array") { + val conf = new SparkConf(false) + conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + conf.set("spark.kryoserializer.buffer.mb", "1") + conf.set("spark.kryoserializer.buffer.max.mb", "1") + val sc = new SparkContext("local", "test", conf) + intercept[SparkException](sc.parallelize(x).collect()) + LocalSparkContext.stop(sc) + } + + test("kryo with resizable output buffer should succeed on large array") { + val conf = new SparkConf(false) + conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + conf.set("spark.kryoserializer.buffer.mb", "1") + conf.set("spark.kryoserializer.buffer.max.mb", "2") + val sc = new SparkContext("local", "test", conf) + assert(sc.parallelize(x).collect() === x) + LocalSparkContext.stop(sc) + } +} diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala index 79280d1a06653..e1e35b688d581 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala @@ -23,9 +23,10 @@ import scala.reflect.ClassTag import com.esotericsoftware.kryo.Kryo import org.scalatest.FunSuite -import org.apache.spark.SharedSparkContext +import org.apache.spark.{SparkConf, SharedSparkContext} import org.apache.spark.serializer.KryoTest._ + class KryoSerializerSuite extends FunSuite with SharedSparkContext { conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") conf.set("spark.kryo.registrator", classOf[MyRegistrator].getName) @@ -207,9 +208,43 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext { .fold(new ClassWithoutNoArgConstructor(10))((t1, t2) => new ClassWithoutNoArgConstructor(t1.x + t2.x)).x assert(10 + control.sum === result) } + + test("kryo with nonexistent custom registrator should fail") { + import org.apache.spark.{SparkConf, SparkException} + + val conf = new SparkConf(false) + conf.set("spark.kryo.registrator", "this.class.does.not.exist") + + val thrown = intercept[SparkException](new KryoSerializer(conf).newInstance()) + assert(thrown.getMessage.contains("Failed to invoke this.class.does.not.exist")) + } + + test("default class loader can be set by a different thread") { + val ser = new KryoSerializer(new SparkConf) + + // First serialize the object + val serInstance = ser.newInstance() + val bytes = serInstance.serialize(new ClassLoaderTestingObject) + + // Deserialize the object to make sure normal deserialization works + serInstance.deserialize[ClassLoaderTestingObject](bytes) + + // Set a special, broken ClassLoader and make sure we get an exception on deserialization + ser.setDefaultClassLoader(new ClassLoader() { + override def loadClass(name: String) = throw new UnsupportedOperationException + }) + intercept[UnsupportedOperationException] { + ser.newInstance().deserialize[ClassLoaderTestingObject](bytes) + } + } } + +class ClassLoaderTestingObject + + object KryoTest { + case class CaseClass(i: Int, s: String) {} class ClassWithNoArgConstructor { diff --git a/core/src/test/scala/org/apache/spark/serializer/ProactiveClosureSerializationSuite.scala b/core/src/test/scala/org/apache/spark/serializer/ProactiveClosureSerializationSuite.scala index 5d15a68ac7e4f..aad6599589420 100644 --- a/core/src/test/scala/org/apache/spark/serializer/ProactiveClosureSerializationSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/ProactiveClosureSerializationSuite.scala @@ -15,15 +15,12 @@ * limitations under the License. */ -package org.apache.spark.serializer; - -import java.io.NotSerializableException +package org.apache.spark.serializer import org.scalatest.FunSuite +import org.apache.spark.{SharedSparkContext, SparkException} import org.apache.spark.rdd.RDD -import org.apache.spark.SparkException -import org.apache.spark.SharedSparkContext /* A trivial (but unserializable) container for trivial functions */ class UnserializableClass { @@ -38,52 +35,50 @@ class ProactiveClosureSerializationSuite extends FunSuite with SharedSparkContex test("throws expected serialization exceptions on actions") { val (data, uc) = fixture - val ex = intercept[SparkException] { - data.map(uc.op(_)).count + data.map(uc.op(_)).count() } - assert(ex.getMessage.contains("Task not serializable")) } // There is probably a cleaner way to eliminate boilerplate here, but we're // iterating over a map from transformation names to functions that perform that // transformation on a given RDD, creating one test case for each - + for (transformation <- - Map("map" -> xmap _, "flatMap" -> xflatMap _, "filter" -> xfilter _, - "mapWith" -> xmapWith _, "mapPartitions" -> xmapPartitions _, + Map("map" -> xmap _, + "flatMap" -> xflatMap _, + "filter" -> xfilter _, + "mapPartitions" -> xmapPartitions _, "mapPartitionsWithIndex" -> xmapPartitionsWithIndex _, - "mapPartitionsWithContext" -> xmapPartitionsWithContext _, - "filterWith" -> xfilterWith _)) { + "mapPartitionsWithContext" -> xmapPartitionsWithContext _)) { val (name, xf) = transformation - + test(s"$name transformations throw proactive serialization exceptions") { val (data, uc) = fixture - val ex = intercept[SparkException] { xf(data, uc) } - assert(ex.getMessage.contains("Task not serializable"), s"RDD.$name doesn't proactively throw NotSerializableException") } } - + private def xmap(x: RDD[String], uc: UnserializableClass): RDD[String] = x.map(y=>uc.op(y)) - private def xmapWith(x: RDD[String], uc: UnserializableClass): RDD[String] = - x.mapWith(x => x.toString)((x,y)=>x + uc.op(y)) + private def xflatMap(x: RDD[String], uc: UnserializableClass): RDD[String] = x.flatMap(y=>Seq(uc.op(y))) + private def xfilter(x: RDD[String], uc: UnserializableClass): RDD[String] = x.filter(y=>uc.pred(y)) - private def xfilterWith(x: RDD[String], uc: UnserializableClass): RDD[String] = - x.filterWith(x => x.toString)((x,y)=>uc.pred(y)) + private def xmapPartitions(x: RDD[String], uc: UnserializableClass): RDD[String] = x.mapPartitions(_.map(y=>uc.op(y))) + private def xmapPartitionsWithIndex(x: RDD[String], uc: UnserializableClass): RDD[String] = x.mapPartitionsWithIndex((_, it) => it.map(y=>uc.op(y))) + private def xmapPartitionsWithContext(x: RDD[String], uc: UnserializableClass): RDD[String] = x.mapPartitionsWithContext((_, it) => it.map(y=>uc.op(y))) diff --git a/core/src/test/scala/org/apache/spark/shuffle/ShuffleMemoryManagerSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/ShuffleMemoryManagerSuite.scala new file mode 100644 index 0000000000000..d31bc22ee74f7 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/shuffle/ShuffleMemoryManagerSuite.scala @@ -0,0 +1,294 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.scalatest.FunSuite +import org.scalatest.concurrent.Timeouts +import org.scalatest.time.SpanSugar._ +import java.util.concurrent.atomic.AtomicBoolean +import java.util.concurrent.CountDownLatch + +class ShuffleMemoryManagerSuite extends FunSuite with Timeouts { + /** Launch a thread with the given body block and return it. */ + private def startThread(name: String)(body: => Unit): Thread = { + val thread = new Thread("ShuffleMemorySuite " + name) { + override def run() { + body + } + } + thread.start() + thread + } + + test("single thread requesting memory") { + val manager = new ShuffleMemoryManager(1000L) + + assert(manager.tryToAcquire(100L) === 100L) + assert(manager.tryToAcquire(400L) === 400L) + assert(manager.tryToAcquire(400L) === 400L) + assert(manager.tryToAcquire(200L) === 100L) + assert(manager.tryToAcquire(100L) === 0L) + assert(manager.tryToAcquire(100L) === 0L) + + manager.release(500L) + assert(manager.tryToAcquire(300L) === 300L) + assert(manager.tryToAcquire(300L) === 200L) + + manager.releaseMemoryForThisThread() + assert(manager.tryToAcquire(1000L) === 1000L) + assert(manager.tryToAcquire(100L) === 0L) + } + + test("two threads requesting full memory") { + // Two threads request 500 bytes first, wait for each other to get it, and then request + // 500 more; we should immediately return 0 as both are now at 1 / N + + val manager = new ShuffleMemoryManager(1000L) + + class State { + var t1Result1 = -1L + var t2Result1 = -1L + var t1Result2 = -1L + var t2Result2 = -1L + } + val state = new State + + val t1 = startThread("t1") { + val r1 = manager.tryToAcquire(500L) + state.synchronized { + state.t1Result1 = r1 + state.notifyAll() + while (state.t2Result1 === -1L) { + state.wait() + } + } + val r2 = manager.tryToAcquire(500L) + state.synchronized { state.t1Result2 = r2 } + } + + val t2 = startThread("t2") { + val r1 = manager.tryToAcquire(500L) + state.synchronized { + state.t2Result1 = r1 + state.notifyAll() + while (state.t1Result1 === -1L) { + state.wait() + } + } + val r2 = manager.tryToAcquire(500L) + state.synchronized { state.t2Result2 = r2 } + } + + failAfter(20 seconds) { + t1.join() + t2.join() + } + + assert(state.t1Result1 === 500L) + assert(state.t2Result1 === 500L) + assert(state.t1Result2 === 0L) + assert(state.t2Result2 === 0L) + } + + + test("threads cannot grow past 1 / N") { + // Two threads request 250 bytes first, wait for each other to get it, and then request + // 500 more; we should only grant 250 bytes to each of them on this second request + + val manager = new ShuffleMemoryManager(1000L) + + class State { + var t1Result1 = -1L + var t2Result1 = -1L + var t1Result2 = -1L + var t2Result2 = -1L + } + val state = new State + + val t1 = startThread("t1") { + val r1 = manager.tryToAcquire(250L) + state.synchronized { + state.t1Result1 = r1 + state.notifyAll() + while (state.t2Result1 === -1L) { + state.wait() + } + } + val r2 = manager.tryToAcquire(500L) + state.synchronized { state.t1Result2 = r2 } + } + + val t2 = startThread("t2") { + val r1 = manager.tryToAcquire(250L) + state.synchronized { + state.t2Result1 = r1 + state.notifyAll() + while (state.t1Result1 === -1L) { + state.wait() + } + } + val r2 = manager.tryToAcquire(500L) + state.synchronized { state.t2Result2 = r2 } + } + + failAfter(20 seconds) { + t1.join() + t2.join() + } + + assert(state.t1Result1 === 250L) + assert(state.t2Result1 === 250L) + assert(state.t1Result2 === 250L) + assert(state.t2Result2 === 250L) + } + + test("threads can block to get at least 1 / 2N memory") { + // t1 grabs 1000 bytes and then waits until t2 is ready to make a request. It sleeps + // for a bit and releases 250 bytes, which should then be greanted to t2. Further requests + // by t2 will return false right away because it now has 1 / 2N of the memory. + + val manager = new ShuffleMemoryManager(1000L) + + class State { + var t1Requested = false + var t2Requested = false + var t1Result = -1L + var t2Result = -1L + var t2Result2 = -1L + var t2WaitTime = 0L + } + val state = new State + + val t1 = startThread("t1") { + state.synchronized { + state.t1Result = manager.tryToAcquire(1000L) + state.t1Requested = true + state.notifyAll() + while (!state.t2Requested) { + state.wait() + } + } + // Sleep a bit before releasing our memory; this is hacky but it would be difficult to make + // sure the other thread blocks for some time otherwise + Thread.sleep(300) + manager.release(250L) + } + + val t2 = startThread("t2") { + state.synchronized { + while (!state.t1Requested) { + state.wait() + } + state.t2Requested = true + state.notifyAll() + } + val startTime = System.currentTimeMillis() + val result = manager.tryToAcquire(250L) + val endTime = System.currentTimeMillis() + state.synchronized { + state.t2Result = result + // A second call should return 0 because we're now already at 1 / 2N + state.t2Result2 = manager.tryToAcquire(100L) + state.t2WaitTime = endTime - startTime + } + } + + failAfter(20 seconds) { + t1.join() + t2.join() + } + + // Both threads should've been able to acquire their memory; the second one will have waited + // until the first one acquired 1000 bytes and then released 250 + state.synchronized { + assert(state.t1Result === 1000L, "t1 could not allocate memory") + assert(state.t2Result === 250L, "t2 could not allocate memory") + assert(state.t2WaitTime > 200, s"t2 waited less than 200 ms (${state.t2WaitTime})") + assert(state.t2Result2 === 0L, "t1 got extra memory the second time") + } + } + + test("releaseMemoryForThisThread") { + // t1 grabs 1000 bytes and then waits until t2 is ready to make a request. It sleeps + // for a bit and releases all its memory. t2 should now be able to grab all the memory. + + val manager = new ShuffleMemoryManager(1000L) + + class State { + var t1Requested = false + var t2Requested = false + var t1Result = -1L + var t2Result1 = -1L + var t2Result2 = -1L + var t2Result3 = -1L + var t2WaitTime = 0L + } + val state = new State + + val t1 = startThread("t1") { + state.synchronized { + state.t1Result = manager.tryToAcquire(1000L) + state.t1Requested = true + state.notifyAll() + while (!state.t2Requested) { + state.wait() + } + } + // Sleep a bit before releasing our memory; this is hacky but it would be difficult to make + // sure the other thread blocks for some time otherwise + Thread.sleep(300) + manager.releaseMemoryForThisThread() + } + + val t2 = startThread("t2") { + state.synchronized { + while (!state.t1Requested) { + state.wait() + } + state.t2Requested = true + state.notifyAll() + } + val startTime = System.currentTimeMillis() + val r1 = manager.tryToAcquire(500L) + val endTime = System.currentTimeMillis() + val r2 = manager.tryToAcquire(500L) + val r3 = manager.tryToAcquire(500L) + state.synchronized { + state.t2Result1 = r1 + state.t2Result2 = r2 + state.t2Result3 = r3 + state.t2WaitTime = endTime - startTime + } + } + + failAfter(20 seconds) { + t1.join() + t2.join() + } + + // Both threads should've been able to acquire their memory; the second one will have waited + // until the first one acquired 1000 bytes and then released all of it + state.synchronized { + assert(state.t1Result === 1000L, "t1 could not allocate memory") + assert(state.t2Result1 === 500L, "t2 didn't get 500 bytes the first time") + assert(state.t2Result2 === 500L, "t2 didn't get 500 bytes the second time") + assert(state.t2Result3 === 0L, s"t2 got more bytes a third time (${state.t2Result3})") + assert(state.t2WaitTime > 200, s"t2 waited less than 200 ms (${state.t2WaitTime})") + } + } +} diff --git a/core/src/test/scala/org/apache/spark/storage/BlockFetcherIteratorSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockFetcherIteratorSuite.scala new file mode 100644 index 0000000000000..bcbfe8baf36ad --- /dev/null +++ b/core/src/test/scala/org/apache/spark/storage/BlockFetcherIteratorSuite.scala @@ -0,0 +1,231 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.storage + +import java.io.IOException +import java.nio.ByteBuffer + +import scala.collection.mutable.ArrayBuffer +import scala.concurrent.future +import scala.concurrent.ExecutionContext.Implicits.global + +import org.scalatest.{FunSuite, Matchers} + +import org.mockito.Mockito._ +import org.mockito.Matchers.{any, eq => meq} +import org.mockito.stubbing.Answer +import org.mockito.invocation.InvocationOnMock + +import org.apache.spark.storage.BlockFetcherIterator._ +import org.apache.spark.network.{ConnectionManager, Message} +import org.apache.spark.executor.ShuffleReadMetrics + +class BlockFetcherIteratorSuite extends FunSuite with Matchers { + + test("block fetch from local fails using BasicBlockFetcherIterator") { + val blockManager = mock(classOf[BlockManager]) + val connManager = mock(classOf[ConnectionManager]) + doReturn(connManager).when(blockManager).connectionManager + doReturn(BlockManagerId("test-client", "test-client", 1, 0)).when(blockManager).blockManagerId + + doReturn((48 * 1024 * 1024).asInstanceOf[Long]).when(blockManager).maxBytesInFlight + + val blIds = Array[BlockId]( + ShuffleBlockId(0,0,0), + ShuffleBlockId(0,1,0), + ShuffleBlockId(0,2,0), + ShuffleBlockId(0,3,0), + ShuffleBlockId(0,4,0)) + + val optItr = mock(classOf[Option[Iterator[Any]]]) + val answer = new Answer[Option[Iterator[Any]]] { + override def answer(invocation: InvocationOnMock) = Option[Iterator[Any]] { + throw new Exception + } + } + + // 3rd block is going to fail + doReturn(optItr).when(blockManager).getLocalFromDisk(meq(blIds(0)), any()) + doReturn(optItr).when(blockManager).getLocalFromDisk(meq(blIds(1)), any()) + doAnswer(answer).when(blockManager).getLocalFromDisk(meq(blIds(2)), any()) + doReturn(optItr).when(blockManager).getLocalFromDisk(meq(blIds(3)), any()) + doReturn(optItr).when(blockManager).getLocalFromDisk(meq(blIds(4)), any()) + + val bmId = BlockManagerId("test-client", "test-client",1 , 0) + val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( + (bmId, blIds.map(blId => (blId, 1.asInstanceOf[Long])).toSeq) + ) + + val iterator = new BasicBlockFetcherIterator(blockManager, blocksByAddress, null, + new ShuffleReadMetrics()) + + iterator.initialize() + + // 3rd getLocalFromDisk invocation should be failed + verify(blockManager, times(3)).getLocalFromDisk(any(), any()) + + assert(iterator.hasNext, "iterator should have 5 elements but actually has no elements") + // the 2nd element of the tuple returned by iterator.next should be defined when fetching successfully + assert(iterator.next._2.isDefined, "1st element should be defined but is not actually defined") + assert(iterator.hasNext, "iterator should have 5 elements but actually has 1 element") + assert(iterator.next._2.isDefined, "2nd element should be defined but is not actually defined") + assert(iterator.hasNext, "iterator should have 5 elements but actually has 2 elements") + // 3rd fetch should be failed + assert(!iterator.next._2.isDefined, "3rd element should not be defined but is actually defined") + assert(iterator.hasNext, "iterator should have 5 elements but actually has 3 elements") + // Don't call next() after fetching non-defined element even if thare are rest of elements in the iterator. + // Otherwise, BasicBlockFetcherIterator hangs up. + } + + + test("block fetch from local succeed using BasicBlockFetcherIterator") { + val blockManager = mock(classOf[BlockManager]) + val connManager = mock(classOf[ConnectionManager]) + doReturn(connManager).when(blockManager).connectionManager + doReturn(BlockManagerId("test-client", "test-client", 1, 0)).when(blockManager).blockManagerId + + doReturn((48 * 1024 * 1024).asInstanceOf[Long]).when(blockManager).maxBytesInFlight + + val blIds = Array[BlockId]( + ShuffleBlockId(0,0,0), + ShuffleBlockId(0,1,0), + ShuffleBlockId(0,2,0), + ShuffleBlockId(0,3,0), + ShuffleBlockId(0,4,0)) + + val optItr = mock(classOf[Option[Iterator[Any]]]) + + // All blocks should be fetched successfully + doReturn(optItr).when(blockManager).getLocalFromDisk(meq(blIds(0)), any()) + doReturn(optItr).when(blockManager).getLocalFromDisk(meq(blIds(1)), any()) + doReturn(optItr).when(blockManager).getLocalFromDisk(meq(blIds(2)), any()) + doReturn(optItr).when(blockManager).getLocalFromDisk(meq(blIds(3)), any()) + doReturn(optItr).when(blockManager).getLocalFromDisk(meq(blIds(4)), any()) + + val bmId = BlockManagerId("test-client", "test-client",1 , 0) + val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( + (bmId, blIds.map(blId => (blId, 1.asInstanceOf[Long])).toSeq) + ) + + val iterator = new BasicBlockFetcherIterator(blockManager, blocksByAddress, null, + new ShuffleReadMetrics()) + + iterator.initialize() + + // getLocalFromDis should be invoked for all of 5 blocks + verify(blockManager, times(5)).getLocalFromDisk(any(), any()) + + assert(iterator.hasNext, "iterator should have 5 elements but actually has no elements") + assert(iterator.next._2.isDefined, "All elements should be defined but 1st element is not actually defined") + assert(iterator.hasNext, "iterator should have 5 elements but actually has 1 element") + assert(iterator.next._2.isDefined, "All elements should be defined but 2nd element is not actually defined") + assert(iterator.hasNext, "iterator should have 5 elements but actually has 2 elements") + assert(iterator.next._2.isDefined, "All elements should be defined but 3rd element is not actually defined") + assert(iterator.hasNext, "iterator should have 5 elements but actually has 3 elements") + assert(iterator.next._2.isDefined, "All elements should be defined but 4th element is not actually defined") + assert(iterator.hasNext, "iterator should have 5 elements but actually has 4 elements") + assert(iterator.next._2.isDefined, "All elements should be defined but 5th element is not actually defined") + } + + test("block fetch from remote fails using BasicBlockFetcherIterator") { + val blockManager = mock(classOf[BlockManager]) + val connManager = mock(classOf[ConnectionManager]) + when(blockManager.connectionManager).thenReturn(connManager) + + val f = future { + throw new IOException("Send failed or we received an error ACK") + } + when(connManager.sendMessageReliably(any(), + any())).thenReturn(f) + when(blockManager.futureExecContext).thenReturn(global) + + when(blockManager.blockManagerId).thenReturn( + BlockManagerId("test-client", "test-client", 1, 0)) + when(blockManager.maxBytesInFlight).thenReturn(48 * 1024 * 1024) + + val blId1 = ShuffleBlockId(0,0,0) + val blId2 = ShuffleBlockId(0,1,0) + val bmId = BlockManagerId("test-server", "test-server",1 , 0) + val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( + (bmId, Seq((blId1, 1L), (blId2, 1L))) + ) + + val iterator = new BasicBlockFetcherIterator(blockManager, + blocksByAddress, null, new ShuffleReadMetrics()) + + iterator.initialize() + iterator.foreach{ + case (_, r) => { + (!r.isDefined) should be(true) + } + } + } + + test("block fetch from remote succeed using BasicBlockFetcherIterator") { + val blockManager = mock(classOf[BlockManager]) + val connManager = mock(classOf[ConnectionManager]) + when(blockManager.connectionManager).thenReturn(connManager) + + val blId1 = ShuffleBlockId(0,0,0) + val blId2 = ShuffleBlockId(0,1,0) + val buf1 = ByteBuffer.allocate(4) + val buf2 = ByteBuffer.allocate(4) + buf1.putInt(1) + buf1.flip() + buf2.putInt(1) + buf2.flip() + val blockMessage1 = BlockMessage.fromGotBlock(GotBlock(blId1, buf1)) + val blockMessage2 = BlockMessage.fromGotBlock(GotBlock(blId2, buf2)) + val blockMessageArray = new BlockMessageArray( + Seq(blockMessage1, blockMessage2)) + + val bufferMessage = blockMessageArray.toBufferMessage + val buffer = ByteBuffer.allocate(bufferMessage.size) + val arrayBuffer = new ArrayBuffer[ByteBuffer] + bufferMessage.buffers.foreach{ b => + buffer.put(b) + } + buffer.flip() + arrayBuffer += buffer + + val f = future { + Message.createBufferMessage(arrayBuffer) + } + when(connManager.sendMessageReliably(any(), + any())).thenReturn(f) + when(blockManager.futureExecContext).thenReturn(global) + + when(blockManager.blockManagerId).thenReturn( + BlockManagerId("test-client", "test-client", 1, 0)) + when(blockManager.maxBytesInFlight).thenReturn(48 * 1024 * 1024) + + val bmId = BlockManagerId("test-server", "test-server",1 , 0) + val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( + (bmId, Seq((blId1, 1L), (blId2, 1L))) + ) + + val iterator = new BasicBlockFetcherIterator(blockManager, + blocksByAddress, null, new ShuffleReadMetrics()) + iterator.initialize() + iterator.foreach{ + case (_, r) => { + (r.isDefined) should be(true) + } + } + } +} diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 23cb6905bfdeb..20bac66105a69 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -19,30 +19,40 @@ package org.apache.spark.storage import java.nio.{ByteBuffer, MappedByteBuffer} import java.util.Arrays +import java.util.concurrent.TimeUnit import akka.actor._ -import org.apache.spark.SparkConf -import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} -import org.apache.spark.util.{AkkaUtils, ByteBufferInputStream, SizeEstimator, Utils} -import org.mockito.Mockito.{mock, when} +import akka.pattern.ask +import akka.util.Timeout +import org.apache.spark.shuffle.hash.HashShuffleManager + +import org.mockito.invocation.InvocationOnMock +import org.mockito.Matchers.any +import org.mockito.Mockito.{doAnswer, mock, spy, when} +import org.mockito.stubbing.Answer + import org.scalatest.{BeforeAndAfter, FunSuite, PrivateMethodTester} import org.scalatest.concurrent.Eventually._ import org.scalatest.concurrent.Timeouts._ import org.scalatest.Matchers -import org.scalatest.time.SpanSugar._ -import org.apache.spark.{MapOutputTrackerMaster, SecurityManager, SparkConf, SparkContext} +import org.apache.spark.{MapOutputTrackerMaster, SecurityManager, SparkConf} import org.apache.spark.executor.DataReadMethod +import org.apache.spark.network.{Message, ConnectionManagerId} import org.apache.spark.scheduler.LiveListenerBus import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} +import org.apache.spark.storage.BlockManagerMessages.BlockManagerHeartbeat import org.apache.spark.util.{AkkaUtils, ByteBufferInputStream, SizeEstimator, Utils} import scala.collection.mutable.ArrayBuffer +import scala.concurrent.Await +import scala.concurrent.duration._ import scala.language.implicitConversions import scala.language.postfixOps class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter with PrivateMethodTester { + private val conf = new SparkConf(false) var store: BlockManager = null var store2: BlockManager = null @@ -52,6 +62,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter conf.set("spark.authenticate", "false") val securityMgr = new SecurityManager(conf) val mapOutputTracker = new MapOutputTrackerMaster(conf) + val shuffleManager = new HashShuffleManager(conf) // Reuse a serializer across tests to avoid creating a new thread-local buffer on each test conf.set("spark.kryoserializer.buffer.mb", "1") @@ -61,21 +72,28 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter implicit def StringToBlockId(value: String): BlockId = new TestBlockId(value) def rdd(rddId: Int, splitId: Int) = RDDBlockId(rddId, splitId) + private def makeBlockManager(maxMem: Long, name: String = ""): BlockManager = { + new BlockManager(name, actorSystem, master, serializer, maxMem, conf, securityMgr, + mapOutputTracker, shuffleManager) + } + before { - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("test", "localhost", 0, conf = conf, - securityManager = securityMgr) + val (actorSystem, boundPort) = AkkaUtils.createActorSystem( + "test", "localhost", 0, conf = conf, securityManager = securityMgr) this.actorSystem = actorSystem + + // Set the arch to 64-bit and compressedOops to true to get a deterministic test-case + oldArch = System.setProperty("os.arch", "amd64") + conf.set("os.arch", "amd64") + conf.set("spark.test.useCompressedOops", "true") conf.set("spark.driver.port", boundPort.toString) + conf.set("spark.storage.unrollFraction", "0.4") + conf.set("spark.storage.unrollMemoryThreshold", "512") master = new BlockManagerMaster( actorSystem.actorOf(Props(new BlockManagerMasterActor(true, conf, new LiveListenerBus))), conf) - // Set the arch to 64-bit and compressedOops to true to get a deterministic test-case - oldArch = System.setProperty("os.arch", "amd64") - conf.set("os.arch", "amd64") - conf.set("spark.test.useCompressedOops", "true") - conf.set("spark.storage.disableBlockManagerHeartBeat", "true") val initialize = PrivateMethod[Unit]('initialize) SizeEstimator invokePrivate initialize() } @@ -138,11 +156,10 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("master + 1 manager interaction") { - store = new BlockManager("", actorSystem, master, serializer, 2000, conf, - securityMgr, mapOutputTracker) - val a1 = new Array[Byte](400) - val a2 = new Array[Byte](400) - val a3 = new Array[Byte](400) + store = makeBlockManager(20000) + val a1 = new Array[Byte](4000) + val a2 = new Array[Byte](4000) + val a3 = new Array[Byte](4000) // Putting a1, a2 and a3 in memory and telling master only about a1 and a2 store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY) @@ -169,10 +186,8 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("master + 2 managers interaction") { - store = new BlockManager("exec1", actorSystem, master, serializer, 2000, conf, - securityMgr, mapOutputTracker) - store2 = new BlockManager("exec2", actorSystem, master, new KryoSerializer(conf), 2000, conf, - securityMgr, mapOutputTracker) + store = makeBlockManager(2000, "exec1") + store2 = makeBlockManager(2000, "exec2") val peers = master.getPeers(store.blockManagerId, 1) assert(peers.size === 1, "master did not return the other manager as a peer") @@ -187,11 +202,10 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("removing block") { - store = new BlockManager("", actorSystem, master, serializer, 2000, conf, - securityMgr, mapOutputTracker) - val a1 = new Array[Byte](400) - val a2 = new Array[Byte](400) - val a3 = new Array[Byte](400) + store = makeBlockManager(20000) + val a1 = new Array[Byte](4000) + val a2 = new Array[Byte](4000) + val a3 = new Array[Byte](4000) // Putting a1, a2 and a3 in memory and telling master only about a1 and a2 store.putSingle("a1-to-remove", a1, StorageLevel.MEMORY_ONLY) @@ -200,8 +214,8 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter // Checking whether blocks are in memory and memory size val memStatus = master.getMemoryStatus.head._2 - assert(memStatus._1 == 2000L, "total memory " + memStatus._1 + " should equal 2000") - assert(memStatus._2 <= 1200L, "remaining memory " + memStatus._2 + " should <= 1200") + assert(memStatus._1 == 20000L, "total memory " + memStatus._1 + " should equal 20000") + assert(memStatus._2 <= 12000L, "remaining memory " + memStatus._2 + " should <= 12000") assert(store.getSingle("a1-to-remove").isDefined, "a1 was not in store") assert(store.getSingle("a2-to-remove").isDefined, "a2 was not in store") assert(store.getSingle("a3-to-remove").isDefined, "a3 was not in store") @@ -230,17 +244,16 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { val memStatus = master.getMemoryStatus.head._2 - memStatus._1 should equal (2000L) - memStatus._2 should equal (2000L) + memStatus._1 should equal (20000L) + memStatus._2 should equal (20000L) } } test("removing rdd") { - store = new BlockManager("", actorSystem, master, serializer, 2000, conf, - securityMgr, mapOutputTracker) - val a1 = new Array[Byte](400) - val a2 = new Array[Byte](400) - val a3 = new Array[Byte](400) + store = makeBlockManager(20000) + val a1 = new Array[Byte](4000) + val a2 = new Array[Byte](4000) + val a3 = new Array[Byte](4000) // Putting a1, a2 and a3 in memory. store.putSingle(rdd(0, 0), a1, StorageLevel.MEMORY_ONLY) store.putSingle(rdd(0, 1), a2, StorageLevel.MEMORY_ONLY) @@ -270,11 +283,9 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("removing broadcast") { - store = new BlockManager("", actorSystem, master, serializer, 2000, conf, - securityMgr, mapOutputTracker) + store = makeBlockManager(2000) val driverStore = store - val executorStore = new BlockManager("executor", actorSystem, master, serializer, 2000, conf, - securityMgr, mapOutputTracker) + val executorStore = makeBlockManager(2000, "executor") val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -342,9 +353,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("reregistration on heart beat") { - val heartBeat = PrivateMethod[Unit]('heartBeat) - store = new BlockManager("", actorSystem, master, serializer, 2000, conf, - securityMgr, mapOutputTracker) + store = makeBlockManager(2000) val a1 = new Array[Byte](400) store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY) @@ -355,13 +364,15 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter master.removeExecutor(store.blockManagerId.executorId) assert(master.getLocations("a1").size == 0, "a1 was not removed from master") - store invokePrivate heartBeat() - assert(master.getLocations("a1").size > 0, "a1 was not reregistered with master") + implicit val timeout = Timeout(30, TimeUnit.SECONDS) + val reregister = !Await.result( + master.driverActor ? BlockManagerHeartbeat(store.blockManagerId), + timeout.duration).asInstanceOf[Boolean] + assert(reregister == true) } test("reregistration on block update") { - store = new BlockManager("", actorSystem, master, serializer, 2000, conf, - securityMgr, mapOutputTracker) + store = makeBlockManager(2000) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) @@ -379,9 +390,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("reregistration doesn't dead lock") { - val heartBeat = PrivateMethod[Unit]('heartBeat) - store = new BlockManager("", actorSystem, master, serializer, 2000, conf, - securityMgr, mapOutputTracker) + store = makeBlockManager(2000) val a1 = new Array[Byte](400) val a2 = List(new Array[Byte](400)) @@ -390,7 +399,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter master.removeExecutor(store.blockManagerId.executorId) val t1 = new Thread { override def run() { - store.put("a2", a2.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIterator("a2", a2.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) } } val t2 = new Thread { @@ -400,7 +409,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } val t3 = new Thread { override def run() { - store invokePrivate heartBeat() + store.reregister() } } @@ -418,19 +427,14 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("correct BlockResult returned from get() calls") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr, - mapOutputTracker) - val list1 = List(new Array[Byte](200), new Array[Byte](200)) - val list1ForSizeEstimate = new ArrayBuffer[Any] - list1ForSizeEstimate ++= list1.iterator - val list1SizeEstimate = SizeEstimator.estimate(list1ForSizeEstimate) - val list2 = List(new Array[Byte](50), new Array[Byte](100), new Array[Byte](150)) - val list2ForSizeEstimate = new ArrayBuffer[Any] - list2ForSizeEstimate ++= list2.iterator - val list2SizeEstimate = SizeEstimator.estimate(list2ForSizeEstimate) - store.put("list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) - store.put("list2memory", list2.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) - store.put("list2disk", list2.iterator, StorageLevel.DISK_ONLY, tellMaster = true) + store = makeBlockManager(12000) + val list1 = List(new Array[Byte](2000), new Array[Byte](2000)) + val list2 = List(new Array[Byte](500), new Array[Byte](1000), new Array[Byte](1500)) + val list1SizeEstimate = SizeEstimator.estimate(list1.iterator.toArray) + val list2SizeEstimate = SizeEstimator.estimate(list2.iterator.toArray) + store.putIterator("list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIterator("list2memory", list2.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIterator("list2disk", list2.iterator, StorageLevel.DISK_ONLY, tellMaster = true) val list1Get = store.get("list1") assert(list1Get.isDefined, "list1 expected to be in store") assert(list1Get.get.data.size === 2) @@ -451,11 +455,10 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("in-memory LRU storage") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker) - val a1 = new Array[Byte](400) - val a2 = new Array[Byte](400) - val a3 = new Array[Byte](400) + store = makeBlockManager(12000) + val a1 = new Array[Byte](4000) + val a2 = new Array[Byte](4000) + val a3 = new Array[Byte](4000) store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY) store.putSingle("a2", a2, StorageLevel.MEMORY_ONLY) store.putSingle("a3", a3, StorageLevel.MEMORY_ONLY) @@ -471,11 +474,10 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("in-memory LRU storage with serialization") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker) - val a1 = new Array[Byte](400) - val a2 = new Array[Byte](400) - val a3 = new Array[Byte](400) + store = makeBlockManager(12000) + val a1 = new Array[Byte](4000) + val a2 = new Array[Byte](4000) + val a3 = new Array[Byte](4000) store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY_SER) store.putSingle("a2", a2, StorageLevel.MEMORY_ONLY_SER) store.putSingle("a3", a3, StorageLevel.MEMORY_ONLY_SER) @@ -491,11 +493,10 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("in-memory LRU for partitions of same RDD") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker) - val a1 = new Array[Byte](400) - val a2 = new Array[Byte](400) - val a3 = new Array[Byte](400) + store = makeBlockManager(12000) + val a1 = new Array[Byte](4000) + val a2 = new Array[Byte](4000) + val a3 = new Array[Byte](4000) store.putSingle(rdd(0, 1), a1, StorageLevel.MEMORY_ONLY) store.putSingle(rdd(0, 2), a2, StorageLevel.MEMORY_ONLY) store.putSingle(rdd(0, 3), a3, StorageLevel.MEMORY_ONLY) @@ -511,11 +512,10 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("in-memory LRU for partitions of multiple RDDs") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker) - store.putSingle(rdd(0, 1), new Array[Byte](400), StorageLevel.MEMORY_ONLY) - store.putSingle(rdd(0, 2), new Array[Byte](400), StorageLevel.MEMORY_ONLY) - store.putSingle(rdd(1, 1), new Array[Byte](400), StorageLevel.MEMORY_ONLY) + store = makeBlockManager(12000) + store.putSingle(rdd(0, 1), new Array[Byte](4000), StorageLevel.MEMORY_ONLY) + store.putSingle(rdd(0, 2), new Array[Byte](4000), StorageLevel.MEMORY_ONLY) + store.putSingle(rdd(1, 1), new Array[Byte](4000), StorageLevel.MEMORY_ONLY) // At this point rdd_1_1 should've replaced rdd_0_1 assert(store.memoryStore.contains(rdd(1, 1)), "rdd_1_1 was not in store") assert(!store.memoryStore.contains(rdd(0, 1)), "rdd_0_1 was in store") @@ -523,8 +523,8 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter // Do a get() on rdd_0_2 so that it is the most recently used item assert(store.getSingle(rdd(0, 2)).isDefined, "rdd_0_2 was not in store") // Put in more partitions from RDD 0; they should replace rdd_1_1 - store.putSingle(rdd(0, 3), new Array[Byte](400), StorageLevel.MEMORY_ONLY) - store.putSingle(rdd(0, 4), new Array[Byte](400), StorageLevel.MEMORY_ONLY) + store.putSingle(rdd(0, 3), new Array[Byte](4000), StorageLevel.MEMORY_ONLY) + store.putSingle(rdd(0, 4), new Array[Byte](4000), StorageLevel.MEMORY_ONLY) // Now rdd_1_1 should be dropped to add rdd_0_3, but then rdd_0_2 should *not* be dropped // when we try to add rdd_0_4. assert(!store.memoryStore.contains(rdd(1, 1)), "rdd_1_1 was in store") @@ -538,8 +538,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter // TODO Make the spark.test.tachyon.enable true after using tachyon 0.5.0 testing jar. val tachyonUnitTestEnabled = conf.getBoolean("spark.test.tachyon.enable", false) if (tachyonUnitTestEnabled) { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker) + store = makeBlockManager(1200) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -555,8 +554,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("on-disk storage") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker) + store = makeBlockManager(1200) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -569,11 +567,10 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("disk and memory storage") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker) - val a1 = new Array[Byte](400) - val a2 = new Array[Byte](400) - val a3 = new Array[Byte](400) + store = makeBlockManager(12000) + val a1 = new Array[Byte](4000) + val a2 = new Array[Byte](4000) + val a3 = new Array[Byte](4000) store.putSingle("a1", a1, StorageLevel.MEMORY_AND_DISK) store.putSingle("a2", a2, StorageLevel.MEMORY_AND_DISK) store.putSingle("a3", a3, StorageLevel.MEMORY_AND_DISK) @@ -585,11 +582,10 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("disk and memory storage with getLocalBytes") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker) - val a1 = new Array[Byte](400) - val a2 = new Array[Byte](400) - val a3 = new Array[Byte](400) + store = makeBlockManager(12000) + val a1 = new Array[Byte](4000) + val a2 = new Array[Byte](4000) + val a3 = new Array[Byte](4000) store.putSingle("a1", a1, StorageLevel.MEMORY_AND_DISK) store.putSingle("a2", a2, StorageLevel.MEMORY_AND_DISK) store.putSingle("a3", a3, StorageLevel.MEMORY_AND_DISK) @@ -601,11 +597,10 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("disk and memory storage with serialization") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker) - val a1 = new Array[Byte](400) - val a2 = new Array[Byte](400) - val a3 = new Array[Byte](400) + store = makeBlockManager(12000) + val a1 = new Array[Byte](4000) + val a2 = new Array[Byte](4000) + val a3 = new Array[Byte](4000) store.putSingle("a1", a1, StorageLevel.MEMORY_AND_DISK_SER) store.putSingle("a2", a2, StorageLevel.MEMORY_AND_DISK_SER) store.putSingle("a3", a3, StorageLevel.MEMORY_AND_DISK_SER) @@ -617,11 +612,10 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("disk and memory storage with serialization and getLocalBytes") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker) - val a1 = new Array[Byte](400) - val a2 = new Array[Byte](400) - val a3 = new Array[Byte](400) + store = makeBlockManager(12000) + val a1 = new Array[Byte](4000) + val a2 = new Array[Byte](4000) + val a3 = new Array[Byte](4000) store.putSingle("a1", a1, StorageLevel.MEMORY_AND_DISK_SER) store.putSingle("a2", a2, StorageLevel.MEMORY_AND_DISK_SER) store.putSingle("a3", a3, StorageLevel.MEMORY_AND_DISK_SER) @@ -633,12 +627,11 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("LRU with mixed storage levels") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker) - val a1 = new Array[Byte](400) - val a2 = new Array[Byte](400) - val a3 = new Array[Byte](400) - val a4 = new Array[Byte](400) + store = makeBlockManager(12000) + val a1 = new Array[Byte](4000) + val a2 = new Array[Byte](4000) + val a3 = new Array[Byte](4000) + val a4 = new Array[Byte](4000) // First store a1 and a2, both in memory, and a3, on disk only store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY_SER) store.putSingle("a2", a2, StorageLevel.MEMORY_ONLY_SER) @@ -656,14 +649,13 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("in-memory LRU with streams") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker) - val list1 = List(new Array[Byte](200), new Array[Byte](200)) - val list2 = List(new Array[Byte](200), new Array[Byte](200)) - val list3 = List(new Array[Byte](200), new Array[Byte](200)) - store.put("list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) - store.put("list2", list2.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) - store.put("list3", list3.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store = makeBlockManager(12000) + val list1 = List(new Array[Byte](2000), new Array[Byte](2000)) + val list2 = List(new Array[Byte](2000), new Array[Byte](2000)) + val list3 = List(new Array[Byte](2000), new Array[Byte](2000)) + store.putIterator("list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIterator("list2", list2.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIterator("list3", list3.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) assert(store.get("list2").isDefined, "list2 was not in store") assert(store.get("list2").get.data.size === 2) assert(store.get("list3").isDefined, "list3 was not in store") @@ -672,7 +664,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter assert(store.get("list2").isDefined, "list2 was not in store") assert(store.get("list2").get.data.size === 2) // At this point list2 was gotten last, so LRU will getSingle rid of list3 - store.put("list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIterator("list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) assert(store.get("list1").isDefined, "list1 was not in store") assert(store.get("list1").get.data.size === 2) assert(store.get("list2").isDefined, "list2 was not in store") @@ -681,16 +673,15 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("LRU with mixed storage levels and streams") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker) - val list1 = List(new Array[Byte](200), new Array[Byte](200)) - val list2 = List(new Array[Byte](200), new Array[Byte](200)) - val list3 = List(new Array[Byte](200), new Array[Byte](200)) - val list4 = List(new Array[Byte](200), new Array[Byte](200)) + store = makeBlockManager(12000) + val list1 = List(new Array[Byte](2000), new Array[Byte](2000)) + val list2 = List(new Array[Byte](2000), new Array[Byte](2000)) + val list3 = List(new Array[Byte](2000), new Array[Byte](2000)) + val list4 = List(new Array[Byte](2000), new Array[Byte](2000)) // First store list1 and list2, both in memory, and list3, on disk only - store.put("list1", list1.iterator, StorageLevel.MEMORY_ONLY_SER, tellMaster = true) - store.put("list2", list2.iterator, StorageLevel.MEMORY_ONLY_SER, tellMaster = true) - store.put("list3", list3.iterator, StorageLevel.DISK_ONLY, tellMaster = true) + store.putIterator("list1", list1.iterator, StorageLevel.MEMORY_ONLY_SER, tellMaster = true) + store.putIterator("list2", list2.iterator, StorageLevel.MEMORY_ONLY_SER, tellMaster = true) + store.putIterator("list3", list3.iterator, StorageLevel.DISK_ONLY, tellMaster = true) val listForSizeEstimate = new ArrayBuffer[Any] listForSizeEstimate ++= list1.iterator val listSize = SizeEstimator.estimate(listForSizeEstimate) @@ -708,7 +699,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter assert(store.get("list3").isDefined, "list3 was not in store") assert(store.get("list3").get.data.size === 2) // Now let's add in list4, which uses both disk and memory; list1 should drop out - store.put("list4", list4.iterator, StorageLevel.MEMORY_AND_DISK_SER, tellMaster = true) + store.putIterator("list4", list4.iterator, StorageLevel.MEMORY_AND_DISK_SER, tellMaster = true) assert(store.get("list1") === None, "list1 was in store") assert(store.get("list2").isDefined, "list2 was not in store") assert(store.get("list2").get.data.size === 2) @@ -731,11 +722,10 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("overly large block") { - store = new BlockManager("", actorSystem, master, serializer, 500, conf, - securityMgr, mapOutputTracker) - store.putSingle("a1", new Array[Byte](1000), StorageLevel.MEMORY_ONLY) + store = makeBlockManager(5000) + store.putSingle("a1", new Array[Byte](10000), StorageLevel.MEMORY_ONLY) assert(store.getSingle("a1") === None, "a1 was in store") - store.putSingle("a2", new Array[Byte](1000), StorageLevel.MEMORY_AND_DISK) + store.putSingle("a2", new Array[Byte](10000), StorageLevel.MEMORY_AND_DISK) assert(store.memoryStore.getValues("a2") === None, "a2 was in memory store") assert(store.getSingle("a2").isDefined, "a2 was not in store") } @@ -743,8 +733,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter test("block compression") { try { conf.set("spark.shuffle.compress", "true") - store = new BlockManager("exec1", actorSystem, master, serializer, 2000, conf, - securityMgr, mapOutputTracker) + store = makeBlockManager(20000, "exec1") store.putSingle(ShuffleBlockId(0, 0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(ShuffleBlockId(0, 0, 0)) <= 100, "shuffle_0_0_0 was not compressed") @@ -752,52 +741,46 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter store = null conf.set("spark.shuffle.compress", "false") - store = new BlockManager("exec2", actorSystem, master, serializer, 2000, conf, - securityMgr, mapOutputTracker) - store.putSingle(ShuffleBlockId(0, 0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) - assert(store.memoryStore.getSize(ShuffleBlockId(0, 0, 0)) >= 1000, + store = makeBlockManager(20000, "exec2") + store.putSingle(ShuffleBlockId(0, 0, 0), new Array[Byte](10000), StorageLevel.MEMORY_ONLY_SER) + assert(store.memoryStore.getSize(ShuffleBlockId(0, 0, 0)) >= 10000, "shuffle_0_0_0 was compressed") store.stop() store = null conf.set("spark.broadcast.compress", "true") - store = new BlockManager("exec3", actorSystem, master, serializer, 2000, conf, - securityMgr, mapOutputTracker) - store.putSingle(BroadcastBlockId(0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) - assert(store.memoryStore.getSize(BroadcastBlockId(0)) <= 100, + store = makeBlockManager(20000, "exec3") + store.putSingle(BroadcastBlockId(0), new Array[Byte](10000), StorageLevel.MEMORY_ONLY_SER) + assert(store.memoryStore.getSize(BroadcastBlockId(0)) <= 1000, "broadcast_0 was not compressed") store.stop() store = null conf.set("spark.broadcast.compress", "false") - store = new BlockManager("exec4", actorSystem, master, serializer, 2000, conf, - securityMgr, mapOutputTracker) - store.putSingle(BroadcastBlockId(0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) - assert(store.memoryStore.getSize(BroadcastBlockId(0)) >= 1000, "broadcast_0 was compressed") + store = makeBlockManager(20000, "exec4") + store.putSingle(BroadcastBlockId(0), new Array[Byte](10000), StorageLevel.MEMORY_ONLY_SER) + assert(store.memoryStore.getSize(BroadcastBlockId(0)) >= 10000, "broadcast_0 was compressed") store.stop() store = null conf.set("spark.rdd.compress", "true") - store = new BlockManager("exec5", actorSystem, master, serializer, 2000, conf, - securityMgr, mapOutputTracker) - store.putSingle(rdd(0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) - assert(store.memoryStore.getSize(rdd(0, 0)) <= 100, "rdd_0_0 was not compressed") + store = makeBlockManager(20000, "exec5") + store.putSingle(rdd(0, 0), new Array[Byte](10000), StorageLevel.MEMORY_ONLY_SER) + assert(store.memoryStore.getSize(rdd(0, 0)) <= 1000, "rdd_0_0 was not compressed") store.stop() store = null conf.set("spark.rdd.compress", "false") - store = new BlockManager("exec6", actorSystem, master, serializer, 2000, conf, - securityMgr, mapOutputTracker) - store.putSingle(rdd(0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) - assert(store.memoryStore.getSize(rdd(0, 0)) >= 1000, "rdd_0_0 was compressed") + store = makeBlockManager(20000, "exec6") + store.putSingle(rdd(0, 0), new Array[Byte](10000), StorageLevel.MEMORY_ONLY_SER) + assert(store.memoryStore.getSize(rdd(0, 0)) >= 10000, "rdd_0_0 was compressed") store.stop() store = null // Check that any other block types are also kept uncompressed - store = new BlockManager("exec7", actorSystem, master, serializer, 2000, conf, - securityMgr, mapOutputTracker) - store.putSingle("other_block", new Array[Byte](1000), StorageLevel.MEMORY_ONLY) - assert(store.memoryStore.getSize("other_block") >= 1000, "other_block was compressed") + store = makeBlockManager(20000, "exec7") + store.putSingle("other_block", new Array[Byte](10000), StorageLevel.MEMORY_ONLY) + assert(store.memoryStore.getSize("other_block") >= 10000, "other_block was compressed") store.stop() store = null } finally { @@ -810,7 +793,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter test("block store put failure") { // Use Java serializer so we can create an unserializable error. store = new BlockManager("", actorSystem, master, new JavaSerializer(conf), 1200, conf, - securityMgr, mapOutputTracker) + securityMgr, mapOutputTracker, shuffleManager) // The put should fail since a1 is not serializable. class UnserializableClass @@ -871,30 +854,29 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter assert(Arrays.equals(mappedAsArray, bytes)) assert(Arrays.equals(notMappedAsArray, bytes)) } - + test("updated block statuses") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker) - val list = List.fill(2)(new Array[Byte](200)) - val bigList = List.fill(8)(new Array[Byte](200)) + store = makeBlockManager(12000) + val list = List.fill(2)(new Array[Byte](2000)) + val bigList = List.fill(8)(new Array[Byte](2000)) // 1 updated block (i.e. list1) val updatedBlocks1 = - store.put("list1", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIterator("list1", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) assert(updatedBlocks1.size === 1) assert(updatedBlocks1.head._1 === TestBlockId("list1")) assert(updatedBlocks1.head._2.storageLevel === StorageLevel.MEMORY_ONLY) // 1 updated block (i.e. list2) val updatedBlocks2 = - store.put("list2", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) + store.putIterator("list2", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) assert(updatedBlocks2.size === 1) assert(updatedBlocks2.head._1 === TestBlockId("list2")) assert(updatedBlocks2.head._2.storageLevel === StorageLevel.MEMORY_ONLY) // 2 updated blocks - list1 is kicked out of memory while list3 is added val updatedBlocks3 = - store.put("list3", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIterator("list3", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) assert(updatedBlocks3.size === 2) updatedBlocks3.foreach { case (id, status) => id match { @@ -903,11 +885,11 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter case _ => fail("Updated block is neither list1 nor list3") } } - assert(store.get("list3").isDefined, "list3 was not in store") + assert(store.memoryStore.contains("list3"), "list3 was not in memory store") // 2 updated blocks - list2 is kicked out of memory (but put on disk) while list4 is added val updatedBlocks4 = - store.put("list4", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIterator("list4", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) assert(updatedBlocks4.size === 2) updatedBlocks4.foreach { case (id, status) => id match { @@ -916,26 +898,37 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter case _ => fail("Updated block is neither list2 nor list4") } } - assert(store.get("list4").isDefined, "list4 was not in store") + assert(store.diskStore.contains("list2"), "list2 was not in disk store") + assert(store.memoryStore.contains("list4"), "list4 was not in memory store") - // No updated blocks - nothing is kicked out of memory because list5 is too big to be added + // No updated blocks - list5 is too big to fit in store and nothing is kicked out val updatedBlocks5 = - store.put("list5", bigList.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIterator("list5", bigList.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) assert(updatedBlocks5.size === 0) - assert(store.get("list2").isDefined, "list2 was not in store") - assert(store.get("list4").isDefined, "list4 was not in store") - assert(!store.get("list5").isDefined, "list5 was in store") + + // memory store contains only list3 and list4 + assert(!store.memoryStore.contains("list1"), "list1 was in memory store") + assert(!store.memoryStore.contains("list2"), "list2 was in memory store") + assert(store.memoryStore.contains("list3"), "list3 was not in memory store") + assert(store.memoryStore.contains("list4"), "list4 was not in memory store") + assert(!store.memoryStore.contains("list5"), "list5 was in memory store") + + // disk store contains only list2 + assert(!store.diskStore.contains("list1"), "list1 was in disk store") + assert(store.diskStore.contains("list2"), "list2 was not in disk store") + assert(!store.diskStore.contains("list3"), "list3 was in disk store") + assert(!store.diskStore.contains("list4"), "list4 was in disk store") + assert(!store.diskStore.contains("list5"), "list5 was in disk store") } test("query block statuses") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker) - val list = List.fill(2)(new Array[Byte](200)) + store = makeBlockManager(12000) + val list = List.fill(2)(new Array[Byte](2000)) // Tell master. By LRU, only list2 and list3 remains. - store.put("list1", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) - store.put("list2", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) - store.put("list3", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIterator("list1", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIterator("list2", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) + store.putIterator("list3", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) // getLocations and getBlockStatus should yield the same locations assert(store.master.getLocations("list1").size === 0) @@ -949,9 +942,9 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter assert(store.master.getBlockStatus("list3", askSlaves = true).size === 1) // This time don't tell master and see what happens. By LRU, only list5 and list6 remains. - store.put("list4", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = false) - store.put("list5", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = false) - store.put("list6", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = false) + store.putIterator("list4", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = false) + store.putIterator("list5", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = false) + store.putIterator("list6", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = false) // getLocations should return nothing because the master is not informed // getBlockStatus without asking slaves should have the same result @@ -968,23 +961,22 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("get matching blocks") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker) - val list = List.fill(2)(new Array[Byte](10)) + store = makeBlockManager(12000) + val list = List.fill(2)(new Array[Byte](100)) // insert some blocks - store.put("list1", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) - store.put("list2", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) - store.put("list3", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) + store.putIterator("list1", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) + store.putIterator("list2", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) + store.putIterator("list3", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) // getLocations and getBlockStatus should yield the same locations assert(store.master.getMatchingBlockIds(_.toString.contains("list"), askSlaves = false).size === 3) assert(store.master.getMatchingBlockIds(_.toString.contains("list1"), askSlaves = false).size === 1) // insert some more blocks - store.put("newlist1", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) - store.put("newlist2", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = false) - store.put("newlist3", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = false) + store.putIterator("newlist1", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) + store.putIterator("newlist2", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = false) + store.putIterator("newlist3", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = false) // getLocations and getBlockStatus should yield the same locations assert(store.master.getMatchingBlockIds(_.toString.contains("newlist"), askSlaves = false).size === 1) @@ -992,7 +984,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter val blockIds = Seq(RDDBlockId(1, 0), RDDBlockId(1, 1), RDDBlockId(2, 0)) blockIds.foreach { blockId => - store.put(blockId, list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIterator(blockId, list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) } val matchedBlockIds = store.master.getMatchingBlockIds(_ match { case RDDBlockId(1, _) => true @@ -1002,17 +994,343 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("SPARK-1194 regression: fix the same-RDD rule for cache replacement") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker) - store.putSingle(rdd(0, 0), new Array[Byte](400), StorageLevel.MEMORY_ONLY) - store.putSingle(rdd(1, 0), new Array[Byte](400), StorageLevel.MEMORY_ONLY) + store = makeBlockManager(12000) + store.putSingle(rdd(0, 0), new Array[Byte](4000), StorageLevel.MEMORY_ONLY) + store.putSingle(rdd(1, 0), new Array[Byte](4000), StorageLevel.MEMORY_ONLY) // Access rdd_1_0 to ensure it's not least recently used. assert(store.getSingle(rdd(1, 0)).isDefined, "rdd_1_0 was not in store") // According to the same-RDD rule, rdd_1_0 should be replaced here. - store.putSingle(rdd(0, 1), new Array[Byte](400), StorageLevel.MEMORY_ONLY) + store.putSingle(rdd(0, 1), new Array[Byte](4000), StorageLevel.MEMORY_ONLY) // rdd_1_0 should have been replaced, even it's not least recently used. assert(store.memoryStore.contains(rdd(0, 0)), "rdd_0_0 was not in store") assert(store.memoryStore.contains(rdd(0, 1)), "rdd_0_1 was not in store") assert(!store.memoryStore.contains(rdd(1, 0)), "rdd_1_0 was in store") } + + test("return error message when error occurred in BlockManagerWorker#onBlockMessageReceive") { + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, + securityMgr, mapOutputTracker, shuffleManager) + + val worker = spy(new BlockManagerWorker(store)) + val connManagerId = mock(classOf[ConnectionManagerId]) + + // setup request block messages + val reqBlId1 = ShuffleBlockId(0,0,0) + val reqBlId2 = ShuffleBlockId(0,1,0) + val reqBlockMessage1 = BlockMessage.fromGetBlock(GetBlock(reqBlId1)) + val reqBlockMessage2 = BlockMessage.fromGetBlock(GetBlock(reqBlId2)) + val reqBlockMessages = new BlockMessageArray( + Seq(reqBlockMessage1, reqBlockMessage2)) + val reqBufferMessage = reqBlockMessages.toBufferMessage + + val answer = new Answer[Option[BlockMessage]] { + override def answer(invocation: InvocationOnMock) + :Option[BlockMessage]= { + throw new Exception + } + } + + doAnswer(answer).when(worker).processBlockMessage(any()) + + // Test when exception was thrown during processing block messages + var ackMessage = worker.onBlockMessageReceive(reqBufferMessage, connManagerId) + + assert(ackMessage.isDefined, "When Exception was thrown in " + + "BlockManagerWorker#processBlockMessage, " + + "ackMessage should be defined") + assert(ackMessage.get.hasError, "When Exception was thown in " + + "BlockManagerWorker#processBlockMessage, " + + "ackMessage should have error") + + val notBufferMessage = mock(classOf[Message]) + + // Test when not BufferMessage was received + ackMessage = worker.onBlockMessageReceive(notBufferMessage, connManagerId) + assert(ackMessage.isDefined, "When not BufferMessage was passed to " + + "BlockManagerWorker#onBlockMessageReceive, " + + "ackMessage should be defined") + assert(ackMessage.get.hasError, "When not BufferMessage was passed to " + + "BlockManagerWorker#onBlockMessageReceive, " + + "ackMessage should have error") + } + + test("return ack message when no error occurred in BlocManagerWorker#onBlockMessageReceive") { + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, + securityMgr, mapOutputTracker, shuffleManager) + + val worker = spy(new BlockManagerWorker(store)) + val connManagerId = mock(classOf[ConnectionManagerId]) + + // setup request block messages + val reqBlId1 = ShuffleBlockId(0,0,0) + val reqBlId2 = ShuffleBlockId(0,1,0) + val reqBlockMessage1 = BlockMessage.fromGetBlock(GetBlock(reqBlId1)) + val reqBlockMessage2 = BlockMessage.fromGetBlock(GetBlock(reqBlId2)) + val reqBlockMessages = new BlockMessageArray( + Seq(reqBlockMessage1, reqBlockMessage2)) + + val tmpBufferMessage = reqBlockMessages.toBufferMessage + val buffer = ByteBuffer.allocate(tmpBufferMessage.size) + val arrayBuffer = new ArrayBuffer[ByteBuffer] + tmpBufferMessage.buffers.foreach{ b => + buffer.put(b) + } + buffer.flip() + arrayBuffer += buffer + val reqBufferMessage = Message.createBufferMessage(arrayBuffer) + + // setup ack block messages + val buf1 = ByteBuffer.allocate(4) + val buf2 = ByteBuffer.allocate(4) + buf1.putInt(1) + buf1.flip() + buf2.putInt(1) + buf2.flip() + val ackBlockMessage1 = BlockMessage.fromGotBlock(GotBlock(reqBlId1, buf1)) + val ackBlockMessage2 = BlockMessage.fromGotBlock(GotBlock(reqBlId2, buf2)) + + val answer = new Answer[Option[BlockMessage]] { + override def answer(invocation: InvocationOnMock) + :Option[BlockMessage]= { + if (invocation.getArguments()(0).asInstanceOf[BlockMessage].eq( + reqBlockMessage1)) { + return Some(ackBlockMessage1) + } else { + return Some(ackBlockMessage2) + } + } + } + + doAnswer(answer).when(worker).processBlockMessage(any()) + + val ackMessage = worker.onBlockMessageReceive(reqBufferMessage, connManagerId) + assert(ackMessage.isDefined, "When BlockManagerWorker#onBlockMessageReceive " + + "was executed successfully, ackMessage should be defined") + assert(!ackMessage.get.hasError, "When BlockManagerWorker#onBlockMessageReceive " + + "was executed successfully, ackMessage should not have error") + } + + test("reserve/release unroll memory") { + store = makeBlockManager(12000) + val memoryStore = store.memoryStore + assert(memoryStore.currentUnrollMemory === 0) + assert(memoryStore.currentUnrollMemoryForThisThread === 0) + + // Reserve + memoryStore.reserveUnrollMemoryForThisThread(100) + assert(memoryStore.currentUnrollMemoryForThisThread === 100) + memoryStore.reserveUnrollMemoryForThisThread(200) + assert(memoryStore.currentUnrollMemoryForThisThread === 300) + memoryStore.reserveUnrollMemoryForThisThread(500) + assert(memoryStore.currentUnrollMemoryForThisThread === 800) + memoryStore.reserveUnrollMemoryForThisThread(1000000) + assert(memoryStore.currentUnrollMemoryForThisThread === 800) // not granted + // Release + memoryStore.releaseUnrollMemoryForThisThread(100) + assert(memoryStore.currentUnrollMemoryForThisThread === 700) + memoryStore.releaseUnrollMemoryForThisThread(100) + assert(memoryStore.currentUnrollMemoryForThisThread === 600) + // Reserve again + memoryStore.reserveUnrollMemoryForThisThread(4400) + assert(memoryStore.currentUnrollMemoryForThisThread === 5000) + memoryStore.reserveUnrollMemoryForThisThread(20000) + assert(memoryStore.currentUnrollMemoryForThisThread === 5000) // not granted + // Release again + memoryStore.releaseUnrollMemoryForThisThread(1000) + assert(memoryStore.currentUnrollMemoryForThisThread === 4000) + memoryStore.releaseUnrollMemoryForThisThread() // release all + assert(memoryStore.currentUnrollMemoryForThisThread === 0) + } + + /** + * Verify the result of MemoryStore#unrollSafely is as expected. + */ + private def verifyUnroll( + expected: Iterator[Any], + result: Either[Array[Any], Iterator[Any]], + shouldBeArray: Boolean): Unit = { + val actual: Iterator[Any] = result match { + case Left(arr: Array[Any]) => + assert(shouldBeArray, "expected iterator from unroll!") + arr.iterator + case Right(it: Iterator[Any]) => + assert(!shouldBeArray, "expected array from unroll!") + it + case _ => + fail("unroll returned neither an iterator nor an array...") + } + expected.zip(actual).foreach { case (e, a) => + assert(e === a, "unroll did not return original values!") + } + } + + test("safely unroll blocks") { + store = makeBlockManager(12000) + val smallList = List.fill(40)(new Array[Byte](100)) + val bigList = List.fill(40)(new Array[Byte](1000)) + val memoryStore = store.memoryStore + val droppedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] + assert(memoryStore.currentUnrollMemoryForThisThread === 0) + + // Unroll with all the space in the world. This should succeed and return an array. + var unrollResult = memoryStore.unrollSafely("unroll", smallList.iterator, droppedBlocks) + verifyUnroll(smallList.iterator, unrollResult, shouldBeArray = true) + assert(memoryStore.currentUnrollMemoryForThisThread === 0) + + // Unroll with not enough space. This should succeed after kicking out someBlock1. + store.putIterator("someBlock1", smallList.iterator, StorageLevel.MEMORY_ONLY) + store.putIterator("someBlock2", smallList.iterator, StorageLevel.MEMORY_ONLY) + unrollResult = memoryStore.unrollSafely("unroll", smallList.iterator, droppedBlocks) + verifyUnroll(smallList.iterator, unrollResult, shouldBeArray = true) + assert(memoryStore.currentUnrollMemoryForThisThread === 0) + assert(droppedBlocks.size === 1) + assert(droppedBlocks.head._1 === TestBlockId("someBlock1")) + droppedBlocks.clear() + + // Unroll huge block with not enough space. Even after ensuring free space of 12000 * 0.4 = + // 4800 bytes, there is still not enough room to unroll this block. This returns an iterator. + // In the mean time, however, we kicked out someBlock2 before giving up. + store.putIterator("someBlock3", smallList.iterator, StorageLevel.MEMORY_ONLY) + unrollResult = memoryStore.unrollSafely("unroll", bigList.iterator, droppedBlocks) + verifyUnroll(bigList.iterator, unrollResult, shouldBeArray = false) + assert(memoryStore.currentUnrollMemoryForThisThread > 0) // we returned an iterator + assert(droppedBlocks.size === 1) + assert(droppedBlocks.head._1 === TestBlockId("someBlock2")) + droppedBlocks.clear() + } + + test("safely unroll blocks through putIterator") { + store = makeBlockManager(12000) + val memOnly = StorageLevel.MEMORY_ONLY + val memoryStore = store.memoryStore + val smallList = List.fill(40)(new Array[Byte](100)) + val bigList = List.fill(40)(new Array[Byte](1000)) + def smallIterator = smallList.iterator.asInstanceOf[Iterator[Any]] + def bigIterator = bigList.iterator.asInstanceOf[Iterator[Any]] + assert(memoryStore.currentUnrollMemoryForThisThread === 0) + + // Unroll with plenty of space. This should succeed and cache both blocks. + val result1 = memoryStore.putIterator("b1", smallIterator, memOnly, returnValues = true) + val result2 = memoryStore.putIterator("b2", smallIterator, memOnly, returnValues = true) + assert(memoryStore.contains("b1")) + assert(memoryStore.contains("b2")) + assert(result1.size > 0) // unroll was successful + assert(result2.size > 0) + assert(result1.data.isLeft) // unroll did not drop this block to disk + assert(result2.data.isLeft) + assert(memoryStore.currentUnrollMemoryForThisThread === 0) + + // Re-put these two blocks so block manager knows about them too. Otherwise, block manager + // would not know how to drop them from memory later. + memoryStore.remove("b1") + memoryStore.remove("b2") + store.putIterator("b1", smallIterator, memOnly) + store.putIterator("b2", smallIterator, memOnly) + + // Unroll with not enough space. This should succeed but kick out b1 in the process. + val result3 = memoryStore.putIterator("b3", smallIterator, memOnly, returnValues = true) + assert(result3.size > 0) + assert(result3.data.isLeft) + assert(!memoryStore.contains("b1")) + assert(memoryStore.contains("b2")) + assert(memoryStore.contains("b3")) + assert(memoryStore.currentUnrollMemoryForThisThread === 0) + memoryStore.remove("b3") + store.putIterator("b3", smallIterator, memOnly) + + // Unroll huge block with not enough space. This should fail and kick out b2 in the process. + val result4 = memoryStore.putIterator("b4", bigIterator, memOnly, returnValues = true) + assert(result4.size === 0) // unroll was unsuccessful + assert(result4.data.isLeft) + assert(!memoryStore.contains("b1")) + assert(!memoryStore.contains("b2")) + assert(memoryStore.contains("b3")) + assert(!memoryStore.contains("b4")) + assert(memoryStore.currentUnrollMemoryForThisThread > 0) // we returned an iterator + } + + /** + * This test is essentially identical to the preceding one, except that it uses MEMORY_AND_DISK. + */ + test("safely unroll blocks through putIterator (disk)") { + store = makeBlockManager(12000) + val memAndDisk = StorageLevel.MEMORY_AND_DISK + val memoryStore = store.memoryStore + val diskStore = store.diskStore + val smallList = List.fill(40)(new Array[Byte](100)) + val bigList = List.fill(40)(new Array[Byte](1000)) + def smallIterator = smallList.iterator.asInstanceOf[Iterator[Any]] + def bigIterator = bigList.iterator.asInstanceOf[Iterator[Any]] + assert(memoryStore.currentUnrollMemoryForThisThread === 0) + + store.putIterator("b1", smallIterator, memAndDisk) + store.putIterator("b2", smallIterator, memAndDisk) + + // Unroll with not enough space. This should succeed but kick out b1 in the process. + // Memory store should contain b2 and b3, while disk store should contain only b1 + val result3 = memoryStore.putIterator("b3", smallIterator, memAndDisk, returnValues = true) + assert(result3.size > 0) + assert(!memoryStore.contains("b1")) + assert(memoryStore.contains("b2")) + assert(memoryStore.contains("b3")) + assert(diskStore.contains("b1")) + assert(!diskStore.contains("b2")) + assert(!diskStore.contains("b3")) + memoryStore.remove("b3") + store.putIterator("b3", smallIterator, StorageLevel.MEMORY_ONLY) + assert(memoryStore.currentUnrollMemoryForThisThread === 0) + + // Unroll huge block with not enough space. This should fail and drop the new block to disk + // directly in addition to kicking out b2 in the process. Memory store should contain only + // b3, while disk store should contain b1, b2 and b4. + val result4 = memoryStore.putIterator("b4", bigIterator, memAndDisk, returnValues = true) + assert(result4.size > 0) + assert(result4.data.isRight) // unroll returned bytes from disk + assert(!memoryStore.contains("b1")) + assert(!memoryStore.contains("b2")) + assert(memoryStore.contains("b3")) + assert(!memoryStore.contains("b4")) + assert(diskStore.contains("b1")) + assert(diskStore.contains("b2")) + assert(!diskStore.contains("b3")) + assert(diskStore.contains("b4")) + assert(memoryStore.currentUnrollMemoryForThisThread > 0) // we returned an iterator + } + + test("multiple unrolls by the same thread") { + store = makeBlockManager(12000) + val memOnly = StorageLevel.MEMORY_ONLY + val memoryStore = store.memoryStore + val smallList = List.fill(40)(new Array[Byte](100)) + def smallIterator = smallList.iterator.asInstanceOf[Iterator[Any]] + assert(memoryStore.currentUnrollMemoryForThisThread === 0) + + // All unroll memory used is released because unrollSafely returned an array + memoryStore.putIterator("b1", smallIterator, memOnly, returnValues = true) + assert(memoryStore.currentUnrollMemoryForThisThread === 0) + memoryStore.putIterator("b2", smallIterator, memOnly, returnValues = true) + assert(memoryStore.currentUnrollMemoryForThisThread === 0) + + // Unroll memory is not released because unrollSafely returned an iterator + // that still depends on the underlying vector used in the process + memoryStore.putIterator("b3", smallIterator, memOnly, returnValues = true) + val unrollMemoryAfterB3 = memoryStore.currentUnrollMemoryForThisThread + assert(unrollMemoryAfterB3 > 0) + + // The unroll memory owned by this thread builds on top of its value after the previous unrolls + memoryStore.putIterator("b4", smallIterator, memOnly, returnValues = true) + val unrollMemoryAfterB4 = memoryStore.currentUnrollMemoryForThisThread + assert(unrollMemoryAfterB4 > unrollMemoryAfterB3) + + // ... but only to a certain extent (until we run out of free space to grant new unroll memory) + memoryStore.putIterator("b5", smallIterator, memOnly, returnValues = true) + val unrollMemoryAfterB5 = memoryStore.currentUnrollMemoryForThisThread + memoryStore.putIterator("b6", smallIterator, memOnly, returnValues = true) + val unrollMemoryAfterB6 = memoryStore.currentUnrollMemoryForThisThread + memoryStore.putIterator("b7", smallIterator, memOnly, returnValues = true) + val unrollMemoryAfterB7 = memoryStore.currentUnrollMemoryForThisThread + assert(unrollMemoryAfterB5 === unrollMemoryAfterB4) + assert(unrollMemoryAfterB6 === unrollMemoryAfterB4) + assert(unrollMemoryAfterB7 === unrollMemoryAfterB4) + } } diff --git a/core/src/test/scala/org/apache/spark/storage/BlockObjectWriterSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockObjectWriterSuite.scala new file mode 100644 index 0000000000000..bbc7e1357b90d --- /dev/null +++ b/core/src/test/scala/org/apache/spark/storage/BlockObjectWriterSuite.scala @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.storage + +import org.scalatest.FunSuite +import java.io.File +import org.apache.spark.executor.ShuffleWriteMetrics +import org.apache.spark.serializer.JavaSerializer +import org.apache.spark.SparkConf + +class BlockObjectWriterSuite extends FunSuite { + test("verify write metrics") { + val file = new File("somefile") + file.deleteOnExit() + val writeMetrics = new ShuffleWriteMetrics() + val writer = new DiskBlockObjectWriter(new TestBlockId("0"), file, + new JavaSerializer(new SparkConf()), 1024, os => os, true, writeMetrics) + + writer.write(Long.box(20)) + // Metrics don't update on every write + assert(writeMetrics.shuffleBytesWritten == 0) + // After 32 writes, metrics should update + for (i <- 0 until 32) { + writer.flush() + writer.write(Long.box(i)) + } + assert(writeMetrics.shuffleBytesWritten > 0) + writer.commitAndClose() + assert(file.length() == writeMetrics.shuffleBytesWritten) + } + + test("verify write metrics on revert") { + val file = new File("somefile") + file.deleteOnExit() + val writeMetrics = new ShuffleWriteMetrics() + val writer = new DiskBlockObjectWriter(new TestBlockId("0"), file, + new JavaSerializer(new SparkConf()), 1024, os => os, true, writeMetrics) + + writer.write(Long.box(20)) + // Metrics don't update on every write + assert(writeMetrics.shuffleBytesWritten == 0) + // After 32 writes, metrics should update + for (i <- 0 until 32) { + writer.flush() + writer.write(Long.box(i)) + } + assert(writeMetrics.shuffleBytesWritten > 0) + writer.revertPartialWritesAndClose() + assert(writeMetrics.shuffleBytesWritten == 0) + } +} diff --git a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala index aaa7714049732..777579bc570db 100644 --- a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala @@ -19,14 +19,20 @@ package org.apache.spark.storage import java.io.{File, FileWriter} +import org.apache.spark.shuffle.hash.HashShuffleManager + import scala.collection.mutable import scala.language.reflectiveCalls +import akka.actor.Props import com.google.common.io.Files import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite} import org.apache.spark.SparkConf -import org.apache.spark.util.Utils +import org.apache.spark.scheduler.LiveListenerBus +import org.apache.spark.serializer.JavaSerializer +import org.apache.spark.util.{AkkaUtils, Utils} +import org.apache.spark.executor.ShuffleWriteMetrics class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach with BeforeAndAfterAll { private val testConf = new SparkConf(false) @@ -38,7 +44,9 @@ class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach with Before // so we coerce consolidation if not already enabled. testConf.set("spark.shuffle.consolidateFiles", "true") - val shuffleBlockManager = new ShuffleBlockManager(null) { + private val shuffleManager = new HashShuffleManager(testConf.clone) + + val shuffleBlockManager = new ShuffleBlockManager(null, shuffleManager) { override def conf = testConf.clone var idToSegmentMap = mutable.Map[ShuffleBlockId, FileSegment]() override def getBlockLocation(id: ShuffleBlockId) = idToSegmentMap(id) @@ -121,6 +129,90 @@ class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach with Before newFile.delete() } + private def checkSegments(segment1: FileSegment, segment2: FileSegment) { + assert (segment1.file.getCanonicalPath === segment2.file.getCanonicalPath) + assert (segment1.offset === segment2.offset) + assert (segment1.length === segment2.length) + } + + test("consolidated shuffle can write to shuffle group without messing existing offsets/lengths") { + + val serializer = new JavaSerializer(testConf) + val confCopy = testConf.clone + // reset after EACH object write. This is to ensure that there are bytes appended after + // an object is written. So if the codepaths assume writeObject is end of data, this should + // flush those bugs out. This was common bug in ExternalAppendOnlyMap, etc. + confCopy.set("spark.serializer.objectStreamReset", "1") + + val securityManager = new org.apache.spark.SecurityManager(confCopy) + // Do not use the shuffleBlockManager above ! + val (actorSystem, boundPort) = AkkaUtils.createActorSystem("test", "localhost", 0, confCopy, + securityManager) + val master = new BlockManagerMaster( + actorSystem.actorOf(Props(new BlockManagerMasterActor(true, confCopy, new LiveListenerBus))), + confCopy) + val store = new BlockManager("", actorSystem, master , serializer, confCopy, + securityManager, null, shuffleManager) + + try { + + val shuffleManager = store.shuffleBlockManager + + val shuffle1 = shuffleManager.forMapTask(1, 1, 1, serializer, new ShuffleWriteMetrics) + for (writer <- shuffle1.writers) { + writer.write("test1") + writer.write("test2") + } + for (writer <- shuffle1.writers) { + writer.commitAndClose() + } + + val shuffle1Segment = shuffle1.writers(0).fileSegment() + shuffle1.releaseWriters(success = true) + + val shuffle2 = shuffleManager.forMapTask(1, 2, 1, new JavaSerializer(testConf), + new ShuffleWriteMetrics) + + for (writer <- shuffle2.writers) { + writer.write("test3") + writer.write("test4") + } + for (writer <- shuffle2.writers) { + writer.commitAndClose() + } + val shuffle2Segment = shuffle2.writers(0).fileSegment() + shuffle2.releaseWriters(success = true) + + // Now comes the test : + // Write to shuffle 3; and close it, but before registering it, check if the file lengths for + // previous task (forof shuffle1) is the same as 'segments'. Earlier, we were inferring length + // of block based on remaining data in file : which could mess things up when there is concurrent read + // and writes happening to the same shuffle group. + + val shuffle3 = shuffleManager.forMapTask(1, 3, 1, new JavaSerializer(testConf), + new ShuffleWriteMetrics) + for (writer <- shuffle3.writers) { + writer.write("test3") + writer.write("test4") + } + for (writer <- shuffle3.writers) { + writer.commitAndClose() + } + // check before we register. + checkSegments(shuffle2Segment, shuffleManager.getBlockLocation(ShuffleBlockId(1, 2, 0))) + shuffle3.releaseWriters(success = true) + checkSegments(shuffle2Segment, shuffleManager.getBlockLocation(ShuffleBlockId(1, 2, 0))) + shuffleManager.removeShuffle(1) + } finally { + + if (store != null) { + store.stop() + } + actorSystem.shutdown() + actorSystem.awaitTermination() + } + } + def assertSegmentEquals(blockId: BlockId, filename: String, offset: Int, length: Int) { val segment = diskBlockManager.getBlockLocation(blockId) assert(segment.file.getName === filename) diff --git a/core/src/test/scala/org/apache/spark/storage/StorageStatusListenerSuite.scala b/core/src/test/scala/org/apache/spark/storage/StorageStatusListenerSuite.scala index 2179c6dd3302e..51fb646a3cb61 100644 --- a/core/src/test/scala/org/apache/spark/storage/StorageStatusListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/StorageStatusListenerSuite.scala @@ -41,13 +41,13 @@ class StorageStatusListenerSuite extends FunSuite { assert(listener.executorIdToStorageStatus.get("big").isDefined) assert(listener.executorIdToStorageStatus("big").blockManagerId === bm1) assert(listener.executorIdToStorageStatus("big").maxMem === 1000L) - assert(listener.executorIdToStorageStatus("big").blocks.isEmpty) + assert(listener.executorIdToStorageStatus("big").numBlocks === 0) listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(bm2, 2000L)) assert(listener.executorIdToStorageStatus.size === 2) assert(listener.executorIdToStorageStatus.get("fat").isDefined) assert(listener.executorIdToStorageStatus("fat").blockManagerId === bm2) assert(listener.executorIdToStorageStatus("fat").maxMem === 2000L) - assert(listener.executorIdToStorageStatus("fat").blocks.isEmpty) + assert(listener.executorIdToStorageStatus("fat").numBlocks === 0) // Block manager remove listener.onBlockManagerRemoved(SparkListenerBlockManagerRemoved(bm1)) @@ -67,14 +67,14 @@ class StorageStatusListenerSuite extends FunSuite { val taskMetrics = new TaskMetrics // Task end with no updated blocks - assert(listener.executorIdToStorageStatus("big").blocks.isEmpty) - assert(listener.executorIdToStorageStatus("fat").blocks.isEmpty) + assert(listener.executorIdToStorageStatus("big").numBlocks === 0) + assert(listener.executorIdToStorageStatus("fat").numBlocks === 0) listener.onTaskEnd(SparkListenerTaskEnd(1, "obliteration", Success, taskInfo1, taskMetrics)) - assert(listener.executorIdToStorageStatus("big").blocks.isEmpty) - assert(listener.executorIdToStorageStatus("fat").blocks.isEmpty) + assert(listener.executorIdToStorageStatus("big").numBlocks === 0) + assert(listener.executorIdToStorageStatus("fat").numBlocks === 0) listener.onTaskEnd(SparkListenerTaskEnd(1, "obliteration", Success, taskInfo2, taskMetrics)) - assert(listener.executorIdToStorageStatus("big").blocks.isEmpty) - assert(listener.executorIdToStorageStatus("fat").blocks.isEmpty) + assert(listener.executorIdToStorageStatus("big").numBlocks === 0) + assert(listener.executorIdToStorageStatus("fat").numBlocks === 0) } test("task end with updated blocks") { @@ -90,20 +90,20 @@ class StorageStatusListenerSuite extends FunSuite { taskMetrics2.updatedBlocks = Some(Seq(block3)) // Task end with new blocks - assert(listener.executorIdToStorageStatus("big").blocks.isEmpty) - assert(listener.executorIdToStorageStatus("fat").blocks.isEmpty) + assert(listener.executorIdToStorageStatus("big").numBlocks === 0) + assert(listener.executorIdToStorageStatus("fat").numBlocks === 0) listener.onTaskEnd(SparkListenerTaskEnd(1, "obliteration", Success, taskInfo1, taskMetrics1)) - assert(listener.executorIdToStorageStatus("big").blocks.size === 2) - assert(listener.executorIdToStorageStatus("fat").blocks.size === 0) - assert(listener.executorIdToStorageStatus("big").blocks.contains(RDDBlockId(1, 1))) - assert(listener.executorIdToStorageStatus("big").blocks.contains(RDDBlockId(1, 2))) - assert(listener.executorIdToStorageStatus("fat").blocks.isEmpty) + assert(listener.executorIdToStorageStatus("big").numBlocks === 2) + assert(listener.executorIdToStorageStatus("fat").numBlocks === 0) + assert(listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 1))) + assert(listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 2))) + assert(listener.executorIdToStorageStatus("fat").numBlocks === 0) listener.onTaskEnd(SparkListenerTaskEnd(1, "obliteration", Success, taskInfo2, taskMetrics2)) - assert(listener.executorIdToStorageStatus("big").blocks.size === 2) - assert(listener.executorIdToStorageStatus("fat").blocks.size === 1) - assert(listener.executorIdToStorageStatus("big").blocks.contains(RDDBlockId(1, 1))) - assert(listener.executorIdToStorageStatus("big").blocks.contains(RDDBlockId(1, 2))) - assert(listener.executorIdToStorageStatus("fat").blocks.contains(RDDBlockId(4, 0))) + assert(listener.executorIdToStorageStatus("big").numBlocks === 2) + assert(listener.executorIdToStorageStatus("fat").numBlocks === 1) + assert(listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 1))) + assert(listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 2))) + assert(listener.executorIdToStorageStatus("fat").containsBlock(RDDBlockId(4, 0))) // Task end with dropped blocks val droppedBlock1 = (RDDBlockId(1, 1), BlockStatus(StorageLevel.NONE, 0L, 0L, 0L)) @@ -112,17 +112,17 @@ class StorageStatusListenerSuite extends FunSuite { taskMetrics1.updatedBlocks = Some(Seq(droppedBlock1, droppedBlock3)) taskMetrics2.updatedBlocks = Some(Seq(droppedBlock2, droppedBlock3)) listener.onTaskEnd(SparkListenerTaskEnd(1, "obliteration", Success, taskInfo1, taskMetrics1)) - assert(listener.executorIdToStorageStatus("big").blocks.size === 1) - assert(listener.executorIdToStorageStatus("fat").blocks.size === 1) - assert(!listener.executorIdToStorageStatus("big").blocks.contains(RDDBlockId(1, 1))) - assert(listener.executorIdToStorageStatus("big").blocks.contains(RDDBlockId(1, 2))) - assert(listener.executorIdToStorageStatus("fat").blocks.contains(RDDBlockId(4, 0))) + assert(listener.executorIdToStorageStatus("big").numBlocks === 1) + assert(listener.executorIdToStorageStatus("fat").numBlocks === 1) + assert(!listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 1))) + assert(listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 2))) + assert(listener.executorIdToStorageStatus("fat").containsBlock(RDDBlockId(4, 0))) listener.onTaskEnd(SparkListenerTaskEnd(1, "obliteration", Success, taskInfo2, taskMetrics2)) - assert(listener.executorIdToStorageStatus("big").blocks.size === 1) - assert(listener.executorIdToStorageStatus("fat").blocks.size === 0) - assert(!listener.executorIdToStorageStatus("big").blocks.contains(RDDBlockId(1, 1))) - assert(listener.executorIdToStorageStatus("big").blocks.contains(RDDBlockId(1, 2))) - assert(listener.executorIdToStorageStatus("fat").blocks.isEmpty) + assert(listener.executorIdToStorageStatus("big").numBlocks === 1) + assert(listener.executorIdToStorageStatus("fat").numBlocks === 0) + assert(!listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 1))) + assert(listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 2))) + assert(listener.executorIdToStorageStatus("fat").numBlocks === 0) } test("unpersist RDD") { @@ -137,16 +137,16 @@ class StorageStatusListenerSuite extends FunSuite { taskMetrics2.updatedBlocks = Some(Seq(block3)) listener.onTaskEnd(SparkListenerTaskEnd(1, "obliteration", Success, taskInfo1, taskMetrics1)) listener.onTaskEnd(SparkListenerTaskEnd(1, "obliteration", Success, taskInfo1, taskMetrics2)) - assert(listener.executorIdToStorageStatus("big").blocks.size === 3) + assert(listener.executorIdToStorageStatus("big").numBlocks === 3) // Unpersist RDD listener.onUnpersistRDD(SparkListenerUnpersistRDD(9090)) - assert(listener.executorIdToStorageStatus("big").blocks.size === 3) + assert(listener.executorIdToStorageStatus("big").numBlocks === 3) listener.onUnpersistRDD(SparkListenerUnpersistRDD(4)) - assert(listener.executorIdToStorageStatus("big").blocks.size === 2) - assert(listener.executorIdToStorageStatus("big").blocks.contains(RDDBlockId(1, 1))) - assert(listener.executorIdToStorageStatus("big").blocks.contains(RDDBlockId(1, 2))) + assert(listener.executorIdToStorageStatus("big").numBlocks === 2) + assert(listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 1))) + assert(listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 2))) listener.onUnpersistRDD(SparkListenerUnpersistRDD(1)) - assert(listener.executorIdToStorageStatus("big").blocks.isEmpty) + assert(listener.executorIdToStorageStatus("big").numBlocks === 0) } } diff --git a/core/src/test/scala/org/apache/spark/storage/StorageSuite.scala b/core/src/test/scala/org/apache/spark/storage/StorageSuite.scala new file mode 100644 index 0000000000000..38678bbd1dd28 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/storage/StorageSuite.scala @@ -0,0 +1,354 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.storage + +import org.scalatest.FunSuite + +/** + * Test various functionalities in StorageUtils and StorageStatus. + */ +class StorageSuite extends FunSuite { + private val memAndDisk = StorageLevel.MEMORY_AND_DISK + + // For testing add, update, and remove (for non-RDD blocks) + private def storageStatus1: StorageStatus = { + val status = new StorageStatus(BlockManagerId("big", "dog", 1, 1), 1000L) + assert(status.blocks.isEmpty) + assert(status.rddBlocks.isEmpty) + assert(status.memUsed === 0L) + assert(status.memRemaining === 1000L) + assert(status.diskUsed === 0L) + assert(status.offHeapUsed === 0L) + status.addBlock(TestBlockId("foo"), BlockStatus(memAndDisk, 10L, 20L, 1L)) + status.addBlock(TestBlockId("fee"), BlockStatus(memAndDisk, 10L, 20L, 1L)) + status.addBlock(TestBlockId("faa"), BlockStatus(memAndDisk, 10L, 20L, 1L)) + status + } + + test("storage status add non-RDD blocks") { + val status = storageStatus1 + assert(status.blocks.size === 3) + assert(status.blocks.contains(TestBlockId("foo"))) + assert(status.blocks.contains(TestBlockId("fee"))) + assert(status.blocks.contains(TestBlockId("faa"))) + assert(status.rddBlocks.isEmpty) + assert(status.memUsed === 30L) + assert(status.memRemaining === 970L) + assert(status.diskUsed === 60L) + assert(status.offHeapUsed === 3L) + } + + test("storage status update non-RDD blocks") { + val status = storageStatus1 + status.updateBlock(TestBlockId("foo"), BlockStatus(memAndDisk, 50L, 100L, 1L)) + status.updateBlock(TestBlockId("fee"), BlockStatus(memAndDisk, 100L, 20L, 0L)) + assert(status.blocks.size === 3) + assert(status.memUsed === 160L) + assert(status.memRemaining === 840L) + assert(status.diskUsed === 140L) + assert(status.offHeapUsed === 2L) + } + + test("storage status remove non-RDD blocks") { + val status = storageStatus1 + status.removeBlock(TestBlockId("foo")) + status.removeBlock(TestBlockId("faa")) + assert(status.blocks.size === 1) + assert(status.blocks.contains(TestBlockId("fee"))) + assert(status.memUsed === 10L) + assert(status.memRemaining === 990L) + assert(status.diskUsed === 20L) + assert(status.offHeapUsed === 1L) + } + + // For testing add, update, remove, get, and contains etc. for both RDD and non-RDD blocks + private def storageStatus2: StorageStatus = { + val status = new StorageStatus(BlockManagerId("big", "dog", 1, 1), 1000L) + assert(status.rddBlocks.isEmpty) + status.addBlock(TestBlockId("dan"), BlockStatus(memAndDisk, 10L, 20L, 0L)) + status.addBlock(TestBlockId("man"), BlockStatus(memAndDisk, 10L, 20L, 0L)) + status.addBlock(RDDBlockId(0, 0), BlockStatus(memAndDisk, 10L, 20L, 1L)) + status.addBlock(RDDBlockId(1, 1), BlockStatus(memAndDisk, 100L, 200L, 1L)) + status.addBlock(RDDBlockId(2, 2), BlockStatus(memAndDisk, 10L, 20L, 1L)) + status.addBlock(RDDBlockId(2, 3), BlockStatus(memAndDisk, 10L, 20L, 0L)) + status.addBlock(RDDBlockId(2, 4), BlockStatus(memAndDisk, 10L, 40L, 0L)) + status + } + + test("storage status add RDD blocks") { + val status = storageStatus2 + assert(status.blocks.size === 7) + assert(status.rddBlocks.size === 5) + assert(status.rddBlocks.contains(RDDBlockId(0, 0))) + assert(status.rddBlocks.contains(RDDBlockId(1, 1))) + assert(status.rddBlocks.contains(RDDBlockId(2, 2))) + assert(status.rddBlocks.contains(RDDBlockId(2, 3))) + assert(status.rddBlocks.contains(RDDBlockId(2, 4))) + assert(status.rddBlocksById(0).size === 1) + assert(status.rddBlocksById(0).contains(RDDBlockId(0, 0))) + assert(status.rddBlocksById(1).size === 1) + assert(status.rddBlocksById(1).contains(RDDBlockId(1, 1))) + assert(status.rddBlocksById(2).size === 3) + assert(status.rddBlocksById(2).contains(RDDBlockId(2, 2))) + assert(status.rddBlocksById(2).contains(RDDBlockId(2, 3))) + assert(status.rddBlocksById(2).contains(RDDBlockId(2, 4))) + assert(status.memUsedByRdd(0) === 10L) + assert(status.memUsedByRdd(1) === 100L) + assert(status.memUsedByRdd(2) === 30L) + assert(status.diskUsedByRdd(0) === 20L) + assert(status.diskUsedByRdd(1) === 200L) + assert(status.diskUsedByRdd(2) === 80L) + assert(status.offHeapUsedByRdd(0) === 1L) + assert(status.offHeapUsedByRdd(1) === 1L) + assert(status.offHeapUsedByRdd(2) === 1L) + assert(status.rddStorageLevel(0) === Some(memAndDisk)) + assert(status.rddStorageLevel(1) === Some(memAndDisk)) + assert(status.rddStorageLevel(2) === Some(memAndDisk)) + + // Verify default values for RDDs that don't exist + assert(status.rddBlocksById(10).isEmpty) + assert(status.memUsedByRdd(10) === 0L) + assert(status.diskUsedByRdd(10) === 0L) + assert(status.offHeapUsedByRdd(10) === 0L) + assert(status.rddStorageLevel(10) === None) + } + + test("storage status update RDD blocks") { + val status = storageStatus2 + status.updateBlock(TestBlockId("dan"), BlockStatus(memAndDisk, 5000L, 0L, 0L)) + status.updateBlock(RDDBlockId(0, 0), BlockStatus(memAndDisk, 0L, 0L, 0L)) + status.updateBlock(RDDBlockId(2, 2), BlockStatus(memAndDisk, 0L, 1000L, 0L)) + assert(status.blocks.size === 7) + assert(status.rddBlocks.size === 5) + assert(status.rddBlocksById(0).size === 1) + assert(status.rddBlocksById(1).size === 1) + assert(status.rddBlocksById(2).size === 3) + assert(status.memUsedByRdd(0) === 0L) + assert(status.memUsedByRdd(1) === 100L) + assert(status.memUsedByRdd(2) === 20L) + assert(status.diskUsedByRdd(0) === 0L) + assert(status.diskUsedByRdd(1) === 200L) + assert(status.diskUsedByRdd(2) === 1060L) + assert(status.offHeapUsedByRdd(0) === 0L) + assert(status.offHeapUsedByRdd(1) === 1L) + assert(status.offHeapUsedByRdd(2) === 0L) + } + + test("storage status remove RDD blocks") { + val status = storageStatus2 + status.removeBlock(TestBlockId("man")) + status.removeBlock(RDDBlockId(1, 1)) + status.removeBlock(RDDBlockId(2, 2)) + status.removeBlock(RDDBlockId(2, 4)) + assert(status.blocks.size === 3) + assert(status.rddBlocks.size === 2) + assert(status.rddBlocks.contains(RDDBlockId(0, 0))) + assert(status.rddBlocks.contains(RDDBlockId(2, 3))) + assert(status.rddBlocksById(0).size === 1) + assert(status.rddBlocksById(0).contains(RDDBlockId(0, 0))) + assert(status.rddBlocksById(1).size === 0) + assert(status.rddBlocksById(2).size === 1) + assert(status.rddBlocksById(2).contains(RDDBlockId(2, 3))) + assert(status.memUsedByRdd(0) === 10L) + assert(status.memUsedByRdd(1) === 0L) + assert(status.memUsedByRdd(2) === 10L) + assert(status.diskUsedByRdd(0) === 20L) + assert(status.diskUsedByRdd(1) === 0L) + assert(status.diskUsedByRdd(2) === 20L) + assert(status.offHeapUsedByRdd(0) === 1L) + assert(status.offHeapUsedByRdd(1) === 0L) + assert(status.offHeapUsedByRdd(2) === 0L) + } + + test("storage status containsBlock") { + val status = storageStatus2 + // blocks that actually exist + assert(status.blocks.contains(TestBlockId("dan")) === status.containsBlock(TestBlockId("dan"))) + assert(status.blocks.contains(TestBlockId("man")) === status.containsBlock(TestBlockId("man"))) + assert(status.blocks.contains(RDDBlockId(0, 0)) === status.containsBlock(RDDBlockId(0, 0))) + assert(status.blocks.contains(RDDBlockId(1, 1)) === status.containsBlock(RDDBlockId(1, 1))) + assert(status.blocks.contains(RDDBlockId(2, 2)) === status.containsBlock(RDDBlockId(2, 2))) + assert(status.blocks.contains(RDDBlockId(2, 3)) === status.containsBlock(RDDBlockId(2, 3))) + assert(status.blocks.contains(RDDBlockId(2, 4)) === status.containsBlock(RDDBlockId(2, 4))) + // blocks that don't exist + assert(status.blocks.contains(TestBlockId("fan")) === status.containsBlock(TestBlockId("fan"))) + assert(status.blocks.contains(RDDBlockId(100, 0)) === status.containsBlock(RDDBlockId(100, 0))) + } + + test("storage status getBlock") { + val status = storageStatus2 + // blocks that actually exist + assert(status.blocks.get(TestBlockId("dan")) === status.getBlock(TestBlockId("dan"))) + assert(status.blocks.get(TestBlockId("man")) === status.getBlock(TestBlockId("man"))) + assert(status.blocks.get(RDDBlockId(0, 0)) === status.getBlock(RDDBlockId(0, 0))) + assert(status.blocks.get(RDDBlockId(1, 1)) === status.getBlock(RDDBlockId(1, 1))) + assert(status.blocks.get(RDDBlockId(2, 2)) === status.getBlock(RDDBlockId(2, 2))) + assert(status.blocks.get(RDDBlockId(2, 3)) === status.getBlock(RDDBlockId(2, 3))) + assert(status.blocks.get(RDDBlockId(2, 4)) === status.getBlock(RDDBlockId(2, 4))) + // blocks that don't exist + assert(status.blocks.get(TestBlockId("fan")) === status.getBlock(TestBlockId("fan"))) + assert(status.blocks.get(RDDBlockId(100, 0)) === status.getBlock(RDDBlockId(100, 0))) + } + + test("storage status num[Rdd]Blocks") { + val status = storageStatus2 + assert(status.blocks.size === status.numBlocks) + assert(status.rddBlocks.size === status.numRddBlocks) + status.addBlock(TestBlockId("Foo"), BlockStatus(memAndDisk, 0L, 0L, 100L)) + status.addBlock(RDDBlockId(4, 4), BlockStatus(memAndDisk, 0L, 0L, 100L)) + status.addBlock(RDDBlockId(4, 8), BlockStatus(memAndDisk, 0L, 0L, 100L)) + assert(status.blocks.size === status.numBlocks) + assert(status.rddBlocks.size === status.numRddBlocks) + assert(status.rddBlocksById(4).size === status.numRddBlocksById(4)) + assert(status.rddBlocksById(10).size === status.numRddBlocksById(10)) + status.updateBlock(TestBlockId("Foo"), BlockStatus(memAndDisk, 0L, 10L, 400L)) + status.updateBlock(RDDBlockId(4, 0), BlockStatus(memAndDisk, 0L, 0L, 100L)) + status.updateBlock(RDDBlockId(4, 8), BlockStatus(memAndDisk, 0L, 0L, 100L)) + status.updateBlock(RDDBlockId(10, 10), BlockStatus(memAndDisk, 0L, 0L, 100L)) + assert(status.blocks.size === status.numBlocks) + assert(status.rddBlocks.size === status.numRddBlocks) + assert(status.rddBlocksById(4).size === status.numRddBlocksById(4)) + assert(status.rddBlocksById(10).size === status.numRddBlocksById(10)) + assert(status.rddBlocksById(100).size === status.numRddBlocksById(100)) + status.removeBlock(RDDBlockId(4, 0)) + status.removeBlock(RDDBlockId(10, 10)) + assert(status.blocks.size === status.numBlocks) + assert(status.rddBlocks.size === status.numRddBlocks) + assert(status.rddBlocksById(4).size === status.numRddBlocksById(4)) + assert(status.rddBlocksById(10).size === status.numRddBlocksById(10)) + // remove a block that doesn't exist + status.removeBlock(RDDBlockId(1000, 999)) + assert(status.blocks.size === status.numBlocks) + assert(status.rddBlocks.size === status.numRddBlocks) + assert(status.rddBlocksById(4).size === status.numRddBlocksById(4)) + assert(status.rddBlocksById(10).size === status.numRddBlocksById(10)) + assert(status.rddBlocksById(1000).size === status.numRddBlocksById(1000)) + } + + test("storage status memUsed, diskUsed, tachyonUsed") { + val status = storageStatus2 + def actualMemUsed: Long = status.blocks.values.map(_.memSize).sum + def actualDiskUsed: Long = status.blocks.values.map(_.diskSize).sum + def actualOffHeapUsed: Long = status.blocks.values.map(_.tachyonSize).sum + assert(status.memUsed === actualMemUsed) + assert(status.diskUsed === actualDiskUsed) + assert(status.offHeapUsed === actualOffHeapUsed) + status.addBlock(TestBlockId("fire"), BlockStatus(memAndDisk, 4000L, 5000L, 6000L)) + status.addBlock(TestBlockId("wire"), BlockStatus(memAndDisk, 400L, 500L, 600L)) + status.addBlock(RDDBlockId(25, 25), BlockStatus(memAndDisk, 40L, 50L, 60L)) + assert(status.memUsed === actualMemUsed) + assert(status.diskUsed === actualDiskUsed) + assert(status.offHeapUsed === actualOffHeapUsed) + status.updateBlock(TestBlockId("dan"), BlockStatus(memAndDisk, 4L, 5L, 6L)) + status.updateBlock(RDDBlockId(0, 0), BlockStatus(memAndDisk, 4L, 5L, 6L)) + status.updateBlock(RDDBlockId(1, 1), BlockStatus(memAndDisk, 4L, 5L, 6L)) + assert(status.memUsed === actualMemUsed) + assert(status.diskUsed === actualDiskUsed) + assert(status.offHeapUsed === actualOffHeapUsed) + status.removeBlock(TestBlockId("fire")) + status.removeBlock(TestBlockId("man")) + status.removeBlock(RDDBlockId(2, 2)) + status.removeBlock(RDDBlockId(2, 3)) + assert(status.memUsed === actualMemUsed) + assert(status.diskUsed === actualDiskUsed) + assert(status.offHeapUsed === actualOffHeapUsed) + } + + // For testing StorageUtils.updateRddInfo and StorageUtils.getRddBlockLocations + private def stockStorageStatuses: Seq[StorageStatus] = { + val status1 = new StorageStatus(BlockManagerId("big", "dog", 1, 1), 1000L) + val status2 = new StorageStatus(BlockManagerId("fat", "duck", 2, 2), 2000L) + val status3 = new StorageStatus(BlockManagerId("fat", "cat", 3, 3), 3000L) + status1.addBlock(RDDBlockId(0, 0), BlockStatus(memAndDisk, 1L, 2L, 0L)) + status1.addBlock(RDDBlockId(0, 1), BlockStatus(memAndDisk, 1L, 2L, 0L)) + status2.addBlock(RDDBlockId(0, 2), BlockStatus(memAndDisk, 1L, 2L, 0L)) + status2.addBlock(RDDBlockId(0, 3), BlockStatus(memAndDisk, 1L, 2L, 0L)) + status2.addBlock(RDDBlockId(1, 0), BlockStatus(memAndDisk, 1L, 2L, 0L)) + status2.addBlock(RDDBlockId(1, 1), BlockStatus(memAndDisk, 1L, 2L, 0L)) + status3.addBlock(RDDBlockId(0, 4), BlockStatus(memAndDisk, 1L, 2L, 0L)) + status3.addBlock(RDDBlockId(1, 2), BlockStatus(memAndDisk, 1L, 2L, 0L)) + Seq(status1, status2, status3) + } + + // For testing StorageUtils.updateRddInfo + private def stockRDDInfos: Seq[RDDInfo] = { + val info0 = new RDDInfo(0, "0", 10, memAndDisk) + val info1 = new RDDInfo(1, "1", 3, memAndDisk) + Seq(info0, info1) + } + + test("StorageUtils.updateRddInfo") { + val storageStatuses = stockStorageStatuses + val rddInfos = stockRDDInfos + StorageUtils.updateRddInfo(rddInfos, storageStatuses) + assert(rddInfos(0).storageLevel === memAndDisk) + assert(rddInfos(0).numCachedPartitions === 5) + assert(rddInfos(0).memSize === 5L) + assert(rddInfos(0).diskSize === 10L) + assert(rddInfos(0).tachyonSize === 0L) + assert(rddInfos(1).storageLevel === memAndDisk) + assert(rddInfos(1).numCachedPartitions === 3) + assert(rddInfos(1).memSize === 3L) + assert(rddInfos(1).diskSize === 6L) + assert(rddInfos(1).tachyonSize === 0L) + } + + test("StorageUtils.getRddBlockLocations") { + val storageStatuses = stockStorageStatuses + val blockLocations0 = StorageUtils.getRddBlockLocations(0, storageStatuses) + val blockLocations1 = StorageUtils.getRddBlockLocations(1, storageStatuses) + assert(blockLocations0.size === 5) + assert(blockLocations1.size === 3) + assert(blockLocations0.contains(RDDBlockId(0, 0))) + assert(blockLocations0.contains(RDDBlockId(0, 1))) + assert(blockLocations0.contains(RDDBlockId(0, 2))) + assert(blockLocations0.contains(RDDBlockId(0, 3))) + assert(blockLocations0.contains(RDDBlockId(0, 4))) + assert(blockLocations1.contains(RDDBlockId(1, 0))) + assert(blockLocations1.contains(RDDBlockId(1, 1))) + assert(blockLocations1.contains(RDDBlockId(1, 2))) + assert(blockLocations0(RDDBlockId(0, 0)) === Seq("dog:1")) + assert(blockLocations0(RDDBlockId(0, 1)) === Seq("dog:1")) + assert(blockLocations0(RDDBlockId(0, 2)) === Seq("duck:2")) + assert(blockLocations0(RDDBlockId(0, 3)) === Seq("duck:2")) + assert(blockLocations0(RDDBlockId(0, 4)) === Seq("cat:3")) + assert(blockLocations1(RDDBlockId(1, 0)) === Seq("duck:2")) + assert(blockLocations1(RDDBlockId(1, 1)) === Seq("duck:2")) + assert(blockLocations1(RDDBlockId(1, 2)) === Seq("cat:3")) + } + + test("StorageUtils.getRddBlockLocations with multiple locations") { + val storageStatuses = stockStorageStatuses + storageStatuses(0).addBlock(RDDBlockId(1, 0), BlockStatus(memAndDisk, 1L, 2L, 0L)) + storageStatuses(0).addBlock(RDDBlockId(0, 4), BlockStatus(memAndDisk, 1L, 2L, 0L)) + storageStatuses(2).addBlock(RDDBlockId(0, 0), BlockStatus(memAndDisk, 1L, 2L, 0L)) + val blockLocations0 = StorageUtils.getRddBlockLocations(0, storageStatuses) + val blockLocations1 = StorageUtils.getRddBlockLocations(1, storageStatuses) + assert(blockLocations0.size === 5) + assert(blockLocations1.size === 3) + assert(blockLocations0(RDDBlockId(0, 0)) === Seq("dog:1", "cat:3")) + assert(blockLocations0(RDDBlockId(0, 1)) === Seq("dog:1")) + assert(blockLocations0(RDDBlockId(0, 2)) === Seq("duck:2")) + assert(blockLocations0(RDDBlockId(0, 3)) === Seq("duck:2")) + assert(blockLocations0(RDDBlockId(0, 4)) === Seq("dog:1", "cat:3")) + assert(blockLocations1(RDDBlockId(1, 0)) === Seq("dog:1", "duck:2")) + assert(blockLocations1(RDDBlockId(1, 1)) === Seq("duck:2")) + assert(blockLocations1(RDDBlockId(1, 2)) === Seq("cat:3")) + } + +} diff --git a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala index fa43b66c6cb5a..147ec0bc52e39 100644 --- a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala @@ -21,11 +21,13 @@ import org.scalatest.FunSuite import org.scalatest.Matchers import org.apache.spark._ -import org.apache.spark.executor.{ShuffleReadMetrics, TaskMetrics} +import org.apache.spark.{LocalSparkContext, SparkConf, Success} +import org.apache.spark.executor._ import org.apache.spark.scheduler._ import org.apache.spark.util.Utils class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matchers { + test("test LRU eviction of stages") { val conf = new SparkConf() conf.set("spark.ui.retainedStages", 5.toString) @@ -47,11 +49,11 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc } listener.completedStages.size should be (5) - listener.completedStages.filter(_.stageId == 50).size should be (1) - listener.completedStages.filter(_.stageId == 49).size should be (1) - listener.completedStages.filter(_.stageId == 48).size should be (1) - listener.completedStages.filter(_.stageId == 47).size should be (1) - listener.completedStages.filter(_.stageId == 46).size should be (1) + listener.completedStages.count(_.stageId == 50) should be (1) + listener.completedStages.count(_.stageId == 49) should be (1) + listener.completedStages.count(_.stageId == 48) should be (1) + listener.completedStages.count(_.stageId == 47) should be (1) + listener.completedStages.count(_.stageId == 46) should be (1) } test("test executor id to summary") { @@ -59,48 +61,42 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc val listener = new JobProgressListener(conf) val taskMetrics = new TaskMetrics() val shuffleReadMetrics = new ShuffleReadMetrics() - - // nothing in it - assert(listener.stageIdToExecutorSummaries.size == 0) + assert(listener.stageIdToData.size === 0) // finish this task, should get updated shuffleRead shuffleReadMetrics.remoteBytesRead = 1000 - taskMetrics.shuffleReadMetrics = Some(shuffleReadMetrics) + taskMetrics.setShuffleReadMetrics(Some(shuffleReadMetrics)) var taskInfo = new TaskInfo(1234L, 0, 1, 0L, "exe-1", "host1", TaskLocality.NODE_LOCAL, false) taskInfo.finishTime = 1 - var task = new ShuffleMapTask(0, null, null, 0, null) + var task = new ShuffleMapTask(0) val taskType = Utils.getFormattedClassName(task) listener.onTaskEnd(SparkListenerTaskEnd(task.stageId, taskType, Success, taskInfo, taskMetrics)) - assert(listener.stageIdToExecutorSummaries.getOrElse(0, fail()).getOrElse("exe-1", fail()) - .shuffleRead == 1000) + assert(listener.stageIdToData.getOrElse(0, fail()).executorSummary.getOrElse("exe-1", fail()) + .shuffleRead === 1000) // finish a task with unknown executor-id, nothing should happen taskInfo = new TaskInfo(1234L, 0, 1, 1000L, "exe-unknown", "host1", TaskLocality.NODE_LOCAL, true) taskInfo.finishTime = 1 - task = new ShuffleMapTask(0, null, null, 0, null) + task = new ShuffleMapTask(0) listener.onTaskEnd(SparkListenerTaskEnd(task.stageId, taskType, Success, taskInfo, taskMetrics)) - assert(listener.stageIdToExecutorSummaries.size == 1) + assert(listener.stageIdToData.size === 1) // finish this task, should get updated duration - shuffleReadMetrics.remoteBytesRead = 1000 - taskMetrics.shuffleReadMetrics = Some(shuffleReadMetrics) taskInfo = new TaskInfo(1235L, 0, 1, 0L, "exe-1", "host1", TaskLocality.NODE_LOCAL, false) taskInfo.finishTime = 1 - task = new ShuffleMapTask(0, null, null, 0, null) + task = new ShuffleMapTask(0) listener.onTaskEnd(SparkListenerTaskEnd(task.stageId, taskType, Success, taskInfo, taskMetrics)) - assert(listener.stageIdToExecutorSummaries.getOrElse(0, fail()).getOrElse("exe-1", fail()) - .shuffleRead == 2000) + assert(listener.stageIdToData.getOrElse(0, fail()).executorSummary.getOrElse("exe-1", fail()) + .shuffleRead === 2000) // finish this task, should get updated duration - shuffleReadMetrics.remoteBytesRead = 1000 - taskMetrics.shuffleReadMetrics = Some(shuffleReadMetrics) taskInfo = new TaskInfo(1236L, 0, 2, 0L, "exe-2", "host1", TaskLocality.NODE_LOCAL, false) taskInfo.finishTime = 1 - task = new ShuffleMapTask(0, null, null, 0, null) + task = new ShuffleMapTask(0) listener.onTaskEnd(SparkListenerTaskEnd(task.stageId, taskType, Success, taskInfo, taskMetrics)) - assert(listener.stageIdToExecutorSummaries.getOrElse(0, fail()).getOrElse("exe-2", fail()) - .shuffleRead == 1000) + assert(listener.stageIdToData.getOrElse(0, fail()).executorSummary.getOrElse("exe-2", fail()) + .shuffleRead === 1000) } test("test task success vs failure counting for different task end reasons") { @@ -109,7 +105,7 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc val metrics = new TaskMetrics() val taskInfo = new TaskInfo(1234L, 0, 3, 0L, "exe-1", "host1", TaskLocality.NODE_LOCAL, false) taskInfo.finishTime = 1 - val task = new ShuffleMapTask(0, null, null, 0, null) + val task = new ShuffleMapTask(0) val taskType = Utils.getFormattedClassName(task) // Go through all the failure cases to make sure we are counting them as failures. @@ -121,13 +117,107 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc TaskKilled, ExecutorLostFailure, UnknownReason) + var failCount = 0 for (reason <- taskFailedReasons) { listener.onTaskEnd(SparkListenerTaskEnd(task.stageId, taskType, reason, taskInfo, metrics)) - assert(listener.stageIdToTasksComplete.get(task.stageId) === None) + failCount += 1 + assert(listener.stageIdToData(task.stageId).numCompleteTasks === 0) + assert(listener.stageIdToData(task.stageId).numFailedTasks === failCount) } // Make sure we count success as success. listener.onTaskEnd(SparkListenerTaskEnd(task.stageId, taskType, Success, taskInfo, metrics)) - assert(listener.stageIdToTasksComplete.get(task.stageId) === Some(1)) + assert(listener.stageIdToData(task.stageId).numCompleteTasks === 1) + assert(listener.stageIdToData(task.stageId).numFailedTasks === failCount) + } + + test("test update metrics") { + val conf = new SparkConf() + val listener = new JobProgressListener(conf) + + val taskType = Utils.getFormattedClassName(new ShuffleMapTask(0)) + val execId = "exe-1" + + def makeTaskMetrics(base: Int) = { + val taskMetrics = new TaskMetrics() + val shuffleReadMetrics = new ShuffleReadMetrics() + val shuffleWriteMetrics = new ShuffleWriteMetrics() + taskMetrics.setShuffleReadMetrics(Some(shuffleReadMetrics)) + taskMetrics.shuffleWriteMetrics = Some(shuffleWriteMetrics) + shuffleReadMetrics.remoteBytesRead = base + 1 + shuffleReadMetrics.remoteBlocksFetched = base + 2 + shuffleWriteMetrics.shuffleBytesWritten = base + 3 + taskMetrics.executorRunTime = base + 4 + taskMetrics.diskBytesSpilled = base + 5 + taskMetrics.memoryBytesSpilled = base + 6 + val inputMetrics = new InputMetrics(DataReadMethod.Hadoop) + taskMetrics.inputMetrics = Some(inputMetrics) + inputMetrics.bytesRead = base + 7 + taskMetrics + } + + def makeTaskInfo(taskId: Long, finishTime: Int = 0) = { + val taskInfo = new TaskInfo(taskId, 0, 1, 0L, execId, "host1", TaskLocality.NODE_LOCAL, + false) + taskInfo.finishTime = finishTime + taskInfo + } + + listener.onTaskStart(SparkListenerTaskStart(0, makeTaskInfo(1234L))) + listener.onTaskStart(SparkListenerTaskStart(0, makeTaskInfo(1235L))) + listener.onTaskStart(SparkListenerTaskStart(1, makeTaskInfo(1236L))) + listener.onTaskStart(SparkListenerTaskStart(1, makeTaskInfo(1237L))) + + listener.onExecutorMetricsUpdate(SparkListenerExecutorMetricsUpdate(execId, Array( + (1234L, 0, makeTaskMetrics(0)), + (1235L, 0, makeTaskMetrics(100)), + (1236L, 1, makeTaskMetrics(200))))) + + var stage0Data = listener.stageIdToData.get(0).get + var stage1Data = listener.stageIdToData.get(1).get + assert(stage0Data.shuffleReadBytes == 102) + assert(stage1Data.shuffleReadBytes == 201) + assert(stage0Data.shuffleWriteBytes == 106) + assert(stage1Data.shuffleWriteBytes == 203) + assert(stage0Data.executorRunTime == 108) + assert(stage1Data.executorRunTime == 204) + assert(stage0Data.diskBytesSpilled == 110) + assert(stage1Data.diskBytesSpilled == 205) + assert(stage0Data.memoryBytesSpilled == 112) + assert(stage1Data.memoryBytesSpilled == 206) + assert(stage0Data.inputBytes == 114) + assert(stage1Data.inputBytes == 207) + assert(stage0Data.taskData.get(1234L).get.taskMetrics.get.shuffleReadMetrics.get + .totalBlocksFetched == 2) + assert(stage0Data.taskData.get(1235L).get.taskMetrics.get.shuffleReadMetrics.get + .totalBlocksFetched == 102) + assert(stage1Data.taskData.get(1236L).get.taskMetrics.get.shuffleReadMetrics.get + .totalBlocksFetched == 202) + + // task that was included in a heartbeat + listener.onTaskEnd(SparkListenerTaskEnd(0, taskType, Success, makeTaskInfo(1234L, 1), + makeTaskMetrics(300))) + // task that wasn't included in a heartbeat + listener.onTaskEnd(SparkListenerTaskEnd(1, taskType, Success, makeTaskInfo(1237L, 1), + makeTaskMetrics(400))) + + stage0Data = listener.stageIdToData.get(0).get + stage1Data = listener.stageIdToData.get(1).get + assert(stage0Data.shuffleReadBytes == 402) + assert(stage1Data.shuffleReadBytes == 602) + assert(stage0Data.shuffleWriteBytes == 406) + assert(stage1Data.shuffleWriteBytes == 606) + assert(stage0Data.executorRunTime == 408) + assert(stage1Data.executorRunTime == 608) + assert(stage0Data.diskBytesSpilled == 410) + assert(stage1Data.diskBytesSpilled == 610) + assert(stage0Data.memoryBytesSpilled == 412) + assert(stage1Data.memoryBytesSpilled == 612) + assert(stage0Data.inputBytes == 414) + assert(stage1Data.inputBytes == 614) + assert(stage0Data.taskData.get(1234L).get.taskMetrics.get.shuffleReadMetrics.get + .totalBlocksFetched == 302) + assert(stage1Data.taskData.get(1237L).get.taskMetrics.get.shuffleReadMetrics.get + .totalBlocksFetched == 402) } } diff --git a/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala b/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala new file mode 100644 index 0000000000000..6e68dcb3425aa --- /dev/null +++ b/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala @@ -0,0 +1,165 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.ui.storage + +import org.scalatest.{BeforeAndAfter, FunSuite} +import org.apache.spark.Success +import org.apache.spark.executor.TaskMetrics +import org.apache.spark.scheduler._ +import org.apache.spark.storage._ + +/** + * Test various functionality in the StorageListener that supports the StorageTab. + */ +class StorageTabSuite extends FunSuite with BeforeAndAfter { + private var bus: LiveListenerBus = _ + private var storageStatusListener: StorageStatusListener = _ + private var storageListener: StorageListener = _ + private val memAndDisk = StorageLevel.MEMORY_AND_DISK + private val memOnly = StorageLevel.MEMORY_ONLY + private val none = StorageLevel.NONE + private val taskInfo = new TaskInfo(0, 0, 0, 0, "big", "dog", TaskLocality.ANY, false) + private def rddInfo0 = new RDDInfo(0, "freedom", 100, memOnly) + private def rddInfo1 = new RDDInfo(1, "hostage", 200, memOnly) + private def rddInfo2 = new RDDInfo(2, "sanity", 300, memAndDisk) + private def rddInfo3 = new RDDInfo(3, "grace", 400, memAndDisk) + private val bm1 = BlockManagerId("big", "dog", 1, 1) + + before { + bus = new LiveListenerBus + storageStatusListener = new StorageStatusListener + storageListener = new StorageListener(storageStatusListener) + bus.addListener(storageStatusListener) + bus.addListener(storageListener) + } + + test("stage submitted / completed") { + assert(storageListener._rddInfoMap.isEmpty) + assert(storageListener.rddInfoList.isEmpty) + + // 2 RDDs are known, but none are cached + val stageInfo0 = new StageInfo(0, "0", 100, Seq(rddInfo0, rddInfo1), "details") + bus.postToAll(SparkListenerStageSubmitted(stageInfo0)) + assert(storageListener._rddInfoMap.size === 2) + assert(storageListener.rddInfoList.isEmpty) + + // 4 RDDs are known, but only 2 are cached + val rddInfo2Cached = rddInfo2 + val rddInfo3Cached = rddInfo3 + rddInfo2Cached.numCachedPartitions = 1 + rddInfo3Cached.numCachedPartitions = 1 + val stageInfo1 = new StageInfo(1, "0", 100, Seq(rddInfo2Cached, rddInfo3Cached), "details") + bus.postToAll(SparkListenerStageSubmitted(stageInfo1)) + assert(storageListener._rddInfoMap.size === 4) + assert(storageListener.rddInfoList.size === 2) + + // Submitting RDDInfos with duplicate IDs does nothing + val rddInfo0Cached = new RDDInfo(0, "freedom", 100, StorageLevel.MEMORY_ONLY) + rddInfo0Cached.numCachedPartitions = 1 + val stageInfo0Cached = new StageInfo(0, "0", 100, Seq(rddInfo0), "details") + bus.postToAll(SparkListenerStageSubmitted(stageInfo0Cached)) + assert(storageListener._rddInfoMap.size === 4) + assert(storageListener.rddInfoList.size === 2) + + // We only keep around the RDDs that are cached + bus.postToAll(SparkListenerStageCompleted(stageInfo0)) + assert(storageListener._rddInfoMap.size === 2) + assert(storageListener.rddInfoList.size === 2) + } + + test("unpersist") { + val rddInfo0Cached = rddInfo0 + val rddInfo1Cached = rddInfo1 + rddInfo0Cached.numCachedPartitions = 1 + rddInfo1Cached.numCachedPartitions = 1 + val stageInfo0 = new StageInfo(0, "0", 100, Seq(rddInfo0Cached, rddInfo1Cached), "details") + bus.postToAll(SparkListenerStageSubmitted(stageInfo0)) + assert(storageListener._rddInfoMap.size === 2) + assert(storageListener.rddInfoList.size === 2) + bus.postToAll(SparkListenerUnpersistRDD(0)) + assert(storageListener._rddInfoMap.size === 1) + assert(storageListener.rddInfoList.size === 1) + bus.postToAll(SparkListenerUnpersistRDD(4)) // doesn't exist + assert(storageListener._rddInfoMap.size === 1) + assert(storageListener.rddInfoList.size === 1) + bus.postToAll(SparkListenerUnpersistRDD(1)) + assert(storageListener._rddInfoMap.size === 0) + assert(storageListener.rddInfoList.size === 0) + } + + test("task end") { + val myRddInfo0 = rddInfo0 + val myRddInfo1 = rddInfo1 + val myRddInfo2 = rddInfo2 + val stageInfo0 = new StageInfo(0, "0", 100, Seq(myRddInfo0, myRddInfo1, myRddInfo2), "details") + bus.postToAll(SparkListenerBlockManagerAdded(bm1, 1000L)) + bus.postToAll(SparkListenerStageSubmitted(stageInfo0)) + assert(storageListener._rddInfoMap.size === 3) + assert(storageListener.rddInfoList.size === 0) // not cached + assert(!storageListener._rddInfoMap(0).isCached) + assert(!storageListener._rddInfoMap(1).isCached) + assert(!storageListener._rddInfoMap(2).isCached) + + // Task end with no updated blocks. This should not change anything. + bus.postToAll(SparkListenerTaskEnd(0, "obliteration", Success, taskInfo, new TaskMetrics)) + assert(storageListener._rddInfoMap.size === 3) + assert(storageListener.rddInfoList.size === 0) + + // Task end with a few new persisted blocks, some from the same RDD + val metrics1 = new TaskMetrics + metrics1.updatedBlocks = Some(Seq( + (RDDBlockId(0, 100), BlockStatus(memAndDisk, 400L, 0L, 0L)), + (RDDBlockId(0, 101), BlockStatus(memAndDisk, 0L, 400L, 0L)), + (RDDBlockId(0, 102), BlockStatus(memAndDisk, 400L, 0L, 200L)), + (RDDBlockId(1, 20), BlockStatus(memAndDisk, 0L, 240L, 0L)) + )) + bus.postToAll(SparkListenerTaskEnd(1, "obliteration", Success, taskInfo, metrics1)) + assert(storageListener._rddInfoMap(0).memSize === 800L) + assert(storageListener._rddInfoMap(0).diskSize === 400L) + assert(storageListener._rddInfoMap(0).tachyonSize === 200L) + assert(storageListener._rddInfoMap(0).numCachedPartitions === 3) + assert(storageListener._rddInfoMap(0).isCached) + assert(storageListener._rddInfoMap(1).memSize === 0L) + assert(storageListener._rddInfoMap(1).diskSize === 240L) + assert(storageListener._rddInfoMap(1).tachyonSize === 0L) + assert(storageListener._rddInfoMap(1).numCachedPartitions === 1) + assert(storageListener._rddInfoMap(1).isCached) + assert(!storageListener._rddInfoMap(2).isCached) + assert(storageListener._rddInfoMap(2).numCachedPartitions === 0) + + // Task end with a few dropped blocks + val metrics2 = new TaskMetrics + metrics2.updatedBlocks = Some(Seq( + (RDDBlockId(0, 100), BlockStatus(none, 0L, 0L, 0L)), + (RDDBlockId(1, 20), BlockStatus(none, 0L, 0L, 0L)), + (RDDBlockId(2, 40), BlockStatus(none, 0L, 0L, 0L)), // doesn't actually exist + (RDDBlockId(4, 80), BlockStatus(none, 0L, 0L, 0L)) // doesn't actually exist + )) + bus.postToAll(SparkListenerTaskEnd(2, "obliteration", Success, taskInfo, metrics2)) + assert(storageListener._rddInfoMap(0).memSize === 400L) + assert(storageListener._rddInfoMap(0).diskSize === 400L) + assert(storageListener._rddInfoMap(0).tachyonSize === 200L) + assert(storageListener._rddInfoMap(0).numCachedPartitions === 2) + assert(storageListener._rddInfoMap(0).isCached) + assert(!storageListener._rddInfoMap(1).isCached) + assert(storageListener._rddInfoMap(2).numCachedPartitions === 0) + assert(!storageListener._rddInfoMap(2).isCached) + assert(storageListener._rddInfoMap(2).numCachedPartitions === 0) + } + +} diff --git a/core/src/test/scala/org/apache/spark/AkkaUtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala similarity index 99% rename from core/src/test/scala/org/apache/spark/AkkaUtilsSuite.scala rename to core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala index 4ab870e751778..c4765e53de17b 100644 --- a/core/src/test/scala/org/apache/spark/AkkaUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala @@ -15,14 +15,14 @@ * limitations under the License. */ -package org.apache.spark - -import org.scalatest.FunSuite +package org.apache.spark.util import akka.actor._ +import org.apache.spark._ import org.apache.spark.scheduler.MapStatus import org.apache.spark.storage.BlockManagerId -import org.apache.spark.util.AkkaUtils +import org.scalatest.FunSuite + import scala.concurrent.Await /** diff --git a/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala b/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala index ca37d707b06ca..d2bee448d4d3b 100644 --- a/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala @@ -135,12 +135,11 @@ class FileAppenderSuite extends FunSuite with BeforeAndAfter with Logging { val testOutputStream = new PipedOutputStream() val testInputStream = new PipedInputStream(testOutputStream) val appender = FileAppender(testInputStream, testFile, conf) - assert(appender.isInstanceOf[ExpectedAppender]) + //assert(appender.getClass === classTag[ExpectedAppender].getClass) assert(appender.getClass.getSimpleName === classTag[ExpectedAppender].runtimeClass.getSimpleName) if (appender.isInstanceOf[RollingFileAppender]) { val rollingPolicy = appender.asInstanceOf[RollingFileAppender].rollingPolicy - rollingPolicy.isInstanceOf[ExpectedRollingPolicy] val policyParam = if (rollingPolicy.isInstanceOf[TimeBasedRollingPolicy]) { rollingPolicy.asInstanceOf[TimeBasedRollingPolicy].rolloverIntervalMillis } else { diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 058d31453081a..97ffb07662482 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -123,15 +123,22 @@ class JsonProtocolSuite extends FunSuite { testBlockId(StreamBlockId(1, 2L)) } - test("StageInfo.details backward compatibility") { - // StageInfo.details was added after 1.0.0. + test("StageInfo backward compatibility") { val info = makeStageInfo(1, 2, 3, 4L, 5L) - assert(info.details.nonEmpty) val newJson = JsonProtocol.stageInfoToJson(info) - val oldJson = newJson.removeField { case (field, _) => field == "Details" } + + // Fields added after 1.0.0. + assert(info.details.nonEmpty) + assert(info.accumulables.nonEmpty) + val oldJson = newJson + .removeField { case (field, _) => field == "Details" } + .removeField { case (field, _) => field == "Accumulables" } + val newInfo = JsonProtocol.stageInfoFromJson(oldJson) + assert(info.name === newInfo.name) assert("" === newInfo.details) + assert(0 === newInfo.accumulables.size) } test("InputMetrics backward compatibility") { @@ -261,6 +268,7 @@ class JsonProtocolSuite extends FunSuite { (0 until info1.rddInfos.size).foreach { i => assertEquals(info1.rddInfos(i), info2.rddInfos(i)) } + assert(info1.accumulables === info2.accumulables) assert(info1.details === info2.details) } @@ -293,6 +301,7 @@ class JsonProtocolSuite extends FunSuite { assert(info1.gettingResultTime === info2.gettingResultTime) assert(info1.finishTime === info2.finishTime) assert(info1.failed === info2.failed) + assert(info1.accumulables === info2.accumulables) } private def assertEquals(metrics1: TaskMetrics, metrics2: TaskMetrics) { @@ -314,7 +323,6 @@ class JsonProtocolSuite extends FunSuite { private def assertEquals(metrics1: ShuffleReadMetrics, metrics2: ShuffleReadMetrics) { assert(metrics1.shuffleFinishTime === metrics2.shuffleFinishTime) - assert(metrics1.totalBlocksFetched === metrics2.totalBlocksFetched) assert(metrics1.remoteBlocksFetched === metrics2.remoteBlocksFetched) assert(metrics1.localBlocksFetched === metrics2.localBlocksFetched) assert(metrics1.fetchWaitTime === metrics2.fetchWaitTime) @@ -477,13 +485,27 @@ class JsonProtocolSuite extends FunSuite { private def makeStageInfo(a: Int, b: Int, c: Int, d: Long, e: Long) = { val rddInfos = (0 until a % 5).map { i => makeRddInfo(a + i, b + i, c + i, d + i, e + i) } - new StageInfo(a, "greetings", b, rddInfos, "details") + val stageInfo = new StageInfo(a, "greetings", b, rddInfos, "details") + val (acc1, acc2) = (makeAccumulableInfo(1), makeAccumulableInfo(2)) + stageInfo.accumulables(acc1.id) = acc1 + stageInfo.accumulables(acc2.id) = acc2 + stageInfo } private def makeTaskInfo(a: Long, b: Int, c: Int, d: Long, speculative: Boolean) = { - new TaskInfo(a, b, c, d, "executor", "your kind sir", TaskLocality.NODE_LOCAL, speculative) + val taskInfo = new TaskInfo(a, b, c, d, "executor", "your kind sir", TaskLocality.NODE_LOCAL, + speculative) + val (acc1, acc2, acc3) = + (makeAccumulableInfo(1), makeAccumulableInfo(2), makeAccumulableInfo(3)) + taskInfo.accumulables += acc1 + taskInfo.accumulables += acc2 + taskInfo.accumulables += acc3 + taskInfo } + private def makeAccumulableInfo(id: Int): AccumulableInfo = + AccumulableInfo(id, " Accumulable " + id, Some("delta" + id), "val" + id) + /** * Creates a TaskMetrics object describing a task that read data from Hadoop (if hasHadoopInput is * set to true) or read data from a shuffle otherwise. @@ -513,12 +535,11 @@ class JsonProtocolSuite extends FunSuite { } else { val sr = new ShuffleReadMetrics sr.shuffleFinishTime = b + c - sr.totalBlocksFetched = e + f sr.remoteBytesRead = b + d sr.localBlocksFetched = e sr.fetchWaitTime = a + d sr.remoteBlocksFetched = f - t.shuffleReadMetrics = Some(sr) + t.setShuffleReadMetrics(Some(sr)) } sw.shuffleBytesWritten = a + b + c sw.shuffleWriteTime = b + c + d @@ -538,7 +559,9 @@ class JsonProtocolSuite extends FunSuite { private val stageSubmittedJsonString = """ {"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":100,"Stage Name": - "greetings","Number of Tasks":200,"RDD Info":[],"Details":"details"},"Properties": + "greetings","Number of Tasks":200,"RDD Info":[],"Details":"details", + "Accumulables":[{"ID":2,"Name":"Accumulable2","Update":"delta2","Value":"val2"}, + {"ID":1,"Name":"Accumulable1","Update":"delta1","Value":"val1"}]},"Properties": {"France":"Paris","Germany":"Berlin","Russia":"Moscow","Ukraine":"Kiev"}} """ @@ -548,7 +571,9 @@ class JsonProtocolSuite extends FunSuite { "greetings","Number of Tasks":201,"RDD Info":[{"RDD ID":101,"Name":"mayor","Storage Level":{"Use Disk":true,"Use Memory":true,"Use Tachyon":false,"Deserialized":true, "Replication":1},"Number of Partitions":201,"Number of Cached Partitions":301, - "Memory Size":401,"Tachyon Size":0,"Disk Size":501}],"Details":"details"}} + "Memory Size":401,"Tachyon Size":0,"Disk Size":501}],"Details":"details", + "Accumulables":[{"ID":2,"Name":"Accumulable2","Update":"delta2","Value":"val2"}, + {"ID":1,"Name":"Accumulable1","Update":"delta1","Value":"val1"}]}} """ private val taskStartJsonString = @@ -556,7 +581,9 @@ class JsonProtocolSuite extends FunSuite { |{"Event":"SparkListenerTaskStart","Stage ID":111,"Task Info":{"Task ID":222, |"Index":333,"Attempt":1,"Launch Time":444,"Executor ID":"executor","Host":"your kind sir", |"Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0, - |"Failed":false}} + |"Failed":false,"Accumulables":[{"ID":1,"Name":"Accumulable1","Update":"delta1", + |"Value":"val1"},{"ID":2,"Name":"Accumulable2","Update":"delta2","Value":"val2"}, + |{"ID":3,"Name":"Accumulable3","Update":"delta3","Value":"val3"}]}} """.stripMargin private val taskGettingResultJsonString = @@ -564,7 +591,10 @@ class JsonProtocolSuite extends FunSuite { |{"Event":"SparkListenerTaskGettingResult","Task Info": | {"Task ID":1000,"Index":2000,"Attempt":5,"Launch Time":3000,"Executor ID":"executor", | "Host":"your kind sir","Locality":"NODE_LOCAL","Speculative":true,"Getting Result Time":0, - | "Finish Time":0,"Failed":false + | "Finish Time":0,"Failed":false, + | "Accumulables":[{"ID":1,"Name":"Accumulable1","Update":"delta1", + | "Value":"val1"},{"ID":2,"Name":"Accumulable2","Update":"delta2","Value":"val2"}, + | {"ID":3,"Name":"Accumulable3","Update":"delta3","Value":"val3"}] | } |} """.stripMargin @@ -576,7 +606,10 @@ class JsonProtocolSuite extends FunSuite { |"Task Info":{ | "Task ID":123,"Index":234,"Attempt":67,"Launch Time":345,"Executor ID":"executor", | "Host":"your kind sir","Locality":"NODE_LOCAL","Speculative":false, - | "Getting Result Time":0,"Finish Time":0,"Failed":false + | "Getting Result Time":0,"Finish Time":0,"Failed":false, + | "Accumulables":[{"ID":1,"Name":"Accumulable1","Update":"delta1", + | "Value":"val1"},{"ID":2,"Name":"Accumulable2","Update":"delta2","Value":"val2"}, + | {"ID":3,"Name":"Accumulable3","Update":"delta3","Value":"val3"}] |}, |"Task Metrics":{ | "Host Name":"localhost","Executor Deserialize Time":300,"Executor Run Time":400, @@ -584,7 +617,6 @@ class JsonProtocolSuite extends FunSuite { | "Memory Bytes Spilled":800,"Disk Bytes Spilled":0, | "Shuffle Read Metrics":{ | "Shuffle Finish Time":900, - | "Total Blocks Fetched":1500, | "Remote Blocks Fetched":800, | "Local Blocks Fetched":700, | "Fetch Wait Time":900, @@ -616,7 +648,10 @@ class JsonProtocolSuite extends FunSuite { |"Task Info":{ | "Task ID":123,"Index":234,"Attempt":67,"Launch Time":345,"Executor ID":"executor", | "Host":"your kind sir","Locality":"NODE_LOCAL","Speculative":false, - | "Getting Result Time":0,"Finish Time":0,"Failed":false + | "Getting Result Time":0,"Finish Time":0,"Failed":false, + | "Accumulables":[{"ID":1,"Name":"Accumulable1","Update":"delta1", + | "Value":"val1"},{"ID":2,"Name":"Accumulable2","Update":"delta2","Value":"val2"}, + | {"ID":3,"Name":"Accumulable3","Update":"delta3","Value":"val3"}] |}, |"Task Metrics":{ | "Host Name":"localhost","Executor Deserialize Time":300,"Executor Run Time":400, diff --git a/core/src/test/scala/org/apache/spark/util/SizeTrackingAppendOnlyMapSuite.scala b/core/src/test/scala/org/apache/spark/util/SizeTrackingAppendOnlyMapSuite.scala deleted file mode 100644 index 93f0c6a8e6408..0000000000000 --- a/core/src/test/scala/org/apache/spark/util/SizeTrackingAppendOnlyMapSuite.scala +++ /dev/null @@ -1,120 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.util - -import scala.util.Random - -import org.scalatest.{BeforeAndAfterAll, FunSuite} - -import org.apache.spark.util.SizeTrackingAppendOnlyMapSuite.LargeDummyClass -import org.apache.spark.util.collection.{AppendOnlyMap, SizeTrackingAppendOnlyMap} - -class SizeTrackingAppendOnlyMapSuite extends FunSuite with BeforeAndAfterAll { - val NORMAL_ERROR = 0.20 - val HIGH_ERROR = 0.30 - - test("fixed size insertions") { - testWith[Int, Long](10000, i => (i, i.toLong)) - testWith[Int, (Long, Long)](10000, i => (i, (i.toLong, i.toLong))) - testWith[Int, LargeDummyClass](10000, i => (i, new LargeDummyClass())) - } - - test("variable size insertions") { - val rand = new Random(123456789) - def randString(minLen: Int, maxLen: Int): String = { - "a" * (rand.nextInt(maxLen - minLen) + minLen) - } - testWith[Int, String](10000, i => (i, randString(0, 10))) - testWith[Int, String](10000, i => (i, randString(0, 100))) - testWith[Int, String](10000, i => (i, randString(90, 100))) - } - - test("updates") { - val rand = new Random(123456789) - def randString(minLen: Int, maxLen: Int): String = { - "a" * (rand.nextInt(maxLen - minLen) + minLen) - } - testWith[String, Int](10000, i => (randString(0, 10000), i)) - } - - def testWith[K, V](numElements: Int, makeElement: (Int) => (K, V)) { - val map = new SizeTrackingAppendOnlyMap[K, V]() - for (i <- 0 until numElements) { - val (k, v) = makeElement(i) - map(k) = v - expectWithinError(map, map.estimateSize(), if (i < 32) HIGH_ERROR else NORMAL_ERROR) - } - } - - def expectWithinError(obj: AnyRef, estimatedSize: Long, error: Double) { - val betterEstimatedSize = SizeEstimator.estimate(obj) - assert(betterEstimatedSize * (1 - error) < estimatedSize, - s"Estimated size $estimatedSize was less than expected size $betterEstimatedSize") - assert(betterEstimatedSize * (1 + 2 * error) > estimatedSize, - s"Estimated size $estimatedSize was greater than expected size $betterEstimatedSize") - } -} - -object SizeTrackingAppendOnlyMapSuite { - // Speed test, for reproducibility of results. - // These could be highly non-deterministic in general, however. - // Results: - // AppendOnlyMap: 31 ms - // SizeTracker: 54 ms - // SizeEstimator: 1500 ms - def main(args: Array[String]) { - val numElements = 100000 - - val baseTimes = for (i <- 0 until 10) yield time { - val map = new AppendOnlyMap[Int, LargeDummyClass]() - for (i <- 0 until numElements) { - map(i) = new LargeDummyClass() - } - } - - val sampledTimes = for (i <- 0 until 10) yield time { - val map = new SizeTrackingAppendOnlyMap[Int, LargeDummyClass]() - for (i <- 0 until numElements) { - map(i) = new LargeDummyClass() - map.estimateSize() - } - } - - val unsampledTimes = for (i <- 0 until 3) yield time { - val map = new AppendOnlyMap[Int, LargeDummyClass]() - for (i <- 0 until numElements) { - map(i) = new LargeDummyClass() - SizeEstimator.estimate(map) - } - } - - println("Base: " + baseTimes) - println("SizeTracker (sampled): " + sampledTimes) - println("SizeEstimator (unsampled): " + unsampledTimes) - } - - def time(f: => Unit): Long = { - val start = System.currentTimeMillis() - f - System.currentTimeMillis() - start - } - - private class LargeDummyClass { - val arr = new Array[Int](100) - } -} diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala index 1ee936bc78f49..70d423ba8a04d 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.util import scala.util.Random import java.io.{File, ByteArrayOutputStream, ByteArrayInputStream, FileOutputStream} -import java.net.URI +import java.net.{BindException, ServerSocket, URI} import java.nio.{ByteBuffer, ByteOrder} import com.google.common.base.Charsets @@ -265,4 +265,36 @@ class UtilsSuite extends FunSuite { Array("hdfs:/a.jar", "s3:/another.jar")) } + test("isBindCollision") { + // Negatives + assert(!Utils.isBindCollision(null)) + assert(!Utils.isBindCollision(new Exception)) + assert(!Utils.isBindCollision(new Exception(new Exception))) + assert(!Utils.isBindCollision(new Exception(new BindException))) + assert(!Utils.isBindCollision(new Exception(new BindException("Random message")))) + + // Positives + val be = new BindException("Address already in use") + val be1 = new Exception(new BindException("Address already in use")) + val be2 = new Exception(new Exception(new BindException("Address already in use"))) + assert(Utils.isBindCollision(be)) + assert(Utils.isBindCollision(be1)) + assert(Utils.isBindCollision(be2)) + + // Actual bind exception + var server1: ServerSocket = null + var server2: ServerSocket = null + try { + server1 = new java.net.ServerSocket(0) + server2 = new java.net.ServerSocket(server1.getLocalPort) + } catch { + case e: Exception => + assert(e.isInstanceOf[java.net.BindException]) + assert(Utils.isBindCollision(e)) + } finally { + Option(server1).foreach(_.close()) + Option(server2).foreach(_.close()) + } + } + } diff --git a/core/src/test/scala/org/apache/spark/util/VectorSuite.scala b/core/src/test/scala/org/apache/spark/util/VectorSuite.scala index 7006571ef0ef6..794a55d61750b 100644 --- a/core/src/test/scala/org/apache/spark/util/VectorSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/VectorSuite.scala @@ -24,6 +24,7 @@ import org.scalatest.FunSuite /** * Tests org.apache.spark.util.Vector functionality */ +@deprecated("suppress compile time deprecation warning", "1.0.0") class VectorSuite extends FunSuite { def verifyVector(vector: Vector, expectedLength: Int) = { diff --git a/core/src/test/scala/org/apache/spark/util/collection/AppendOnlyMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/AppendOnlyMapSuite.scala index 52c7288e18b69..cb99d14b27af4 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/AppendOnlyMapSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/AppendOnlyMapSuite.scala @@ -170,10 +170,10 @@ class AppendOnlyMapSuite extends FunSuite { case e: IllegalStateException => fail() } - val it = map.destructiveSortedIterator(new Comparator[(String, String)] { - def compare(kv1: (String, String), kv2: (String, String)): Int = { - val x = if (kv1 != null && kv1._1 != null) kv1._1.toInt else Int.MinValue - val y = if (kv2 != null && kv2._1 != null) kv2._1.toInt else Int.MinValue + val it = map.destructiveSortedIterator(new Comparator[String] { + def compare(key1: String, key2: String): Int = { + val x = if (key1 != null) key1.toInt else Int.MinValue + val y = if (key2 != null) key2.toInt else Int.MinValue x.compareTo(y) } }) diff --git a/core/src/test/scala/org/apache/spark/util/collection/CompactBufferSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/CompactBufferSuite.scala new file mode 100644 index 0000000000000..6c956d93dc80d --- /dev/null +++ b/core/src/test/scala/org/apache/spark/util/collection/CompactBufferSuite.scala @@ -0,0 +1,105 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.collection + +import org.scalatest.FunSuite + +class CompactBufferSuite extends FunSuite { + test("empty buffer") { + val b = new CompactBuffer[Int] + assert(b.size === 0) + assert(b.iterator.toList === Nil) + assert(b.size === 0) + assert(b.iterator.toList === Nil) + intercept[IndexOutOfBoundsException] { b(0) } + intercept[IndexOutOfBoundsException] { b(1) } + intercept[IndexOutOfBoundsException] { b(2) } + intercept[IndexOutOfBoundsException] { b(-1) } + } + + test("basic inserts") { + val b = new CompactBuffer[Int] + assert(b.size === 0) + assert(b.iterator.toList === Nil) + for (i <- 0 until 1000) { + b += i + assert(b.size === i + 1) + assert(b(i) === i) + } + assert(b.iterator.toList === (0 until 1000).toList) + assert(b.iterator.toList === (0 until 1000).toList) + assert(b.size === 1000) + } + + test("adding sequences") { + val b = new CompactBuffer[Int] + assert(b.size === 0) + assert(b.iterator.toList === Nil) + + // Add some simple lists and iterators + b ++= List(0) + assert(b.size === 1) + assert(b.iterator.toList === List(0)) + b ++= Iterator(1) + assert(b.size === 2) + assert(b.iterator.toList === List(0, 1)) + b ++= List(2) + assert(b.size === 3) + assert(b.iterator.toList === List(0, 1, 2)) + b ++= Iterator(3, 4, 5, 6, 7, 8, 9) + assert(b.size === 10) + assert(b.iterator.toList === (0 until 10).toList) + + // Add CompactBuffers + val b2 = new CompactBuffer[Int] + b2 ++= 0 until 10 + b ++= b2 + assert(b.iterator.toList === (1 to 2).flatMap(i => 0 until 10).toList) + b ++= b2 + assert(b.iterator.toList === (1 to 3).flatMap(i => 0 until 10).toList) + b ++= b2 + assert(b.iterator.toList === (1 to 4).flatMap(i => 0 until 10).toList) + + // Add some small CompactBuffers as well + val b3 = new CompactBuffer[Int] + b ++= b3 + assert(b.iterator.toList === (1 to 4).flatMap(i => 0 until 10).toList) + b3 += 0 + b ++= b3 + assert(b.iterator.toList === (1 to 4).flatMap(i => 0 until 10).toList ++ List(0)) + b3 += 1 + b ++= b3 + assert(b.iterator.toList === (1 to 4).flatMap(i => 0 until 10).toList ++ List(0, 0, 1)) + b3 += 2 + b ++= b3 + assert(b.iterator.toList === (1 to 4).flatMap(i => 0 until 10).toList ++ List(0, 0, 1, 0, 1, 2)) + } + + test("adding the same buffer to itself") { + val b = new CompactBuffer[Int] + assert(b.size === 0) + assert(b.iterator.toList === Nil) + b += 1 + assert(b.toList === List(1)) + for (j <- 1 until 8) { + b ++= b + assert(b.size === (1 << j)) + assert(b.iterator.toList === (1 to (1 << j)).map(i => 1).toList) + } + } +} diff --git a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala index 428822949c085..04d7338488628 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala @@ -30,8 +30,19 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { private def mergeValue(buffer: ArrayBuffer[Int], i: Int) = buffer += i private def mergeCombiners(buf1: ArrayBuffer[Int], buf2: ArrayBuffer[Int]) = buf1 ++= buf2 + private def createSparkConf(loadDefaults: Boolean): SparkConf = { + val conf = new SparkConf(loadDefaults) + // Make the Java serializer write a reset instruction (TC_RESET) after each object to test + // for a bug we had with bytes written past the last object in a batch (SPARK-2792) + conf.set("spark.serializer.objectStreamReset", "1") + conf.set("spark.serializer", "org.apache.spark.serializer.JavaSerializer") + // Ensure that we actually have multiple batches per spill file + conf.set("spark.shuffle.spill.batchSize", "10") + conf + } + test("simple insert") { - val conf = new SparkConf(false) + val conf = createSparkConf(false) sc = new SparkContext("local", "test", conf) val map = new ExternalAppendOnlyMap[Int, Int, ArrayBuffer[Int]](createCombiner, @@ -57,18 +68,19 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { } test("insert with collision") { - val conf = new SparkConf(false) + val conf = createSparkConf(false) sc = new SparkContext("local", "test", conf) val map = new ExternalAppendOnlyMap[Int, Int, ArrayBuffer[Int]](createCombiner, mergeValue, mergeCombiners) - map.insert(1, 10) - map.insert(2, 20) - map.insert(3, 30) - map.insert(1, 100) - map.insert(2, 200) - map.insert(1, 1000) + map.insertAll(Seq( + (1, 10), + (2, 20), + (3, 30), + (1, 100), + (2, 200), + (1, 1000))) val it = map.iterator assert(it.hasNext) val result = it.toSet[(Int, ArrayBuffer[Int])].map(kv => (kv._1, kv._2.toSet)) @@ -79,7 +91,7 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { } test("ordering") { - val conf = new SparkConf(false) + val conf = createSparkConf(false) sc = new SparkContext("local", "test", conf) val map1 = new ExternalAppendOnlyMap[Int, Int, ArrayBuffer[Int]](createCombiner, @@ -124,7 +136,7 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { } test("null keys and values") { - val conf = new SparkConf(false) + val conf = createSparkConf(false) sc = new SparkContext("local", "test", conf) val map = new ExternalAppendOnlyMap[Int, Int, ArrayBuffer[Int]](createCombiner, @@ -165,7 +177,7 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { } test("simple aggregator") { - val conf = new SparkConf(false) + val conf = createSparkConf(false) sc = new SparkContext("local", "test", conf) // reduceByKey @@ -180,7 +192,7 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { } test("simple cogroup") { - val conf = new SparkConf(false) + val conf = createSparkConf(false) sc = new SparkContext("local", "test", conf) val rdd1 = sc.parallelize(1 to 4).map(i => (i, i)) val rdd2 = sc.parallelize(1 to 4).map(i => (i%2, i)) @@ -198,7 +210,7 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { } test("spilling") { - val conf = new SparkConf(true) // Load defaults, otherwise SPARK_HOME is not found + val conf = createSparkConf(true) // Load defaults, otherwise SPARK_HOME is not found conf.set("spark.shuffle.memoryFraction", "0.001") sc = new SparkContext("local-cluster[1,1,512]", "test", conf) @@ -207,11 +219,8 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { val resultA = rddA.reduceByKey(math.max).collect() assert(resultA.length == 50000) resultA.foreach { case(k, v) => - k match { - case 0 => assert(v == 1) - case 25000 => assert(v == 50001) - case 49999 => assert(v == 99999) - case _ => + if (v != k * 2 + 1) { + fail(s"Value for ${k} was wrong: expected ${k * 2 + 1}, got ${v}") } } @@ -220,11 +229,9 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { val resultB = rddB.groupByKey().collect() assert(resultB.length == 25000) resultB.foreach { case(i, seq) => - i match { - case 0 => assert(seq.toSet == Set[Int](0, 1, 2, 3)) - case 12500 => assert(seq.toSet == Set[Int](50000, 50001, 50002, 50003)) - case 24999 => assert(seq.toSet == Set[Int](99996, 99997, 99998, 99999)) - case _ => + val expected = Set(i * 4, i * 4 + 1, i * 4 + 2, i * 4 + 3) + if (seq.toSet != expected) { + fail(s"Value for ${i} was wrong: expected ${expected}, got ${seq.toSet}") } } @@ -238,6 +245,9 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { case 0 => assert(seq1.toSet == Set[Int](0)) assert(seq2.toSet == Set[Int](0, 1000, 2000, 3000, 4000, 5000, 6000, 7000, 8000, 9000)) + case 1 => + assert(seq1.toSet == Set[Int](1)) + assert(seq2.toSet == Set[Int](1, 1001, 2001, 3001, 4001, 5001, 6001, 7001, 8001, 9001)) case 5000 => assert(seq1.toSet == Set[Int](5000)) assert(seq2.toSet == Set[Int]()) @@ -250,7 +260,7 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { } test("spilling with hash collisions") { - val conf = new SparkConf(true) + val conf = createSparkConf(true) conf.set("spark.shuffle.memoryFraction", "0.001") sc = new SparkContext("local-cluster[1,1,512]", "test", conf) @@ -282,7 +292,7 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { assert(w1.hashCode === w2.hashCode) } - (1 to 100000).map(_.toString).foreach { i => map.insert(i, i) } + map.insertAll((1 to 100000).iterator.map(_.toString).map(i => (i, i))) collisionPairs.foreach { case (w1, w2) => map.insert(w1, w2) map.insert(w2, w1) @@ -305,7 +315,7 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { } test("spilling with many hash collisions") { - val conf = new SparkConf(true) + val conf = createSparkConf(true) conf.set("spark.shuffle.memoryFraction", "0.0001") sc = new SparkContext("local-cluster[1,1,512]", "test", conf) @@ -330,7 +340,7 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { } test("spilling with hash collisions using the Int.MaxValue key") { - val conf = new SparkConf(true) + val conf = createSparkConf(true) conf.set("spark.shuffle.memoryFraction", "0.001") sc = new SparkContext("local-cluster[1,1,512]", "test", conf) @@ -348,14 +358,14 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { } test("spilling with null keys and values") { - val conf = new SparkConf(true) + val conf = createSparkConf(true) conf.set("spark.shuffle.memoryFraction", "0.001") sc = new SparkContext("local-cluster[1,1,512]", "test", conf) val map = new ExternalAppendOnlyMap[Int, Int, ArrayBuffer[Int]]( createCombiner, mergeValue, mergeCombiners) - (1 to 100000).foreach { i => map.insert(i, i) } + map.insertAll((1 to 100000).iterator.map(i => (i, i))) map.insert(null.asInstanceOf[Int], 1) map.insert(1, null.asInstanceOf[Int]) map.insert(null.asInstanceOf[Int], null.asInstanceOf[Int]) @@ -368,10 +378,3 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { } } - -/** - * A dummy class that always returns the same hash code, to easily test hash collisions - */ -case class FixedHashObject(v: Int, h: Int) extends Serializable { - override def hashCode(): Int = h -} diff --git a/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala new file mode 100644 index 0000000000000..706faed980f31 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala @@ -0,0 +1,710 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.collection + +import scala.collection.mutable.ArrayBuffer + +import org.scalatest.{PrivateMethodTester, FunSuite} + +import org.apache.spark._ +import org.apache.spark.SparkContext._ + +class ExternalSorterSuite extends FunSuite with LocalSparkContext with PrivateMethodTester { + private def createSparkConf(loadDefaults: Boolean): SparkConf = { + val conf = new SparkConf(loadDefaults) + // Make the Java serializer write a reset instruction (TC_RESET) after each object to test + // for a bug we had with bytes written past the last object in a batch (SPARK-2792) + conf.set("spark.serializer.objectStreamReset", "1") + conf.set("spark.serializer", "org.apache.spark.serializer.JavaSerializer") + // Ensure that we actually have multiple batches per spill file + conf.set("spark.shuffle.spill.batchSize", "10") + conf + } + + private def assertBypassedMergeSort(sorter: ExternalSorter[_, _, _]): Unit = { + val bypassMergeSort = PrivateMethod[Boolean]('bypassMergeSort) + assert(sorter.invokePrivate(bypassMergeSort()), "sorter did not bypass merge-sort") + } + + private def assertDidNotBypassMergeSort(sorter: ExternalSorter[_, _, _]): Unit = { + val bypassMergeSort = PrivateMethod[Boolean]('bypassMergeSort) + assert(!sorter.invokePrivate(bypassMergeSort()), "sorter bypassed merge-sort") + } + + test("empty data stream") { + val conf = new SparkConf(false) + conf.set("spark.shuffle.memoryFraction", "0.001") + conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") + sc = new SparkContext("local", "test", conf) + + val agg = new Aggregator[Int, Int, Int](i => i, (i, j) => i + j, (i, j) => i + j) + val ord = implicitly[Ordering[Int]] + + // Both aggregator and ordering + val sorter = new ExternalSorter[Int, Int, Int]( + Some(agg), Some(new HashPartitioner(3)), Some(ord), None) + assert(sorter.iterator.toSeq === Seq()) + sorter.stop() + + // Only aggregator + val sorter2 = new ExternalSorter[Int, Int, Int]( + Some(agg), Some(new HashPartitioner(3)), None, None) + assert(sorter2.iterator.toSeq === Seq()) + sorter2.stop() + + // Only ordering + val sorter3 = new ExternalSorter[Int, Int, Int]( + None, Some(new HashPartitioner(3)), Some(ord), None) + assert(sorter3.iterator.toSeq === Seq()) + sorter3.stop() + + // Neither aggregator nor ordering + val sorter4 = new ExternalSorter[Int, Int, Int]( + None, Some(new HashPartitioner(3)), None, None) + assert(sorter4.iterator.toSeq === Seq()) + sorter4.stop() + } + + test("few elements per partition") { + val conf = createSparkConf(false) + conf.set("spark.shuffle.memoryFraction", "0.001") + conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") + sc = new SparkContext("local", "test", conf) + + val agg = new Aggregator[Int, Int, Int](i => i, (i, j) => i + j, (i, j) => i + j) + val ord = implicitly[Ordering[Int]] + val elements = Set((1, 1), (2, 2), (5, 5)) + val expected = Set( + (0, Set()), (1, Set((1, 1))), (2, Set((2, 2))), (3, Set()), (4, Set()), + (5, Set((5, 5))), (6, Set())) + + // Both aggregator and ordering + val sorter = new ExternalSorter[Int, Int, Int]( + Some(agg), Some(new HashPartitioner(7)), Some(ord), None) + sorter.insertAll(elements.iterator) + assert(sorter.partitionedIterator.map(p => (p._1, p._2.toSet)).toSet === expected) + sorter.stop() + + // Only aggregator + val sorter2 = new ExternalSorter[Int, Int, Int]( + Some(agg), Some(new HashPartitioner(7)), None, None) + sorter2.insertAll(elements.iterator) + assert(sorter2.partitionedIterator.map(p => (p._1, p._2.toSet)).toSet === expected) + sorter2.stop() + + // Only ordering + val sorter3 = new ExternalSorter[Int, Int, Int]( + None, Some(new HashPartitioner(7)), Some(ord), None) + sorter3.insertAll(elements.iterator) + assert(sorter3.partitionedIterator.map(p => (p._1, p._2.toSet)).toSet === expected) + sorter3.stop() + + // Neither aggregator nor ordering + val sorter4 = new ExternalSorter[Int, Int, Int]( + None, Some(new HashPartitioner(7)), None, None) + sorter4.insertAll(elements.iterator) + assert(sorter4.partitionedIterator.map(p => (p._1, p._2.toSet)).toSet === expected) + sorter4.stop() + } + + test("empty partitions with spilling") { + val conf = createSparkConf(false) + conf.set("spark.shuffle.memoryFraction", "0.001") + conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") + sc = new SparkContext("local", "test", conf) + + val ord = implicitly[Ordering[Int]] + val elements = Iterator((1, 1), (5, 5)) ++ (0 until 100000).iterator.map(x => (2, 2)) + + val sorter = new ExternalSorter[Int, Int, Int]( + None, Some(new HashPartitioner(7)), Some(ord), None) + assertDidNotBypassMergeSort(sorter) + sorter.insertAll(elements) + assert(sc.env.blockManager.diskBlockManager.getAllFiles().length > 0) // Make sure it spilled + val iter = sorter.partitionedIterator.map(p => (p._1, p._2.toList)) + assert(iter.next() === (0, Nil)) + assert(iter.next() === (1, List((1, 1)))) + assert(iter.next() === (2, (0 until 100000).map(x => (2, 2)).toList)) + assert(iter.next() === (3, Nil)) + assert(iter.next() === (4, Nil)) + assert(iter.next() === (5, List((5, 5)))) + assert(iter.next() === (6, Nil)) + sorter.stop() + } + + test("empty partitions with spilling, bypass merge-sort") { + val conf = createSparkConf(false) + conf.set("spark.shuffle.memoryFraction", "0.001") + conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") + sc = new SparkContext("local", "test", conf) + + val elements = Iterator((1, 1), (5, 5)) ++ (0 until 100000).iterator.map(x => (2, 2)) + + val sorter = new ExternalSorter[Int, Int, Int]( + None, Some(new HashPartitioner(7)), None, None) + assertBypassedMergeSort(sorter) + sorter.insertAll(elements) + assert(sc.env.blockManager.diskBlockManager.getAllFiles().length > 0) // Make sure it spilled + val iter = sorter.partitionedIterator.map(p => (p._1, p._2.toList)) + assert(iter.next() === (0, Nil)) + assert(iter.next() === (1, List((1, 1)))) + assert(iter.next() === (2, (0 until 100000).map(x => (2, 2)).toList)) + assert(iter.next() === (3, Nil)) + assert(iter.next() === (4, Nil)) + assert(iter.next() === (5, List((5, 5)))) + assert(iter.next() === (6, Nil)) + sorter.stop() + } + + test("spilling in local cluster") { + val conf = createSparkConf(true) // Load defaults, otherwise SPARK_HOME is not found + conf.set("spark.shuffle.memoryFraction", "0.001") + conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") + sc = new SparkContext("local-cluster[1,1,512]", "test", conf) + + // reduceByKey - should spill ~8 times + val rddA = sc.parallelize(0 until 100000).map(i => (i/2, i)) + val resultA = rddA.reduceByKey(math.max).collect() + assert(resultA.length == 50000) + resultA.foreach { case(k, v) => + if (v != k * 2 + 1) { + fail(s"Value for ${k} was wrong: expected ${k * 2 + 1}, got ${v}") + } + } + + // groupByKey - should spill ~17 times + val rddB = sc.parallelize(0 until 100000).map(i => (i/4, i)) + val resultB = rddB.groupByKey().collect() + assert(resultB.length == 25000) + resultB.foreach { case(i, seq) => + val expected = Set(i * 4, i * 4 + 1, i * 4 + 2, i * 4 + 3) + if (seq.toSet != expected) { + fail(s"Value for ${i} was wrong: expected ${expected}, got ${seq.toSet}") + } + } + + // cogroup - should spill ~7 times + val rddC1 = sc.parallelize(0 until 10000).map(i => (i, i)) + val rddC2 = sc.parallelize(0 until 10000).map(i => (i%1000, i)) + val resultC = rddC1.cogroup(rddC2).collect() + assert(resultC.length == 10000) + resultC.foreach { case(i, (seq1, seq2)) => + i match { + case 0 => + assert(seq1.toSet == Set[Int](0)) + assert(seq2.toSet == Set[Int](0, 1000, 2000, 3000, 4000, 5000, 6000, 7000, 8000, 9000)) + case 1 => + assert(seq1.toSet == Set[Int](1)) + assert(seq2.toSet == Set[Int](1, 1001, 2001, 3001, 4001, 5001, 6001, 7001, 8001, 9001)) + case 5000 => + assert(seq1.toSet == Set[Int](5000)) + assert(seq2.toSet == Set[Int]()) + case 9999 => + assert(seq1.toSet == Set[Int](9999)) + assert(seq2.toSet == Set[Int]()) + case _ => + } + } + + // larger cogroup - should spill ~7 times + val rddD1 = sc.parallelize(0 until 10000).map(i => (i/2, i)) + val rddD2 = sc.parallelize(0 until 10000).map(i => (i/2, i)) + val resultD = rddD1.cogroup(rddD2).collect() + assert(resultD.length == 5000) + resultD.foreach { case(i, (seq1, seq2)) => + val expected = Set(i * 2, i * 2 + 1) + if (seq1.toSet != expected) { + fail(s"Value 1 for ${i} was wrong: expected ${expected}, got ${seq1.toSet}") + } + if (seq2.toSet != expected) { + fail(s"Value 2 for ${i} was wrong: expected ${expected}, got ${seq2.toSet}") + } + } + + // sortByKey - should spill ~17 times + val rddE = sc.parallelize(0 until 100000).map(i => (i/4, i)) + val resultE = rddE.sortByKey().collect().toSeq + assert(resultE === (0 until 100000).map(i => (i/4, i)).toSeq) + } + + test("spilling in local cluster with many reduce tasks") { + val conf = createSparkConf(true) // Load defaults, otherwise SPARK_HOME is not found + conf.set("spark.shuffle.memoryFraction", "0.001") + conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") + sc = new SparkContext("local-cluster[2,1,512]", "test", conf) + + // reduceByKey - should spill ~4 times per executor + val rddA = sc.parallelize(0 until 100000).map(i => (i/2, i)) + val resultA = rddA.reduceByKey(math.max _, 100).collect() + assert(resultA.length == 50000) + resultA.foreach { case(k, v) => + if (v != k * 2 + 1) { + fail(s"Value for ${k} was wrong: expected ${k * 2 + 1}, got ${v}") + } + } + + // groupByKey - should spill ~8 times per executor + val rddB = sc.parallelize(0 until 100000).map(i => (i/4, i)) + val resultB = rddB.groupByKey(100).collect() + assert(resultB.length == 25000) + resultB.foreach { case(i, seq) => + val expected = Set(i * 4, i * 4 + 1, i * 4 + 2, i * 4 + 3) + if (seq.toSet != expected) { + fail(s"Value for ${i} was wrong: expected ${expected}, got ${seq.toSet}") + } + } + + // cogroup - should spill ~4 times per executor + val rddC1 = sc.parallelize(0 until 10000).map(i => (i, i)) + val rddC2 = sc.parallelize(0 until 10000).map(i => (i%1000, i)) + val resultC = rddC1.cogroup(rddC2, 100).collect() + assert(resultC.length == 10000) + resultC.foreach { case(i, (seq1, seq2)) => + i match { + case 0 => + assert(seq1.toSet == Set[Int](0)) + assert(seq2.toSet == Set[Int](0, 1000, 2000, 3000, 4000, 5000, 6000, 7000, 8000, 9000)) + case 1 => + assert(seq1.toSet == Set[Int](1)) + assert(seq2.toSet == Set[Int](1, 1001, 2001, 3001, 4001, 5001, 6001, 7001, 8001, 9001)) + case 5000 => + assert(seq1.toSet == Set[Int](5000)) + assert(seq2.toSet == Set[Int]()) + case 9999 => + assert(seq1.toSet == Set[Int](9999)) + assert(seq2.toSet == Set[Int]()) + case _ => + } + } + + // larger cogroup - should spill ~4 times per executor + val rddD1 = sc.parallelize(0 until 10000).map(i => (i/2, i)) + val rddD2 = sc.parallelize(0 until 10000).map(i => (i/2, i)) + val resultD = rddD1.cogroup(rddD2).collect() + assert(resultD.length == 5000) + resultD.foreach { case(i, (seq1, seq2)) => + val expected = Set(i * 2, i * 2 + 1) + if (seq1.toSet != expected) { + fail(s"Value 1 for ${i} was wrong: expected ${expected}, got ${seq1.toSet}") + } + if (seq2.toSet != expected) { + fail(s"Value 2 for ${i} was wrong: expected ${expected}, got ${seq2.toSet}") + } + } + + // sortByKey - should spill ~8 times per executor + val rddE = sc.parallelize(0 until 100000).map(i => (i/4, i)) + val resultE = rddE.sortByKey().collect().toSeq + assert(resultE === (0 until 100000).map(i => (i/4, i)).toSeq) + } + + test("cleanup of intermediate files in sorter") { + val conf = createSparkConf(true) // Load defaults, otherwise SPARK_HOME is not found + conf.set("spark.shuffle.memoryFraction", "0.001") + conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") + sc = new SparkContext("local", "test", conf) + val diskBlockManager = SparkEnv.get.blockManager.diskBlockManager + + val ord = implicitly[Ordering[Int]] + + val sorter = new ExternalSorter[Int, Int, Int]( + None, Some(new HashPartitioner(3)), Some(ord), None) + assertDidNotBypassMergeSort(sorter) + sorter.insertAll((0 until 100000).iterator.map(i => (i, i))) + assert(diskBlockManager.getAllFiles().length > 0) + sorter.stop() + assert(diskBlockManager.getAllBlocks().length === 0) + + val sorter2 = new ExternalSorter[Int, Int, Int]( + None, Some(new HashPartitioner(3)), Some(ord), None) + assertDidNotBypassMergeSort(sorter2) + sorter2.insertAll((0 until 100000).iterator.map(i => (i, i))) + assert(diskBlockManager.getAllFiles().length > 0) + assert(sorter2.iterator.toSet === (0 until 100000).map(i => (i, i)).toSet) + sorter2.stop() + assert(diskBlockManager.getAllBlocks().length === 0) + } + + test("cleanup of intermediate files in sorter, bypass merge-sort") { + val conf = createSparkConf(true) // Load defaults, otherwise SPARK_HOME is not found + conf.set("spark.shuffle.memoryFraction", "0.001") + conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") + sc = new SparkContext("local", "test", conf) + val diskBlockManager = SparkEnv.get.blockManager.diskBlockManager + + val sorter = new ExternalSorter[Int, Int, Int](None, Some(new HashPartitioner(3)), None, None) + assertBypassedMergeSort(sorter) + sorter.insertAll((0 until 100000).iterator.map(i => (i, i))) + assert(diskBlockManager.getAllFiles().length > 0) + sorter.stop() + assert(diskBlockManager.getAllBlocks().length === 0) + + val sorter2 = new ExternalSorter[Int, Int, Int](None, Some(new HashPartitioner(3)), None, None) + assertBypassedMergeSort(sorter2) + sorter2.insertAll((0 until 100000).iterator.map(i => (i, i))) + assert(diskBlockManager.getAllFiles().length > 0) + assert(sorter2.iterator.toSet === (0 until 100000).map(i => (i, i)).toSet) + sorter2.stop() + assert(diskBlockManager.getAllBlocks().length === 0) + } + + test("cleanup of intermediate files in sorter if there are errors") { + val conf = createSparkConf(true) // Load defaults, otherwise SPARK_HOME is not found + conf.set("spark.shuffle.memoryFraction", "0.001") + conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") + sc = new SparkContext("local", "test", conf) + val diskBlockManager = SparkEnv.get.blockManager.diskBlockManager + + val ord = implicitly[Ordering[Int]] + + val sorter = new ExternalSorter[Int, Int, Int]( + None, Some(new HashPartitioner(3)), Some(ord), None) + assertDidNotBypassMergeSort(sorter) + intercept[SparkException] { + sorter.insertAll((0 until 100000).iterator.map(i => { + if (i == 99990) { + throw new SparkException("Intentional failure") + } + (i, i) + })) + } + assert(diskBlockManager.getAllFiles().length > 0) + sorter.stop() + assert(diskBlockManager.getAllBlocks().length === 0) + } + + test("cleanup of intermediate files in sorter if there are errors, bypass merge-sort") { + val conf = createSparkConf(true) // Load defaults, otherwise SPARK_HOME is not found + conf.set("spark.shuffle.memoryFraction", "0.001") + conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") + sc = new SparkContext("local", "test", conf) + val diskBlockManager = SparkEnv.get.blockManager.diskBlockManager + + val sorter = new ExternalSorter[Int, Int, Int](None, Some(new HashPartitioner(3)), None, None) + assertBypassedMergeSort(sorter) + intercept[SparkException] { + sorter.insertAll((0 until 100000).iterator.map(i => { + if (i == 99990) { + throw new SparkException("Intentional failure") + } + (i, i) + })) + } + assert(diskBlockManager.getAllFiles().length > 0) + sorter.stop() + assert(diskBlockManager.getAllBlocks().length === 0) + } + + test("cleanup of intermediate files in shuffle") { + val conf = createSparkConf(false) + conf.set("spark.shuffle.memoryFraction", "0.001") + conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") + sc = new SparkContext("local", "test", conf) + val diskBlockManager = SparkEnv.get.blockManager.diskBlockManager + + val data = sc.parallelize(0 until 100000, 2).map(i => (i, i)) + assert(data.reduceByKey(_ + _).count() === 100000) + + // After the shuffle, there should be only 4 files on disk: our two map output files and + // their index files. All other intermediate files should've been deleted. + assert(diskBlockManager.getAllFiles().length === 4) + } + + test("cleanup of intermediate files in shuffle with errors") { + val conf = createSparkConf(false) + conf.set("spark.shuffle.memoryFraction", "0.001") + conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") + sc = new SparkContext("local", "test", conf) + val diskBlockManager = SparkEnv.get.blockManager.diskBlockManager + + val data = sc.parallelize(0 until 100000, 2).map(i => { + if (i == 99990) { + throw new Exception("Intentional failure") + } + (i, i) + }) + intercept[SparkException] { + data.reduceByKey(_ + _).count() + } + + // After the shuffle, there should be only 2 files on disk: the output of task 1 and its index. + // All other files (map 2's output and intermediate merge files) should've been deleted. + assert(diskBlockManager.getAllFiles().length === 2) + } + + test("no partial aggregation or sorting") { + val conf = createSparkConf(false) + conf.set("spark.shuffle.memoryFraction", "0.001") + conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") + sc = new SparkContext("local", "test", conf) + + val sorter = new ExternalSorter[Int, Int, Int](None, Some(new HashPartitioner(3)), None, None) + sorter.insertAll((0 until 100000).iterator.map(i => (i / 4, i))) + val results = sorter.partitionedIterator.map{case (p, vs) => (p, vs.toSet)}.toSet + val expected = (0 until 3).map(p => { + (p, (0 until 100000).map(i => (i / 4, i)).filter(_._1 % 3 == p).toSet) + }).toSet + assert(results === expected) + } + + test("partial aggregation without spill") { + val conf = createSparkConf(false) + conf.set("spark.shuffle.memoryFraction", "0.001") + conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") + sc = new SparkContext("local", "test", conf) + + val agg = new Aggregator[Int, Int, Int](i => i, (i, j) => i + j, (i, j) => i + j) + val sorter = new ExternalSorter(Some(agg), Some(new HashPartitioner(3)), None, None) + sorter.insertAll((0 until 100).iterator.map(i => (i / 2, i))) + val results = sorter.partitionedIterator.map{case (p, vs) => (p, vs.toSet)}.toSet + val expected = (0 until 3).map(p => { + (p, (0 until 50).map(i => (i, i * 4 + 1)).filter(_._1 % 3 == p).toSet) + }).toSet + assert(results === expected) + } + + test("partial aggregation with spill, no ordering") { + val conf = createSparkConf(false) + conf.set("spark.shuffle.memoryFraction", "0.001") + conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") + sc = new SparkContext("local", "test", conf) + + val agg = new Aggregator[Int, Int, Int](i => i, (i, j) => i + j, (i, j) => i + j) + val sorter = new ExternalSorter(Some(agg), Some(new HashPartitioner(3)), None, None) + sorter.insertAll((0 until 100000).iterator.map(i => (i / 2, i))) + val results = sorter.partitionedIterator.map{case (p, vs) => (p, vs.toSet)}.toSet + val expected = (0 until 3).map(p => { + (p, (0 until 50000).map(i => (i, i * 4 + 1)).filter(_._1 % 3 == p).toSet) + }).toSet + assert(results === expected) + } + + test("partial aggregation with spill, with ordering") { + val conf = createSparkConf(false) + conf.set("spark.shuffle.memoryFraction", "0.001") + conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") + sc = new SparkContext("local", "test", conf) + + val agg = new Aggregator[Int, Int, Int](i => i, (i, j) => i + j, (i, j) => i + j) + val ord = implicitly[Ordering[Int]] + val sorter = new ExternalSorter(Some(agg), Some(new HashPartitioner(3)), Some(ord), None) + sorter.insertAll((0 until 100000).iterator.map(i => (i / 2, i))) + val results = sorter.partitionedIterator.map{case (p, vs) => (p, vs.toSet)}.toSet + val expected = (0 until 3).map(p => { + (p, (0 until 50000).map(i => (i, i * 4 + 1)).filter(_._1 % 3 == p).toSet) + }).toSet + assert(results === expected) + } + + test("sorting without aggregation, no spill") { + val conf = createSparkConf(false) + conf.set("spark.shuffle.memoryFraction", "0.001") + conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") + sc = new SparkContext("local", "test", conf) + + val ord = implicitly[Ordering[Int]] + val sorter = new ExternalSorter[Int, Int, Int]( + None, Some(new HashPartitioner(3)), Some(ord), None) + sorter.insertAll((0 until 100).iterator.map(i => (i, i))) + val results = sorter.partitionedIterator.map{case (p, vs) => (p, vs.toSeq)}.toSeq + val expected = (0 until 3).map(p => { + (p, (0 until 100).map(i => (i, i)).filter(_._1 % 3 == p).toSeq) + }).toSeq + assert(results === expected) + } + + test("sorting without aggregation, with spill") { + val conf = createSparkConf(false) + conf.set("spark.shuffle.memoryFraction", "0.001") + conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") + sc = new SparkContext("local", "test", conf) + + val ord = implicitly[Ordering[Int]] + val sorter = new ExternalSorter[Int, Int, Int]( + None, Some(new HashPartitioner(3)), Some(ord), None) + sorter.insertAll((0 until 100000).iterator.map(i => (i, i))) + val results = sorter.partitionedIterator.map{case (p, vs) => (p, vs.toSeq)}.toSeq + val expected = (0 until 3).map(p => { + (p, (0 until 100000).map(i => (i, i)).filter(_._1 % 3 == p).toSeq) + }).toSeq + assert(results === expected) + } + + test("spilling with hash collisions") { + val conf = createSparkConf(true) + conf.set("spark.shuffle.memoryFraction", "0.001") + sc = new SparkContext("local-cluster[1,1,512]", "test", conf) + + def createCombiner(i: String) = ArrayBuffer[String](i) + def mergeValue(buffer: ArrayBuffer[String], i: String) = buffer += i + def mergeCombiners(buffer1: ArrayBuffer[String], buffer2: ArrayBuffer[String]) = + buffer1 ++= buffer2 + + val agg = new Aggregator[String, String, ArrayBuffer[String]]( + createCombiner _, mergeValue _, mergeCombiners _) + + val sorter = new ExternalSorter[String, String, ArrayBuffer[String]]( + Some(agg), None, None, None) + + val collisionPairs = Seq( + ("Aa", "BB"), // 2112 + ("to", "v1"), // 3707 + ("variants", "gelato"), // -1249574770 + ("Teheran", "Siblings"), // 231609873 + ("misused", "horsemints"), // 1069518484 + ("isohel", "epistolaries"), // -1179291542 + ("righto", "buzzards"), // -931102253 + ("hierarch", "crinolines"), // -1732884796 + ("inwork", "hypercatalexes"), // -1183663690 + ("wainages", "presentencing"), // 240183619 + ("trichothecenes", "locular"), // 339006536 + ("pomatoes", "eructation") // 568647356 + ) + + collisionPairs.foreach { case (w1, w2) => + // String.hashCode is documented to use a specific algorithm, but check just in case + assert(w1.hashCode === w2.hashCode) + } + + val toInsert = (1 to 100000).iterator.map(_.toString).map(s => (s, s)) ++ + collisionPairs.iterator ++ collisionPairs.iterator.map(_.swap) + + sorter.insertAll(toInsert) + + // A map of collision pairs in both directions + val collisionPairsMap = (collisionPairs ++ collisionPairs.map(_.swap)).toMap + + // Avoid map.size or map.iterator.length because this destructively sorts the underlying map + var count = 0 + + val it = sorter.iterator + while (it.hasNext) { + val kv = it.next() + val expectedValue = ArrayBuffer[String](collisionPairsMap.getOrElse(kv._1, kv._1)) + assert(kv._2.equals(expectedValue)) + count += 1 + } + assert(count === 100000 + collisionPairs.size * 2) + } + + test("spilling with many hash collisions") { + val conf = createSparkConf(true) + conf.set("spark.shuffle.memoryFraction", "0.0001") + sc = new SparkContext("local-cluster[1,1,512]", "test", conf) + + val agg = new Aggregator[FixedHashObject, Int, Int](_ => 1, _ + _, _ + _) + val sorter = new ExternalSorter[FixedHashObject, Int, Int](Some(agg), None, None, None) + + // Insert 10 copies each of lots of objects whose hash codes are either 0 or 1. This causes + // problems if the map fails to group together the objects with the same code (SPARK-2043). + val toInsert = for (i <- 1 to 10; j <- 1 to 10000) yield (FixedHashObject(j, j % 2), 1) + sorter.insertAll(toInsert.iterator) + + val it = sorter.iterator + var count = 0 + while (it.hasNext) { + val kv = it.next() + assert(kv._2 === 10) + count += 1 + } + assert(count === 10000) + } + + test("spilling with hash collisions using the Int.MaxValue key") { + val conf = createSparkConf(true) + conf.set("spark.shuffle.memoryFraction", "0.001") + sc = new SparkContext("local-cluster[1,1,512]", "test", conf) + + def createCombiner(i: Int) = ArrayBuffer[Int](i) + def mergeValue(buffer: ArrayBuffer[Int], i: Int) = buffer += i + def mergeCombiners(buf1: ArrayBuffer[Int], buf2: ArrayBuffer[Int]) = buf1 ++= buf2 + + val agg = new Aggregator[Int, Int, ArrayBuffer[Int]](createCombiner, mergeValue, mergeCombiners) + val sorter = new ExternalSorter[Int, Int, ArrayBuffer[Int]](Some(agg), None, None, None) + + sorter.insertAll((1 to 100000).iterator.map(i => (i, i)) ++ Iterator((Int.MaxValue, Int.MaxValue))) + + val it = sorter.iterator + while (it.hasNext) { + // Should not throw NoSuchElementException + it.next() + } + } + + test("spilling with null keys and values") { + val conf = createSparkConf(true) + conf.set("spark.shuffle.memoryFraction", "0.001") + sc = new SparkContext("local-cluster[1,1,512]", "test", conf) + + def createCombiner(i: String) = ArrayBuffer[String](i) + def mergeValue(buffer: ArrayBuffer[String], i: String) = buffer += i + def mergeCombiners(buf1: ArrayBuffer[String], buf2: ArrayBuffer[String]) = buf1 ++= buf2 + + val agg = new Aggregator[String, String, ArrayBuffer[String]]( + createCombiner, mergeValue, mergeCombiners) + + val sorter = new ExternalSorter[String, String, ArrayBuffer[String]]( + Some(agg), None, None, None) + + sorter.insertAll((1 to 100000).iterator.map(i => (i.toString, i.toString)) ++ Iterator( + (null.asInstanceOf[String], "1"), + ("1", null.asInstanceOf[String]), + (null.asInstanceOf[String], null.asInstanceOf[String]) + )) + + val it = sorter.iterator + while (it.hasNext) { + // Should not throw NullPointerException + it.next() + } + } + + test("conditions for bypassing merge-sort") { + val conf = createSparkConf(false) + conf.set("spark.shuffle.memoryFraction", "0.001") + conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") + sc = new SparkContext("local", "test", conf) + + val agg = new Aggregator[Int, Int, Int](i => i, (i, j) => i + j, (i, j) => i + j) + val ord = implicitly[Ordering[Int]] + + // Numbers of partitions that are above and below the default bypassMergeThreshold + val FEW_PARTITIONS = 50 + val MANY_PARTITIONS = 10000 + + // Sorters with no ordering or aggregator: should bypass unless # of partitions is high + + val sorter1 = new ExternalSorter[Int, Int, Int]( + None, Some(new HashPartitioner(FEW_PARTITIONS)), None, None) + assertBypassedMergeSort(sorter1) + + val sorter2 = new ExternalSorter[Int, Int, Int]( + None, Some(new HashPartitioner(MANY_PARTITIONS)), None, None) + assertDidNotBypassMergeSort(sorter2) + + // Sorters with an ordering or aggregator: should not bypass even if they have few partitions + + val sorter3 = new ExternalSorter[Int, Int, Int]( + None, Some(new HashPartitioner(FEW_PARTITIONS)), Some(ord), None) + assertDidNotBypassMergeSort(sorter3) + + val sorter4 = new ExternalSorter[Int, Int, Int]( + Some(agg), Some(new HashPartitioner(FEW_PARTITIONS)), None, None) + assertDidNotBypassMergeSort(sorter4) + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BaseRelation.scala b/core/src/test/scala/org/apache/spark/util/collection/FixedHashObject.scala similarity index 77% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BaseRelation.scala rename to core/src/test/scala/org/apache/spark/util/collection/FixedHashObject.scala index 582334aa42590..c787b5f066e00 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BaseRelation.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/FixedHashObject.scala @@ -15,10 +15,11 @@ * limitations under the License. */ -package org.apache.spark.sql.catalyst.plans.logical +package org.apache.spark.util.collection -abstract class BaseRelation extends LeafNode { - self: Product => - - def tableName: String +/** + * A dummy class that always returns the same hash code, to easily test hash collisions + */ +case class FixedHashObject(v: Int, h: Int) extends Serializable { + override def hashCode(): Int = h } diff --git a/core/src/test/scala/org/apache/spark/util/collection/SizeTrackerSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/SizeTrackerSuite.scala new file mode 100644 index 0000000000000..1f33967249654 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/util/collection/SizeTrackerSuite.scala @@ -0,0 +1,204 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.collection + +import scala.reflect.ClassTag +import scala.util.Random + +import org.scalatest.FunSuite + +import org.apache.spark.util.SizeEstimator + +class SizeTrackerSuite extends FunSuite { + val NORMAL_ERROR = 0.20 + val HIGH_ERROR = 0.30 + + import SizeTrackerSuite._ + + test("vector fixed size insertions") { + testVector[Long](10000, i => i.toLong) + testVector[(Long, Long)](10000, i => (i.toLong, i.toLong)) + testVector[LargeDummyClass](10000, i => new LargeDummyClass) + } + + test("vector variable size insertions") { + val rand = new Random(123456789) + def randString(minLen: Int, maxLen: Int): String = { + "a" * (rand.nextInt(maxLen - minLen) + minLen) + } + testVector[String](10000, i => randString(0, 10)) + testVector[String](10000, i => randString(0, 100)) + testVector[String](10000, i => randString(90, 100)) + } + + test("map fixed size insertions") { + testMap[Int, Long](10000, i => (i, i.toLong)) + testMap[Int, (Long, Long)](10000, i => (i, (i.toLong, i.toLong))) + testMap[Int, LargeDummyClass](10000, i => (i, new LargeDummyClass)) + } + + test("map variable size insertions") { + val rand = new Random(123456789) + def randString(minLen: Int, maxLen: Int): String = { + "a" * (rand.nextInt(maxLen - minLen) + minLen) + } + testMap[Int, String](10000, i => (i, randString(0, 10))) + testMap[Int, String](10000, i => (i, randString(0, 100))) + testMap[Int, String](10000, i => (i, randString(90, 100))) + } + + test("map updates") { + val rand = new Random(123456789) + def randString(minLen: Int, maxLen: Int): String = { + "a" * (rand.nextInt(maxLen - minLen) + minLen) + } + testMap[String, Int](10000, i => (randString(0, 10000), i)) + } + + def testVector[T: ClassTag](numElements: Int, makeElement: Int => T) { + val vector = new SizeTrackingVector[T] + for (i <- 0 until numElements) { + val item = makeElement(i) + vector += item + expectWithinError(vector, vector.estimateSize(), if (i < 32) HIGH_ERROR else NORMAL_ERROR) + } + } + + def testMap[K, V](numElements: Int, makeElement: (Int) => (K, V)) { + val map = new SizeTrackingAppendOnlyMap[K, V] + for (i <- 0 until numElements) { + val (k, v) = makeElement(i) + map(k) = v + expectWithinError(map, map.estimateSize(), if (i < 32) HIGH_ERROR else NORMAL_ERROR) + } + } + + def expectWithinError(obj: AnyRef, estimatedSize: Long, error: Double) { + val betterEstimatedSize = SizeEstimator.estimate(obj) + assert(betterEstimatedSize * (1 - error) < estimatedSize, + s"Estimated size $estimatedSize was less than expected size $betterEstimatedSize") + assert(betterEstimatedSize * (1 + 2 * error) > estimatedSize, + s"Estimated size $estimatedSize was greater than expected size $betterEstimatedSize") + } +} + +private object SizeTrackerSuite { + + /** + * Run speed tests for size tracking collections. + */ + def main(args: Array[String]): Unit = { + if (args.size < 1) { + println("Usage: SizeTrackerSuite [num elements]") + System.exit(1) + } + val numElements = args(0).toInt + vectorSpeedTest(numElements) + mapSpeedTest(numElements) + } + + /** + * Speed test for SizeTrackingVector. + * + * Results for 100000 elements (possibly non-deterministic): + * PrimitiveVector 15 ms + * SizeTracker 51 ms + * SizeEstimator 2000 ms + */ + def vectorSpeedTest(numElements: Int): Unit = { + val baseTimes = for (i <- 0 until 10) yield time { + val vector = new PrimitiveVector[LargeDummyClass] + for (i <- 0 until numElements) { + vector += new LargeDummyClass + } + } + val sampledTimes = for (i <- 0 until 10) yield time { + val vector = new SizeTrackingVector[LargeDummyClass] + for (i <- 0 until numElements) { + vector += new LargeDummyClass + vector.estimateSize() + } + } + val unsampledTimes = for (i <- 0 until 3) yield time { + val vector = new PrimitiveVector[LargeDummyClass] + for (i <- 0 until numElements) { + vector += new LargeDummyClass + SizeEstimator.estimate(vector) + } + } + printSpeedTestResult("SizeTrackingVector", baseTimes, sampledTimes, unsampledTimes) + } + + /** + * Speed test for SizeTrackingAppendOnlyMap. + * + * Results for 100000 elements (possibly non-deterministic): + * AppendOnlyMap 30 ms + * SizeTracker 41 ms + * SizeEstimator 1666 ms + */ + def mapSpeedTest(numElements: Int): Unit = { + val baseTimes = for (i <- 0 until 10) yield time { + val map = new AppendOnlyMap[Int, LargeDummyClass] + for (i <- 0 until numElements) { + map(i) = new LargeDummyClass + } + } + val sampledTimes = for (i <- 0 until 10) yield time { + val map = new SizeTrackingAppendOnlyMap[Int, LargeDummyClass] + for (i <- 0 until numElements) { + map(i) = new LargeDummyClass + map.estimateSize() + } + } + val unsampledTimes = for (i <- 0 until 3) yield time { + val map = new AppendOnlyMap[Int, LargeDummyClass] + for (i <- 0 until numElements) { + map(i) = new LargeDummyClass + SizeEstimator.estimate(map) + } + } + printSpeedTestResult("SizeTrackingAppendOnlyMap", baseTimes, sampledTimes, unsampledTimes) + } + + def printSpeedTestResult( + testName: String, + baseTimes: Seq[Long], + sampledTimes: Seq[Long], + unsampledTimes: Seq[Long]): Unit = { + println(s"Average times for $testName (ms):") + println(" Base - " + averageTime(baseTimes)) + println(" SizeTracker (sampled) - " + averageTime(sampledTimes)) + println(" SizeEstimator (unsampled) - " + averageTime(unsampledTimes)) + println() + } + + def time(f: => Unit): Long = { + val start = System.currentTimeMillis() + f + System.currentTimeMillis() - start + } + + def averageTime(v: Seq[Long]): Long = { + v.sum / v.size + } + + private class LargeDummyClass { + val arr = new Array[Int](100) + } +} diff --git a/core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala new file mode 100644 index 0000000000000..6fe1079c2719a --- /dev/null +++ b/core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala @@ -0,0 +1,167 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.collection + +import java.lang.{Float => JFloat} +import java.util.{Arrays, Comparator} + +import org.scalatest.FunSuite + +import org.apache.spark.util.random.XORShiftRandom + +class SorterSuite extends FunSuite { + + test("equivalent to Arrays.sort") { + val rand = new XORShiftRandom(123) + val data0 = Array.tabulate[Int](10000) { i => rand.nextInt() } + val data1 = data0.clone() + + Arrays.sort(data0) + new Sorter(new IntArraySortDataFormat).sort(data1, 0, data1.length, Ordering.Int) + + data0.zip(data1).foreach { case (x, y) => assert(x === y) } + } + + test("KVArraySorter") { + val rand = new XORShiftRandom(456) + + // Construct an array of keys (to Java sort) and an array where the keys and values + // alternate. Keys are random doubles, values are ordinals from 0 to length. + val keys = Array.tabulate[Double](5000) { i => rand.nextDouble() } + val keyValueArray = Array.tabulate[Number](10000) { i => + if (i % 2 == 0) keys(i / 2) else new Integer(i / 2) + } + + // Map from generated keys to values, to verify correctness later + val kvMap = + keyValueArray.grouped(2).map { case Array(k, v) => k.doubleValue() -> v.intValue() }.toMap + + Arrays.sort(keys) + new Sorter(new KVArraySortDataFormat[Double, Number]) + .sort(keyValueArray, 0, keys.length, Ordering.Double) + + keys.zipWithIndex.foreach { case (k, i) => + assert(k === keyValueArray(2 * i)) + assert(kvMap(k) === keyValueArray(2 * i + 1)) + } + } + + /** + * This provides a simple benchmark for comparing the Sorter with Java internal sorting. + * Ideally these would be executed one at a time, each in their own JVM, so their listing + * here is mainly to have the code. + * + * The goal of this code is to sort an array of key-value pairs, where the array physically + * has the keys and values alternating. The basic Java sorts work only on the keys, so the + * real Java solution is to make Tuple2s to store the keys and values and sort an array of + * those, while the Sorter approach can work directly on the input data format. + * + * Note that the Java implementation varies tremendously between Java 6 and Java 7, when + * the Java sort changed from merge sort to Timsort. + */ + ignore("Sorter benchmark") { + + /** Runs an experiment several times. */ + def runExperiment(name: String)(f: => Unit): Unit = { + val firstTry = org.apache.spark.util.Utils.timeIt(1)(f) + System.gc() + + var i = 0 + var next10: Long = 0 + while (i < 10) { + val time = org.apache.spark.util.Utils.timeIt(1)(f) + next10 += time + println(s"$name: Took $time ms") + i += 1 + } + + println(s"$name: ($firstTry ms first try, ${next10 / 10} ms average)") + } + + val numElements = 25000000 // 25 mil + val rand = new XORShiftRandom(123) + + val keys = Array.tabulate[JFloat](numElements) { i => + new JFloat(rand.nextFloat()) + } + + // Test our key-value pairs where each element is a Tuple2[Float, Integer) + val kvTupleArray = Array.tabulate[AnyRef](numElements) { i => + (keys(i / 2): Float, i / 2: Int) + } + runExperiment("Tuple-sort using Arrays.sort()") { + Arrays.sort(kvTupleArray, new Comparator[AnyRef] { + override def compare(x: AnyRef, y: AnyRef): Int = + Ordering.Float.compare(x.asInstanceOf[(Float, _)]._1, y.asInstanceOf[(Float, _)]._1) + }) + } + + // Test our Sorter where each element alternates between Float and Integer, non-primitive + val keyValueArray = Array.tabulate[AnyRef](numElements * 2) { i => + if (i % 2 == 0) keys(i / 2) else new Integer(i / 2) + } + val sorter = new Sorter(new KVArraySortDataFormat[JFloat, AnyRef]) + runExperiment("KV-sort using Sorter") { + sorter.sort(keyValueArray, 0, keys.length, new Comparator[JFloat] { + override def compare(x: JFloat, y: JFloat): Int = Ordering.Float.compare(x, y) + }) + } + + // Test non-primitive sort on float array + runExperiment("Java Arrays.sort()") { + Arrays.sort(keys, new Comparator[JFloat] { + override def compare(x: JFloat, y: JFloat): Int = Ordering.Float.compare(x, y) + }) + } + + // Test primitive sort on float array + val primitiveKeys = Array.tabulate[Float](numElements) { i => rand.nextFloat() } + runExperiment("Java Arrays.sort() on primitive keys") { + Arrays.sort(primitiveKeys) + } + } +} + + +/** Format to sort a simple Array[Int]. Could be easily generified and specialized. */ +class IntArraySortDataFormat extends SortDataFormat[Int, Array[Int]] { + override protected def getKey(data: Array[Int], pos: Int): Int = { + data(pos) + } + + override protected def swap(data: Array[Int], pos0: Int, pos1: Int): Unit = { + val tmp = data(pos0) + data(pos0) = data(pos1) + data(pos1) = tmp + } + + override protected def copyElement(src: Array[Int], srcPos: Int, dst: Array[Int], dstPos: Int) { + dst(dstPos) = src(srcPos) + } + + /** Copy a range of elements starting at src(srcPos) to dest, starting at destPos. */ + override protected def copyRange(src: Array[Int], srcPos: Int, + dst: Array[Int], dstPos: Int, length: Int) { + System.arraycopy(src, srcPos, dst, dstPos, length) + } + + /** Allocates a new structure that can hold up to 'length' elements. */ + override protected def allocate(length: Int): Array[Int] = { + new Array[Int](length) + } +} diff --git a/core/src/test/scala/org/apache/spark/util/random/SamplingUtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/random/SamplingUtilsSuite.scala index accfe2e9b7f2a..73a9d029b0248 100644 --- a/core/src/test/scala/org/apache/spark/util/random/SamplingUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/random/SamplingUtilsSuite.scala @@ -17,11 +17,32 @@ package org.apache.spark.util.random +import scala.util.Random + import org.apache.commons.math3.distribution.{BinomialDistribution, PoissonDistribution} import org.scalatest.FunSuite class SamplingUtilsSuite extends FunSuite { + test("reservoirSampleAndCount") { + val input = Seq.fill(100)(Random.nextInt()) + + // input size < k + val (sample1, count1) = SamplingUtils.reservoirSampleAndCount(input.iterator, 150) + assert(count1 === 100) + assert(input === sample1.toSeq) + + // input size == k + val (sample2, count2) = SamplingUtils.reservoirSampleAndCount(input.iterator, 100) + assert(count2 === 100) + assert(input === sample2.toSeq) + + // input size > k + val (sample3, count3) = SamplingUtils.reservoirSampleAndCount(input.iterator, 10) + assert(count3 === 100) + assert(sample3.length === 10) + } + test("computeFraction") { // test that the computed fraction guarantees enough data points // in the sample with a failure rate <= 0.0001 diff --git a/mllib/data/als/test.data b/data/mllib/als/test.data similarity index 100% rename from mllib/data/als/test.data rename to data/mllib/als/test.data diff --git a/data/kmeans_data.txt b/data/mllib/kmeans_data.txt similarity index 100% rename from data/kmeans_data.txt rename to data/mllib/kmeans_data.txt diff --git a/mllib/data/lr-data/random.data b/data/mllib/lr-data/random.data similarity index 100% rename from mllib/data/lr-data/random.data rename to data/mllib/lr-data/random.data diff --git a/data/lr_data.txt b/data/mllib/lr_data.txt similarity index 100% rename from data/lr_data.txt rename to data/mllib/lr_data.txt diff --git a/data/pagerank_data.txt b/data/mllib/pagerank_data.txt similarity index 100% rename from data/pagerank_data.txt rename to data/mllib/pagerank_data.txt diff --git a/mllib/data/ridge-data/lpsa.data b/data/mllib/ridge-data/lpsa.data similarity index 100% rename from mllib/data/ridge-data/lpsa.data rename to data/mllib/ridge-data/lpsa.data diff --git a/mllib/data/sample_libsvm_data.txt b/data/mllib/sample_libsvm_data.txt similarity index 100% rename from mllib/data/sample_libsvm_data.txt rename to data/mllib/sample_libsvm_data.txt diff --git a/mllib/data/sample_naive_bayes_data.txt b/data/mllib/sample_naive_bayes_data.txt similarity index 100% rename from mllib/data/sample_naive_bayes_data.txt rename to data/mllib/sample_naive_bayes_data.txt diff --git a/mllib/data/sample_svm_data.txt b/data/mllib/sample_svm_data.txt similarity index 100% rename from mllib/data/sample_svm_data.txt rename to data/mllib/sample_svm_data.txt diff --git a/mllib/data/sample_tree_data.csv b/data/mllib/sample_tree_data.csv similarity index 100% rename from mllib/data/sample_tree_data.csv rename to data/mllib/sample_tree_data.csv diff --git a/dev/audit-release/audit_release.py b/dev/audit-release/audit_release.py index 230e900ecd4de..16ea1a71290dc 100755 --- a/dev/audit-release/audit_release.py +++ b/dev/audit-release/audit_release.py @@ -105,7 +105,7 @@ def get_url(url): "spark-core", "spark-bagel", "spark-mllib", "spark-streaming", "spark-repl", "spark-graphx", "spark-streaming-flume", "spark-streaming-kafka", "spark-streaming-mqtt", "spark-streaming-twitter", "spark-streaming-zeromq", - "spark-catalyst", "spark-sql", "spark-hive" + "spark-catalyst", "spark-sql", "spark-hive", "spark-streaming-kinesis-asl" ] modules = map(lambda m: "%s_%s" % (m, SCALA_BINARY_VERSION), modules) @@ -136,7 +136,7 @@ def ensure_path_not_present(x): os.chdir(original_dir) # SBT application tests -for app in ["sbt_app_core", "sbt_app_graphx", "sbt_app_streaming", "sbt_app_sql", "sbt_app_hive"]: +for app in ["sbt_app_core", "sbt_app_graphx", "sbt_app_streaming", "sbt_app_sql", "sbt_app_hive", "sbt_app_kinesis"]: os.chdir(app) ret = run_cmd("sbt clean run", exit_on_failure=False) test(ret == 0, "sbt application (%s)" % app) diff --git a/dev/audit-release/sbt_app_core/src/main/scala/SparkApp.scala b/dev/audit-release/sbt_app_core/src/main/scala/SparkApp.scala index 77bbd167b199a..fc03fec9866a6 100644 --- a/dev/audit-release/sbt_app_core/src/main/scala/SparkApp.scala +++ b/dev/audit-release/sbt_app_core/src/main/scala/SparkApp.scala @@ -50,5 +50,12 @@ object SimpleApp { println("Ganglia sink was loaded via spark-core") System.exit(-1) } + + // Remove kinesis from default build due to ASL license issue + val foundKinesis = Try(Class.forName("org.apache.spark.streaming.kinesis.KinesisUtils")).isSuccess + if (foundKinesis) { + println("Kinesis was loaded via spark-core") + System.exit(-1) + } } } diff --git a/dev/audit-release/sbt_app_hive/src/main/scala/HiveApp.scala b/dev/audit-release/sbt_app_hive/src/main/scala/HiveApp.scala index 7257d17d10116..5111bc0adb772 100644 --- a/dev/audit-release/sbt_app_hive/src/main/scala/HiveApp.scala +++ b/dev/audit-release/sbt_app_hive/src/main/scala/HiveApp.scala @@ -22,7 +22,7 @@ import scala.collection.mutable.{ListBuffer, Queue} import org.apache.spark.SparkConf import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD -import org.apache.spark.sql.hive.LocalHiveContext +import org.apache.spark.sql.hive.HiveContext case class Person(name: String, age: Int) @@ -34,13 +34,13 @@ object SparkSqlExample { case None => new SparkConf().setAppName("Simple Sql App") } val sc = new SparkContext(conf) - val hiveContext = new LocalHiveContext(sc) + val hiveContext = new HiveContext(sc) import hiveContext._ - hql("DROP TABLE IF EXISTS src") - hql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") - hql("LOAD DATA LOCAL INPATH 'data.txt' INTO TABLE src") - val results = hql("FROM src SELECT key, value WHERE key >= 0 AND KEY < 5").collect() + sql("DROP TABLE IF EXISTS src") + sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") + sql("LOAD DATA LOCAL INPATH 'data.txt' INTO TABLE src") + val results = sql("FROM src SELECT key, value WHERE key >= 0 AND KEY < 5").collect() results.foreach(println) def test(f: => Boolean, failureMsg: String) = { diff --git a/dev/audit-release/sbt_app_kinesis/build.sbt b/dev/audit-release/sbt_app_kinesis/build.sbt new file mode 100644 index 0000000000000..981bc7957b5ed --- /dev/null +++ b/dev/audit-release/sbt_app_kinesis/build.sbt @@ -0,0 +1,28 @@ +// +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. +// + +name := "Kinesis Test" + +version := "1.0" + +scalaVersion := System.getenv.get("SCALA_VERSION") + +libraryDependencies += "org.apache.spark" %% "spark-streaming-kinesis-asl" % System.getenv.get("SPARK_VERSION") + +resolvers ++= Seq( + "Spark Release Repository" at System.getenv.get("SPARK_RELEASE_REPOSITORY"), + "Spray Repository" at "http://repo.spray.cc/") diff --git a/dev/audit-release/sbt_app_kinesis/src/main/scala/SparkApp.scala b/dev/audit-release/sbt_app_kinesis/src/main/scala/SparkApp.scala new file mode 100644 index 0000000000000..9f85066501472 --- /dev/null +++ b/dev/audit-release/sbt_app_kinesis/src/main/scala/SparkApp.scala @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package main.scala + +import scala.util.Try + +import org.apache.spark.SparkContext +import org.apache.spark.SparkContext._ + +object SimpleApp { + def main(args: Array[String]) { + val foundKinesis = Try(Class.forName("org.apache.spark.streaming.kinesis.KinesisUtils")).isSuccess + if (!foundKinesis) { + println("Kinesis not loaded via kinesis-asl") + System.exit(-1) + } + } +} diff --git a/dev/audit-release/sbt_app_sql/src/main/scala/SqlApp.scala b/dev/audit-release/sbt_app_sql/src/main/scala/SqlApp.scala index 50af90c213b5a..d888de929fdda 100644 --- a/dev/audit-release/sbt_app_sql/src/main/scala/SqlApp.scala +++ b/dev/audit-release/sbt_app_sql/src/main/scala/SqlApp.scala @@ -38,7 +38,7 @@ object SparkSqlExample { import sqlContext._ val people = sc.makeRDD(1 to 100, 10).map(x => Person(s"Name$x", x)) - people.registerAsTable("people") + people.registerTempTable("people") val teenagers = sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") val teenagerNames = teenagers.map(t => "Name: " + t(0)).collect() teenagerNames.foreach(println) diff --git a/dev/check-license b/dev/check-license index fbd2dd465bb18..625ec161bc571 100755 --- a/dev/check-license +++ b/dev/check-license @@ -27,23 +27,23 @@ acquire_rat_jar () { if [[ ! -f "$rat_jar" ]]; then # Download rat launch jar if it hasn't been downloaded yet - if [ ! -f ${JAR} ]; then + if [ ! -f "$JAR" ]; then # Download printf "Attempting to fetch rat\n" JAR_DL=${JAR}.part if hash curl 2>/dev/null; then - (curl --progress-bar ${URL1} > ${JAR_DL} || curl --progress-bar ${URL2} > ${JAR_DL}) && mv ${JAR_DL} ${JAR} + (curl --progress-bar ${URL1} > "$JAR_DL" || curl --progress-bar ${URL2} > "$JAR_DL") && mv "$JAR_DL" "$JAR" elif hash wget 2>/dev/null; then - (wget --progress=bar ${URL1} -O ${JAR_DL} || wget --progress=bar ${URL2} -O ${JAR_DL}) && mv ${JAR_DL} ${JAR} + (wget --progress=bar ${URL1} -O "$JAR_DL" || wget --progress=bar ${URL2} -O "$JAR_DL") && mv "$JAR_DL" "$JAR" else printf "You do not have curl or wget installed, please install rat manually.\n" exit -1 fi fi - if [ ! -f ${JAR} ]; then - # We failed to download - printf "Our attempt to download rat locally to ${JAR} failed. Please install rat manually.\n" - exit -1 + if [ ! -f "$JAR" ]; then + # We failed to download + printf "Our attempt to download rat locally to ${JAR} failed. Please install rat manually.\n" + exit -1 fi printf "Launching rat from ${JAR}\n" fi @@ -51,7 +51,7 @@ acquire_rat_jar () { # Go to the Spark project root directory FWDIR="$(cd `dirname $0`/..; pwd)" -cd $FWDIR +cd "$FWDIR" if test -x "$JAVA_HOME/bin/java"; then declare java_cmd="$JAVA_HOME/bin/java" diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index 49bf78f60763a..28f26d2368254 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -35,6 +35,12 @@ RELEASE_VERSION=${RELEASE_VERSION:-1.0.0} RC_NAME=${RC_NAME:-rc2} USER_NAME=${USER_NAME:-pwendell} +if [ -z "$JAVA_HOME" ]; then + echo "Error: JAVA_HOME is not set, cannot proceed." + exit -1 +fi +JAVA_7_HOME=${JAVA_7_HOME:-$JAVA_HOME} + set -e GIT_TAG=v$RELEASE_VERSION-$RC_NAME @@ -53,15 +59,15 @@ if [[ ! "$@" =~ --package-only ]]; then -Dusername=$GIT_USERNAME -Dpassword=$GIT_PASSWORD \ -Dmaven.javadoc.skip=true \ -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ - -Pyarn -Phive -Phadoop-2.2 -Pspark-ganglia-lgpl\ -Dtag=$GIT_TAG -DautoVersionSubmodules=true \ + -Pyarn -Phive -Phive-thriftserver -Phadoop-2.2 -Pspark-ganglia-lgpl -Pkinesis-asl \ --batch-mode release:prepare mvn -DskipTests \ -Darguments="-DskipTests=true -Dmaven.javadoc.skip=true -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -Dgpg.passphrase=${GPG_PASSPHRASE}" \ -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ -Dmaven.javadoc.skip=true \ - -Pyarn -Phive -Phadoop-2.2 -Pspark-ganglia-lgpl\ + -Pyarn -Phive -Phive-thriftserver -Phadoop-2.2 -Pspark-ganglia-lgpl -Pkinesis-asl \ release:perform cd .. @@ -95,7 +101,7 @@ make_binary_release() { cp -r spark spark-$RELEASE_VERSION-bin-$NAME cd spark-$RELEASE_VERSION-bin-$NAME - ./make-distribution.sh $FLAGS --name $NAME --tgz + ./make-distribution.sh --name $NAME --tgz $FLAGS cd .. cp spark-$RELEASE_VERSION-bin-$NAME/spark-$RELEASE_VERSION-bin-$NAME.tgz . rm -rf spark-$RELEASE_VERSION-bin-$NAME @@ -111,9 +117,13 @@ make_binary_release() { spark-$RELEASE_VERSION-bin-$NAME.tgz.sha } -make_binary_release "hadoop1" "--with-hive --hadoop 1.0.4" -make_binary_release "cdh4" "--with-hive --hadoop 2.0.0-mr1-cdh4.2.0" -make_binary_release "hadoop2" "--with-hive --with-yarn --hadoop 2.2.0" +make_binary_release "hadoop1" "-Phive -Phive-thriftserver -Dhadoop.version=1.0.4" & +make_binary_release "cdh4" "-Phive -Phive-thriftserver -Dhadoop.version=2.0.0-mr1-cdh4.2.0" & +make_binary_release "hadoop2" \ + "-Phive -Phive-thriftserver -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 -Pyarn.version=2.2.0" & +make_binary_release "hadoop2-without-hive" \ + "-Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 -Pyarn.version=2.2.0" & +wait # Copy data echo "Copying release tarballs" @@ -127,7 +137,8 @@ scp spark-* \ cd spark sbt/sbt clean cd docs -PRODUCTION=1 jekyll build +# Compile docs with Java 7 to use nicer format +JAVA_HOME=$JAVA_7_HOME PRODUCTION=1 jekyll build echo "Copying release documentation" rc_docs_folder=${rc_folder}-docs ssh $USER_NAME@people.apache.org \ diff --git a/dev/github_jira_sync.py b/dev/github_jira_sync.py new file mode 100755 index 0000000000000..8051080117062 --- /dev/null +++ b/dev/github_jira_sync.py @@ -0,0 +1,146 @@ +#!/usr/bin/env python + +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +# Utility for updating JIRA's with information about Github pull requests + +import json +import os +import re +import sys +import urllib2 + +try: + import jira.client +except ImportError: + print "This tool requires the jira-python library" + print "Install using 'sudo pip install jira-python'" + sys.exit(-1) + +# User facing configs +GITHUB_API_BASE = os.environ.get("GITHUB_API_BASE", "https://api.github.com/repos/apache/spark") +JIRA_API_BASE = os.environ.get("JIRA_API_BASE", "https://issues.apache.org/jira") +JIRA_USERNAME = os.environ.get("JIRA_USERNAME", "apachespark") +JIRA_PASSWORD = os.environ.get("JIRA_PASSWORD", "XXX") +# Maximum number of updates to perform in one run +MAX_UPDATES = int(os.environ.get("MAX_UPDATES", "100000")) +# Cut-off for oldest PR on which to comment. Useful for avoiding +# "notification overload" when running for the first time. +MIN_COMMENT_PR = int(os.environ.get("MIN_COMMENT_PR", "1496")) + +# File used as an opitimization to store maximum previously seen PR +# Used mostly because accessing ASF JIRA is slow, so we want to avoid checking +# the state of JIRA's that are tied to PR's we've already looked at. +MAX_FILE = ".github-jira-max" + +def get_url(url): + try: + return urllib2.urlopen(url) + except urllib2.HTTPError as e: + print "Unable to fetch URL, exiting: %s" % url + sys.exit(-1) + +def get_json(urllib_response): + return json.load(urllib_response) + +# Return a list of (JIRA id, JSON dict) tuples: +# e.g. [('SPARK-1234', {.. json ..}), ('SPARK-5687', {.. json ..})} +def get_jira_prs(): + result = [] + has_next_page = True + page_num = 0 + while has_next_page: + page = get_url(GITHUB_API_BASE + "/pulls?page=%s&per_page=100" % page_num) + page_json = get_json(page) + + for pull in page_json: + jiras = re.findall("SPARK-[0-9]{4,5}", pull['title']) + for jira in jiras: + result = result + [(jira, pull)] + + # Check if there is another page + link_header = filter(lambda k: k.startswith("Link"), page.info().headers)[0] + if not "next"in link_header: + has_next_page = False + else: + page_num = page_num + 1 + return result + +def set_max_pr(max_val): + f = open(MAX_FILE, 'w') + f.write("%s" % max_val) + f.close() + print "Writing largest PR number seen: %s" % max_val + +def get_max_pr(): + if os.path.exists(MAX_FILE): + result = int(open(MAX_FILE, 'r').read()) + print "Read largest PR number previously seen: %s" % result + return result + else: + return 0 + +jira_client = jira.client.JIRA({'server': JIRA_API_BASE}, + basic_auth=(JIRA_USERNAME, JIRA_PASSWORD)) + +jira_prs = get_jira_prs() + +previous_max = get_max_pr() +print "Retrieved %s JIRA PR's from Github" % len(jira_prs) +jira_prs = [(k, v) for k, v in jira_prs if int(v['number']) > previous_max] +print "%s PR's remain after excluding visted ones" % len(jira_prs) + +num_updates = 0 +considered = [] +for issue, pr in sorted(jira_prs, key=lambda (k, v): int(v['number'])): + if num_updates >= MAX_UPDATES: + break + pr_num = int(pr['number']) + + print "Checking issue %s" % issue + considered = considered + [pr_num] + + url = pr['html_url'] + title = "[Github] Pull Request #%s (%s)" % (pr['number'], pr['user']['login']) + try: + existing_links = map(lambda l: l.raw['object']['url'], jira_client.remote_links(issue)) + except: + print "Failure reading JIRA %s (does it exist?)" % issue + print sys.exc_info()[0] + continue + + if url in existing_links: + continue + + icon = {"title": "Pull request #%s" % pr['number'], + "url16x16": "https://assets-cdn.github.com/favicon.ico"} + destination = {"title": title, "url": url, "icon": icon} + # For all possible fields see: + # https://developer.atlassian.com/display/JIRADEV/Fields+in+Remote+Issue+Links + # application = {"name": "Github pull requests", "type": "org.apache.spark.jira.github"} + jira_client.add_remote_link(issue, destination) + + comment = "User '%s' has created a pull request for this issue:" % pr['user']['login'] + comment = comment + ("\n%s" % pr['html_url']) + if pr_num >= MIN_COMMENT_PR: + jira_client.add_comment(issue, comment) + + print "Added link %s <-> PR #%s" % (issue, pr['number']) + num_updates = num_updates + 1 + +if len(considered) > 0: + set_max_pr(max(considered)) diff --git a/dev/lint-python b/dev/lint-python new file mode 100755 index 0000000000000..4efddad839387 --- /dev/null +++ b/dev/lint-python @@ -0,0 +1,60 @@ +#!/usr/bin/env bash + +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +SCRIPT_DIR="$( cd "$( dirname "$0" )" && pwd )" +SPARK_ROOT_DIR="$(dirname $SCRIPT_DIR)" +PEP8_REPORT_PATH="$SPARK_ROOT_DIR/dev/pep8-report.txt" + +cd $SPARK_ROOT_DIR + +# Get pep8 at runtime so that we don't rely on it being installed on the build server. +#+ See: https://github.com/apache/spark/pull/1744#issuecomment-50982162 +#+ TODOs: +#+ - Dynamically determine latest release version of pep8 and use that. +#+ - Download this from a more reliable source. (GitHub raw can be flaky, apparently. (?)) +PEP8_SCRIPT_PATH="$SPARK_ROOT_DIR/dev/pep8.py" +PEP8_SCRIPT_REMOTE_PATH="https://raw.githubusercontent.com/jcrocholl/pep8/1.5.7/pep8.py" + +curl --silent -o "$PEP8_SCRIPT_PATH" "$PEP8_SCRIPT_REMOTE_PATH" +curl_status=$? + +if [ $curl_status -ne 0 ]; then + echo "Failed to download pep8.py from \"$PEP8_SCRIPT_REMOTE_PATH\"." + exit $curl_status +fi + + +# There is no need to write this output to a file +#+ first, but we do so so that the check status can +#+ be output before the report, like with the +#+ scalastyle and RAT checks. +python $PEP8_SCRIPT_PATH ./python > "$PEP8_REPORT_PATH" +pep8_status=${PIPESTATUS[0]} #$? + +if [ $pep8_status -ne 0 ]; then + echo "PEP 8 checks failed." + cat "$PEP8_REPORT_PATH" +else + echo "PEP 8 checks passed." +fi + +rm -f "$PEP8_REPORT_PATH" +rm "$PEP8_SCRIPT_PATH" + +exit $pep8_status diff --git a/dev/lint-scala b/dev/lint-scala new file mode 100755 index 0000000000000..c676dfdf4f44e --- /dev/null +++ b/dev/lint-scala @@ -0,0 +1,23 @@ +#!/usr/bin/env bash + +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +SCRIPT_DIR="$( cd "$( dirname "$0" )" && pwd )" +SPARK_ROOT_DIR="$(dirname $SCRIPT_DIR)" + +"$SCRIPT_DIR/scalastyle" diff --git a/dev/merge_spark_pr.py b/dev/merge_spark_pr.py index c44320239bbbf..d48c8bde12905 100755 --- a/dev/merge_spark_pr.py +++ b/dev/merge_spark_pr.py @@ -29,7 +29,6 @@ import re import subprocess import sys -import tempfile import urllib2 try: @@ -39,15 +38,15 @@ JIRA_IMPORTED = False # Location of your Spark git development area -SPARK_HOME = os.environ.get("SPARK_HOME", "/home/patrick/Documents/spark") +SPARK_HOME = os.environ.get("SPARK_HOME", os.getcwd()) # Remote name which points to the Gihub site PR_REMOTE_NAME = os.environ.get("PR_REMOTE_NAME", "apache-github") # Remote name which points to Apache git PUSH_REMOTE_NAME = os.environ.get("PUSH_REMOTE_NAME", "apache") # ASF JIRA username -JIRA_USERNAME = os.environ.get("JIRA_USERNAME", "pwendell") +JIRA_USERNAME = os.environ.get("JIRA_USERNAME", "") # ASF JIRA password -JIRA_PASSWORD = os.environ.get("JIRA_PASSWORD", "1234") +JIRA_PASSWORD = os.environ.get("JIRA_PASSWORD", "") GITHUB_BASE = "https://github.com/apache/spark/pull" GITHUB_API_BASE = "https://api.github.com/repos/apache/spark" @@ -75,8 +74,10 @@ def fail(msg): def run_cmd(cmd): if isinstance(cmd, list): + print " ".join(cmd) return subprocess.check_output(cmd) else: + print cmd return subprocess.check_output(cmd.split(" ")) @@ -129,7 +130,7 @@ def merge_pr(pr_num, target_ref): merge_message_flags = [] merge_message_flags += ["-m", title] - if body != None: + if body is not None: # We remove @ symbols from the body to avoid triggering e-mails # to people every time someone creates a public fork of Spark. merge_message_flags += ["-m", body.replace("@", "")] @@ -179,7 +180,14 @@ def cherry_pick(pr_num, merge_hash, default_branch): run_cmd("git fetch %s %s:%s" % (PUSH_REMOTE_NAME, pick_ref, pick_branch_name)) run_cmd("git checkout %s" % pick_branch_name) - run_cmd("git cherry-pick -sx %s" % merge_hash) + + try: + run_cmd("git cherry-pick -sx %s" % merge_hash) + except Exception as e: + msg = "Error cherry-picking: %s\nWould you like to manually fix-up this merge?" % e + continue_maybe(msg) + msg = "Okay, please fix any conflicts and finish the cherry-pick. Finished?" + continue_maybe(msg) continue_maybe("Pick complete (local ref %s). Push to %s?" % ( pick_branch_name, PUSH_REMOTE_NAME)) @@ -280,6 +288,7 @@ def get_version_json(version_str): pr_num = raw_input("Which pull request would you like to merge? (e.g. 34): ") pr = get_json("%s/pulls/%s" % (GITHUB_API_BASE, pr_num)) +pr_events = get_json("%s/issues/%s/events" % (GITHUB_API_BASE, pr_num)) url = pr["url"] title = pr["title"] @@ -289,19 +298,23 @@ def get_version_json(version_str): base_ref = pr["head"]["ref"] pr_repo_desc = "%s/%s" % (user_login, base_ref) -if pr["merged"] is True: +# Merged pull requests don't appear as merged in the GitHub API; +# Instead, they're closed by asfgit. +merge_commits = \ + [e for e in pr_events if e["actor"]["login"] == "asfgit" and e["event"] == "closed"] + +if merge_commits: + merge_hash = merge_commits[0]["commit_id"] + message = get_json("%s/commits/%s" % (GITHUB_API_BASE, merge_hash))["commit"]["message"] + print "Pull request %s has already been merged, assuming you want to backport" % pr_num - merge_commit_desc = run_cmd([ - 'git', 'log', '--merges', '--first-parent', - '--grep=pull request #%s' % pr_num, '--oneline']).split("\n")[0] - if merge_commit_desc == "": + commit_is_downloaded = run_cmd(['git', 'rev-parse', '--quiet', '--verify', + "%s^{commit}" % merge_hash]).strip() != "" + if not commit_is_downloaded: fail("Couldn't find any merge commit for #%s, you may need to update HEAD." % pr_num) - merge_hash = merge_commit_desc[:7] - message = merge_commit_desc[8:] - - print "Found: %s" % message - maybe_cherry_pick(pr_num, merge_hash, latest_branch) + print "Found commit %s:\n%s" % (merge_hash, message) + cherry_pick(pr_num, merge_hash, latest_branch) sys.exit(0) if not bool(pr["mergeable"]): @@ -323,9 +336,13 @@ def get_version_json(version_str): merged_refs = merged_refs + [cherry_pick(pr_num, merge_hash, latest_branch)] if JIRA_IMPORTED: - continue_maybe("Would you like to update an associated JIRA?") - jira_comment = "Issue resolved by pull request %s\n[%s/%s]" % (pr_num, GITHUB_BASE, pr_num) - resolve_jira(title, merged_refs, jira_comment) + if JIRA_USERNAME and JIRA_PASSWORD: + continue_maybe("Would you like to update an associated JIRA?") + jira_comment = "Issue resolved by pull request %s\n[%s/%s]" % (pr_num, GITHUB_BASE, pr_num) + resolve_jira(title, merged_refs, jira_comment) + else: + print "JIRA_USERNAME and JIRA_PASSWORD not set" + print "Exiting without trying to close the associated JIRA." else: print "Could not find jira-python library. Run 'sudo pip install jira-python' to install." print "Exiting without trying to close the associated JIRA." diff --git a/dev/mima b/dev/mima index 7857294f61caf..09e4482af5f3d 100755 --- a/dev/mima +++ b/dev/mima @@ -22,11 +22,13 @@ set -e # Go to the Spark project root directory FWDIR="$(cd `dirname $0`/..; pwd)" -cd $FWDIR +cd "$FWDIR" echo -e "q\n" | sbt/sbt oldDeps/update -export SPARK_CLASSPATH=`find lib_managed \( -name '*spark*jar' -a -type f \) -printf "%p:" ` +export SPARK_CLASSPATH=`find lib_managed \( -name '*spark*jar' -a -type f \) | tr "\\n" ":"` +echo "SPARK_CLASSPATH=$SPARK_CLASSPATH" + ./bin/spark-class org.apache.spark.tools.GenerateMIMAIgnore echo -e "q\n" | sbt/sbt mima-report-binary-issues | grep -v -e "info.*Resolving" ret_val=$? diff --git a/dev/run-tests b/dev/run-tests index d9df020f7563c..0e24515d1376c 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -19,10 +19,27 @@ # Go to the Spark project root directory FWDIR="$(cd `dirname $0`/..; pwd)" -cd $FWDIR +cd "$FWDIR" -export SPARK_HADOOP_VERSION=2.3.0 -export SPARK_YARN=true +if [ -n "$AMPLAB_JENKINS_BUILD_PROFILE" ]; then + if [ "$AMPLAB_JENKINS_BUILD_PROFILE" = "hadoop1.0" ]; then + export SBT_MAVEN_PROFILES_ARGS="-Dhadoop.version=1.0.4" + elif [ "$AMPLAB_JENKINS_BUILD_PROFILE" = "hadoop2.0" ]; then + export SBT_MAVEN_PROFILES_ARGS="-Dhadoop.version=2.0.0-mr1-cdh4.1.1" + elif [ "$AMPLAB_JENKINS_BUILD_PROFILE" = "hadoop2.2" ]; then + export SBT_MAVEN_PROFILES_ARGS="-Pyarn -Dhadoop.version=2.2.0" + elif [ "$AMPLAB_JENKINS_BUILD_PROFILE" = "hadoop2.3" ]; then + export SBT_MAVEN_PROFILES_ARGS="-Pyarn -Phadoop-2.3 -Dhadoop.version=2.3.0" + fi +fi + +if [ -z "$SBT_MAVEN_PROFILES_ARGS" ]; then + export SBT_MAVEN_PROFILES_ARGS="-Pyarn -Phadoop-2.3 -Dhadoop.version=2.3.0" +fi + +export SBT_MAVEN_PROFILES_ARGS="$SBT_MAVEN_PROFILES_ARGS -Pkinesis-asl" + +echo "SBT_MAVEN_PROFILES_ARGS=\"$SBT_MAVEN_PROFILES_ARGS\"" # Remove work directory rm -rf ./work @@ -38,7 +55,7 @@ JAVA_VERSION=$($java_cmd -version 2>&1 | sed 's/java version "\(.*\)\.\(.*\)\..* # Partial solution for SPARK-1455. Only run Hive tests if there are sql changes. if [ -n "$AMPLAB_JENKINS" ]; then git fetch origin master:master - diffs=`git diff --dirstat master | awk '{ print $2; }' | grep "^sql/"` + diffs=`git diff --name-only master | grep "^sql/"` if [ -n "$diffs" ]; then echo "Detected changes in SQL. Will run Hive test suite." export _RUN_SQL_TESTS=true # exported for PySpark tests @@ -49,35 +66,45 @@ fi set -e set -o pipefail +echo "" echo "=========================================================================" echo "Running Apache RAT checks" echo "=========================================================================" dev/check-license +echo "" echo "=========================================================================" echo "Running Scala style checks" echo "=========================================================================" -dev/scalastyle +dev/lint-scala + +echo "" +echo "=========================================================================" +echo "Running Python style checks" +echo "=========================================================================" +dev/lint-python +echo "" echo "=========================================================================" echo "Running Spark unit tests" echo "=========================================================================" + +if [ -n "$_RUN_SQL_TESTS" ]; then + SBT_MAVEN_PROFILES_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive -Phive-thriftserver" +fi # echo "q" is needed because sbt on encountering a build file with failure # (either resolution or compilation) prompts the user for input either q, r, # etc to quit or retry. This echo is there to make it not block. -if [ -n "$_RUN_SQL_TESTS" ]; then - echo -e "q\n" | SPARK_HIVE=true sbt/sbt clean assembly test | \ - grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" -else - echo -e "q\n" | sbt/sbt clean assembly test | \ - grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" -fi +echo -e "q\n" | sbt/sbt $SBT_MAVEN_PROFILES_ARGS clean package assembly/assembly test | \ + grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" +echo "" echo "=========================================================================" echo "Running PySpark tests" echo "=========================================================================" ./python/run-tests +echo "" echo "=========================================================================" echo "Detecting binary incompatibilites with MiMa" echo "=========================================================================" diff --git a/dev/run-tests-jenkins b/dev/run-tests-jenkins new file mode 100755 index 0000000000000..31506e28e05af --- /dev/null +++ b/dev/run-tests-jenkins @@ -0,0 +1,166 @@ +#!/usr/bin/env bash + +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Wrapper script that runs the Spark tests then reports QA results +# to github via its API. +# Environment variables are populated by the code here: +#+ https://github.com/jenkinsci/ghprb-plugin/blob/master/src/main/java/org/jenkinsci/plugins/ghprb/GhprbTrigger.java#L139 + +# Go to the Spark project root directory +FWDIR="$(cd `dirname $0`/..; pwd)" +cd "$FWDIR" + +COMMENTS_URL="https://api.github.com/repos/apache/spark/issues/$ghprbPullId/comments" +PULL_REQUEST_URL="https://github.com/apache/spark/pull/$ghprbPullId" + +COMMIT_URL="https://github.com/apache/spark/commit/${ghprbActualCommit}" +# GitHub doesn't auto-link short hashes when submitted via the API, unfortunately. :( +SHORT_COMMIT_HASH="${ghprbActualCommit:0:7}" + +# NOTE: Jenkins will kill the whole build after 120 minutes. +# Tests are a large part of that, but not all of it. +TESTS_TIMEOUT="120m" + +function post_message () { + local message=$1 + local data="{\"body\": \"$message\"}" + local HTTP_CODE_HEADER="HTTP Response Code: " + + echo "Attempting to post to Github..." + + local curl_output=$( + curl `#--dump-header -` \ + --silent \ + --user x-oauth-basic:$GITHUB_OAUTH_KEY \ + --request POST \ + --data "$data" \ + --write-out "${HTTP_CODE_HEADER}%{http_code}\n" \ + --header "Content-Type: application/json" \ + "$COMMENTS_URL" #> /dev/null #| "$FWDIR/dev/jq" .id #| head -n 8 + ) + local curl_status=${PIPESTATUS[0]} + + if [ "$curl_status" -ne 0 ]; then + echo "Failed to post message to GitHub." >&2 + echo " > curl_status: ${curl_status}" >&2 + echo " > curl_output: ${curl_output}" >&2 + echo " > data: ${data}" >&2 + # exit $curl_status + fi + + local api_response=$( + echo "${curl_output}" \ + | grep -v -e "^${HTTP_CODE_HEADER}" + ) + + local http_code=$( + echo "${curl_output}" \ + | grep -e "^${HTTP_CODE_HEADER}" \ + | sed -r -e "s/^${HTTP_CODE_HEADER}//g" + ) + + if [ -n "$http_code" ] && [ "$http_code" -ne "201" ]; then + echo " > http_code: ${http_code}." >&2 + echo " > api_response: ${api_response}" >&2 + echo " > data: ${data}" >&2 + fi + + if [ "$curl_status" -eq 0 ] && [ "$http_code" -eq "201" ]; then + echo " > Post successful." + fi +} + +# check PR merge-ability and check for new public classes +{ + if [ "$sha1" == "$ghprbActualCommit" ]; then + merge_note=" * This patch **does not** merge cleanly!" + else + merge_note=" * This patch merges cleanly." + + non_test_files=$(git diff master --name-only | grep -v "\/test" | tr "\n" " ") + new_public_classes=$( + git diff master ${non_test_files} `# diff this patch against master and...` \ + | grep "^\+" `# filter in only added lines` \ + | sed -r -e "s/^\+//g" `# remove the leading +` \ + | grep -e "trait " -e "class " `# filter in lines with these key words` \ + | grep -e "{" -e "(" `# filter in lines with these key words, too` \ + | grep -v -e "\@\@" -e "private" `# exclude lines with these words` \ + | grep -v -e "^// " -e "^/\*" -e "^ \* " `# exclude comment lines` \ + | sed -r -e "s/\{.*//g" `# remove from the { onwards` \ + | sed -r -e "s/\}//g" `# just in case, remove }; they mess the JSON` \ + | sed -r -e "s/\"/\\\\\"/g" `# escape double quotes; they mess the JSON` \ + | sed -r -e "s/^(.*)$/\`\1\`/g" `# surround with backticks for style` \ + | sed -r -e "s/^/ \* /g" `# prepend ' *' to start of line` \ + | sed -r -e "s/$/\\\n/g" `# append newline to end of line` \ + | tr -d "\n" `# remove actual LF characters` + ) + + if [ "$new_public_classes" == "" ]; then + public_classes_note=" * This patch adds no public classes." + else + public_classes_note=" * This patch adds the following public classes _(experimental)_:" + public_classes_note="${public_classes_note}\n${new_public_classes}" + fi + fi +} + +# post start message +{ + start_message="\ + [QA tests have started](${BUILD_URL}consoleFull) for \ + PR $ghprbPullId at commit [\`${SHORT_COMMIT_HASH}\`](${COMMIT_URL})." + + start_message="${start_message}\n${merge_note}" + # start_message="${start_message}\n${public_classes_note}" + + post_message "$start_message" +} + +# run tests +{ + timeout "${TESTS_TIMEOUT}" ./dev/run-tests + test_result="$?" + + if [ "$test_result" -eq "124" ]; then + fail_message="**Tests timed out** after a configured wait of \`${TESTS_TIMEOUT}\`." + post_message "$fail_message" + exit $test_result + else + if [ "$test_result" -eq "0" ]; then + test_result_note=" * This patch **passes** unit tests." + else + test_result_note=" * This patch **fails** unit tests." + fi + fi +} + +# post end message +{ + result_message="\ + [QA tests have finished](${BUILD_URL}consoleFull) for \ + PR $ghprbPullId at commit [\`${SHORT_COMMIT_HASH}\`](${COMMIT_URL})." + + result_message="${result_message}\n${test_result_note}" + result_message="${result_message}\n${merge_note}" + result_message="${result_message}\n${public_classes_note}" + + post_message "$result_message" +} + +exit $test_result diff --git a/dev/scalastyle b/dev/scalastyle index 0e8fd5cc8d64c..b53053a04ff42 100755 --- a/dev/scalastyle +++ b/dev/scalastyle @@ -17,12 +17,12 @@ # limitations under the License. # -echo -e "q\n" | SPARK_HIVE=true sbt/sbt scalastyle > scalastyle.txt +echo -e "q\n" | sbt/sbt -Phive -Phive-thriftserver scalastyle > scalastyle.txt # Check style with YARN alpha built too -echo -e "q\n" | SPARK_HADOOP_VERSION=0.23.9 SPARK_YARN=true sbt/sbt yarn-alpha/scalastyle \ +echo -e "q\n" | sbt/sbt -Pyarn -Phadoop-0.23 -Dhadoop.version=0.23.9 yarn-alpha/scalastyle \ >> scalastyle.txt # Check style with YARN built too -echo -e "q\n" | SPARK_HADOOP_VERSION=2.2.0 SPARK_YARN=true sbt/sbt yarn/scalastyle \ +echo -e "q\n" | sbt/sbt -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 yarn/scalastyle \ >> scalastyle.txt ERRORS=$(cat scalastyle.txt | grep -e "\") @@ -30,5 +30,5 @@ if test ! -z "$ERRORS"; then echo -e "Scalastyle checks failed at following occurrences:\n$ERRORS" exit 1 else - echo -e "Scalastyle checks passed.\n" + echo -e "Scalastyle checks passed." fi diff --git a/docs/bagel-programming-guide.md b/docs/bagel-programming-guide.md index b280df0c8eeb8..7e55131754a3f 100644 --- a/docs/bagel-programming-guide.md +++ b/docs/bagel-programming-guide.md @@ -46,7 +46,7 @@ import org.apache.spark.bagel.Bagel._ Next, we load a sample graph from a text file as a distributed dataset and package it into `PRVertex` objects. We also cache the distributed dataset because Bagel will use it multiple times and we'd like to avoid recomputing it. {% highlight scala %} -val input = sc.textFile("data/pagerank_data.txt") +val input = sc.textFile("data/mllib/pagerank_data.txt") val numVerts = input.count() diff --git a/docs/building-with-maven.md b/docs/building-with-maven.md index 55a9e37dfed83..4d87ab92cec5b 100644 --- a/docs/building-with-maven.md +++ b/docs/building-with-maven.md @@ -96,9 +96,23 @@ mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=2.4.0 -DskipTests clean package mvn -Pyarn-alpha -Phadoop-2.3 -Dhadoop.version=2.3.0 -Dyarn.version=0.23.7 -DskipTests clean package {% endhighlight %} +# Building Thrift JDBC server and CLI for Spark SQL + +Spark SQL supports Thrift JDBC server and CLI. +See sql-programming-guide.md for more information about those features. +You can use those features by setting `-Phive-thriftserver` when building Spark as follows. +{% highlight bash %} +mvn -Phive-thriftserver assembly +{% endhighlight %} + # Spark Tests in Maven -Tests are run by default via the [ScalaTest Maven plugin](http://www.scalatest.org/user_guide/using_the_scalatest_maven_plugin). Some of the require Spark to be packaged first, so always run `mvn package` with `-DskipTests` the first time. You can then run the tests with `mvn -Dhadoop.version=... test`. +Tests are run by default via the [ScalaTest Maven plugin](http://www.scalatest.org/user_guide/using_the_scalatest_maven_plugin). + +Some of the tests require Spark to be packaged first, so always run `mvn package` with `-DskipTests` the first time. The following is an example of a correct (build, test) sequence: + + mvn -Pyarn -Phadoop-2.3 -DskipTests -Phive clean package + mvn -Pyarn -Phadoop-2.3 -Phive test The ScalaTest plugin also supports running only a specific test suite as follows: diff --git a/docs/configuration.md b/docs/configuration.md index b84104cc7e653..981170d8b49b7 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -42,13 +42,15 @@ val sc = new SparkContext(new SparkConf()) Then, you can supply configuration values at runtime: {% highlight bash %} -./bin/spark-submit --name "My fancy app" --master local[4] myApp.jar +./bin/spark-submit --name "My app" --master local[4] --conf spark.shuffle.spill=false + --conf "spark.executor.extraJavaOptions=-XX:+PrintGCDetails -XX:+PrintGCTimeStamps" myApp.jar {% endhighlight %} The Spark shell and [`spark-submit`](cluster-overview.html#launching-applications-with-spark-submit) tool support two ways to load configurations dynamically. The first are command line options, -such as `--master`, as shown above. Running `./bin/spark-submit --help` will show the entire list -of options. +such as `--master`, as shown above. `spark-submit` can accept any Spark property using the `--conf` +flag, but uses special flags for properties that play a part in launching the Spark application. +Running `./bin/spark-submit --help` will show the entire list of these options. `bin/spark-submit` will also read configuration options from `conf/spark-defaults.conf`, in which each line consists of a key and a value separated by whitespace. For example: @@ -195,6 +197,23 @@ Apart from these, the following properties are also available, and may be useful Spark's dependencies and user dependencies. It is currently an experimental feature. + + spark.python.worker.memory + 512m + + Amount of memory to use per python worker process during aggregation, in the same + format as JVM memory strings (e.g. 512m, 2g). If the memory + used during aggregation goes above this amount, it will spill the data into disks. + + + + spark.executorEnv.[EnvironmentVariableName] + (none) + + Add the environment variable specified by EnvironmentVariableName to the Executor + process. The user can specify multiple of these and to set multiple environment variables. + + #### Shuffle Behavior @@ -228,7 +247,7 @@ Apart from these, the following properties are also available, and may be useful spark.shuffle.memoryFraction - 0.3 + 0.2 Fraction of Java heap to use for aggregation and cogroups during shuffles, if spark.shuffle.spill is true. At any given time, the collective size of @@ -247,7 +266,7 @@ Apart from these, the following properties are also available, and may be useful spark.shuffle.file.buffer.kb - 100 + 32 Size of the in-memory buffer for each shuffle file output stream, in kilobytes. These buffers reduce the number of disk seeks and system calls made in creating intermediate shuffle files. @@ -262,6 +281,24 @@ Apart from these, the following properties are also available, and may be useful overhead per reduce task, so keep it small unless you have a large amount of memory. + + spark.shuffle.manager + HASH + + Implementation to use for shuffling data. A hash-based shuffle manager is the default, but + starting in Spark 1.1 there is an experimental sort-based shuffle manager that is more + memory-efficient in environments with small executors, such as YARN. To use that, change + this value to SORT. + + + + spark.shuffle.sort.bypassMergeThreshold + 200 + + (Advanced) In the sort-based shuffle manager, avoid merge-sorting data if there is no + map-side aggregation and there are at most this many reduce partitions. + + #### Spark UI @@ -336,13 +373,14 @@ Apart from these, the following properties are also available, and may be useful spark.io.compression.codec - org.apache.spark.io.
    LZFCompressionCodec + snappy - The codec used to compress internal data such as RDD partitions and shuffle outputs. - By default, Spark provides two codecs: org.apache.spark.io.LZFCompressionCodec - and org.apache.spark.io.SnappyCompressionCodec. Of these two choices, - Snappy offers faster compression and decompression, while LZF offers a better compression - ratio. + The codec used to compress internal data such as RDD partitions and shuffle outputs. By default, + Spark provides three codecs: lz4, lzf, and snappy. You + can also use fully qualified class names to specify the codec, e.g. + org.apache.spark.io.LZ4CompressionCodec, + org.apache.spark.io.LZFCompressionCodec, + and org.apache.spark.io.SnappyCompressionCodec. @@ -350,7 +388,15 @@ Apart from these, the following properties are also available, and may be useful 32768 Block size (in bytes) used in Snappy compression, in the case when Snappy compression codec - is used. + is used. Lowering this block size will also lower shuffle memory usage when Snappy is used. + + + + spark.io.compression.lz4.block.size + 32768 + + Block size (in bytes) used in LZ4 compression, in the case when LZ4 compression codec + is used. Lowering this block size will also lower shuffle memory usage when LZ4 is used. @@ -362,13 +408,13 @@ Apart from these, the following properties are also available, and may be useful spark.serializer.objectStreamReset - 10000 + 100 When serializing using org.apache.spark.serializer.JavaSerializer, the serializer caches objects to prevent writing redundant data, however that stops garbage collection of those objects. By calling 'reset' you flush that info from the serializer, and allow old - objects to be collected. To turn off this periodic reset set it to a value <= 0. - By default it will reset the serializer every 10,000 objects. + objects to be collected. To turn off this periodic reset set it to -1. + By default it will reset the serializer every 100 objects. @@ -381,14 +427,33 @@ Apart from these, the following properties are also available, and may be useful case. + + spark.kryo.registrationRequired + false + + Whether to require registration with Kryo. If set to 'true', Kryo will throw an exception + if an unregistered class is serialized. If set to false (the default), Kryo will write + unregistered class names along with each object. Writing class names can cause + significant performance overhead, so enabling this option can enforce strictly that a + user has not omitted classes from registration. + + spark.kryoserializer.buffer.mb - 2 + 0.064 - Maximum object size to allow within Kryo (the library needs to create a buffer at least as - large as the largest single object you'll serialize). Increase this if you get a "buffer limit - exceeded" exception inside Kryo. Note that there will be one buffer per core on each - worker. + Initial size of Kryo's serialization buffer, in megabytes. Note that there will be one buffer + per core on each worker. This buffer will grow up to + spark.kryoserializer.buffer.max.mb if needed. + + + + spark.kryoserializer.buffer.max.mb + 64 + + Maximum allowable size of Kryo serialization buffer, in megabytes. This must be larger than any + object you attempt to serialize. Increase this if you get a "buffer limit exceeded" exception + inside Kryo. @@ -412,7 +477,7 @@ Apart from these, the following properties are also available, and may be useful spark.broadcast.factory - org.apache.spark.broadcast.
    HttpBroadcastFactory + org.apache.spark.broadcast.
    TorrentBroadcastFactory Which broadcast implementation to use. @@ -451,6 +516,15 @@ Apart from these, the following properties are also available, and may be useful increase it if you configure your own old generation size. + + spark.storage.unrollFraction + 0.2 + + Fraction of spark.storage.memoryFraction to use for unrolling blocks in memory. + This is dynamically allocated by dropping existing blocks when there is not enough free + storage space to unroll the new block in its entirety. + + spark.tachyonStore.baseDir System.getProperty("java.io.tmpdir") @@ -488,13 +562,20 @@ Apart from these, the following properties are also available, and may be useful - spark.hadoop.validateOutputSpecs + spark.hadoop.validateOutputSpecs true - If set to true, validates the output specification (e.g. checking if the output directory already exists) - used in saveAsHadoopFile and other variants. This can be disabled to silence exceptions due to pre-existing - output directories. We recommend that users do not disable this except if trying to achieve compatibility with + If set to true, validates the output specification (e.g. checking if the output directory already exists) + used in saveAsHadoopFile and other variants. This can be disabled to silence exceptions due to pre-existing + output directories. We recommend that users do not disable this except if trying to achieve compatibility with previous versions of Spark. Simply use Hadoop's FileSystem API to delete output directories by hand. + + spark.executor.heartbeatInterval + 10000 + Interval (milliseconds) between each executor's heartbeats to the driver. Heartbeats let + the driver know that the executor is still alive and update it with metrics for in-progress + tasks. + #### Networking @@ -505,6 +586,7 @@ Apart from these, the following properties are also available, and may be useful (local hostname) Hostname or IP address for the driver to listen on. + This is used for communicating with the executors and the standalone Master. @@ -512,6 +594,51 @@ Apart from these, the following properties are also available, and may be useful (random) Port for the driver to listen on. + This is used for communicating with the executors and the standalone Master. + + + + spark.fileserver.port + (random) + + Port for the driver's HTTP file server to listen on. + + + + spark.broadcast.port + (random) + + Port for the driver's HTTP broadcast server to listen on. + This is not relevant for torrent broadcast. + + + + spark.replClassServer.port + (random) + + Port for the driver's HTTP class server to listen on. + This is only relevant for the Spark shell. + + + + spark.blockManager.port + (random) + + Port for all block managers to listen on. These exist on both the driver and the executors. + + + + spark.executor.port + (random) + + Port for the executor to listen on. This is used for communicating with the driver. + + + + spark.port.maxRetries + 16 + + Maximum number of retries when binding to a port before giving up. @@ -699,6 +826,35 @@ Apart from these, the following properties are also available, and may be useful (in milliseconds) + + spark.scheduler.minRegisteredResourcesRatio + 0 + + The minimum ratio of registered resources (registered resources / total expected resources) + (resources are executors in yarn mode, CPU cores in standalone mode) + to wait for before scheduling begins. Specified as a double between 0 and 1. + Regardless of whether the minimum ratio of resources has been reached, + the maximum amount of time it will wait before scheduling begins is controlled by config + spark.scheduler.maxRegisteredResourcesWaitingTime + + + + spark.scheduler.maxRegisteredResourcesWaitingTime + 30000 + + Maximum amount of time to wait for resources to register before scheduling begins + (in milliseconds). + + + + spark.localExecution.enabled + false + + Enables Spark to run certain jobs, such as first() or take() on the driver, without sending + tasks to the cluster. This can make certain jobs execute very quickly, but may require + shipping a whole partition of data to the driver. + + #### Security @@ -728,6 +884,15 @@ Apart from these, the following properties are also available, and may be useful out and giving up. + + spark.core.connection.ack.wait.timeout + 60 + + Number of seconds for the connection to wait for ack to occur before timing + out and giving up. To avoid unwilling timeout caused by long pause like GC, + you can set larger value. + + spark.ui.filters None @@ -743,13 +908,13 @@ Apart from these, the following properties are also available, and may be useful - spark.ui.acls.enable + spark.acls.enable false - Whether Spark web ui acls should are enabled. If enabled, this checks to see if the user has - access permissions to view the web ui. See spark.ui.view.acls for more details. - Also note this requires the user to be known, if the user comes across as null no checks - are done. Filters can be used to authenticate and set the user. + Whether Spark acls should are enabled. If enabled, this checks to see if the user has + access permissions to view or modify the job. Note this requires the user to be known, + so if the user comes across as null no checks are done. Filters can be used with the UI + to authenticate and set the user. @@ -760,6 +925,23 @@ Apart from these, the following properties are also available, and may be useful user that started the Spark job has view access. + + spark.modify.acls + Empty + + Comma separated list of users that have modify access to the Spark job. By default only the + user that started the Spark job has access to modify it (kill it for example). + + + + spark.admin.acls + Empty + + Comma separated list of users/administrators that have view and modify access to all Spark jobs. + This can be used if you run on a shared cluster and have a set of administrators or devs who + help debug when things work. + + #### Spark Streaming @@ -773,6 +955,15 @@ Apart from these, the following properties are also available, and may be useful into blocks of data before storing them in Spark. + + spark.streaming.receiver.maxRate + infinite + + Maximum rate (per second) at which each receiver will push data into blocks. Effectively, + each stream will consume at most this number of records per second. + Setting this configuration to 0 or a negative number will put no limit on the rate. + + spark.streaming.unpersist true @@ -826,7 +1017,7 @@ Apart from these, the following properties are also available, and may be useful #### Cluster Managers -Each cluster manager in Spark has additional configuration options. Configurations +Each cluster manager in Spark has additional configuration options. Configurations can be found on the pages for each mode: * [YARN](running-on-yarn.html#configuration) diff --git a/docs/hadoop-third-party-distributions.md b/docs/hadoop-third-party-distributions.md index 32403bc6957a2..ab1023b8f1842 100644 --- a/docs/hadoop-third-party-distributions.md +++ b/docs/hadoop-third-party-distributions.md @@ -48,9 +48,9 @@ the _exact_ Hadoop version you are running to avoid any compatibility errors. -In SBT, the equivalent can be achieved by setting the SPARK_HADOOP_VERSION flag: +In SBT, the equivalent can be achieved by setting the the `hadoop.version` property: - SPARK_HADOOP_VERSION=1.0.4 sbt/sbt assembly + sbt/sbt -Dhadoop.version=1.0.4 assembly # Linking Applications to the Hadoop Version diff --git a/docs/mllib-basics.md b/docs/mllib-basics.md index 5796e16e8f99c..8752df412950a 100644 --- a/docs/mllib-basics.md +++ b/docs/mllib-basics.md @@ -9,17 +9,17 @@ displayTitle: MLlib - Basics MLlib supports local vectors and matrices stored on a single machine, as well as distributed matrices backed by one or more RDDs. -In the current implementation, local vectors and matrices are simple data models -to serve public interfaces. The underlying linear algebra operations are provided by +Local vectors and local matrices are simple data models +that serve as public interfaces. The underlying linear algebra operations are provided by [Breeze](http://www.scalanlp.org/) and [jblas](http://jblas.org/). -A training example used in supervised learning is called "labeled point" in MLlib. +A training example used in supervised learning is called a "labeled point" in MLlib. ## Local vector A local vector has integer-typed and 0-based indices and double-typed values, stored on a single machine. MLlib supports two types of local vectors: dense and sparse. A dense vector is backed by a double array representing its entry values, while a sparse vector is backed by two parallel -arrays: indices and values. For example, a vector $(1.0, 0.0, 3.0)$ can be represented in dense +arrays: indices and values. For example, a vector `(1.0, 0.0, 3.0)` can be represented in dense format as `[1.0, 0.0, 3.0]` or in sparse format as `(3, [0, 2], [1.0, 3.0])`, where `3` is the size of the vector. @@ -44,8 +44,7 @@ val sv1: Vector = Vectors.sparse(3, Array(0, 2), Array(1.0, 3.0)) val sv2: Vector = Vectors.sparse(3, Seq((0, 1.0), (2, 3.0))) {% endhighlight %} -***Note*** - +***Note:*** Scala imports `scala.collection.immutable.Vector` by default, so you have to import `org.apache.spark.mllib.linalg.Vector` explicitly to use MLlib's `Vector`. @@ -110,8 +109,8 @@ sv2 = sps.csc_matrix((np.array([1.0, 3.0]), np.array([0, 2]), np.array([0, 2])), A labeled point is a local vector, either dense or sparse, associated with a label/response. In MLlib, labeled points are used in supervised learning algorithms. We use a double to store a label, so we can use labeled points in both regression and classification. -For binary classification, label should be either $0$ (negative) or $1$ (positive). -For multiclass classification, labels should be class indices staring from zero: $0, 1, 2, \ldots$. +For binary classification, a label should be either `0` (negative) or `1` (positive). +For multiclass classification, labels should be class indices starting from zero: `0, 1, 2, ...`.
    @@ -172,7 +171,7 @@ neg = LabeledPoint(0.0, SparseVector(3, [0, 2], [1.0, 3.0])) It is very common in practice to have sparse training data. MLlib supports reading training examples stored in `LIBSVM` format, which is the default format used by [`LIBSVM`](http://www.csie.ntu.edu.tw/~cjlin/libsvm/) and -[`LIBLINEAR`](http://www.csie.ntu.edu.tw/~cjlin/liblinear/). It is a text format. Each line +[`LIBLINEAR`](http://www.csie.ntu.edu.tw/~cjlin/liblinear/). It is a text format in which each line represents a labeled sparse feature vector using the following format: ~~~ @@ -193,7 +192,7 @@ import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD -val examples: RDD[LabeledPoint] = MLUtils.loadLibSVMFile(sc, "mllib/data/sample_libsvm_data.txt") +val examples: RDD[LabeledPoint] = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") {% endhighlight %}
    @@ -207,7 +206,7 @@ import org.apache.spark.mllib.util.MLUtils; import org.apache.spark.api.java.JavaRDD; JavaRDD examples = - MLUtils.loadLibSVMFile(jsc.sc(), "mllib/data/sample_libsvm_data.txt").toJavaRDD(); + MLUtils.loadLibSVMFile(jsc.sc(), "data/mllib/sample_libsvm_data.txt").toJavaRDD(); {% endhighlight %} @@ -218,7 +217,7 @@ examples stored in LIBSVM format. {% highlight python %} from pyspark.mllib.util import MLUtils -examples = MLUtils.loadLibSVMFile(sc, "mllib/data/sample_libsvm_data.txt") +examples = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") {% endhighlight %} @@ -226,7 +225,7 @@ examples = MLUtils.loadLibSVMFile(sc, "mllib/data/sample_libsvm_data.txt") ## Local matrix A local matrix has integer-typed row and column indices and double-typed values, stored on a single -machine. MLlib supports dense matrix, whose entry values are stored in a single double array in +machine. MLlib supports dense matrices, whose entry values are stored in a single double array in column major. For example, the following matrix `\[ \begin{pmatrix} 1.0 & 2.0 \\ 3.0 & 4.0 \\ @@ -234,7 +233,6 @@ column major. For example, the following matrix `\[ \begin{pmatrix} \end{pmatrix} \]` is stored in a one-dimensional array `[1.0, 3.0, 5.0, 2.0, 4.0, 6.0]` with the matrix size `(3, 2)`. -We are going to add sparse matrix in the next release.
    @@ -242,7 +240,7 @@ We are going to add sparse matrix in the next release. The base class of local matrices is [`Matrix`](api/scala/index.html#org.apache.spark.mllib.linalg.Matrix), and we provide one implementation: [`DenseMatrix`](api/scala/index.html#org.apache.spark.mllib.linalg.DenseMatrix). -Sparse matrix will be added in the next release. We recommend using the factory methods implemented +We recommend using the factory methods implemented in [`Matrices`](api/scala/index.html#org.apache.spark.mllib.linalg.Matrices) to create local matrices. @@ -259,7 +257,7 @@ val dm: Matrix = Matrices.dense(3, 2, Array(1.0, 3.0, 5.0, 2.0, 4.0, 6.0)) The base class of local matrices is [`Matrix`](api/java/org/apache/spark/mllib/linalg/Matrix.html), and we provide one implementation: [`DenseMatrix`](api/java/org/apache/spark/mllib/linalg/DenseMatrix.html). -Sparse matrix will be added in the next release. We recommend using the factory methods implemented +We recommend using the factory methods implemented in [`Matrices`](api/java/org/apache/spark/mllib/linalg/Matrices.html) to create local matrices. @@ -279,28 +277,30 @@ Matrix dm = Matrices.dense(3, 2, new double[] {1.0, 3.0, 5.0, 2.0, 4.0, 6.0}); A distributed matrix has long-typed row and column indices and double-typed values, stored distributively in one or more RDDs. It is very important to choose the right format to store large and distributed matrices. Converting a distributed matrix to a different format may require a -global shuffle, which is quite expensive. We implemented three types of distributed matrices in -this release and will add more types in the future. +global shuffle, which is quite expensive. Three types of distributed matrices have been implemented +so far. The basic type is called `RowMatrix`. A `RowMatrix` is a row-oriented distributed matrix without meaningful row indices, e.g., a collection of feature vectors. It is backed by an RDD of its rows, where each row is a local vector. -We assume that the number of columns is not huge for a `RowMatrix`. +We assume that the number of columns is not huge for a `RowMatrix` so that a single +local vector can be reasonably communicated to the driver and can also be stored / +operated on using a single node. An `IndexedRowMatrix` is similar to a `RowMatrix` but with row indices, -which can be used for identifying rows and joins. -A `CoordinateMatrix` is a distributed matrix stored in [coordinate list (COO)](https://en.wikipedia.org/wiki/Sparse_matrix) format, +which can be used for identifying rows and executing joins. +A `CoordinateMatrix` is a distributed matrix stored in [coordinate list (COO)](https://en.wikipedia.org/wiki/Sparse_matrix#Coordinate_list_.28COO.29) format, backed by an RDD of its entries. ***Note*** The underlying RDDs of a distributed matrix must be deterministic, because we cache the matrix size. -It is always error-prone to have non-deterministic RDDs. +In general the use of non-deterministic RDDs can lead to errors. ### RowMatrix A `RowMatrix` is a row-oriented distributed matrix without meaningful row indices, backed by an RDD -of its rows, where each row is a local vector. This is similar to `data matrix` in the context of -multivariate statistics. Since each row is represented by a local vector, the number of columns is +of its rows, where each row is a local vector. +Since each row is represented by a local vector, the number of columns is limited by the integer range but it should be much smaller in practice.
    @@ -344,70 +344,10 @@ long n = mat.numCols();
    -#### Multivariate summary statistics - -We provide column summary statistics for `RowMatrix`. -If the number of columns is not large, say, smaller than 3000, you can also compute -the covariance matrix as a local matrix, which requires $\mathcal{O}(n^2)$ storage where $n$ is the -number of columns. The total CPU time is $\mathcal{O}(m n^2)$, where $m$ is the number of rows, -which could be faster if the rows are sparse. - -
    -
    - -[`RowMatrix#computeColumnSummaryStatistics`](api/scala/index.html#org.apache.spark.mllib.linalg.distributed.RowMatrix) returns an instance of -[`MultivariateStatisticalSummary`](api/scala/index.html#org.apache.spark.mllib.stat.MultivariateStatisticalSummary), -which contains the column-wise max, min, mean, variance, and number of nonzeros, as well as the -total count. - -{% highlight scala %} -import org.apache.spark.mllib.linalg.Matrix -import org.apache.spark.mllib.linalg.distributed.RowMatrix -import org.apache.spark.mllib.stat.MultivariateStatisticalSummary - -val mat: RowMatrix = ... // a RowMatrix - -// Compute column summary statistics. -val summary: MultivariateStatisticalSummary = mat.computeColumnSummaryStatistics() -println(summary.mean) // a dense vector containing the mean value for each column -println(summary.variance) // column-wise variance -println(summary.numNonzeros) // number of nonzeros in each column - -// Compute the covariance matrix. -val cov: Matrix = mat.computeCovariance() -{% endhighlight %} -
    - -
    - -[`RowMatrix#computeColumnSummaryStatistics`](api/java/org/apache/spark/mllib/linalg/distributed/RowMatrix.html#computeColumnSummaryStatistics()) returns an instance of -[`MultivariateStatisticalSummary`](api/java/org/apache/spark/mllib/stat/MultivariateStatisticalSummary.html), -which contains the column-wise max, min, mean, variance, and number of nonzeros, as well as the -total count. - -{% highlight java %} -import org.apache.spark.mllib.linalg.Matrix; -import org.apache.spark.mllib.linalg.distributed.RowMatrix; -import org.apache.spark.mllib.stat.MultivariateStatisticalSummary; - -RowMatrix mat = ... // a RowMatrix - -// Compute column summary statistics. -MultivariateStatisticalSummary summary = mat.computeColumnSummaryStatistics(); -System.out.println(summary.mean()); // a dense vector containing the mean value for each column -System.out.println(summary.variance()); // column-wise variance -System.out.println(summary.numNonzeros()); // number of nonzeros in each column - -// Compute the covariance matrix. -Matrix cov = mat.computeCovariance(); -{% endhighlight %} -
    -
    - ### IndexedRowMatrix An `IndexedRowMatrix` is similar to a `RowMatrix` but with meaningful row indices. It is backed by -an RDD of indexed rows, which each row is represented by its index (long-typed) and a local vector. +an RDD of indexed rows, so that each row is represented by its index (long-typed) and a local vector.
    @@ -467,7 +407,7 @@ RowMatrix rowMat = mat.toRowMatrix(); A `CoordinateMatrix` is a distributed matrix backed by an RDD of its entries. Each entry is a tuple of `(i: Long, j: Long, value: Double)`, where `i` is the row index, `j` is the column index, and -`value` is the entry value. A `CoordinateMatrix` should be used only in the case when both +`value` is the entry value. A `CoordinateMatrix` should be used only when both dimensions of the matrix are huge and the matrix is very sparse.
    @@ -477,9 +417,9 @@ A [`CoordinateMatrix`](api/scala/index.html#org.apache.spark.mllib.linalg.distributed.CoordinateMatrix) can be created from an `RDD[MatrixEntry]` instance, where [`MatrixEntry`](api/scala/index.html#org.apache.spark.mllib.linalg.distributed.MatrixEntry) is a -wrapper over `(Long, Long, Double)`. A `CoordinateMatrix` can be converted to a `IndexedRowMatrix` -with sparse rows by calling `toIndexedRowMatrix`. In this release, we do not provide other -computation for `CoordinateMatrix`. +wrapper over `(Long, Long, Double)`. A `CoordinateMatrix` can be converted to an `IndexedRowMatrix` +with sparse rows by calling `toIndexedRowMatrix`. Other computations for +`CoordinateMatrix` are not currently supported. {% highlight scala %} import org.apache.spark.mllib.linalg.distributed.{CoordinateMatrix, MatrixEntry} @@ -503,8 +443,9 @@ A [`CoordinateMatrix`](api/java/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.html) can be created from a `JavaRDD` instance, where [`MatrixEntry`](api/java/org/apache/spark/mllib/linalg/distributed/MatrixEntry.html) is a -wrapper over `(long, long, double)`. A `CoordinateMatrix` can be converted to a `IndexedRowMatrix` -with sparse rows by calling `toIndexedRowMatrix`. +wrapper over `(long, long, double)`. A `CoordinateMatrix` can be converted to an `IndexedRowMatrix` +with sparse rows by calling `toIndexedRowMatrix`. Other computations for +`CoordinateMatrix` are not currently supported. {% highlight java %} import org.apache.spark.api.java.JavaRDD; diff --git a/docs/mllib-classification-regression.md b/docs/mllib-classification-regression.md new file mode 100644 index 0000000000000..719cc95767b00 --- /dev/null +++ b/docs/mllib-classification-regression.md @@ -0,0 +1,37 @@ +--- +layout: global +title: Classification and Regression - MLlib +displayTitle: MLlib - Classification and Regression +--- + +MLlib supports various methods for +[binary classification](http://en.wikipedia.org/wiki/Binary_classification), +[multiclass +classification](http://en.wikipedia.org/wiki/Multiclass_classification), and +[regression analysis](http://en.wikipedia.org/wiki/Regression_analysis). The table below outlines +the supported algorithms for each type of problem. + + + + + + + + + + + + + + + + +
    Problem TypeSupported Methods
    Binary Classificationlinear SVMs, logistic regression, decision trees, naive Bayes
    Multiclass Classificationdecision trees, naive Bayes
    Regressionlinear least squares, Lasso, ridge regression, decision trees
    + +More details for these methods can be found here: + +* [Linear models](mllib-linear-methods.html) + * [binary classification (SVMs, logistic regression)](mllib-linear-methods.html#binary-classification) + * [linear regression (least squares, Lasso, ridge)](mllib-linear-methods.html#linear-least-squares-lasso-and-ridge-regression) +* [Decision trees](mllib-decision-tree.html) +* [Naive Bayes](mllib-naive-bayes.html) diff --git a/docs/mllib-clustering.md b/docs/mllib-clustering.md index 429cdf8d40cec..dfd9cd572888c 100644 --- a/docs/mllib-clustering.md +++ b/docs/mllib-clustering.md @@ -38,7 +38,7 @@ a given dataset, the algorithm returns the best clustering result).
    -Following code snippets can be executed in `spark-shell`. +The following code snippets can be executed in `spark-shell`. In the following example after loading and parsing data, we use the [`KMeans`](api/scala/index.html#org.apache.spark.mllib.clustering.KMeans) object to cluster the data @@ -51,7 +51,7 @@ import org.apache.spark.mllib.clustering.KMeans import org.apache.spark.mllib.linalg.Vectors // Load and parse the data -val data = sc.textFile("data/kmeans_data.txt") +val data = sc.textFile("data/mllib/kmeans_data.txt") val parsedData = data.map(s => Vectors.dense(s.split(' ').map(_.toDouble))) // Cluster the data into two classes using KMeans @@ -69,11 +69,59 @@ println("Within Set Sum of Squared Errors = " + WSSSE) All of MLlib's methods use Java-friendly types, so you can import and call them there the same way you do in Scala. The only caveat is that the methods take Scala RDD objects, while the Spark Java API uses a separate `JavaRDD` class. You can convert a Java RDD to a Scala one by -calling `.rdd()` on your `JavaRDD` object. +calling `.rdd()` on your `JavaRDD` object. A standalone application example +that is equivalent to the provided example in Scala is given below: + +{% highlight java %} +import org.apache.spark.api.java.*; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.mllib.clustering.KMeans; +import org.apache.spark.mllib.clustering.KMeansModel; +import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.SparkConf; + +public class KMeansExample { + public static void main(String[] args) { + SparkConf conf = new SparkConf().setAppName("K-means Example"); + JavaSparkContext sc = new JavaSparkContext(conf); + + // Load and parse data + String path = "data/mllib/kmeans_data.txt"; + JavaRDD data = sc.textFile(path); + JavaRDD parsedData = data.map( + new Function() { + public Vector call(String s) { + String[] sarray = s.split(" "); + double[] values = new double[sarray.length]; + for (int i = 0; i < sarray.length; i++) + values[i] = Double.parseDouble(sarray[i]); + return Vectors.dense(values); + } + } + ); + + // Cluster the data into two classes using KMeans + int numClusters = 2; + int numIterations = 20; + KMeansModel clusters = KMeans.train(parsedData.rdd(), numClusters, numIterations); + + // Evaluate clustering by computing Within Set Sum of Squared Errors + double WSSSE = clusters.computeCost(parsedData.rdd()); + System.out.println("Within Set Sum of Squared Errors = " + WSSSE); + } +} +{% endhighlight %} + +In order to run the above standalone application, follow the instructions +provided in the [Standalone +Applications](quick-start.html#standalone-applications) section of the Spark +quick-start guide. Be sure to also include *spark-mllib* to your build file as +a dependency.
    -Following examples can be tested in the PySpark shell. +The following examples can be tested in the PySpark shell. In the following example after loading and parsing data, we use the KMeans object to cluster the data into two clusters. The number of desired clusters is passed to the algorithm. We then compute @@ -86,7 +134,7 @@ from numpy import array from math import sqrt # Load and parse the data -data = sc.textFile("data/kmeans_data.txt") +data = sc.textFile("data/mllib/kmeans_data.txt") parsedData = data.map(lambda line: array([float(x) for x in line.split(' ')])) # Build the model (cluster the data) diff --git a/docs/mllib-collaborative-filtering.md b/docs/mllib-collaborative-filtering.md index d51002f015670..ab10b2f01f87b 100644 --- a/docs/mllib-collaborative-filtering.md +++ b/docs/mllib-collaborative-filtering.md @@ -14,13 +14,13 @@ is commonly used for recommender systems. These techniques aim to fill in the missing entries of a user-item association matrix. MLlib currently supports model-based collaborative filtering, in which users and products are described by a small set of latent factors that can be used to predict missing entries. -In particular, we implement the [alternating least squares +MLlib uses the [alternating least squares (ALS)](http://dl.acm.org/citation.cfm?id=1608614) algorithm to learn these latent factors. The implementation in MLlib has the following parameters: * *numBlocks* is the number of blocks used to parallelize computation (set to -1 to auto-configure). -* *rank* is the number of latent factors in our model. +* *rank* is the number of latent factors in the model. * *iterations* is the number of iterations to run. * *lambda* specifies the regularization parameter in ALS. * *implicitPrefs* specifies whether to use the *explicit feedback* ALS variant or one adapted for @@ -58,7 +58,7 @@ import org.apache.spark.mllib.recommendation.ALS import org.apache.spark.mllib.recommendation.Rating // Load and parse the data -val data = sc.textFile("mllib/data/als/test.data") +val data = sc.textFile("data/mllib/als/test.data") val ratings = data.map(_.split(',') match { case Array(user, item, rate) => Rating(user.toInt, item.toInt, rate.toDouble) }) @@ -86,8 +86,8 @@ val MSE = ratesAndPreds.map { case ((user, product), (r1, r2)) => println("Mean Squared Error = " + MSE) {% endhighlight %} -If the rating matrix is derived from other source of information (i.e., it is inferred from -other signals), you can use the trainImplicit method to get better results. +If the rating matrix is derived from another source of information (e.g., it is inferred from +other signals), you can use the `trainImplicit` method to get better results. {% highlight scala %} val alpha = 0.01 @@ -99,7 +99,86 @@ val model = ALS.trainImplicit(ratings, rank, numIterations, alpha) All of MLlib's methods use Java-friendly types, so you can import and call them there the same way you do in Scala. The only caveat is that the methods take Scala RDD objects, while the Spark Java API uses a separate `JavaRDD` class. You can convert a Java RDD to a Scala one by -calling `.rdd()` on your `JavaRDD` object. +calling `.rdd()` on your `JavaRDD` object. A standalone application example +that is equivalent to the provided example in Scala is given bellow: + +{% highlight java %} +import scala.Tuple2; + +import org.apache.spark.api.java.*; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.mllib.recommendation.ALS; +import org.apache.spark.mllib.recommendation.MatrixFactorizationModel; +import org.apache.spark.mllib.recommendation.Rating; +import org.apache.spark.SparkConf; + +public class CollaborativeFiltering { + public static void main(String[] args) { + SparkConf conf = new SparkConf().setAppName("Collaborative Filtering Example"); + JavaSparkContext sc = new JavaSparkContext(conf); + + // Load and parse the data + String path = "data/mllib/als/test.data"; + JavaRDD data = sc.textFile(path); + JavaRDD ratings = data.map( + new Function() { + public Rating call(String s) { + String[] sarray = s.split(","); + return new Rating(Integer.parseInt(sarray[0]), Integer.parseInt(sarray[1]), + Double.parseDouble(sarray[2])); + } + } + ); + + // Build the recommendation model using ALS + int rank = 10; + int numIterations = 20; + MatrixFactorizationModel model = ALS.train(JavaRDD.toRDD(ratings), rank, numIterations, 0.01); + + // Evaluate the model on rating data + JavaRDD> userProducts = ratings.map( + new Function>() { + public Tuple2 call(Rating r) { + return new Tuple2(r.user(), r.product()); + } + } + ); + JavaPairRDD, Double> predictions = JavaPairRDD.fromJavaRDD( + model.predict(JavaRDD.toRDD(userProducts)).toJavaRDD().map( + new Function, Double>>() { + public Tuple2, Double> call(Rating r){ + return new Tuple2, Double>( + new Tuple2(r.user(), r.product()), r.rating()); + } + } + )); + JavaRDD> ratesAndPreds = + JavaPairRDD.fromJavaRDD(ratings.map( + new Function, Double>>() { + public Tuple2, Double> call(Rating r){ + return new Tuple2, Double>( + new Tuple2(r.user(), r.product()), r.rating()); + } + } + )).join(predictions).values(); + double MSE = JavaDoubleRDD.fromRDD(ratesAndPreds.map( + new Function, Object>() { + public Object call(Tuple2 pair) { + Double err = pair._1() - pair._2(); + return err * err; + } + } + ).rdd()).mean(); + System.out.println("Mean Squared Error = " + MSE); + } +} +{% endhighlight %} + +In order to run the above standalone application, follow the instructions +provided in the [Standalone +Applications](quick-start.html#standalone-applications) section of the Spark +quick-start guide. Be sure to also include *spark-mllib* to your build file as +a dependency.
    @@ -112,7 +191,7 @@ from pyspark.mllib.recommendation import ALS from numpy import array # Load and parse the data -data = sc.textFile("mllib/data/als/test.data") +data = sc.textFile("data/mllib/als/test.data") ratings = data.map(lambda line: array([float(x) for x in line.split(',')])) # Build the recommendation model using Alternating Least Squares @@ -141,5 +220,5 @@ model = ALS.trainImplicit(ratings, rank, numIterations, alpha = 0.01) ## Tutorial -[AMP Camp](http://ampcamp.berkeley.edu/) provides a hands-on tutorial for -[personalized movie recommendation with MLlib](http://ampcamp.berkeley.edu/big-data-mini-course/movie-recommendation-with-mllib.html). +The [training exercises](https://databricks-training.s3.amazonaws.com/index.html) from the Spark Summit 2014 include a hands-on tutorial for +[personalized movie recommendation with MLlib](https://databricks-training.s3.amazonaws.com/movie-recommendation-with-mllib.html). diff --git a/docs/mllib-decision-tree.md b/docs/mllib-decision-tree.md index 3002a66a4fdb3..9cbd880897578 100644 --- a/docs/mllib-decision-tree.md +++ b/docs/mllib-decision-tree.md @@ -77,15 +77,17 @@ bins if the condition is not satisfied. **Categorical features** -For `$M$` categorical features, one could come up with `$2^M-1$` split candidates. However, for -binary classification, the number of split candidates can be reduced to `$M-1$` by ordering the +For `$M$` categorical feature values, one could come up with `$2^(M-1)-1$` split candidates. For +binary classification, we can reduce the number of split candidates to `$M-1$` by ordering the categorical feature values by the proportion of labels falling in one of the two classes (see Section 9.2.4 in [Elements of Statistical Machine Learning](http://statweb.stanford.edu/~tibs/ElemStatLearn/) for details). For example, for a binary classification problem with one categorical feature with three categories A, B and C with corresponding proportion of label 1 as 0.2, 0.6 and 0.4, the categorical features are ordered as A followed by C followed B or A, B, C. The two split candidates are A \| C, B -and A , B \| C where \| denotes the split. +and A , B \| C where \| denotes the split. A similar heuristic is used for multiclass classification +when `$2^(M-1)-1$` is greater than the number of bins -- the impurity for each categorical feature value +is used for ordering. ### Stopping rule @@ -122,7 +124,7 @@ import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.impurity.Gini // Load and parse the data file -val data = sc.textFile("mllib/data/sample_tree_data.csv") +val data = sc.textFile("data/mllib/sample_tree_data.csv") val parsedData = data.map { line => val parts = line.split(',').map(_.toDouble) LabeledPoint(parts(0), Vectors.dense(parts.tail)) @@ -161,7 +163,7 @@ import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.impurity.Variance // Load and parse the data file -val data = sc.textFile("mllib/data/sample_tree_data.csv") +val data = sc.textFile("data/mllib/sample_tree_data.csv") val parsedData = data.map { line => val parts = line.split(',').map(_.toDouble) LabeledPoint(parts(0), Vectors.dense(parts.tail)) diff --git a/docs/mllib-dimensionality-reduction.md b/docs/mllib-dimensionality-reduction.md index e3608075fbb13..065d646496131 100644 --- a/docs/mllib-dimensionality-reduction.md +++ b/docs/mllib-dimensionality-reduction.md @@ -9,9 +9,9 @@ displayTitle: MLlib - Dimensionality Reduction [Dimensionality reduction](http://en.wikipedia.org/wiki/Dimensionality_reduction) is the process of reducing the number of variables under consideration. -It is used to extract latent features from raw and noisy features, +It can be used to extract latent features from raw and noisy features or compress data while maintaining the structure. -In this release, we provide preliminary support for dimensionality reduction on tall-and-skinny matrices. +MLlib provides support for dimensionality reduction on tall-and-skinny matrices. ## Singular value decomposition (SVD) @@ -30,17 +30,17 @@ where * $V$ is an orthonormal matrix, whose columns are called right singular vectors. For large matrices, usually we don't need the complete factorization but only the top singular -values and its associated singular vectors. This can save storage, and more importantly, de-noise +values and its associated singular vectors. This can save storage, de-noise and recover the low-rank structure of the matrix. -If we keep the top $k$ singular values, then the dimensions of the return will be: +If we keep the top $k$ singular values, then the dimensions of the resulting low-rank matrix will be: * `$U$`: `$m \times k$`, * `$\Sigma$`: `$k \times k$`, * `$V$`: `$n \times k$`. -In this release, we provide SVD computation to row-oriented matrices that have only a few columns, -say, less than $1000$, but many rows, which we call *tall-and-skinny*. +MLlib provides SVD functionality to row-oriented matrices that have only a few columns, +say, less than $1000$, but many rows, i.e., *tall-and-skinny* matrices.
    @@ -57,9 +57,59 @@ val U: RowMatrix = svd.U // The U factor is a RowMatrix. val s: Vector = svd.s // The singular values are stored in a local dense vector. val V: Matrix = svd.V // The V factor is a local dense matrix. {% endhighlight %} + +The same code applies to `IndexedRowMatrix` if `U` is defined as an +`IndexedRowMatrix`. +
    +
    +{% highlight java %} +import java.util.LinkedList; + +import org.apache.spark.api.java.*; +import org.apache.spark.mllib.linalg.distributed.RowMatrix; +import org.apache.spark.mllib.linalg.Matrix; +import org.apache.spark.mllib.linalg.SingularValueDecomposition; +import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.rdd.RDD; +import org.apache.spark.SparkConf; +import org.apache.spark.SparkContext; + +public class SVD { + public static void main(String[] args) { + SparkConf conf = new SparkConf().setAppName("SVD Example"); + SparkContext sc = new SparkContext(conf); + + double[][] array = ... + LinkedList rowsList = new LinkedList(); + for (int i = 0; i < array.length; i++) { + Vector currentRow = Vectors.dense(array[i]); + rowsList.add(currentRow); + } + JavaRDD rows = JavaSparkContext.fromSparkContext(sc).parallelize(rowsList); + + // Create a RowMatrix from JavaRDD. + RowMatrix mat = new RowMatrix(rows.rdd()); + + // Compute the top 4 singular values and corresponding singular vectors. + SingularValueDecomposition svd = mat.computeSVD(4, true, 1.0E-9d); + RowMatrix U = svd.U(); + Vector s = svd.s(); + Matrix V = svd.V(); + } +} +{% endhighlight %} + +The same code applies to `IndexedRowMatrix` if `U` is defined as an +`IndexedRowMatrix`. + +In order to run the above standalone application, follow the instructions +provided in the [Standalone +Applications](quick-start.html#standalone-applications) section of the Spark +quick-start guide. Be sure to also include *spark-mllib* to your build file as +a dependency. +
    -Same code applies to `IndexedRowMatrix`. -The only difference that the `U` matrix becomes an `IndexedRowMatrix`.
    ## Principal component analysis (PCA) @@ -69,7 +119,7 @@ statistical method to find a rotation such that the first coordinate has the lar possible, and each succeeding coordinate in turn has the largest variance possible. The columns of the rotation matrix are called principal components. PCA is used widely in dimensionality reduction. -In this release, we implement PCA for tall-and-skinny matrices stored in row-oriented format. +MLlib supports PCA for tall-and-skinny matrices stored in row-oriented format.
    @@ -91,4 +141,52 @@ val pc: Matrix = mat.computePrincipalComponents(10) // Principal components are val projected: RowMatrix = mat.multiply(pc) {% endhighlight %}
    + +
    + +The following code demonstrates how to compute principal components on a tall-and-skinny `RowMatrix` +and use them to project the vectors into a low-dimensional space. +The number of columns should be small, e.g, less than 1000. + +{% highlight java %} +import java.util.LinkedList; + +import org.apache.spark.api.java.*; +import org.apache.spark.mllib.linalg.distributed.RowMatrix; +import org.apache.spark.mllib.linalg.Matrix; +import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.rdd.RDD; +import org.apache.spark.SparkConf; +import org.apache.spark.SparkContext; + +public class PCA { + public static void main(String[] args) { + SparkConf conf = new SparkConf().setAppName("PCA Example"); + SparkContext sc = new SparkContext(conf); + + double[][] array = ... + LinkedList rowsList = new LinkedList(); + for (int i = 0; i < array.length; i++) { + Vector currentRow = Vectors.dense(array[i]); + rowsList.add(currentRow); + } + JavaRDD rows = JavaSparkContext.fromSparkContext(sc).parallelize(rowsList); + + // Create a RowMatrix from JavaRDD. + RowMatrix mat = new RowMatrix(rows.rdd()); + + // Compute the top 3 principal components. + Matrix pc = mat.computePrincipalComponents(3); + RowMatrix projected = mat.multiply(pc); + } +} +{% endhighlight %} + +In order to run the above standalone application, follow the instructions +provided in the [Standalone +Applications](quick-start.html#standalone-applications) section of the Spark +quick-start guide. Be sure to also include *spark-mllib* to your build file as +a dependency. +
    diff --git a/docs/mllib-feature-extraction.md b/docs/mllib-feature-extraction.md new file mode 100644 index 0000000000000..4b3cb715c58c7 --- /dev/null +++ b/docs/mllib-feature-extraction.md @@ -0,0 +1,73 @@ +--- +layout: global +title: Feature Extraction - MLlib +displayTitle: MLlib - Feature Extraction +--- + +* Table of contents +{:toc} + +## Word2Vec + +Word2Vec computes distributed vector representation of words. The main advantage of the distributed +representations is that similar words are close in the vector space, which makes generalization to +novel patterns easier and model estimation more robust. Distributed vector representation is +showed to be useful in many natural language processing applications such as named entity +recognition, disambiguation, parsing, tagging and machine translation. + +### Model + +In our implementation of Word2Vec, we used skip-gram model. The training objective of skip-gram is +to learn word vector representations that are good at predicting its context in the same sentence. +Mathematically, given a sequence of training words `$w_1, w_2, \dots, w_T$`, the objective of the +skip-gram model is to maximize the average log-likelihood +`\[ +\frac{1}{T} \sum_{t = 1}^{T}\sum_{j=-k}^{j=k} \log p(w_{t+j} | w_t) +\]` +where $k$ is the size of the training window. + +In the skip-gram model, every word $w$ is associated with two vectors $u_w$ and $v_w$ which are +vector representations of $w$ as word and context respectively. The probability of correctly +predicting word $w_i$ given word $w_j$ is determined by the softmax model, which is +`\[ +p(w_i | w_j ) = \frac{\exp(u_{w_i}^{\top}v_{w_j})}{\sum_{l=1}^{V} \exp(u_l^{\top}v_{w_j})} +\]` +where $V$ is the vocabulary size. + +The skip-gram model with softmax is expensive because the cost of computing $\log p(w_i | w_j)$ +is proportional to $V$, which can be easily in order of millions. To speed up training of Word2Vec, +we used hierarchical softmax, which reduced the complexity of computing of $\log p(w_i | w_j)$ to +$O(\log(V))$ + +### Example + +The example below demonstrates how to load a text file, parse it as an RDD of `Seq[String]`, +construct a `Word2Vec` instance and then fit a `Word2VecModel` with the input data. Finally, +we display the top 40 synonyms of the specified word. To run the example, first download +the [text8](http://mattmahoney.net/dc/text8.zip) data and extract it to your preferred directory. +Here we assume the extracted file is `text8` and in same directory as you run the spark shell. + +
    +
    +{% highlight scala %} +import org.apache.spark._ +import org.apache.spark.rdd._ +import org.apache.spark.SparkContext._ +import org.apache.spark.mllib.feature.Word2Vec + +val input = sc.textFile("text8").map(line => line.split(" ").toSeq) + +val word2vec = new Word2Vec() + +val model = word2vec.fit(input) + +val synonyms = model.findSynonyms("china", 40) + +for((synonym, cosineSimilarity) <- synonyms) { + println(s"$synonym $cosineSimilarity") +} +{% endhighlight %} +
    +
    + +## TFIDF \ No newline at end of file diff --git a/docs/mllib-guide.md b/docs/mllib-guide.md index 95ee6bc96801f..23d5a0c4607af 100644 --- a/docs/mllib-guide.md +++ b/docs/mllib-guide.md @@ -3,18 +3,19 @@ layout: global title: Machine Learning Library (MLlib) --- -MLlib is a Spark implementation of some common machine learning algorithms and utilities, +MLlib is Spark's scalable machine learning library consisting of common learning algorithms and utilities, including classification, regression, clustering, collaborative -filtering, dimensionality reduction, as well as underlying optimization primitives: +filtering, dimensionality reduction, as well as underlying optimization primitives, as outlined below: -* [Basics](mllib-basics.html) - * data types +* [Data types](mllib-basics.html) +* [Basic statistics](mllib-stats.html) + * data generators + * stratified sampling * summary statistics -* Classification and regression - * [linear support vector machine (SVM)](mllib-linear-methods.html#linear-support-vector-machine-svm) - * [logistic regression](mllib-linear-methods.html#logistic-regression) - * [linear least squares, Lasso, and ridge regression](mllib-linear-methods.html#linear-least-squares-lasso-and-ridge-regression) - * [decision tree](mllib-decision-tree.html) + * hypothesis testing +* [Classification and regression](mllib-classification-regression.html) + * [linear models (SVMs, logistic regression, linear regression)](mllib-linear-methods.html) + * [decision trees](mllib-decision-tree.html) * [naive Bayes](mllib-naive-bayes.html) * [Collaborative filtering](mllib-collaborative-filtering.html) * alternating least squares (ALS) @@ -23,17 +24,18 @@ filtering, dimensionality reduction, as well as underlying optimization primitiv * [Dimensionality reduction](mllib-dimensionality-reduction.html) * singular value decomposition (SVD) * principal component analysis (PCA) -* [Optimization](mllib-optimization.html) +* [Feature extraction and transformation](mllib-feature-extraction.html) +* [Optimization (developer)](mllib-optimization.html) * stochastic gradient descent * limited-memory BFGS (L-BFGS) -MLlib is a new component under active development. +MLlib is under active development. The APIs marked `Experimental`/`DeveloperApi` may change in future releases, -and we will provide migration guide between releases. +and the migration guide below will explain all changes between releases. # Dependencies -MLlib uses linear algebra packages [Breeze](http://www.scalanlp.org/), which depends on +MLlib uses the linear algebra package [Breeze](http://www.scalanlp.org/), which depends on [netlib-java](https://github.com/fommil/netlib-java), and [jblas](https://github.com/mikiobraun/jblas). `netlib-java` and `jblas` depend on native Fortran routines. @@ -56,7 +58,7 @@ To use MLlib in Python, you will need [NumPy](http://www.numpy.org) version 1.4 In MLlib v1.0, we support both dense and sparse input in a unified way, which introduces a few breaking changes. If your data is sparse, please store it in a sparse format instead of dense to -take advantage of sparsity in both storage and computation. +take advantage of sparsity in both storage and computation. Details are described below.
    diff --git a/docs/mllib-linear-methods.md b/docs/mllib-linear-methods.md index 4dfbebbcd04b7..e504cd7f0f578 100644 --- a/docs/mllib-linear-methods.md +++ b/docs/mllib-linear-methods.md @@ -33,24 +33,24 @@ the task of finding a minimizer of a convex function `$f$` that depends on a var Formally, we can write this as the optimization problem `$\min_{\wv \in\R^d} \; f(\wv)$`, where the objective function is of the form `\begin{equation} - f(\wv) := - \frac1n \sum_{i=1}^n L(\wv;\x_i,y_i) + - \lambda\, R(\wv_i) + f(\wv) := \lambda\, R(\wv) + + \frac1n \sum_{i=1}^n L(\wv;\x_i,y_i) \label{eq:regPrimal} \ . \end{equation}` Here the vectors `$\x_i\in\R^d$` are the training data examples, for `$1\le i\le n$`, and `$y_i\in\R$` are their corresponding labels, which we want to predict. We call the method *linear* if $L(\wv; \x, y)$ can be expressed as a function of $\wv^T x$ and $y$. -Several MLlib's classification and regression algorithms fall into this category, +Several of MLlib's classification and regression algorithms fall into this category, and are discussed here. The objective function `$f$` has two parts: -the loss that measures the error of the model on the training data, -and the regularizer that measures the complexity of the model. -The loss function `$L(\wv;.)$` must be a convex function in `$\wv$`. -The fixed regularization parameter `$\lambda \ge 0$` (`regParam` in the code) defines the trade-off -between the two goals of small loss and small model complexity. +the regularizer that controls the complexity of the model, +and the loss that measures the error of the model on the training data. +The loss function `$L(\wv;.)$` is typically a convex function in `$\wv$`. The +fixed regularization parameter `$\lambda \ge 0$` (`regParam` in the code) +defines the trade-off between the two goals of minimizing the loss (i.e., +training error) and minimizing model complexity (i.e., to avoid overfitting). ### Loss functions @@ -80,10 +80,10 @@ methods MLlib supports: ### Regularizers -The purpose of the [regularizer](http://en.wikipedia.org/wiki/Regularization_(mathematics)) is to -encourage simple models, by punishing the complexity of the model `$\wv$`, in order to e.g. avoid -over-fitting. -We support the following regularizers in MLlib: +The purpose of the +[regularizer](http://en.wikipedia.org/wiki/Regularization_(mathematics)) is to +encourage simple models and avoid overfitting. We support the following +regularizers in MLlib: @@ -106,27 +106,28 @@ Here `$\mathrm{sign}(\wv)$` is the vector consisting of the signs (`$\pm1$`) of of `$\wv$`. L2-regularized problems are generally easier to solve than L1-regularized due to smoothness. -However, L1 regularization can help promote sparsity in weights, leading to simpler models, which is -also used for feature selection. It is not recommended to train models without any regularization, +However, L1 regularization can help promote sparsity in weights leading to smaller and more interpretable models, the latter of which can be useful for feature selection. +It is not recommended to train models without any regularization, especially when the number of training examples is small. ## Binary classification -[Binary classification](http://en.wikipedia.org/wiki/Binary_classification) is to divide items into -two categories: positive and negative. MLlib supports two linear methods for binary classification: -linear support vector machine (SVM) and logistic regression. The training data set is represented -by an RDD of [LabeledPoint](mllib-data-types.html) in MLlib. Note that, in the mathematical -formulation, a training label $y$ is either $+1$ (positive) or $-1$ (negative), which is convenient -for the formulation. *However*, the negative label is represented by $0$ in MLlib instead of $-1$, -to be consistent with multiclass labeling. +[Binary classification](http://en.wikipedia.org/wiki/Binary_classification) +aims to divide items into two categories: positive and negative. MLlib +supports two linear methods for binary classification: linear support vector +machines (SVMs) and logistic regression. For both methods, MLlib supports +L1 and L2 regularized variants. The training data set is represented by an RDD +of [LabeledPoint](mllib-data-types.html) in MLlib. Note that, in the +mathematical formulation in this guide, a training label $y$ is denoted as +either $+1$ (positive) or $-1$ (negative), which is convenient for the +formulation. *However*, the negative label is represented by $0$ in MLlib +instead of $-1$, to be consistent with multiclass labeling. -### Linear support vector machine (SVM) +### Linear support vector machines (SVMs) The [linear SVM](http://en.wikipedia.org/wiki/Support_vector_machine#Linear_SVM) -has become a standard choice for large-scale classification tasks. -The name "linear SVM" is actually ambiguous. -By "linear SVM", we mean specifically the linear method with the loss function in formulation -`$\eqref{eq:regPrimal}$` given by the hinge loss +is a standard method for large-scale classification tasks. It is a linear method as described above in equation `$\eqref{eq:regPrimal}$`, with the loss function in the formulation given by the hinge loss: + `\[ L(\wv;\x,y) := \max \{0, 1-y \wv^T \x \}. \]` @@ -134,39 +135,44 @@ By default, linear SVMs are trained with an L2 regularization. We also support alternative L1 regularization. In this case, the problem becomes a [linear program](http://en.wikipedia.org/wiki/Linear_programming). -Linear SVM algorithm outputs a SVM model, which makes predictions based on the value of $\wv^T \x$. -By the default, if $\wv^T \x \geq 0$, the outcome is positive, or negative otherwise. -However, quite often in practice, the default threshold $0$ is not a good choice. -The threshold should be determined via model evaluation. +The linear SVMs algorithm outputs an SVM model. Given a new data point, +denoted by $\x$, the model makes predictions based on the value of $\wv^T \x$. +By the default, if $\wv^T \x \geq 0$ then the outcome is positive, and negative +otherwise. ### Logistic regression [Logistic regression](http://en.wikipedia.org/wiki/Logistic_regression) is widely used to predict a -binary response. It is a linear method with the loss function in formulation -`$\eqref{eq:regPrimal}$` given by the logistic loss +binary response. +It is a linear method as described above in equation `$\eqref{eq:regPrimal}$`, with the loss +function in the formulation given by the logistic loss: `\[ L(\wv;\x,y) := \log(1+\exp( -y \wv^T \x)). \]` -Logistic regression algorithm outputs a logistic regression model, which makes predictions by +The logistic regression algorithm outputs a logistic regression model. Given a +new data point, denoted by $\x$, the model makes predictions by applying the logistic function `\[ -\mathrm{logit}(z) = \frac{1}{1 + e^{-z}} +\mathrm{f}(z) = \frac{1}{1 + e^{-z}} \]` -$\wv^T \x$. -By default, if $\mathrm{logit}(\wv^T x) > 0.5$, the outcome is positive, or negative otherwise. -For the same reason mentioned above, quite often in practice, this default threshold is not a good choice. -The threshold should be determined via model evaluation. +where $z = \wv^T \x$. +By default, if $\mathrm{f}(\wv^T x) > 0.5$, the outcome is positive, or +negative otherwise, though unlike linear SVMs, the raw output of the logistic regression +model, $\mathrm{f}(z)$, has a probabilistic interpretation (i.e., the probability +that $\x$ is positive). ### Evaluation metrics -MLlib supports common evaluation metrics for binary classification (not available in Python). This +MLlib supports common evaluation metrics for binary classification (not available in PySpark). +This includes precision, recall, [F-measure](http://en.wikipedia.org/wiki/F1_score), [receiver operating characteristic (ROC)](http://en.wikipedia.org/wiki/Receiver_operating_characteristic), precision-recall curve, and [area under the curves (AUC)](http://en.wikipedia.org/wiki/Receiver_operating_characteristic#Area_under_the_curve). -Among the metrics, area under ROC is commonly used to compare models and precision/recall/F-measure -can help determine the threshold to use. +AUC is commonly used to compare the performance of various models while +precision/recall/F-measure can help determine the appropriate threshold to use +for prediction purposes. ### Examples @@ -187,7 +193,7 @@ import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.util.MLUtils // Load training data in LIBSVM format. -val data = MLUtils.loadLibSVMFile(sc, "mllib/data/sample_libsvm_data.txt") +val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") // Split data into training (60%) and test (40%). val splits = data.randomSplit(Array(0.6, 0.4), seed = 11L) @@ -233,8 +239,7 @@ svmAlg.optimizer. val modelL1 = svmAlg.run(training) {% endhighlight %} -Similarly, you can use replace `SVMWithSGD` by -[`LogisticRegressionWithSGD`](api/scala/index.html#org.apache.spark.mllib.classification.LogisticRegressionWithSGD). +[`LogisticRegressionWithSGD`](api/scala/index.html#org.apache.spark.mllib.classification.LogisticRegressionWithSGD) can be used in a similar fashion as `SVMWithSGD`. @@ -242,7 +247,87 @@ Similarly, you can use replace `SVMWithSGD` by All of MLlib's methods use Java-friendly types, so you can import and call them there the same way you do in Scala. The only caveat is that the methods take Scala RDD objects, while the Spark Java API uses a separate `JavaRDD` class. You can convert a Java RDD to a Scala one by -calling `.rdd()` on your `JavaRDD` object. +calling `.rdd()` on your `JavaRDD` object. A standalone application example +that is equivalent to the provided example in Scala is given bellow: + +{% highlight java %} +import java.util.Random; + +import scala.Tuple2; + +import org.apache.spark.api.java.*; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.mllib.classification.*; +import org.apache.spark.mllib.evaluation.BinaryClassificationMetrics; +import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.util.MLUtils; +import org.apache.spark.SparkConf; +import org.apache.spark.SparkContext; + +public class SVMClassifier { + public static void main(String[] args) { + SparkConf conf = new SparkConf().setAppName("SVM Classifier Example"); + SparkContext sc = new SparkContext(conf); + String path = "data/mllib/sample_libsvm_data.txt"; + JavaRDD data = MLUtils.loadLibSVMFile(sc, path).toJavaRDD(); + + // Split initial RDD into two... [60% training data, 40% testing data]. + JavaRDD training = data.sample(false, 0.6, 11L); + training.cache(); + JavaRDD test = data.subtract(training); + + // Run training algorithm to build the model. + int numIterations = 100; + final SVMModel model = SVMWithSGD.train(training.rdd(), numIterations); + + // Clear the default threshold. + model.clearThreshold(); + + // Compute raw scores on the test set. + JavaRDD> scoreAndLabels = test.map( + new Function>() { + public Tuple2 call(LabeledPoint p) { + Double score = model.predict(p.features()); + return new Tuple2(score, p.label()); + } + } + ); + + // Get evaluation metrics. + BinaryClassificationMetrics metrics = + new BinaryClassificationMetrics(JavaRDD.toRDD(scoreAndLabels)); + double auROC = metrics.areaUnderROC(); + + System.out.println("Area under ROC = " + auROC); + } +} +{% endhighlight %} + +The `SVMWithSGD.train()` method by default performs L2 regularization with the +regularization parameter set to 1.0. If we want to configure this algorithm, we +can customize `SVMWithSGD` further by creating a new object directly and +calling setter methods. All other MLlib algorithms support customization in +this way as well. For example, the following code produces an L1 regularized +variant of SVMs with regularization parameter set to 0.1, and runs the training +algorithm for 200 iterations. + +{% highlight java %} +import org.apache.spark.mllib.optimization.L1Updater; + +SVMWithSGD svmAlg = new SVMWithSGD(); +svmAlg.optimizer() + .setNumIterations(200) + .setRegParam(0.1) + .setUpdater(new L1Updater()); +final SVMModel modelL1 = svmAlg.run(training.rdd()); +{% endhighlight %} + +In order to run the above standalone application, follow the instructions +provided in the [Standalone +Applications](quick-start.html#standalone-applications) section of the Spark +quick-start guide. Be sure to also include *spark-mllib* to your build file as +a dependency.
    @@ -259,7 +344,7 @@ def parsePoint(line): values = [float(x) for x in line.split(' ')] return LabeledPoint(values[0], values[1:]) -data = sc.textFile("mllib/data/sample_svm_data.txt") +data = sc.textFile("data/mllib/sample_svm_data.txt") parsedData = data.map(parsePoint) # Build the model @@ -275,24 +360,22 @@ print("Training Error = " + str(trainErr)) ## Linear least squares, Lasso, and ridge regression -Linear least squares is a family of linear methods with the loss function in formulation -`$\eqref{eq:regPrimal}$` given by the squared loss +Linear least squares is the most common formulation for regression problems. +It is a linear method as described above in equation `$\eqref{eq:regPrimal}$`, with the loss +function in the formulation given by the squared loss: `\[ L(\wv;\x,y) := \frac{1}{2} (\wv^T \x - y)^2. \]` -Depending on the regularization type, we call the method -[*ordinary least squares*](http://en.wikipedia.org/wiki/Ordinary_least_squares) or simply -[*linear least squares*](http://en.wikipedia.org/wiki/Linear_least_squares_(mathematics)) if there -is no regularization, [*ridge regression*](http://en.wikipedia.org/wiki/Ridge_regression) if L2 -regularization is used, and [*Lasso*](http://en.wikipedia.org/wiki/Lasso_(statistics)) if L1 -regularization is used. This average loss $\frac{1}{n} \sum_{i=1}^n (\wv^T x_i - y_i)^2$ is also +Various related regression methods are derived by using different types of regularization: +[*ordinary least squares*](http://en.wikipedia.org/wiki/Ordinary_least_squares) or +[*linear least squares*](http://en.wikipedia.org/wiki/Linear_least_squares_(mathematics)) uses + no regularization; [*ridge regression*](http://en.wikipedia.org/wiki/Ridge_regression) uses L2 +regularization; and [*Lasso*](http://en.wikipedia.org/wiki/Lasso_(statistics)) uses L1 +regularization. For all of these models, the average loss or training error, $\frac{1}{n} \sum_{i=1}^n (\wv^T x_i - y_i)^2$, is known as the [mean squared error](http://en.wikipedia.org/wiki/Mean_squared_error). -Note that the squared loss is sensitive to outliers. -Regularization or a robust alternative (e.g., $\ell_1$ regression) is usually necessary in practice. - ### Examples
    @@ -300,7 +383,7 @@ Regularization or a robust alternative (e.g., $\ell_1$ regression) is usually ne
    The following example demonstrate how to load training data, parse it as an RDD of LabeledPoint. The example then uses LinearRegressionWithSGD to build a simple linear model to predict label -values. We compute the Mean Squared Error at the end to evaluate +values. We compute the mean squared error at the end to evaluate [goodness of fit](http://en.wikipedia.org/wiki/Goodness_of_fit). {% highlight scala %} @@ -309,7 +392,7 @@ import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.linalg.Vectors // Load and parse the data -val data = sc.textFile("mllib/data/ridge-data/lpsa.data") +val data = sc.textFile("data/mllib/ridge-data/lpsa.data") val parsedData = data.map { line => val parts = line.split(',') LabeledPoint(parts(0).toDouble, Vectors.dense(parts(1).split(' ').map(_.toDouble))) @@ -328,9 +411,8 @@ val MSE = valuesAndPreds.map{case(v, p) => math.pow((v - p), 2)}.mean() println("training Mean Squared Error = " + MSE) {% endhighlight %} -Similarly you can use [`RidgeRegressionWithSGD`](api/scala/index.html#org.apache.spark.mllib.regression.RidgeRegressionWithSGD) -and [`LassoWithSGD`](api/scala/index.html#org.apache.spark.mllib.regression.LassoWithSGD). +and [`LassoWithSGD`](api/scala/index.html#org.apache.spark.mllib.regression.LassoWithSGD) can be used in a similar fashion as `LinearRegressionWithSGD`.
    @@ -338,13 +420,79 @@ and [`LassoWithSGD`](api/scala/index.html#org.apache.spark.mllib.regression.Lass All of MLlib's methods use Java-friendly types, so you can import and call them there the same way you do in Scala. The only caveat is that the methods take Scala RDD objects, while the Spark Java API uses a separate `JavaRDD` class. You can convert a Java RDD to a Scala one by -calling `.rdd()` on your `JavaRDD` object. +calling `.rdd()` on your `JavaRDD` object. The corresponding Java example to +the Scala snippet provided, is presented bellow: + +{% highlight java %} +import scala.Tuple2; + +import org.apache.spark.api.java.*; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.regression.LinearRegressionModel; +import org.apache.spark.mllib.regression.LinearRegressionWithSGD; +import org.apache.spark.SparkConf; + +public class LinearRegression { + public static void main(String[] args) { + SparkConf conf = new SparkConf().setAppName("Linear Regression Example"); + JavaSparkContext sc = new JavaSparkContext(conf); + + // Load and parse the data + String path = "data/mllib/ridge-data/lpsa.data"; + JavaRDD data = sc.textFile(path); + JavaRDD parsedData = data.map( + new Function() { + public LabeledPoint call(String line) { + String[] parts = line.split(","); + String[] features = parts[1].split(" "); + double[] v = new double[features.length]; + for (int i = 0; i < features.length - 1; i++) + v[i] = Double.parseDouble(features[i]); + return new LabeledPoint(Double.parseDouble(parts[0]), Vectors.dense(v)); + } + } + ); + + // Building the model + int numIterations = 100; + final LinearRegressionModel model = + LinearRegressionWithSGD.train(JavaRDD.toRDD(parsedData), numIterations); + + // Evaluate model on training examples and compute training error + JavaRDD> valuesAndPreds = parsedData.map( + new Function>() { + public Tuple2 call(LabeledPoint point) { + double prediction = model.predict(point.features()); + return new Tuple2(prediction, point.label()); + } + } + ); + JavaRDD MSE = new JavaDoubleRDD(valuesAndPreds.map( + new Function, Object>() { + public Object call(Tuple2 pair) { + return Math.pow(pair._1() - pair._2(), 2.0); + } + } + ).rdd()).mean(); + System.out.println("training Mean Squared Error = " + MSE); + } +} +{% endhighlight %} + +In order to run the above standalone application, follow the instructions +provided in the [Standalone +Applications](quick-start.html#standalone-applications) section of the Spark +quick-start guide. Be sure to also include *spark-mllib* to your build file as +a dependency.
    The following example demonstrate how to load training data, parse it as an RDD of LabeledPoint. The example then uses LinearRegressionWithSGD to build a simple linear model to predict label -values. We compute the Mean Squared Error at the end to evaluate +values. We compute the mean squared error at the end to evaluate [goodness of fit](http://en.wikipedia.org/wiki/Goodness_of_fit). {% highlight python %} @@ -356,7 +504,7 @@ def parsePoint(line): values = [float(x) for x in line.replace(',', ' ').split(' ')] return LabeledPoint(values[0], values[1:]) -data = sc.textFile("mllib/data/ridge-data/lpsa.data") +data = sc.textFile("data/mllib/ridge-data/lpsa.data") parsedData = data.map(parsePoint) # Build the model diff --git a/docs/mllib-naive-bayes.md b/docs/mllib-naive-bayes.md index 1d1d7dcf6ffcb..86d94aebd9442 100644 --- a/docs/mllib-naive-bayes.md +++ b/docs/mllib-naive-bayes.md @@ -4,23 +4,23 @@ title: Naive Bayes - MLlib displayTitle: MLlib - Naive Bayes --- -Naive Bayes is a simple multiclass classification algorithm with the assumption of independence -between every pair of features. Naive Bayes can be trained very efficiently. Within a single pass to -the training data, it computes the conditional probability distribution of each feature given label, -and then it applies Bayes' theorem to compute the conditional probability distribution of label -given an observation and use it for prediction. For more details, please visit the Wikipedia page -[Naive Bayes classifier](http://en.wikipedia.org/wiki/Naive_Bayes_classifier). - -In MLlib, we implemented multinomial naive Bayes, which is typically used for document -classification. Within that context, each observation is a document, each feature represents a term, -whose value is the frequency of the term. For its formulation, please visit the Wikipedia page -[Multinomial Naive Bayes](http://en.wikipedia.org/wiki/Naive_Bayes_classifier#Multinomial_naive_Bayes) -or the section -[Naive Bayes text classification](http://nlp.stanford.edu/IR-book/html/htmledition/naive-bayes-text-classification-1.html) -from the book Introduction to Information -Retrieval. [Additive smoothing](http://en.wikipedia.org/wiki/Lidstone_smoothing) can be used by +[Naive Bayes](http://en.wikipedia.org/wiki/Naive_Bayes_classifier) is a simple +multiclass classification algorithm with the assumption of independence between +every pair of features. Naive Bayes can be trained very efficiently. Within a +single pass to the training data, it computes the conditional probability +distribution of each feature given label, and then it applies Bayes' theorem to +compute the conditional probability distribution of label given an observation +and use it for prediction. + +MLlib supports [multinomial naive +Bayes](http://en.wikipedia.org/wiki/Naive_Bayes_classifier#Multinomial_naive_Bayes), +which is typically used for [document +classification](http://nlp.stanford.edu/IR-book/html/htmledition/naive-bayes-text-classification-1.html). +Within that context, each observation is a document and each +feature represents a term whose value is the frequency of the term. +[Additive smoothing](http://en.wikipedia.org/wiki/Lidstone_smoothing) can be used by setting the parameter $\lambda$ (default to $1.0$). For document classification, the input feature -vectors are usually sparse. Please supply sparse vectors as input to take advantage of +vectors are usually sparse, and sparse vectors should be supplied as input to take advantage of sparsity. Since the training data is only used once, it is not necessary to cache it. ## Examples @@ -40,7 +40,7 @@ import org.apache.spark.mllib.classification.NaiveBayes import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.regression.LabeledPoint -val data = sc.textFile("mllib/data/sample_naive_bayes_data.txt") +val data = sc.textFile("data/mllib/sample_naive_bayes_data.txt") val parsedData = data.map { line => val parts = line.split(',') LabeledPoint(parts(0).toDouble, Vectors.dense(parts(1).split(' ').map(_.toDouble))) diff --git a/docs/mllib-optimization.md b/docs/mllib-optimization.md index ae9ede58e8e60..26ce5f3c501ff 100644 --- a/docs/mllib-optimization.md +++ b/docs/mllib-optimization.md @@ -207,6 +207,10 @@ the loss computed for every iteration. Here is an example to train binary logistic regression with L2 regularization using L-BFGS optimizer. + +
    + +
    {% highlight scala %} import org.apache.spark.SparkContext import org.apache.spark.mllib.evaluation.BinaryClassificationMetrics @@ -214,7 +218,7 @@ import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.util.MLUtils import org.apache.spark.mllib.classification.LogisticRegressionModel -val data = MLUtils.loadLibSVMFile(sc, "mllib/data/sample_libsvm_data.txt") +val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") val numFeatures = data.take(1)(0).features.size // Split data into training (60%) and test (40%). @@ -263,7 +267,97 @@ println("Loss of each step in training process") loss.foreach(println) println("Area under ROC = " + auROC) {% endhighlight %} - +
    + +
    +{% highlight java %} +import java.util.Arrays; +import java.util.Random; + +import scala.Tuple2; + +import org.apache.spark.api.java.*; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.mllib.classification.LogisticRegressionModel; +import org.apache.spark.mllib.evaluation.BinaryClassificationMetrics; +import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.mllib.optimization.*; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.util.MLUtils; +import org.apache.spark.SparkConf; +import org.apache.spark.SparkContext; + +public class LBFGSExample { + public static void main(String[] args) { + SparkConf conf = new SparkConf().setAppName("L-BFGS Example"); + SparkContext sc = new SparkContext(conf); + String path = "data/mllib/sample_libsvm_data.txt"; + JavaRDD data = MLUtils.loadLibSVMFile(sc, path).toJavaRDD(); + int numFeatures = data.take(1).get(0).features().size(); + + // Split initial RDD into two... [60% training data, 40% testing data]. + JavaRDD trainingInit = data.sample(false, 0.6, 11L); + JavaRDD test = data.subtract(trainingInit); + + // Append 1 into the training data as intercept. + JavaRDD> training = data.map( + new Function>() { + public Tuple2 call(LabeledPoint p) { + return new Tuple2(p.label(), MLUtils.appendBias(p.features())); + } + }); + training.cache(); + + // Run training algorithm to build the model. + int numCorrections = 10; + double convergenceTol = 1e-4; + int maxNumIterations = 20; + double regParam = 0.1; + Vector initialWeightsWithIntercept = Vectors.dense(new double[numFeatures + 1]); + + Tuple2 result = LBFGS.runLBFGS( + training.rdd(), + new LogisticGradient(), + new SquaredL2Updater(), + numCorrections, + convergenceTol, + maxNumIterations, + regParam, + initialWeightsWithIntercept); + Vector weightsWithIntercept = result._1(); + double[] loss = result._2(); + + final LogisticRegressionModel model = new LogisticRegressionModel( + Vectors.dense(Arrays.copyOf(weightsWithIntercept.toArray(), weightsWithIntercept.size() - 1)), + (weightsWithIntercept.toArray())[weightsWithIntercept.size() - 1]); + + // Clear the default threshold. + model.clearThreshold(); + + // Compute raw scores on the test set. + JavaRDD> scoreAndLabels = test.map( + new Function>() { + public Tuple2 call(LabeledPoint p) { + Double score = model.predict(p.features()); + return new Tuple2(score, p.label()); + } + }); + + // Get evaluation metrics. + BinaryClassificationMetrics metrics = + new BinaryClassificationMetrics(scoreAndLabels.rdd()); + double auROC = metrics.areaUnderROC(); + + System.out.println("Loss of each step in training process"); + for (double l : loss) + System.out.println(l); + System.out.println("Area under ROC = " + auROC); + } +} +{% endhighlight %} +
    +
    #### Developer's note Since the Hessian is constructed approximately from previous gradient evaluations, the objective function can not be changed during the optimization process. diff --git a/docs/mllib-stats.md b/docs/mllib-stats.md new file mode 100644 index 0000000000000..ca9ef46c15186 --- /dev/null +++ b/docs/mllib-stats.md @@ -0,0 +1,95 @@ +--- +layout: global +title: Statistics Functionality - MLlib +displayTitle: MLlib - Statistics Functionality +--- + +* Table of contents +{:toc} + + +`\[ +\newcommand{\R}{\mathbb{R}} +\newcommand{\E}{\mathbb{E}} +\newcommand{\x}{\mathbf{x}} +\newcommand{\y}{\mathbf{y}} +\newcommand{\wv}{\mathbf{w}} +\newcommand{\av}{\mathbf{\alpha}} +\newcommand{\bv}{\mathbf{b}} +\newcommand{\N}{\mathbb{N}} +\newcommand{\id}{\mathbf{I}} +\newcommand{\ind}{\mathbf{1}} +\newcommand{\0}{\mathbf{0}} +\newcommand{\unit}{\mathbf{e}} +\newcommand{\one}{\mathbf{1}} +\newcommand{\zero}{\mathbf{0}} +\]` + +## Data Generators + +## Stratified Sampling + +## Summary Statistics + +### Multivariate summary statistics + +We provide column summary statistics for `RowMatrix` (note: this functionality is not currently supported in `IndexedRowMatrix` or `CoordinateMatrix`). +If the number of columns is not large, e.g., on the order of thousands, then the +covariance matrix can also be computed as a local matrix, which requires $\mathcal{O}(n^2)$ storage where $n$ is the +number of columns. The total CPU time is $\mathcal{O}(m n^2)$, where $m$ is the number of rows, +and is faster if the rows are sparse. + +
    +
    + +[`computeColumnSummaryStatistics()`](api/scala/index.html#org.apache.spark.mllib.linalg.distributed.RowMatrix) returns an instance of +[`MultivariateStatisticalSummary`](api/scala/index.html#org.apache.spark.mllib.stat.MultivariateStatisticalSummary), +which contains the column-wise max, min, mean, variance, and number of nonzeros, as well as the +total count. + +{% highlight scala %} +import org.apache.spark.mllib.linalg.Matrix +import org.apache.spark.mllib.linalg.distributed.RowMatrix +import org.apache.spark.mllib.stat.MultivariateStatisticalSummary + +val mat: RowMatrix = ... // a RowMatrix + +// Compute column summary statistics. +val summary: MultivariateStatisticalSummary = mat.computeColumnSummaryStatistics() +println(summary.mean) // a dense vector containing the mean value for each column +println(summary.variance) // column-wise variance +println(summary.numNonzeros) // number of nonzeros in each column + +// Compute the covariance matrix. +val cov: Matrix = mat.computeCovariance() +{% endhighlight %} +
    + +
    + +[`RowMatrix#computeColumnSummaryStatistics`](api/java/org/apache/spark/mllib/linalg/distributed/RowMatrix.html#computeColumnSummaryStatistics()) returns an instance of +[`MultivariateStatisticalSummary`](api/java/org/apache/spark/mllib/stat/MultivariateStatisticalSummary.html), +which contains the column-wise max, min, mean, variance, and number of nonzeros, as well as the +total count. + +{% highlight java %} +import org.apache.spark.mllib.linalg.Matrix; +import org.apache.spark.mllib.linalg.distributed.RowMatrix; +import org.apache.spark.mllib.stat.MultivariateStatisticalSummary; + +RowMatrix mat = ... // a RowMatrix + +// Compute column summary statistics. +MultivariateStatisticalSummary summary = mat.computeColumnSummaryStatistics(); +System.out.println(summary.mean()); // a dense vector containing the mean value for each column +System.out.println(summary.variance()); // column-wise variance +System.out.println(summary.numNonzeros()); // number of nonzeros in each column + +// Compute the covariance matrix. +Matrix cov = mat.computeCovariance(); +{% endhighlight %} +
    +
    + + +## Hypothesis Testing diff --git a/docs/monitoring.md b/docs/monitoring.md index 84073fe4d949a..d07ec4a57a2cc 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -33,7 +33,7 @@ application's UI after the application has finished. If Spark is run on Mesos or YARN, it is still possible to reconstruct the UI of a finished application through Spark's history server, provided that the application's event logs exist. -You can start a the history server by executing: +You can start the history server by executing: ./sbin/start-history-server.sh @@ -106,7 +106,7 @@ follows:
    diff --git a/docs/programming-guide.md b/docs/programming-guide.md index b09d6347cd1b2..6ae780d94046a 100644 --- a/docs/programming-guide.md +++ b/docs/programming-guide.md @@ -383,16 +383,16 @@ Apart from text files, Spark's Python API also supports several other data forma * `RDD.saveAsPickleFile` and `SparkContext.pickleFile` support saving an RDD in a simple format consisting of pickled Python objects. Batching is used on pickle serialization, with default batch size 10. -* Details on reading `SequenceFile` and arbitrary Hadoop `InputFormat` are given below. - -### SequenceFile and Hadoop InputFormats +* SequenceFile and Hadoop Input/Output Formats **Note** this feature is currently marked ```Experimental``` and is intended for advanced users. It may be replaced in future with read/write support based on SparkSQL, in which case SparkSQL is the preferred approach. -#### Writable Support +**Writable Support** -PySpark SequenceFile support loads an RDD within Java, and pickles the resulting Java objects using -[Pyrolite](https://github.com/irmen/Pyrolite/). The following Writables are automatically converted: +PySpark SequenceFile support loads an RDD of key-value pairs within Java, converts Writables to base Java types, and pickles the +resulting Java objects using [Pyrolite](https://github.com/irmen/Pyrolite/). When saving an RDD of key-value pairs to SequenceFile, +PySpark does the reverse. It unpickles Python objects into Java objects and then converts them to Writables. The following +Writables are automatically converted:
    Indicates whether the history server should use kerberos to login. This is useful if the history server is accessing HDFS files on a secure Hadoop cluster. If this is - true it looks uses the configs spark.history.kerberos.principal and + true, it uses the configs spark.history.kerberos.principal and spark.history.kerberos.keytab.
    @@ -403,32 +403,30 @@ PySpark SequenceFile support loads an RDD within Java, and pickles the resulting - - -
    Writable TypePython Type
    BooleanWritablebool
    BytesWritablebytearray
    NullWritableNone
    ArrayWritablelist of primitives, or tuple of objects
    MapWritabledict
    Custom Class conforming to Java Bean conventionsdict of public properties (via JavaBean getters and setters) + __class__ for the class type
    -#### Loading SequenceFiles +Arrays are not handled out-of-the-box. Users need to specify custom `ArrayWritable` subtypes when reading or writing. When writing, +users also need to specify custom converters that convert arrays to custom `ArrayWritable` subtypes. When reading, the default +converter will convert custom `ArrayWritable` subtypes to Java `Object[]`, which then get pickled to Python tuples. To get +Python `array.array` for arrays of primitive types, users need to specify custom converters. + +**Saving and Loading SequenceFiles** -Similarly to text files, SequenceFiles can be loaded by specifying the path. The key and value +Similarly to text files, SequenceFiles can be saved and loaded by specifying the path. The key and value classes can be specified, but for standard Writables this is not required. {% highlight python %} ->>> rdd = sc.sequenceFile("path/to/sequencefile/of/doubles") ->>> rdd.collect() # this example has DoubleWritable keys and Text values -[(1.0, u'aa'), - (2.0, u'bb'), - (2.0, u'aa'), - (3.0, u'cc'), - (2.0, u'bb'), - (1.0, u'aa')] +>>> rdd = sc.parallelize(range(1, 4)).map(lambda x: (x, "a" * x )) +>>> rdd.saveAsSequenceFile("path/to/file") +>>> sorted(sc.sequenceFile("path/to/file").collect()) +[(1, u'a'), (2, u'aa'), (3, u'aaa')] {% endhighlight %} -#### Loading Other Hadoop InputFormats +**Saving and Loading Other Hadoop Input/Output Formats** -PySpark can also read any Hadoop InputFormat, for both 'new' and 'old' Hadoop APIs. If required, -a Hadoop configuration can be passed in as a Python dict. Here is an example using the +PySpark can also read any Hadoop InputFormat or write any Hadoop OutputFormat, for both 'new' and 'old' Hadoop MapReduce APIs. +If required, a Hadoop configuration can be passed in as a Python dict. Here is an example using the Elasticsearch ESInputFormat: {% highlight python %} @@ -447,8 +445,7 @@ Note that, if the InputFormat simply depends on a Hadoop configuration and/or in the key and value classes can easily be converted according to the above table, then this approach should work well for such cases. -If you have custom serialized binary data (such as loading data from Cassandra / HBase) or custom -classes that don't conform to the JavaBean requirements, then you will first need to +If you have custom serialized binary data (such as loading data from Cassandra / HBase), then you will first need to transform that data on the Scala/Java side to something which can be handled by Pyrolite's pickler. A [Converter](api/scala/index.html#org.apache.spark.api.python.Converter) trait is provided for this. Simply extend this trait and implement your transformation code in the ```convert``` @@ -456,11 +453,8 @@ method. Remember to ensure that this class, along with any dependencies required classpath. See the [Python examples]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/python) and -the [Converter examples]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/scala/pythonconverters) -for examples of using HBase and Cassandra ```InputFormat```. - -Future support for writing data out as ```SequenceFileOutputFormat``` and other ```OutputFormats```, -is forthcoming. +the [Converter examples]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/scala/org/apache/spark/examples/pythonconverters) +for examples of using Cassandra / HBase ```InputFormat``` and ```OutputFormat``` with custom converters.
    @@ -739,7 +733,7 @@ def doStuff(self, rdd): While most Spark operations work on RDDs containing any type of objects, a few special operations are only available on RDDs of key-value pairs. -The most common ones are distibuted "shuffle" operations, such as grouping or aggregating the elements +The most common ones are distributed "shuffle" operations, such as grouping or aggregating the elements by a key. In Scala, these operations are automatically available on RDDs containing @@ -773,7 +767,7 @@ documentation](http://docs.oracle.com/javase/7/docs/api/java/lang/Object.html#ha While most Spark operations work on RDDs containing any type of objects, a few special operations are only available on RDDs of key-value pairs. -The most common ones are distibuted "shuffle" operations, such as grouping or aggregating the elements +The most common ones are distributed "shuffle" operations, such as grouping or aggregating the elements by a key. In Java, key-value pairs are represented using the @@ -810,7 +804,7 @@ documentation](http://docs.oracle.com/javase/7/docs/api/java/lang/Object.html#ha While most Spark operations work on RDDs containing any type of objects, a few special operations are only available on RDDs of key-value pairs. -The most common ones are distibuted "shuffle" operations, such as grouping or aggregating the elements +The most common ones are distributed "shuffle" operations, such as grouping or aggregating the elements by a key. In Python, these operations work on RDDs containing built-in Python tuples such as `(1, 2)`. @@ -1180,7 +1174,9 @@ value of the broadcast variable (e.g. if the variable is shipped to a new node l Accumulators are variables that are only "added" to through an associative operation and can therefore be efficiently supported in parallel. They can be used to implement counters (as in MapReduce) or sums. Spark natively supports accumulators of numeric types, and programmers -can add support for new types. +can add support for new types. If accumulators are created with a name, they will be +displayed in Spark's UI. This can can be useful for understanding the progress of +running stages (NOTE: this is not yet supported in Python). An accumulator is created from an initial value `v` by calling `SparkContext.accumulator(v)`. Tasks running on the cluster can then add to it using the `add` method or the `+=` operator (in Scala and Python). @@ -1194,7 +1190,7 @@ The code below shows an accumulator being used to add up the elements of an arra
    {% highlight scala %} -scala> val accum = sc.accumulator(0) +scala> val accum = sc.accumulator(0, "My Accumulator") accum: spark.Accumulator[Int] = 0 scala> sc.parallelize(Array(1, 2, 3, 4)).foreach(x => accum += x) diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 0362f5a223319..9bc20dbf926b2 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -17,10 +17,6 @@ To build Spark yourself, refer to the [building with Maven guide](building-with- Most of the configs are the same for Spark on YARN as for other deployment modes. See the [configuration page](configuration.html) for more information on those. These are configs that are specific to Spark on YARN. -#### Environment Variables - -* `SPARK_YARN_USER_ENV`, to add environment variables to the Spark processes launched on YARN. This can be a comma separated list of environment variables, e.g. `SPARK_YARN_USER_ENV="JAVA_HOME=/jdk64,FOO=bar"`. - #### Spark Properties @@ -106,6 +102,29 @@ Most of the configs are the same for Spark on YARN as for other deployment modes set this configuration to "hdfs:///some/path". + + + + + + + + + +
    spark.yarn.access.namenodes(none) + A list of secure HDFS namenodes your Spark application is going to access. For + example, `spark.yarn.access.namenodes=hdfs://nn1.com:8032,hdfs://nn2.com:8032`. + The Spark application must have acess to the namenodes listed and Kerberos must + be properly configured to be able to access them (either in the same realm or in + a trusted realm). Spark acquires security tokens for each of the namenodes so that + the Spark application can access those remote HDFS clusters. +
    spark.yarn.appMasterEnv.[EnvironmentVariableName](none) + Add the environment variable specified by EnvironmentVariableName to the + Application Master process launched on YARN. The user can specify multiple of + these and to set multiple environment variables. In yarn-cluster mode this controls + the environment of the SPARK driver and in yarn-client mode it only controls + the environment of the executor launcher. +
    # Launching Spark on YARN diff --git a/docs/security.md b/docs/security.md index 90ba678033b19..ec0523184d665 100644 --- a/docs/security.md +++ b/docs/security.md @@ -7,14 +7,142 @@ Spark currently supports authentication via a shared secret. Authentication can * For Spark on [YARN](running-on-yarn.html) deployments, configuring `spark.authenticate` to `true` will automatically handle generating and distributing the shared secret. Each application will use a unique shared secret. * For other types of Spark deployments, the Spark parameter `spark.authenticate.secret` should be configured on each of the nodes. This secret will be used by all the Master/Workers and applications. +* **IMPORTANT NOTE:** *The experimental Netty shuffle path (`spark.shuffle.use.netty`) is not secured, so do not use Netty for shuffles if running with authentication.* -The Spark UI can also be secured by using [javax servlet filters](http://docs.oracle.com/javaee/6/api/javax/servlet/Filter.html) via the `spark.ui.filters` setting. A user may want to secure the UI if it has data that other users should not be allowed to see. The javax servlet filter specified by the user can authenticate the user and then once the user is logged in, Spark can compare that user versus the view ACLs to make sure they are authorized to view the UI. The configs `spark.ui.acls.enable` and `spark.ui.view.acls` control the behavior of the ACLs. Note that the user who started the application always has view access to the UI. -On YARN, the Spark UI uses the standard YARN web application proxy mechanism and will authenticate via any installed Hadoop filters. +## Web UI + +The Spark UI can also be secured by using [javax servlet filters](http://docs.oracle.com/javaee/6/api/javax/servlet/Filter.html) via the `spark.ui.filters` setting. A user may want to secure the UI if it has data that other users should not be allowed to see. The javax servlet filter specified by the user can authenticate the user and then once the user is logged in, Spark can compare that user versus the view ACLs to make sure they are authorized to view the UI. The configs `spark.acls.enable` and `spark.ui.view.acls` control the behavior of the ACLs. Note that the user who started the application always has view access to the UI. On YARN, the Spark UI uses the standard YARN web application proxy mechanism and will authenticate via any installed Hadoop filters. + +Spark also supports modify ACLs to control who has access to modify a running Spark application. This includes things like killing the application or a task. This is controlled by the configs `spark.acls.enable` and `spark.modify.acls`. Note that if you are authenticating the web UI, in order to use the kill button on the web UI it might be necessary to add the users in the modify acls to the view acls also. On YARN, the modify acls are passed in and control who has modify access via YARN interfaces. + +Spark allows for a set of administrators to be specified in the acls who always have view and modify permissions to all the applications. is controlled by the config `spark.admin.acls`. This is useful on a shared cluster where you might have administrators or support staff who help users debug applications. + +## Event Logging If your applications are using event logging, the directory where the event logs go (`spark.eventLog.dir`) should be manually created and have the proper permissions set on it. If you want those log files secured, the permissions should be set to `drwxrwxrwxt` for that directory. The owner of the directory should be the super user who is running the history server and the group permissions should be restricted to super user group. This will allow all users to write to the directory but will prevent unprivileged users from removing or renaming a file unless they own the file or directory. The event log files will be created by Spark with permissions such that only the user and group have read and write access. -**IMPORTANT NOTE:** *The experimental Netty shuffle path (`spark.shuffle.use.netty`) is not secured, so do not use Netty for shuffles if running with authentication.* +## Configuring Ports for Network Security + +Spark makes heavy use of the network, and some environments have strict requirements for using tight +firewall settings. Below are the primary ports that Spark uses for its communication and how to +configure those ports. + +### Standalone mode only + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    FromToDefault PortPurposeConfiguration + SettingNotes
    BrowserStandalone Master8080Web UIspark.master.ui.port /
    SPARK_MASTER_WEBUI_PORT
    Jetty-based. Standalone mode only.
    BrowserStandalone Worker8081Web UIspark.worker.ui.port /
    SPARK_WORKER_WEBUI_PORT
    Jetty-based. Standalone mode only.
    Driver /
    Standalone Worker
    Standalone Master7077Submit job to cluster /
    Join cluster
    SPARK_MASTER_PORTAkka-based. Set to "0" to choose a port randomly. Standalone mode only.
    Standalone MasterStandalone Worker(random)Schedule executorsSPARK_WORKER_PORTAkka-based. Set to "0" to choose a port randomly. Standalone mode only.
    + +### All cluster managers + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    FromToDefault PortPurposeConfiguration + SettingNotes
    BrowserApplication4040Web UIspark.ui.portJetty-based
    BrowserHistory Server18080Web UIspark.history.ui.portJetty-based
    Executor /
    Standalone Master
    Driver(random)Connect to application /
    Notify executor state changes
    spark.driver.portAkka-based. Set to "0" to choose a port randomly.
    DriverExecutor(random)Schedule tasksspark.executor.portAkka-based. Set to "0" to choose a port randomly.
    ExecutorDriver(random)File server for files and jarsspark.fileserver.portJetty-based
    ExecutorDriver(random)HTTP Broadcastspark.broadcast.portJetty-based. Not used by TorrentBroadcast, which sends data through the block manager + instead.
    ExecutorDriver(random)Class file serverspark.replClassServer.portJetty-based. Only used in Spark shells.
    Executor / DriverExecutor / Driver(random)Block Manager portspark.blockManager.portRaw socket via ServerSocketChannel
    -See the [configuration page](configuration.html) for more details on the security configuration parameters. -See org.apache.spark.SecurityManager for implementation details about security. +See the [configuration page](configuration.html) for more details on the security configuration +parameters, and +org.apache.spark.SecurityManager for implementation details about security. diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index f5c0f7cef83d2..c791c81f8bfd0 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -156,6 +156,20 @@ SPARK_MASTER_OPTS supports the following system properties: + + + + + + + + + + @@ -228,9 +242,6 @@ To run an interactive Spark shell against the cluster, run the following command ./bin/spark-shell --master spark://IP:PORT -Note that if you are running spark-shell from one of the spark cluster machines, the `bin/spark-shell` script will -automatically set MASTER from the `SPARK_MASTER_IP` and `SPARK_MASTER_PORT` variables in `conf/spark-env.sh`. - You can also pass an option `--cores ` to control the number of cores that spark-shell uses on the cluster. # Launching Compiled Spark Applications @@ -288,97 +299,15 @@ You can run Spark alongside your existing Hadoop cluster by just launching it as # Configuring Ports for Network Security -Spark makes heavy use of the network, and some environments have strict requirements for using tight -firewall settings. Below are the primary ports that Spark uses for its communication and how to -configure those ports. - -
    Property NameDefaultMeaning
    spark.deploy.retainedApplications200 + The maximum number of completed applications to display. Older applications will be dropped from the UI to maintain this limit.
    +
    spark.deploy.retainedDrivers200 + The maximum number of completed drivers to display. Older drivers will be dropped from the UI to maintain this limit.
    +
    spark.deploy.spreadOut true
    - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    FromToDefault PortPurposeConfiguration - SettingNotes
    BrowserStandalone Cluster Master8080Web UImaster.ui.portJetty-based
    BrowserDriver4040Web UIspark.ui.portJetty-based
    BrowserHistory Server18080Web UIspark.history.ui.portJetty-based
    BrowserWorker8081Web UIworker.ui.portJetty-based
    ApplicationStandalone Cluster Master7077Submit job to clusterspark.driver.portAkka-based. Set to "0" to choose a port randomly
    WorkerStandalone Cluster Master7077Join clusterspark.driver.portAkka-based. Set to "0" to choose a port randomly
    ApplicationWorker(random)Join clusterSPARK_WORKER_PORT (standalone cluster)Akka-based
    Driver and other WorkersWorker(random) -
      -
    • File server for file and jars
    • -
    • Http Broadcast
    • -
    • Class file server (Spark Shell only)
    • -
    -
    NoneJetty-based. Each of these services starts on a random port that cannot be configured
    +Spark makes heavy use of the network, and some environments have strict requirements for using +tight firewall settings. For a complete list of ports to configure, see the +[security page](security.html#configuring-ports-for-network-security). # High Availability By default, standalone scheduling clusters are resilient to Worker failures (insofar as Spark itself is resilient to losing work by moving it to other workers). However, the scheduler uses a Master to make scheduling decisions, and this (by default) creates a single point of failure: if the Master crashes, no new applications can be created. In order to circumvent this, we have two high availability schemes, detailed below. -## Standby Masters with ZooKeeper +# Standby Masters with ZooKeeper **Overview** @@ -418,7 +347,7 @@ There's an important distinction to be made between "registering with a Master" Due to this property, new Masters can be created at any time, and the only thing you need to worry about is that _new_ applications and Workers can find it to register with in case it becomes the leader. Once registered, you're taken care of. -## Single-Node Recovery with Local File System +# Single-Node Recovery with Local File System **Overview** diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 522c83884ef42..cd6543945c385 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -136,13 +136,13 @@ val sqlContext = new org.apache.spark.sql.SQLContext(sc) import sqlContext.createSchemaRDD // Define the schema using a case class. -// Note: Case classes in Scala 2.10 can support only up to 22 fields. To work around this limit, +// Note: Case classes in Scala 2.10 can support only up to 22 fields. To work around this limit, // you can use custom classes that implement the Product interface. case class Person(name: String, age: Int) // Create an RDD of Person objects and register it as a table. val people = sc.textFile("examples/src/main/resources/people.txt").map(_.split(",")).map(p => Person(p(0), p(1).trim.toInt)) -people.registerAsTable("people") +people.registerTempTable("people") // SQL statements can be run by using the sql methods provided by sqlContext. val teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") @@ -210,7 +210,7 @@ JavaRDD people = sc.textFile("examples/src/main/resources/people.txt").m // Apply a schema to an RDD of JavaBeans and register it as a table. JavaSchemaRDD schemaPeople = sqlContext.applySchema(people, Person.class); -schemaPeople.registerAsTable("people"); +schemaPeople.registerTempTable("people"); // SQL can be run over RDDs that have been registered as tables. JavaSchemaRDD teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") @@ -248,7 +248,7 @@ people = parts.map(lambda p: {"name": p[0], "age": int(p[1])}) # In future versions of PySpark we would like to add support for registering RDDs with other # datatypes as tables schemaPeople = sqlContext.inferSchema(people) -schemaPeople.registerAsTable("people") +schemaPeople.registerTempTable("people") # SQL can be run over SchemaRDDs that have been registered as a table. teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") @@ -292,7 +292,7 @@ people.saveAsParquetFile("people.parquet") val parquetFile = sqlContext.parquetFile("people.parquet") //Parquet files can also be registered as tables and then used in SQL statements. -parquetFile.registerAsTable("parquetFile") +parquetFile.registerTempTable("parquetFile") val teenagers = sqlContext.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19") teenagers.map(t => "Name: " + t(0)).collect().foreach(println) {% endhighlight %} @@ -314,7 +314,7 @@ schemaPeople.saveAsParquetFile("people.parquet"); JavaSchemaRDD parquetFile = sqlContext.parquetFile("people.parquet"); //Parquet files can also be registered as tables and then used in SQL statements. -parquetFile.registerAsTable("parquetFile"); +parquetFile.registerTempTable("parquetFile"); JavaSchemaRDD teenagers = sqlContext.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19"); List teenagerNames = teenagers.map(new Function() { public String call(Row row) { @@ -340,7 +340,7 @@ schemaPeople.saveAsParquetFile("people.parquet") parquetFile = sqlContext.parquetFile("people.parquet") # Parquet files can also be registered as tables and then used in SQL statements. -parquetFile.registerAsTable("parquetFile"); +parquetFile.registerTempTable("parquetFile"); teenagers = sqlContext.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19") teenNames = teenagers.map(lambda p: "Name: " + p.name) for teenName in teenNames.collect(): @@ -378,7 +378,7 @@ people.printSchema() // |-- name: StringType // Register this SchemaRDD as a table. -people.registerAsTable("people") +people.registerTempTable("people") // SQL statements can be run by using the sql methods provided by sqlContext. val teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") @@ -416,7 +416,7 @@ people.printSchema(); // |-- name: StringType // Register this JavaSchemaRDD as a table. -people.registerAsTable("people"); +people.registerTempTable("people"); // SQL statements can be run by using the sql methods provided by sqlContext. JavaSchemaRDD teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19"); @@ -455,7 +455,7 @@ people.printSchema() # |-- name: StringType # Register this SchemaRDD as a table. -people.registerAsTable("people") +people.registerTempTable("people") # SQL statements can be run by using the sql methods provided by sqlContext. teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") @@ -474,7 +474,7 @@ anotherPeople = sqlContext.jsonRDD(anotherPeopleRDD) Spark SQL also supports reading and writing data stored in [Apache Hive](http://hive.apache.org/). However, since Hive has a large number of dependencies, it is not included in the default Spark assembly. -In order to use Hive you must first run '`SPARK_HIVE=true sbt/sbt assembly/assembly`' (or use `-Phive` for maven). +In order to use Hive you must first run '`sbt/sbt -Phive assembly/assembly`' (or use `-Phive` for maven). This command builds a new assembly jar that includes Hive. Note that this Hive assembly jar must also be present on all of the worker nodes, as they will need access to the Hive serialization and deserialization libraries (SerDes) in order to acccess data stored in Hive. @@ -487,19 +487,19 @@ Configuration of Hive is done by placing your `hive-site.xml` file in `conf/`. When working with Hive one must construct a `HiveContext`, which inherits from `SQLContext`, and adds support for finding tables in in the MetaStore and writing queries using HiveQL. Users who do -not have an existing Hive deployment can also experiment with the `LocalHiveContext`, -which is similar to `HiveContext`, but creates a local copy of the `metastore` and `warehouse` -automatically. +not have an existing Hive deployment can still create a HiveContext. When not configured by the +hive-site.xml, the context automatically creates `metastore_db` and `warehouse` in the current +directory. {% highlight scala %} // sc is an existing SparkContext. val hiveContext = new org.apache.spark.sql.hive.HiveContext(sc) -hiveContext.hql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") -hiveContext.hql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src") +hiveContext.sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") +hiveContext.sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src") // Queries are expressed in HiveQL -hiveContext.hql("FROM src SELECT key, value").collect().foreach(println) +hiveContext.sql("FROM src SELECT key, value").collect().foreach(println) {% endhighlight %}
    @@ -515,11 +515,11 @@ expressed in HiveQL. // sc is an existing JavaSparkContext. JavaHiveContext hiveContext = new org.apache.spark.sql.hive.api.java.HiveContext(sc); -hiveContext.hql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)"); -hiveContext.hql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src"); +hiveContext.sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)"); +hiveContext.sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src"); // Queries are expressed in HiveQL. -Row[] results = hiveContext.hql("FROM src SELECT key, value").collect(); +Row[] results = hiveContext.sql("FROM src SELECT key, value").collect(); {% endhighlight %} @@ -537,18 +537,17 @@ expressed in HiveQL. from pyspark.sql import HiveContext hiveContext = HiveContext(sc) -hiveContext.hql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") -hiveContext.hql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src") +hiveContext.sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") +hiveContext.sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src") # Queries can be expressed in HiveQL. -results = hiveContext.hql("FROM src SELECT key, value").collect() +results = hiveContext.sql("FROM src SELECT key, value").collect() {% endhighlight %}
    - # Writing Language-Integrated Relational Queries **Language-Integrated queries are currently only supported in Scala.** @@ -573,4 +572,210 @@ prefixed with a tick (`'`). Implicit conversions turn these symbols into expres evaluated by the SQL execution engine. A full list of the functions supported can be found in the [ScalaDoc](api/scala/index.html#org.apache.spark.sql.SchemaRDD). - \ No newline at end of file + + +## Running the Thrift JDBC server + +The Thrift JDBC server implemented here corresponds to the [`HiveServer2`] +(https://cwiki.apache.org/confluence/display/Hive/Setting+Up+HiveServer2) in Hive 0.12. You can test +the JDBC server with the beeline script comes with either Spark or Hive 0.12. In order to use Hive +you must first run '`sbt/sbt -Phive-thriftserver assembly/assembly`' (or use `-Phive-thriftserver` +for maven). + +To start the JDBC server, run the following in the Spark directory: + + ./sbin/start-thriftserver.sh + +The default port the server listens on is 10000. To listen on customized host and port, please set +the `HIVE_SERVER2_THRIFT_PORT` and `HIVE_SERVER2_THRIFT_BIND_HOST` environment variables. You may +run `./sbin/start-thriftserver.sh --help` for a complete list of all available options. Now you can +use beeline to test the Thrift JDBC server: + + ./bin/beeline + +Connect to the JDBC server in beeline with: + + beeline> !connect jdbc:hive2://localhost:10000 + +Beeline will ask you for a username and password. In non-secure mode, simply enter the username on +your machine and a blank password. For secure mode, please follow the instructions given in the +[beeline documentation](https://cwiki.apache.org/confluence/display/Hive/HiveServer2+Clients) + +Configuration of Hive is done by placing your `hive-site.xml` file in `conf/`. + +You may also use the beeline script comes with Hive. + +### Migration Guide for Shark Users + +#### Reducer number + +In Shark, default reducer number is 1 and is controlled by the property `mapred.reduce.tasks`. Spark +SQL deprecates this property by a new property `spark.sql.shuffle.partitions`, whose default value +is 200. Users may customize this property via `SET`: + +``` +SET spark.sql.shuffle.partitions=10; +SELECT page, count(*) c FROM logs_last_month_cached +GROUP BY page ORDER BY c DESC LIMIT 10; +``` + +You may also put this property in `hive-site.xml` to override the default value. + +For now, the `mapred.reduce.tasks` property is still recognized, and is converted to +`spark.sql.shuffle.partitions` automatically. + +#### Caching + +The `shark.cache` table property no longer exists, and tables whose name end with `_cached` are no +longer automcatically cached. Instead, we provide `CACHE TABLE` and `UNCACHE TABLE` statements to +let user control table caching explicitly: + +``` +CACHE TABLE logs_last_month; +UNCACHE TABLE logs_last_month; +``` + +**NOTE** `CACHE TABLE tbl` is lazy, it only marks table `tbl` as "need to by cached if necessary", +but doesn't actually cache it until a query that touches `tbl` is executed. To force the table to be +cached, you may simply count the table immediately after executing `CACHE TABLE`: + +``` +CACHE TABLE logs_last_month; +SELECT COUNT(1) FROM logs_last_month; +``` + +Several caching related features are not supported yet: + +* User defined partition level cache eviction policy +* RDD reloading +* In-memory cache write through policy + +### Compatibility with Apache Hive + +#### Deploying in Exising Hive Warehouses + +Spark SQL Thrift JDBC server is designed to be "out of the box" compatible with existing Hive +installations. You do not need to modify your existing Hive Metastore or change the data placement +or partitioning of your tables. + +#### Supported Hive Features + +Spark SQL supports the vast majority of Hive features, such as: + +* Hive query statements, including: + * `SELECT` + * `GROUP BY + * `ORDER BY` + * `CLUSTER BY` + * `SORT BY` +* All Hive operators, including: + * Relational operators (`=`, `⇔`, `==`, `<>`, `<`, `>`, `>=`, `<=`, etc) + * Arthimatic operators (`+`, `-`, `*`, `/`, `%`, etc) + * Logical operators (`AND`, `&&`, `OR`, `||`, etc) + * Complex type constructors + * Mathemtatical functions (`sign`, `ln`, `cos`, etc) + * String functions (`instr`, `length`, `printf`, etc) +* User defined functions (UDF) +* User defined aggregation functions (UDAF) +* User defined serialization formats (SerDe's) +* Joins + * `JOIN` + * `{LEFT|RIGHT|FULL} OUTER JOIN` + * `LEFT SEMI JOIN` + * `CROSS JOIN` +* Unions +* Sub queries + * `SELECT col FROM ( SELECT a + b AS col from t1) t2` +* Sampling +* Explain +* Partitioned tables +* All Hive DDL Functions, including: + * `CREATE TABLE` + * `CREATE TABLE AS SELECT` + * `ALTER TABLE` +* Most Hive Data types, including: + * `TINYINT` + * `SMALLINT` + * `INT` + * `BIGINT` + * `BOOLEAN` + * `FLOAT` + * `DOUBLE` + * `STRING` + * `BINARY` + * `TIMESTAMP` + * `ARRAY<>` + * `MAP<>` + * `STRUCT<>` + +#### Unsupported Hive Functionality + +Below is a list of Hive features that we don't support yet. Most of these features are rarely used +in Hive deployments. + +**Major Hive Features** + +* Tables with buckets: bucket is the hash partitioning within a Hive table partition. Spark SQL + doesn't support buckets yet. + +**Esoteric Hive Features** + +* Tables with partitions using different input formats: In Spark SQL, all table partitions need to + have the same input format. +* Non-equi outer join: For the uncommon use case of using outer joins with non-equi join conditions + (e.g. condition "`key < 10`"), Spark SQL will output wrong result for the `NULL` tuple. +* `UNIONTYPE` +* Unique join +* Single query multi insert +* Column statistics collecting: Spark SQL does not piggyback scans to collect column statistics at + the moment. + +**Hive Input/Output Formats** + +* File format for CLI: For results showing back to the CLI, Spark SQL only supports TextOutputFormat. +* Hadoop archive + +**Hive Optimizations** + +A handful of Hive optimizations are not yet included in Spark. Some of these (such as indexes) are +not necessary due to Spark SQL's in-memory computational model. Others are slotted for future +releases of Spark SQL. + +* Block level bitmap indexes and virtual columns (used to build indexes) +* Automatically convert a join to map join: For joining a large table with multiple small tables, + Hive automatically converts the join into a map join. We are adding this auto conversion in the + next release. +* Automatically determine the number of reducers for joins and groupbys: Currently in Spark SQL, you + need to control the degree of parallelism post-shuffle using "SET + spark.sql.shuffle.partitions=[num_tasks];". We are going to add auto-setting of parallelism in the + next release. +* Meta-data only query: For queries that can be answered by using only meta data, Spark SQL still + launches tasks to compute the result. +* Skew data flag: Spark SQL does not follow the skew data flags in Hive. +* `STREAMTABLE` hint in join: Spark SQL does not follow the `STREAMTABLE` hint. +* Merge multiple small files for query results: if the result output contains multiple small files, + Hive can optionally merge the small files into fewer large files to avoid overflowing the HDFS + metadata. Spark SQL does not support that. + +## Running the Spark SQL CLI + +The Spark SQL CLI is a convenient tool to run the Hive metastore service in local mode and execute +queries input from command line. Note: the Spark SQL CLI cannot talk to the Thrift JDBC server. + +To start the Spark SQL CLI, run the following in the Spark directory: + + ./bin/spark-sql + +Configuration of Hive is done by placing your `hive-site.xml` file in `conf/`. +You may run `./bin/spark-sql --help` for a complete list of all available +options. + +# Cached tables + +Spark SQL can cache tables using an in-memory columnar format by calling `cacheTable("tableName")`. +Then Spark SQL will scan only required columns and will automatically tune compression to minimize +memory usage and GC pressure. You can call `uncacheTable("tableName")` to remove the table from memory. + +Note that if you just call `cache` rather than `cacheTable`, tables will _not_ be cached in +in-memory columnar format. So we strongly recommend using `cacheTable` whenever you want to +cache tables. diff --git a/docs/streaming-custom-receivers.md b/docs/streaming-custom-receivers.md index a2dc3a8961dfc..1e045a3dd0ca9 100644 --- a/docs/streaming-custom-receivers.md +++ b/docs/streaming-custom-receivers.md @@ -4,7 +4,7 @@ title: Spark Streaming Custom Receivers --- Spark Streaming can receive streaming data from any arbitrary data source beyond -the one's for which it has in-built support (that is, beyond Flume, Kafka, files, sockets, etc.). +the one's for which it has in-built support (that is, beyond Flume, Kafka, Kinesis, files, sockets, etc.). This requires the developer to implement a *receiver* that is customized for receiving data from the concerned data source. This guide walks through the process of implementing a custom receiver and using it in a Spark Streaming application. @@ -174,7 +174,7 @@ val words = lines.flatMap(_.split(" ")) ... {% endhighlight %} -The full source code is in the example [CustomReceiver.scala](https://github.com/apache/spark/blob/master/examples/src/main/scala/org/apache/spark/streaming/examples/CustomReceiver.scala). +The full source code is in the example [CustomReceiver.scala](https://github.com/apache/spark/blob/master/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala).
    diff --git a/docs/streaming-kinesis.md b/docs/streaming-kinesis.md new file mode 100644 index 0000000000000..16ad3222105a2 --- /dev/null +++ b/docs/streaming-kinesis.md @@ -0,0 +1,59 @@ +--- +layout: global +title: Spark Streaming Kinesis Receiver +--- + +## Kinesis +###Design +
  • The KinesisReceiver uses the Kinesis Client Library (KCL) provided by Amazon under the Amazon Software License.
  • +
  • The KCL builds on top of the Apache 2.0 licensed AWS Java SDK and provides load-balancing, fault-tolerance, checkpointing through the concept of Workers, Checkpoints, and Shard Leases.
  • +
  • The KCL uses DynamoDB to maintain all state. A DynamoDB table is created in the us-east-1 region (regardless of Kinesis stream region) during KCL initialization for each Kinesis application name.
  • +
  • A single KinesisReceiver can process many shards of a stream by spinning up multiple KinesisRecordProcessor threads.
  • +
  • You never need more KinesisReceivers than the number of shards in your stream as each will spin up at least one KinesisRecordProcessor thread.
  • +
  • Horizontal scaling is achieved by autoscaling additional KinesisReceiver (separate processes) or spinning up new KinesisRecordProcessor threads within each KinesisReceiver - up to the number of current shards for a given stream, of course. Don't forget to autoscale back down!
  • + +### Build +
  • Spark supports a Streaming KinesisReceiver, but it is not included in the default build due to Amazon Software Licensing (ASL) restrictions.
  • +
  • To build with the Kinesis Streaming Receiver and supporting ASL-licensed code, you must run the maven or sbt builds with the **-Pkinesis-asl** profile.
  • +
  • All KinesisReceiver-related code, examples, tests, and artifacts live in **$SPARK_HOME/extras/kinesis-asl/**.
  • +
  • Kinesis-based Spark Applications will need to link to the **spark-streaming-kinesis-asl** artifact that is built when **-Pkinesis-asl** is specified.
  • +
  • _**Note that by linking to this library, you will include [ASL](https://aws.amazon.com/asl/)-licensed code in your Spark package**_.
  • + +###Example +
  • To build the Kinesis example, you must run the maven or sbt builds with the **-Pkinesis-asl** profile.
  • +
  • You need to setup a Kinesis stream at one of the valid Kinesis endpoints with 1 or more shards per the following: http://docs.aws.amazon.com/kinesis/latest/dev/step-one-create-stream.html
  • +
  • Valid Kinesis endpoints can be found here: http://docs.aws.amazon.com/general/latest/gr/rande.html#ak_region
  • +
  • When running **locally**, the example automatically determines the number of threads and KinesisReceivers to spin up based on the number of shards configured for the stream. Therefore, **local[n]** is not needed when starting the example as with other streaming examples.
  • +
  • While this example could use a single KinesisReceiver which spins up multiple KinesisRecordProcessor threads to process multiple shards, I wanted to demonstrate unioning multiple KinesisReceivers as a single DStream. (It's a bit confusing in local mode.)
  • +
  • **KinesisWordCountProducerASL** is provided to generate random records into the Kinesis stream for testing.
  • +
  • The example has been configured to immediately replicate incoming stream data to another node by using (StorageLevel.MEMORY_AND_DISK_2) +
  • Spark checkpointing is disabled because the example does not use any stateful or window-based DStream operations such as updateStateByKey and reduceByWindow. If those operations are introduced, you would need to enable checkpointing or risk losing data in the case of a failure.
  • +
  • Kinesis checkpointing is enabled. This means that the example will recover from a Kinesis failure.
  • +
  • The example uses InitialPositionInStream.LATEST strategy to pull from the latest tip of the stream if no Kinesis checkpoint info exists.
  • +
  • In our example, **KinesisWordCount** is the Kinesis application name for both the Scala and Java versions. The use of this application name is described next.
  • + +###Deployment and Runtime +
  • A Kinesis application name must be unique for a given account and region.
  • +
  • A DynamoDB table and CloudWatch namespace are created during KCL initialization using this Kinesis application name. http://docs.aws.amazon.com/kinesis/latest/dev/kinesis-record-processor-implementation-app.html#kinesis-record-processor-initialization
  • +
  • This DynamoDB table lives in the us-east-1 region regardless of the Kinesis endpoint URL.
  • +
  • Changing the app name or stream name could lead to Kinesis errors as only a single logical application can process a single stream.
  • +
  • If you are seeing errors after changing the app name or stream name, it may be necessary to manually delete the DynamoDB table and start from scratch.
  • +
  • The Kinesis libraries must be present on all worker nodes, as they will need access to the KCL.
  • +
  • The KinesisReceiver uses the DefaultAWSCredentialsProviderChain for AWS credentials which searches for credentials in the following order of precedence:
    +1) Environment Variables - AWS_ACCESS_KEY_ID and AWS_SECRET_KEY
    +2) Java System Properties - aws.accessKeyId and aws.secretKey
    +3) Credential profiles file - default location (~/.aws/credentials) shared by all AWS SDKs
    +4) Instance profile credentials - delivered through the Amazon EC2 metadata service +
  • + +###Fault-Tolerance +
  • The combination of Spark Streaming and Kinesis creates 2 different checkpoints that may occur at different intervals.
  • +
  • Checkpointing too frequently against Kinesis will cause excess load on the AWS checkpoint storage layer and may lead to AWS throttling. The provided example handles this throttling with a random backoff retry strategy.
  • +
  • Upon startup, a KinesisReceiver will begin processing records with sequence numbers greater than the last Kinesis checkpoint sequence number recorded per shard (stored in the DynamoDB table).
  • +
  • If no Kinesis checkpoint info exists, the KinesisReceiver will start either from the oldest record available (InitialPositionInStream.TRIM_HORIZON) or from the latest tip (InitialPostitionInStream.LATEST). This is configurable.
  • +
  • InitialPositionInStream.LATEST could lead to missed records if data is added to the stream while no KinesisReceivers are running (and no checkpoint info is being stored.)
  • +
  • In production, you'll want to switch to InitialPositionInStream.TRIM_HORIZON which will read up to 24 hours (Kinesis limit) of previous stream data.
  • +
  • InitialPositionInStream.TRIM_HORIZON may lead to duplicate processing of records where the impact is dependent on checkpoint frequency.
  • +
  • Record processing should be idempotent when possible.
  • +
  • A failed or latent KinesisRecordProcessor within the KinesisReceiver will be detected and automatically restarted by the KCL.
  • +
  • If possible, the KinesisReceiver should be shutdown cleanly in order to trigger a final checkpoint of all KinesisRecordProcessors to avoid duplicate record processing.
  • \ No newline at end of file diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 90a0eef60c200..9f331ed50d2a4 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -9,7 +9,7 @@ title: Spark Streaming Programming Guide # Overview Spark Streaming is an extension of the core Spark API that allows enables high-throughput, fault-tolerant stream processing of live data streams. Data can be ingested from many sources -like Kafka, Flume, Twitter, ZeroMQ or plain old TCP sockets and be processed using complex +like Kafka, Flume, Twitter, ZeroMQ, Kinesis or plain old TCP sockets and be processed using complex algorithms expressed with high-level functions like `map`, `reduce`, `join` and `window`. Finally, processed data can be pushed out to filesystems, databases, and live dashboards. In fact, you can apply Spark's in-built @@ -38,7 +38,7 @@ stream of results in batches. Spark Streaming provides a high-level abstraction called *discretized stream* or *DStream*, which represents a continuous stream of data. DStreams can be created either from input data -stream from sources such as Kafka and Flume, or by applying high-level +stream from sources such as Kafka, Flume, and Kinesis, or by applying high-level operations on other DStreams. Internally, a DStream is represented as a sequence of [RDDs](api/scala/index.html#org.apache.spark.rdd.RDD). @@ -313,7 +313,7 @@ To write your own Spark Streaming program, you will have to add the following de artifactId = spark-streaming_{{site.SCALA_BINARY_VERSION}} version = {{site.SPARK_VERSION}} -For ingesting data from sources like Kafka and Flume that are not present in the Spark +For ingesting data from sources like Kafka, Flume, and Kinesis that are not present in the Spark Streaming core API, you will have to add the corresponding artifact `spark-streaming-xyz_{{site.SCALA_BINARY_VERSION}}` to the dependencies. For example, @@ -327,6 +327,7 @@ some of the common ones are as follows. Twitter spark-streaming-twitter_{{site.SCALA_BINARY_VERSION}} ZeroMQ spark-streaming-zeromq_{{site.SCALA_BINARY_VERSION}} MQTT spark-streaming-mqtt_{{site.SCALA_BINARY_VERSION}} + Kinesis
    (built separately) kinesis-asl_{{site.SCALA_BINARY_VERSION}} @@ -442,7 +443,7 @@ see the API documentations of the relevant functions in Scala and [JavaStreamingContext](api/scala/index.html#org.apache.spark.streaming.api.java.JavaStreamingContext) for Java. -Additional functionality for creating DStreams from sources such as Kafka, Flume, and Twitter +Additional functionality for creating DStreams from sources such as Kafka, Flume, Kinesis, and Twitter can be imported by adding the right dependencies as explained in an [earlier](#linking) section. To take the case of Kafka, after adding the artifact `spark-streaming-kafka_{{site.SCALA_BINARY_VERSION}}` to the @@ -467,6 +468,9 @@ For more details on these additional sources, see the corresponding [API documen Furthermore, you can also implement your own custom receiver for your sources. See the [Custom Receiver Guide](streaming-custom-receivers.html). +### Kinesis +[Kinesis](streaming-kinesis.html) + ## Operations There are two kinds of DStream operations - _transformations_ and _output operations_. Similar to RDD transformations, DStream transformations operate on one or more DStreams to create new DStreams @@ -939,7 +943,7 @@ Receiving multiple data streams can therefore be achieved by creating multiple i and configuring them to receive different partitions of the data stream from the source(s). For example, a single Kafka input stream receiving two topics of data can be split into two Kafka input streams, each receiving only one topic. This would run two receivers on two workers, -thus allowing data to received in parallel, and increasing overall throughput. +thus allowing data to be received in parallel, and increasing overall throughput. Another parameter that should be considered is the receiver's blocking interval. For most receivers, the received data is coalesced together into large blocks of data before storing inside Spark's memory. @@ -980,7 +984,7 @@ If the number of tasks launched per second is high (say, 50 or more per second), of sending out tasks to the slaves maybe significant and will make it hard to achieve sub-second latencies. The overhead can be reduced by the following changes: -* **Task Serialization**: Using Kryo serialization for serializing tasks can reduced the task +* **Task Serialization**: Using Kryo serialization for serializing tasks can reduce the task sizes, and therefore reduce the time taken to send them to the slaves. * **Execution mode**: Running Spark in Standalone mode or coarse-grained Mesos mode leads to diff --git a/docs/submitting-applications.md b/docs/submitting-applications.md index e05883072bfa8..45b70b1a5457a 100644 --- a/docs/submitting-applications.md +++ b/docs/submitting-applications.md @@ -33,6 +33,7 @@ dependencies, and can support different cluster managers and deploy modes that S --class --master \ --deploy-mode \ + --conf = \ ... # other options \ [application-arguments] @@ -43,6 +44,7 @@ Some of the commonly used options are: * `--class`: The entry point for your application (e.g. `org.apache.spark.examples.SparkPi`) * `--master`: The [master URL](#master-urls) for the cluster (e.g. `spark://23.195.26.187:7077`) * `--deploy-mode`: Whether to deploy your driver on the worker nodes (`cluster`) or locally as an external client (`client`) (default: `client`)* +* `--conf`: Arbitrary Spark configuration property in key=value format. For values that contain spaces wrap "key=value" in quotes (as shown). * `application-jar`: Path to a bundled jar including your application and all dependencies. The URL must be globally visible inside of your cluster, for instance, an `hdfs://` path or a `file://` path that is present on all nodes. * `application-arguments`: Arguments passed to the main method of your main class, if any diff --git a/docs/tuning.md b/docs/tuning.md index 4917c11bc1147..8fb2a0433b1a8 100644 --- a/docs/tuning.md +++ b/docs/tuning.md @@ -32,7 +32,7 @@ in your operations) and performance. It provides two serialization libraries: [`java.io.Externalizable`](http://docs.oracle.com/javase/6/docs/api/java/io/Externalizable.html). Java serialization is flexible but often quite slow, and leads to large serialized formats for many classes. -* [Kryo serialization](http://code.google.com/p/kryo/): Spark can also use +* [Kryo serialization](https://github.com/EsotericSoftware/kryo): Spark can also use the Kryo library (version 2) to serialize objects more quickly. Kryo is significantly faster and more compact than Java serialization (often as much as 10x), but does not support all `Serializable` types and requires you to *register* the classes you'll use in the program in advance @@ -68,7 +68,7 @@ conf.set("spark.kryo.registrator", "mypackage.MyRegistrator") val sc = new SparkContext(conf) {% endhighlight %} -The [Kryo documentation](http://code.google.com/p/kryo/) describes more advanced +The [Kryo documentation](https://github.com/EsotericSoftware/kryo) describes more advanced registration options, such as adding custom serialization code. If your objects are large, you may also need to increase the `spark.kryoserializer.buffer.mb` diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index f5c2bfb697c81..0c2f85a3868f4 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -135,6 +135,10 @@ def parse_args(): "--master-opts", type="string", default="", help="Extra options to give to master through SPARK_MASTER_OPTS variable " + "(e.g -Dspark.worker.timeout=180)") + parser.add_option( + "--user-data", type="string", default="", + help="Path to a user-data file (most AMI's interpret this as an initialization script)") + (opts, args) = parser.parse_args() if len(args) != 2: @@ -240,7 +244,10 @@ def get_spark_ami(opts): "r3.xlarge": "hvm", "r3.2xlarge": "hvm", "r3.4xlarge": "hvm", - "r3.8xlarge": "hvm" + "r3.8xlarge": "hvm", + "t2.micro": "hvm", + "t2.small": "hvm", + "t2.medium": "hvm" } if opts.instance_type in instance_types: instance_type = instance_types[opts.instance_type] @@ -271,6 +278,12 @@ def launch_cluster(conn, opts, cluster_name): if opts.key_pair is None: print >> stderr, "ERROR: Must provide a key pair name (-k) to use on instances." sys.exit(1) + + user_data_content = None + if opts.user_data: + with open(opts.user_data) as user_data_file: + user_data_content = user_data_file.read() + print "Setting up security groups..." master_group = get_or_make_group(conn, cluster_name + "-master") slave_group = get_or_make_group(conn, cluster_name + "-slaves") @@ -344,7 +357,8 @@ def launch_cluster(conn, opts, cluster_name): key_name=opts.key_pair, security_groups=[slave_group], instance_type=opts.instance_type, - block_device_map=block_map) + block_device_map=block_map, + user_data=user_data_content) my_req_ids += [req.id for req in slave_reqs] i += 1 @@ -395,7 +409,8 @@ def launch_cluster(conn, opts, cluster_name): placement=zone, min_count=num_slaves_this_zone, max_count=num_slaves_this_zone, - block_device_map=block_map) + block_device_map=block_map, + user_data=user_data_content) slave_nodes += slave_res.instances print "Launched %d slaves in %s, regid = %s" % (num_slaves_this_zone, zone, slave_res.id) @@ -428,11 +443,11 @@ def launch_cluster(conn, opts, cluster_name): for master in master_nodes: master.add_tag( key='Name', - value='spark-{cn}-master-{iid}'.format(cn=cluster_name, iid=master.id)) + value='{cn}-master-{iid}'.format(cn=cluster_name, iid=master.id)) for slave in slave_nodes: slave.add_tag( key='Name', - value='spark-{cn}-slave-{iid}'.format(cn=cluster_name, iid=slave.id)) + value='{cn}-slave-{iid}'.format(cn=cluster_name, iid=slave.id)) # Return all the instances return (master_nodes, slave_nodes) @@ -699,6 +714,7 @@ def ssh(host, opts, command): time.sleep(30) tries = tries + 1 + # Backported from Python 2.7 for compatiblity with 2.6 (See SPARK-1990) def _check_output(*popenargs, **kwargs): if 'stdout' in kwargs: diff --git a/examples/pom.xml b/examples/pom.xml index 4f6d7fdb87d47..8c4c128bb484d 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -27,10 +27,26 @@ org.apache.spark spark-examples_2.10 + + examples + jar Spark Project Examples http://spark.apache.org/ + + + kinesis-asl + + + org.apache.spark + spark-streaming-kinesis-asl_${scala.binary.version} + ${project.version} + + + + + org.apache.spark diff --git a/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java b/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java index 607df3eddd550..898297dc658ba 100644 --- a/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java +++ b/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java @@ -74,7 +74,7 @@ public Person call(String line) throws Exception { // Apply a schema to an RDD of Java Beans and register it as a table. JavaSchemaRDD schemaPeople = sqlCtx.applySchema(people, Person.class); - schemaPeople.registerAsTable("people"); + schemaPeople.registerTempTable("people"); // SQL can be run over RDDs that have been registered as tables. JavaSchemaRDD teenagers = sqlCtx.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19"); @@ -100,7 +100,7 @@ public String call(Row row) { JavaSchemaRDD parquetFile = sqlCtx.parquetFile("people.parquet"); //Parquet files can also be registered as tables and then used in SQL statements. - parquetFile.registerAsTable("parquetFile"); + parquetFile.registerTempTable("parquetFile"); JavaSchemaRDD teenagers2 = sqlCtx.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19"); teenagerNames = teenagers2.map(new Function() { @@ -128,7 +128,7 @@ public String call(Row row) { // |-- name: StringType // Register this JavaSchemaRDD as a table. - peopleFromJsonFile.registerAsTable("people"); + peopleFromJsonFile.registerTempTable("people"); // SQL statements can be run by using the sql methods provided by sqlCtx. JavaSchemaRDD teenagers3 = sqlCtx.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19"); @@ -158,7 +158,7 @@ public String call(Row row) { // | |-- state: StringType // |-- name: StringType - peopleFromJsonRDD.registerAsTable("people2"); + peopleFromJsonRDD.registerTempTable("people2"); JavaSchemaRDD peopleWithCity = sqlCtx.sql("SELECT name, address.city FROM people2"); List nameAndCity = peopleWithCity.map(new Function() { diff --git a/examples/src/main/python/als.py b/examples/src/main/python/als.py index 1a7c4c51f48cd..c862650b0aa1d 100755 --- a/examples/src/main/python/als.py +++ b/examples/src/main/python/als.py @@ -16,6 +16,9 @@ # """ +This is an example implementation of ALS for learning how to use Spark. Please refer to +ALS in pyspark.mllib.recommendation for more conventional use. + This example requires numpy (http://www.numpy.org/) """ from os.path import realpath @@ -49,9 +52,15 @@ def update(i, vec, mat, ratings): if __name__ == "__main__": + """ Usage: als [M] [U] [F] [iterations] [slices]" """ + + print >> sys.stderr, """WARN: This is a naive implementation of ALS and is given as an + example. Please use the ALS method found in pyspark.mllib.recommendation for more + conventional use.""" + sc = SparkContext(appName="PythonALS") M = int(sys.argv[1]) if len(sys.argv) > 1 else 100 U = int(sys.argv[2]) if len(sys.argv) > 2 else 500 diff --git a/examples/src/main/python/avro_inputformat.py b/examples/src/main/python/avro_inputformat.py new file mode 100644 index 0000000000000..e902ae29753c0 --- /dev/null +++ b/examples/src/main/python/avro_inputformat.py @@ -0,0 +1,75 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import sys + +from pyspark import SparkContext + +""" +Read data file users.avro in local Spark distro: + +$ cd $SPARK_HOME +$ ./bin/spark-submit --driver-class-path /path/to/example/jar ./examples/src/main/python/avro_inputformat.py \ +> examples/src/main/resources/users.avro +{u'favorite_color': None, u'name': u'Alyssa', u'favorite_numbers': [3, 9, 15, 20]} +{u'favorite_color': u'red', u'name': u'Ben', u'favorite_numbers': []} + +To read name and favorite_color fields only, specify the following reader schema: + +$ cat examples/src/main/resources/user.avsc +{"namespace": "example.avro", + "type": "record", + "name": "User", + "fields": [ + {"name": "name", "type": "string"}, + {"name": "favorite_color", "type": ["string", "null"]} + ] +} + +$ ./bin/spark-submit --driver-class-path /path/to/example/jar ./examples/src/main/python/avro_inputformat.py \ +> examples/src/main/resources/users.avro examples/src/main/resources/user.avsc +{u'favorite_color': None, u'name': u'Alyssa'} +{u'favorite_color': u'red', u'name': u'Ben'} +""" +if __name__ == "__main__": + if len(sys.argv) != 2 and len(sys.argv) != 3: + print >> sys.stderr, """ + Usage: avro_inputformat [reader_schema_file] + + Run with example jar: + ./bin/spark-submit --driver-class-path /path/to/example/jar /path/to/examples/avro_inputformat.py [reader_schema_file] + Assumes you have Avro data stored in . Reader schema can be optionally specified in [reader_schema_file]. + """ + exit(-1) + + path = sys.argv[1] + sc = SparkContext(appName="AvroKeyInputFormat") + + conf = None + if len(sys.argv) == 3: + schema_rdd = sc.textFile(sys.argv[2], 1).collect() + conf = {"avro.schema.input.key" : reduce(lambda x, y: x+y, schema_rdd)} + + avro_rdd = sc.newAPIHadoopFile(path, + "org.apache.avro.mapreduce.AvroKeyInputFormat", + "org.apache.avro.mapred.AvroKey", + "org.apache.hadoop.io.NullWritable", + keyConverter="org.apache.spark.examples.pythonconverters.AvroWrapperToJavaConverter", + conf=conf) + output = avro_rdd.map(lambda x: x[0]).collect() + for k in output: + print k diff --git a/examples/src/main/python/cassandra_outputformat.py b/examples/src/main/python/cassandra_outputformat.py new file mode 100644 index 0000000000000..1dfbf98604425 --- /dev/null +++ b/examples/src/main/python/cassandra_outputformat.py @@ -0,0 +1,83 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import sys + +from pyspark import SparkContext + +""" +Create data in Cassandra fist +(following: https://wiki.apache.org/cassandra/GettingStarted) + +cqlsh> CREATE KEYSPACE test + ... WITH REPLICATION = { 'class' : 'SimpleStrategy', 'replication_factor' : 1 }; +cqlsh> use test; +cqlsh:test> CREATE TABLE users ( + ... user_id int PRIMARY KEY, + ... fname text, + ... lname text + ... ); + +> cassandra_outputformat test users 1745 john smith +> cassandra_outputformat test users 1744 john doe +> cassandra_outputformat test users 1746 john smith + +cqlsh:test> SELECT * FROM users; + + user_id | fname | lname +---------+-------+------- + 1745 | john | smith + 1744 | john | doe + 1746 | john | smith +""" +if __name__ == "__main__": + if len(sys.argv) != 7: + print >> sys.stderr, """ + Usage: cassandra_outputformat + + Run with example jar: + ./bin/spark-submit --driver-class-path /path/to/example/jar /path/to/examples/cassandra_outputformat.py + Assumes you have created the following table in Cassandra already, + running on , in . + + cqlsh:> CREATE TABLE ( + ... user_id int PRIMARY KEY, + ... fname text, + ... lname text + ... ); + """ + exit(-1) + + host = sys.argv[1] + keyspace = sys.argv[2] + cf = sys.argv[3] + sc = SparkContext(appName="CassandraOutputFormat") + + conf = {"cassandra.output.thrift.address":host, + "cassandra.output.thrift.port":"9160", + "cassandra.output.keyspace":keyspace, + "cassandra.output.partitioner.class":"Murmur3Partitioner", + "cassandra.output.cql":"UPDATE " + keyspace + "." + cf + " SET fname = ?, lname = ?", + "mapreduce.output.basename":cf, + "mapreduce.outputformat.class":"org.apache.cassandra.hadoop.cql3.CqlOutputFormat", + "mapreduce.job.output.key.class":"java.util.Map", + "mapreduce.job.output.value.class":"java.util.List"} + key = {"user_id" : int(sys.argv[4])} + sc.parallelize([(key, sys.argv[5:])]).saveAsNewAPIHadoopDataset( + conf=conf, + keyConverter="org.apache.spark.examples.pythonconverters.ToCassandraCQLKeyConverter", + valueConverter="org.apache.spark.examples.pythonconverters.ToCassandraCQLValueConverter") diff --git a/examples/src/main/python/hbase_inputformat.py b/examples/src/main/python/hbase_inputformat.py index 3289d9880a0f5..c9fa8e171c2a1 100644 --- a/examples/src/main/python/hbase_inputformat.py +++ b/examples/src/main/python/hbase_inputformat.py @@ -65,7 +65,8 @@ "org.apache.hadoop.hbase.mapreduce.TableInputFormat", "org.apache.hadoop.hbase.io.ImmutableBytesWritable", "org.apache.hadoop.hbase.client.Result", - valueConverter="org.apache.spark.examples.pythonconverters.HBaseConverter", + keyConverter="org.apache.spark.examples.pythonconverters.ImmutableBytesWritableToStringConverter", + valueConverter="org.apache.spark.examples.pythonconverters.HBaseResultToStringConverter", conf=conf) output = hbase_rdd.collect() for (k, v) in output: diff --git a/examples/src/main/python/hbase_outputformat.py b/examples/src/main/python/hbase_outputformat.py new file mode 100644 index 0000000000000..5e11548fd13f7 --- /dev/null +++ b/examples/src/main/python/hbase_outputformat.py @@ -0,0 +1,65 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import sys + +from pyspark import SparkContext + +""" +Create test table in HBase first: + +hbase(main):001:0> create 'test', 'f1' +0 row(s) in 0.7840 seconds + +> hbase_outputformat test row1 f1 q1 value1 +> hbase_outputformat test row2 f1 q1 value2 +> hbase_outputformat test row3 f1 q1 value3 +> hbase_outputformat test row4 f1 q1 value4 + +hbase(main):002:0> scan 'test' +ROW COLUMN+CELL + row1 column=f1:q1, timestamp=1405659615726, value=value1 + row2 column=f1:q1, timestamp=1405659626803, value=value2 + row3 column=f1:q1, timestamp=1405659640106, value=value3 + row4 column=f1:q1, timestamp=1405659650292, value=value4 +4 row(s) in 0.0780 seconds +""" +if __name__ == "__main__": + if len(sys.argv) != 7: + print >> sys.stderr, """ + Usage: hbase_outputformat + + Run with example jar: + ./bin/spark-submit --driver-class-path /path/to/example/jar /path/to/examples/hbase_outputformat.py + Assumes you have created
    with column family in HBase running on already + """ + exit(-1) + + host = sys.argv[1] + table = sys.argv[2] + sc = SparkContext(appName="HBaseOutputFormat") + + conf = {"hbase.zookeeper.quorum": host, + "hbase.mapred.outputtable": table, + "mapreduce.outputformat.class" : "org.apache.hadoop.hbase.mapreduce.TableOutputFormat", + "mapreduce.job.output.key.class" : "org.apache.hadoop.hbase.io.ImmutableBytesWritable", + "mapreduce.job.output.value.class" : "org.apache.hadoop.io.Writable"} + + sc.parallelize([sys.argv[3:]]).map(lambda x: (x[0], x)).saveAsNewAPIHadoopDataset( + conf=conf, + keyConverter="org.apache.spark.examples.pythonconverters.StringToImmutableBytesWritableConverter", + valueConverter="org.apache.spark.examples.pythonconverters.StringListToPutConverter") diff --git a/examples/src/main/python/kmeans.py b/examples/src/main/python/kmeans.py index 988fc45baf3bc..036bdf4c4f999 100755 --- a/examples/src/main/python/kmeans.py +++ b/examples/src/main/python/kmeans.py @@ -45,9 +45,15 @@ def closestPoint(p, centers): if __name__ == "__main__": + if len(sys.argv) != 4: print >> sys.stderr, "Usage: kmeans " exit(-1) + + print >> sys.stderr, """WARN: This is a naive implementation of KMeans Clustering and is given + as an example! Please refer to examples/src/main/python/mllib/kmeans.py for an example on + how to use MLlib's KMeans implementation.""" + sc = SparkContext(appName="PythonKMeans") lines = sc.textFile(sys.argv[1]) data = lines.map(parseVector).cache() diff --git a/examples/src/main/python/logistic_regression.py b/examples/src/main/python/logistic_regression.py index 6c33deabfd6ea..8456b272f9c05 100755 --- a/examples/src/main/python/logistic_regression.py +++ b/examples/src/main/python/logistic_regression.py @@ -47,9 +47,15 @@ def readPointBatch(iterator): return [matrix] if __name__ == "__main__": + if len(sys.argv) != 3: print >> sys.stderr, "Usage: logistic_regression " exit(-1) + + print >> sys.stderr, """WARN: This is a naive implementation of Logistic Regression and is + given as an example! Please refer to examples/src/main/python/mllib/logistic_regression.py + to see how MLlib's implementation is used.""" + sc = SparkContext(appName="PythonLR") points = sc.textFile(sys.argv[1]).mapPartitions(readPointBatch).cache() iterations = int(sys.argv[2]) diff --git a/examples/src/main/python/mllib/decision_tree_runner.py b/examples/src/main/python/mllib/decision_tree_runner.py new file mode 100755 index 0000000000000..8efadb5223f56 --- /dev/null +++ b/examples/src/main/python/mllib/decision_tree_runner.py @@ -0,0 +1,133 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +""" +Decision tree classification and regression using MLlib. +""" + +import numpy, os, sys + +from operator import add + +from pyspark import SparkContext +from pyspark.mllib.regression import LabeledPoint +from pyspark.mllib.tree import DecisionTree +from pyspark.mllib.util import MLUtils + + +def getAccuracy(dtModel, data): + """ + Return accuracy of DecisionTreeModel on the given RDD[LabeledPoint]. + """ + seqOp = (lambda acc, x: acc + (x[0] == x[1])) + predictions = dtModel.predict(data.map(lambda x: x.features)) + truth = data.map(lambda p: p.label) + trainCorrect = predictions.zip(truth).aggregate(0, seqOp, add) + if data.count() == 0: + return 0 + return trainCorrect / (0.0 + data.count()) + + +def getMSE(dtModel, data): + """ + Return mean squared error (MSE) of DecisionTreeModel on the given + RDD[LabeledPoint]. + """ + seqOp = (lambda acc, x: acc + numpy.square(x[0] - x[1])) + predictions = dtModel.predict(data.map(lambda x: x.features)) + truth = data.map(lambda p: p.label) + trainMSE = predictions.zip(truth).aggregate(0, seqOp, add) + if data.count() == 0: + return 0 + return trainMSE / (0.0 + data.count()) + + +def reindexClassLabels(data): + """ + Re-index class labels in a dataset to the range {0,...,numClasses-1}. + If all labels in that range already appear at least once, + then the returned RDD is the same one (without a mapping). + Note: If a label simply does not appear in the data, + the index will not include it. + Be aware of this when reindexing subsampled data. + :param data: RDD of LabeledPoint where labels are integer values + denoting labels for a classification problem. + :return: Pair (reindexedData, origToNewLabels) where + reindexedData is an RDD of LabeledPoint with labels in + the range {0,...,numClasses-1}, and + origToNewLabels is a dictionary mapping original labels + to new labels. + """ + # classCounts: class --> # examples in class + classCounts = data.map(lambda x: x.label).countByValue() + numExamples = sum(classCounts.values()) + sortedClasses = sorted(classCounts.keys()) + numClasses = len(classCounts) + # origToNewLabels: class --> index in 0,...,numClasses-1 + if (numClasses < 2): + print >> sys.stderr, \ + "Dataset for classification should have at least 2 classes." + \ + " The given dataset had only %d classes." % numClasses + exit(1) + origToNewLabels = dict([(sortedClasses[i], i) for i in range(0, numClasses)]) + + print "numClasses = %d" % numClasses + print "Per-class example fractions, counts:" + print "Class\tFrac\tCount" + for c in sortedClasses: + frac = classCounts[c] / (numExamples + 0.0) + print "%g\t%g\t%d" % (c, frac, classCounts[c]) + + if (sortedClasses[0] == 0 and sortedClasses[-1] == numClasses - 1): + return (data, origToNewLabels) + else: + reindexedData = \ + data.map(lambda x: LabeledPoint(origToNewLabels[x.label], x.features)) + return (reindexedData, origToNewLabels) + + +def usage(): + print >> sys.stderr, \ + "Usage: decision_tree_runner [libsvm format data filepath]\n" + \ + " Note: This only supports binary classification." + exit(1) + + +if __name__ == "__main__": + if len(sys.argv) > 2: + usage() + sc = SparkContext(appName="PythonDT") + + # Load data. + dataPath = 'data/mllib/sample_libsvm_data.txt' + if len(sys.argv) == 2: + dataPath = sys.argv[1] + if not os.path.isfile(dataPath): + usage() + points = MLUtils.loadLibSVMFile(sc, dataPath) + + # Re-index class labels if needed. + (reindexedData, origToNewLabels) = reindexClassLabels(points) + + # Train a classifier. + model = DecisionTree.trainClassifier(reindexedData, numClasses=2) + # Print learned tree and stats. + print "Trained DecisionTree for classification:" + print " Model numNodes: %d\n" % model.numNodes() + print " Model depth: %d\n" % model.depth() + print " Training accuracy: %g\n" % getAccuracy(model, reindexedData) + print model diff --git a/examples/src/main/python/mllib/logistic_regression.py b/examples/src/main/python/mllib/logistic_regression.py index 6e0f7a4ee5a81..9d547ff77c984 100755 --- a/examples/src/main/python/mllib/logistic_regression.py +++ b/examples/src/main/python/mllib/logistic_regression.py @@ -30,8 +30,10 @@ from pyspark.mllib.classification import LogisticRegressionWithSGD -# Parse a line of text into an MLlib LabeledPoint object def parsePoint(line): + """ + Parse a line of text into an MLlib LabeledPoint object. + """ values = [float(s) for s in line.split(' ')] if values[0] == -1: # Convert -1 labels to 0 for MLlib values[0] = 0 diff --git a/examples/src/main/python/streaming/network_wordcount.py b/examples/src/main/python/streaming/network_wordcount.py new file mode 100644 index 0000000000000..cd2a8a73de63b --- /dev/null +++ b/examples/src/main/python/streaming/network_wordcount.py @@ -0,0 +1,20 @@ +import sys + +from pyspark.streaming.context import StreamingContext +from pyspark.streaming.duration import * + +if __name__ == "__main__": + if len(sys.argv) != 3: + print >> sys.stderr, "Usage: wordcount " + exit(-1) + ssc = StreamingContext(appName="PythonStreamingNetworkWordCount", + duration=Seconds(1)) + + lines = ssc.socketTextStream(sys.argv[1], int(sys.argv[2])) + counts = lines.flatMap(lambda line: line.split(" "))\ + .map(lambda word: (word, 1))\ + .reduceByKey(lambda a,b: a+b) + counts.pyprint() + + ssc.start() + ssc.awaitTermination() diff --git a/examples/src/main/python/streaming/wordcount.py b/examples/src/main/python/streaming/wordcount.py new file mode 100644 index 0000000000000..4c62835ed8025 --- /dev/null +++ b/examples/src/main/python/streaming/wordcount.py @@ -0,0 +1,21 @@ +import sys + +from pyspark.streaming.context import StreamingContext +from pyspark.streaming.duration import * + +if __name__ == "__main__": + if len(sys.argv) != 2: + print >> sys.stderr, "Usage: wordcount " + exit(-1) + + ssc = StreamingContext(appName="PythonStreamingWordCount", + duration=Seconds(1)) + + lines = ssc.textFileStream(sys.argv[1]) + counts = lines.flatMap(lambda line: line.split(" "))\ + .map(lambda x: (x, 1))\ + .reduceByKey(lambda a, b: a+b) + counts.pyprint() + + ssc.start() + ssc.awaitTermination() diff --git a/examples/src/main/resources/user.avsc b/examples/src/main/resources/user.avsc new file mode 100644 index 0000000000000..4995357ab3736 --- /dev/null +++ b/examples/src/main/resources/user.avsc @@ -0,0 +1,8 @@ +{"namespace": "example.avro", + "type": "record", + "name": "User", + "fields": [ + {"name": "name", "type": "string"}, + {"name": "favorite_color", "type": ["string", "null"]} + ] +} diff --git a/examples/src/main/resources/users.avro b/examples/src/main/resources/users.avro new file mode 100644 index 0000000000000..27c526ab114b2 Binary files /dev/null and b/examples/src/main/resources/users.avro differ diff --git a/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala b/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala index 4893b017ed819..822673347bdce 100644 --- a/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala @@ -31,12 +31,12 @@ object HBaseTest { val conf = HBaseConfiguration.create() // Other options for configuring scan behavior are available. More information available at // http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/TableInputFormat.html - conf.set(TableInputFormat.INPUT_TABLE, args(1)) + conf.set(TableInputFormat.INPUT_TABLE, args(0)) // Initialize hBase table if necessary val admin = new HBaseAdmin(conf) - if(!admin.isTableAvailable(args(1))) { - val tableDesc = new HTableDescriptor(args(1)) + if (!admin.isTableAvailable(args(0))) { + val tableDesc = new HTableDescriptor(args(0)) admin.createTable(tableDesc) } diff --git a/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala b/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala index 331de3ad1ef53..ed2b38e2ca6f8 100644 --- a/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala @@ -19,16 +19,22 @@ package org.apache.spark.examples import org.apache.spark._ + object HdfsTest { + + /** Usage: HdfsTest [file] */ def main(args: Array[String]) { + if (args.length < 1) { + System.err.println("Usage: HdfsTest ") + System.exit(1) + } val sparkConf = new SparkConf().setAppName("HdfsTest") val sc = new SparkContext(sparkConf) - val file = sc.textFile(args(1)) + val file = sc.textFile(args(0)) val mapped = file.map(s => s.length).cache() for (iter <- 1 to 10) { val start = System.currentTimeMillis() for (x <- mapped) { x + 2 } - // println("Processing: " + x) val end = System.currentTimeMillis() println("Iteration " + iter + " took " + (end-start) + " ms") } diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala b/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala index 658f73d96a86a..1f576319b3ca8 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala @@ -25,6 +25,9 @@ import cern.jet.math._ /** * Alternating least squares matrix factorization. + * + * This is an example implementation for learning how to use Spark. For more conventional use, + * please refer to org.apache.spark.mllib.recommendation.ALS */ object LocalALS { // Parameters set through command line arguments @@ -107,7 +110,16 @@ object LocalALS { solved2D.viewColumn(0) } + def showWarning() { + System.err.println( + """WARN: This is a naive implementation of ALS and is given as an example! + |Please use the ALS method found in org.apache.spark.mllib.recommendation + |for more conventional use. + """.stripMargin) + } + def main(args: Array[String]) { + args match { case Array(m, u, f, iters) => { M = m.toInt @@ -120,6 +132,9 @@ object LocalALS { System.exit(1) } } + + showWarning() + printf("Running with M=%d, U=%d, F=%d, iters=%d\n", M, U, F, ITERATIONS) val R = generateR() diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalFileLR.scala b/examples/src/main/scala/org/apache/spark/examples/LocalFileLR.scala index 0ef3001ca4ccd..931faac5463c4 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LocalFileLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LocalFileLR.scala @@ -21,6 +21,12 @@ import java.util.Random import breeze.linalg.{Vector, DenseVector} +/** + * Logistic regression based classification. + * + * This is an example implementation for learning how to use Spark. For more conventional use, + * please refer to org.apache.spark.mllib.classification.LogisticRegression + */ object LocalFileLR { val D = 10 // Numer of dimensions val rand = new Random(42) @@ -32,7 +38,18 @@ object LocalFileLR { DataPoint(new DenseVector(nums.slice(1, D + 1)), nums(0)) } + def showWarning() { + System.err.println( + """WARN: This is a naive implementation of Logistic Regression and is given as an example! + |Please use the LogisticRegression method found in org.apache.spark.mllib.classification + |for more conventional use. + """.stripMargin) + } + def main(args: Array[String]) { + + showWarning() + val lines = scala.io.Source.fromFile(args(0)).getLines().toArray val points = lines.map(parsePoint _) val ITERATIONS = args(1).toInt diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalKMeans.scala b/examples/src/main/scala/org/apache/spark/examples/LocalKMeans.scala index e33a1b336d163..17624c20cff3d 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LocalKMeans.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LocalKMeans.scala @@ -28,6 +28,9 @@ import org.apache.spark.SparkContext._ /** * K-means clustering. + * + * This is an example implementation for learning how to use Spark. For more conventional use, + * please refer to org.apache.spark.mllib.clustering.KMeans */ object LocalKMeans { val N = 1000 @@ -61,7 +64,18 @@ object LocalKMeans { bestIndex } + def showWarning() { + System.err.println( + """WARN: This is a naive implementation of KMeans Clustering and is given as an example! + |Please use the KMeans method found in org.apache.spark.mllib.clustering + |for more conventional use. + """.stripMargin) + } + def main(args: Array[String]) { + + showWarning() + val data = generateData var points = new HashSet[Vector[Double]] var kPoints = new HashMap[Int, Vector[Double]] diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalLR.scala b/examples/src/main/scala/org/apache/spark/examples/LocalLR.scala index 385b48089d572..2d75b9d2590f8 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LocalLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LocalLR.scala @@ -23,6 +23,9 @@ import breeze.linalg.{Vector, DenseVector} /** * Logistic regression based classification. + * + * This is an example implementation for learning how to use Spark. For more conventional use, + * please refer to org.apache.spark.mllib.classification.LogisticRegression */ object LocalLR { val N = 10000 // Number of data points @@ -42,9 +45,19 @@ object LocalLR { Array.tabulate(N)(generatePoint) } + def showWarning() { + System.err.println( + """WARN: This is a naive implementation of Logistic Regression and is given as an example! + |Please use the LogisticRegression method found in org.apache.spark.mllib.classification + |for more conventional use. + """.stripMargin) + } + def main(args: Array[String]) { - val data = generateData + showWarning() + + val data = generateData // Initialize w to a random value var w = DenseVector.fill(D){2 * rand.nextDouble - 1} println("Initial w: " + w) diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala index 5cbc966bf06ca..fde8ffeedf8b4 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala @@ -27,6 +27,9 @@ import org.apache.spark._ /** * Alternating least squares matrix factorization. + * + * This is an example implementation for learning how to use Spark. For more conventional use, + * please refer to org.apache.spark.mllib.recommendation.ALS */ object SparkALS { // Parameters set through command line arguments @@ -87,7 +90,16 @@ object SparkALS { solved2D.viewColumn(0) } + def showWarning() { + System.err.println( + """WARN: This is a naive implementation of ALS and is given as an example! + |Please use the ALS method found in org.apache.spark.mllib.recommendation + |for more conventional use. + """.stripMargin) + } + def main(args: Array[String]) { + var slices = 0 val options = (0 to 4).map(i => if (i < args.length) Some(args(i)) else None) @@ -103,7 +115,11 @@ object SparkALS { System.err.println("Usage: SparkALS [M] [U] [F] [iters] [slices]") System.exit(1) } + + showWarning() + printf("Running with M=%d, U=%d, F=%d, iters=%d\n", M, U, F, ITERATIONS) + val sparkConf = new SparkConf().setAppName("SparkALS") val sc = new SparkContext(sparkConf) diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala index 4906a696e90a7..d583cf421ed23 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala @@ -30,6 +30,9 @@ import org.apache.spark.scheduler.InputFormatInfo /** * Logistic regression based classification. + * + * This is an example implementation for learning how to use Spark. For more conventional use, + * please refer to org.apache.spark.mllib.classification.LogisticRegression */ object SparkHdfsLR { val D = 10 // Numer of dimensions @@ -48,12 +51,23 @@ object SparkHdfsLR { DataPoint(new DenseVector(x), y) } + def showWarning() { + System.err.println( + """WARN: This is a naive implementation of Logistic Regression and is given as an example! + |Please use the LogisticRegression method found in org.apache.spark.mllib.classification + |for more conventional use. + """.stripMargin) + } + def main(args: Array[String]) { + if (args.length < 2) { System.err.println("Usage: SparkHdfsLR ") System.exit(1) } + showWarning() + val sparkConf = new SparkConf().setAppName("SparkHdfsLR") val inputPath = args(0) val conf = SparkHadoopUtil.get.newConfiguration() diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala b/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala index 4d28e0aad6597..48e8d11cdf95b 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala @@ -17,8 +17,6 @@ package org.apache.spark.examples -import java.util.Random - import breeze.linalg.{Vector, DenseVector, squaredDistance} import org.apache.spark.{SparkConf, SparkContext} @@ -26,17 +24,17 @@ import org.apache.spark.SparkContext._ /** * K-means clustering. + * + * This is an example implementation for learning how to use Spark. For more conventional use, + * please refer to org.apache.spark.mllib.clustering.KMeans */ object SparkKMeans { - val R = 1000 // Scaling factor - val rand = new Random(42) def parseVector(line: String): Vector[Double] = { DenseVector(line.split(' ').map(_.toDouble)) } def closestPoint(p: Vector[Double], centers: Array[Vector[Double]]): Int = { - var index = 0 var bestIndex = 0 var closest = Double.PositiveInfinity @@ -51,11 +49,23 @@ object SparkKMeans { bestIndex } + def showWarning() { + System.err.println( + """WARN: This is a naive implementation of KMeans Clustering and is given as an example! + |Please use the KMeans method found in org.apache.spark.mllib.clustering + |for more conventional use. + """.stripMargin) + } + def main(args: Array[String]) { + if (args.length < 3) { System.err.println("Usage: SparkKMeans ") System.exit(1) } + + showWarning() + val sparkConf = new SparkConf().setAppName("SparkKMeans") val sc = new SparkContext(sparkConf) val lines = sc.textFile(args(0)) diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala index 99ceb3089e9fe..fc23308fc4adf 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala @@ -28,6 +28,9 @@ import org.apache.spark._ /** * Logistic regression based classification. * Usage: SparkLR [slices] + * + * This is an example implementation for learning how to use Spark. For more conventional use, + * please refer to org.apache.spark.mllib.classification.LogisticRegression */ object SparkLR { val N = 10000 // Number of data points @@ -47,7 +50,18 @@ object SparkLR { Array.tabulate(N)(generatePoint) } + def showWarning() { + System.err.println( + """WARN: This is a naive implementation of Logistic Regression and is given as an example! + |Please use the LogisticRegression method found in org.apache.spark.mllib.classification + |for more conventional use. + """.stripMargin) + } + def main(args: Array[String]) { + + showWarning() + val sparkConf = new SparkConf().setAppName("SparkLR") val sc = new SparkContext(sparkConf) val numSlices = if (args.length > 0) args(0).toInt else 2 @@ -66,6 +80,7 @@ object SparkLR { } println("Final w: " + w) + sc.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala b/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala index 40b36c779afd6..4c7e006da0618 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala @@ -31,8 +31,12 @@ import org.apache.spark.{SparkConf, SparkContext} */ object SparkPageRank { def main(args: Array[String]) { + if (args.length < 1) { + System.err.println("Usage: SparkPageRank ") + System.exit(1) + } val sparkConf = new SparkConf().setAppName("PageRank") - var iters = args(1).toInt + val iters = if (args.length > 0) args(1).toInt else 10 val ctx = new SparkContext(sparkConf) val lines = ctx.textFile(args(0), 1) val links = lines.map{ s => diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassification.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassification.scala index 56b02b65d8724..a6f78d2441db1 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassification.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassification.scala @@ -21,7 +21,7 @@ import org.apache.log4j.{Level, Logger} import scopt.OptionParser import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.mllib.classification.{LogisticRegressionWithSGD, SVMWithSGD} +import org.apache.spark.mllib.classification.{LogisticRegressionWithLBFGS, SVMWithSGD} import org.apache.spark.mllib.evaluation.BinaryClassificationMetrics import org.apache.spark.mllib.util.MLUtils import org.apache.spark.mllib.optimization.{SquaredL2Updater, L1Updater} @@ -66,7 +66,8 @@ object BinaryClassification { .text("number of iterations") .action((x, c) => c.copy(numIterations = x)) opt[Double]("stepSize") - .text(s"initial step size, default: ${defaultParams.stepSize}") + .text("initial step size (ignored by logistic regression), " + + s"default: ${defaultParams.stepSize}") .action((x, c) => c.copy(stepSize = x)) opt[String]("algorithm") .text(s"algorithm (${Algorithm.values.mkString(",")}), " + @@ -125,10 +126,9 @@ object BinaryClassification { val model = params.algorithm match { case LR => - val algorithm = new LogisticRegressionWithSGD() + val algorithm = new LogisticRegressionWithLBFGS() algorithm.optimizer .setNumIterations(params.numIterations) - .setStepSize(params.stepSize) .setUpdater(updater) .setRegParam(params.regParam) algorithm.run(training).clearThreshold() diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala index b3cc361154198..cf3d2cca81ff6 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala @@ -21,7 +21,6 @@ import scopt.OptionParser import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.SparkContext._ -import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.{DecisionTree, impurity} import org.apache.spark.mllib.tree.configuration.{Algo, Strategy} @@ -33,9 +32,12 @@ import org.apache.spark.rdd.RDD /** * An example runner for decision tree. Run with * {{{ - * ./bin/spark-example org.apache.spark.examples.mllib.DecisionTreeRunner [options] + * ./bin/run-example org.apache.spark.examples.mllib.DecisionTreeRunner [options] * }}} * If you use it as a template to create your own app, please use `spark-submit` to submit your app. + * + * Note: This script treats all features as real-valued (not categorical). + * To include categorical features, modify categoricalFeaturesInfo. */ object DecisionTreeRunner { @@ -48,10 +50,12 @@ object DecisionTreeRunner { case class Params( input: String = null, + dataFormat: String = "libsvm", algo: Algo = Classification, - maxDepth: Int = 5, + maxDepth: Int = 4, impurity: ImpurityType = Gini, - maxBins: Int = 100) + maxBins: Int = 100, + fracTest: Double = 0.2) def main(args: Array[String]) { val defaultParams = Params() @@ -71,18 +75,28 @@ object DecisionTreeRunner { opt[Int]("maxBins") .text(s"max number of bins, default: ${defaultParams.maxBins}") .action((x, c) => c.copy(maxBins = x)) + opt[Double]("fracTest") + .text(s"fraction of data to hold out for testing, default: ${defaultParams.fracTest}") + .action((x, c) => c.copy(fracTest = x)) + opt[String]("") + .text("data format: libsvm (default), dense (deprecated in Spark v1.1)") + .action((x, c) => c.copy(dataFormat = x)) arg[String]("") .text("input paths to labeled examples in dense format (label,f0 f1 f2 ...)") .required() .action((x, c) => c.copy(input = x)) checkConfig { params => - if (params.algo == Classification && - (params.impurity == Gini || params.impurity == Entropy)) { - success - } else if (params.algo == Regression && params.impurity == Variance) { - success + if (params.fracTest < 0 || params.fracTest > 1) { + failure(s"fracTest ${params.fracTest} value incorrect; should be in [0,1].") } else { - failure(s"Algo ${params.algo} is not compatible with impurity ${params.impurity}.") + if (params.algo == Classification && + (params.impurity == Gini || params.impurity == Entropy)) { + success + } else if (params.algo == Regression && params.impurity == Variance) { + success + } else { + failure(s"Algo ${params.algo} is not compatible with impurity ${params.impurity}.") + } } } } @@ -95,16 +109,57 @@ object DecisionTreeRunner { } def run(params: Params) { + val conf = new SparkConf().setAppName("DecisionTreeRunner") val sc = new SparkContext(conf) // Load training data and cache it. - val examples = MLUtils.loadLabeledPoints(sc, params.input).cache() + val origExamples = params.dataFormat match { + case "dense" => MLUtils.loadLabeledPoints(sc, params.input).cache() + case "libsvm" => MLUtils.loadLibSVMFile(sc, params.input).cache() + } + // For classification, re-index classes if needed. + val (examples, numClasses) = params.algo match { + case Classification => { + // classCounts: class --> # examples in class + val classCounts = origExamples.map(_.label).countByValue() + val sortedClasses = classCounts.keys.toList.sorted + val numClasses = classCounts.size + // classIndexMap: class --> index in 0,...,numClasses-1 + val classIndexMap = { + if (classCounts.keySet != Set(0.0, 1.0)) { + sortedClasses.zipWithIndex.toMap + } else { + Map[Double, Int]() + } + } + val examples = { + if (classIndexMap.isEmpty) { + origExamples + } else { + origExamples.map(lp => LabeledPoint(classIndexMap(lp.label), lp.features)) + } + } + val numExamples = examples.count() + println(s"numClasses = $numClasses.") + println(s"Per-class example fractions, counts:") + println(s"Class\tFrac\tCount") + sortedClasses.foreach { c => + val frac = classCounts(c) / numExamples.toDouble + println(s"$c\t$frac\t${classCounts(c)}") + } + (examples, numClasses) + } + case Regression => + (origExamples, 0) + case _ => + throw new IllegalArgumentException("Algo ${params.algo} not supported.") + } - val splits = examples.randomSplit(Array(0.8, 0.2)) + // Split into training, test. + val splits = examples.randomSplit(Array(1.0 - params.fracTest, params.fracTest)) val training = splits(0).cache() val test = splits(1).cache() - val numTraining = training.count() val numTest = test.count() @@ -118,17 +173,25 @@ object DecisionTreeRunner { case Variance => impurity.Variance } - val strategy = new Strategy(params.algo, impurityCalculator, params.maxDepth, params.maxBins) + val strategy + = new Strategy( + algo = params.algo, + impurity = impurityCalculator, + maxDepth = params.maxDepth, + maxBins = params.maxBins, + numClassesForClassification = numClasses) val model = DecisionTree.train(training, strategy) + println(model) + if (params.algo == Classification) { val accuracy = accuracyScore(model, test) - println(s"Test accuracy = $accuracy.") + println(s"Test accuracy = $accuracy") } if (params.algo == Regression) { val mse = meanSquaredError(model, test) - println(s"Test mean squared error = $mse.") + println(s"Test mean squared error = $mse") } sc.stop() @@ -139,12 +202,8 @@ object DecisionTreeRunner { */ private def accuracyScore( model: DecisionTreeModel, - data: RDD[LabeledPoint], - threshold: Double = 0.5): Double = { - def predictedValue(features: Vector): Double = { - if (model.predict(features) < threshold) 0.0 else 1.0 - } - val correctCount = data.filter(y => predictedValue(y.features) == y.label).count() + data: RDD[LabeledPoint]): Double = { + val correctCount = data.filter(y => model.predict(y.features) == y.label).count() val count = data.count() correctCount.toDouble / count } diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala index 4811bb70e4b28..05b7d66f8dffd 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala @@ -91,7 +91,7 @@ object LinearRegression extends App { Logger.getRootLogger.setLevel(Level.WARN) - val examples = MLUtils.loadLibSVMFile(sc, params.input, multiclass = true).cache() + val examples = MLUtils.loadLibSVMFile(sc, params.input).cache() val splits = examples.randomSplit(Array(0.8, 0.2)) val training = splits(0).cache() diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/SparseNaiveBayes.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/SparseNaiveBayes.scala index 537e68a0991aa..88acd9dbb0878 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/SparseNaiveBayes.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/SparseNaiveBayes.scala @@ -22,7 +22,7 @@ import scopt.OptionParser import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.mllib.classification.NaiveBayes -import org.apache.spark.mllib.util.{MLUtils, MulticlassLabelParser} +import org.apache.spark.mllib.util.MLUtils /** * An example naive Bayes app. Run with @@ -76,7 +76,7 @@ object SparseNaiveBayes { if (params.minPartitions > 0) params.minPartitions else sc.defaultMinPartitions val examples = - MLUtils.loadLibSVMFile(sc, params.input, multiclass = true, params.numFeatures, minPartitions) + MLUtils.loadLibSVMFile(sc, params.input, params.numFeatures, minPartitions) // Cache examples because it will be used in both training and evaluation. examples.cache() diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLinearRegression.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLinearRegression.scala new file mode 100644 index 0000000000000..0e992fa9967bb --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLinearRegression.scala @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.mllib + +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.regression.{LabeledPoint, StreamingLinearRegressionWithSGD} +import org.apache.spark.SparkConf +import org.apache.spark.streaming.{Seconds, StreamingContext} + +/** + * Train a linear regression model on one stream of data and make predictions + * on another stream, where the data streams arrive as text files + * into two different directories. + * + * The rows of the text files must be labeled data points in the form + * `(y,[x1,x2,x3,...,xn])` + * Where n is the number of features. n must be the same for train and test. + * + * Usage: StreamingLinearRegression + * + * To run on your local machine using the two directories `trainingDir` and `testDir`, + * with updates every 5 seconds, and 2 features per data point, call: + * $ bin/run-example \ + * org.apache.spark.examples.mllib.StreamingLinearRegression trainingDir testDir 5 2 + * + * As you add text files to `trainingDir` the model will continuously update. + * Anytime you add text files to `testDir`, you'll see predictions from the current model. + * + */ +object StreamingLinearRegression { + + def main(args: Array[String]) { + + if (args.length != 4) { + System.err.println( + "Usage: StreamingLinearRegression ") + System.exit(1) + } + + val conf = new SparkConf().setMaster("local").setAppName("StreamingLinearRegression") + val ssc = new StreamingContext(conf, Seconds(args(2).toLong)) + + val trainingData = ssc.textFileStream(args(0)).map(LabeledPoint.parse) + val testData = ssc.textFileStream(args(1)).map(LabeledPoint.parse) + + val model = new StreamingLinearRegressionWithSGD() + .setInitialWeights(Vectors.dense(Array.fill[Double](args(3).toInt)(0))) + + model.trainOn(trainingData) + model.predictOn(testData).print() + + ssc.start() + ssc.awaitTermination() + + } + +} diff --git a/examples/src/main/scala/org/apache/spark/examples/pythonconverters/AvroConverters.scala b/examples/src/main/scala/org/apache/spark/examples/pythonconverters/AvroConverters.scala new file mode 100644 index 0000000000000..1b25983a38453 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/pythonconverters/AvroConverters.scala @@ -0,0 +1,130 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.pythonconverters + +import java.util.{Collection => JCollection, Map => JMap} + +import scala.collection.JavaConversions._ + +import org.apache.avro.generic.{GenericFixed, IndexedRecord} +import org.apache.avro.mapred.AvroWrapper +import org.apache.avro.Schema +import org.apache.avro.Schema.Type._ + +import org.apache.spark.api.python.Converter +import org.apache.spark.SparkException + + +/** + * Implementation of [[org.apache.spark.api.python.Converter]] that converts + * an Avro Record wrapped in an AvroKey (or AvroValue) to a Java Map. It tries + * to work with all 3 Avro data mappings (Generic, Specific and Reflect). + */ +class AvroWrapperToJavaConverter extends Converter[Any, Any] { + override def convert(obj: Any): Any = { + if (obj == null) { + return null + } + obj.asInstanceOf[AvroWrapper[_]].datum() match { + case null => null + case record: IndexedRecord => unpackRecord(record) + case other => throw new SparkException( + s"Unsupported top-level Avro data type ${other.getClass.getName}") + } + } + + def unpackRecord(obj: Any): JMap[String, Any] = { + val map = new java.util.HashMap[String, Any] + obj match { + case record: IndexedRecord => + record.getSchema.getFields.zipWithIndex.foreach { case (f, i) => + map.put(f.name, fromAvro(record.get(i), f.schema)) + } + case other => throw new SparkException( + s"Unsupported RECORD type ${other.getClass.getName}") + } + map + } + + def unpackMap(obj: Any, schema: Schema): JMap[String, Any] = { + obj.asInstanceOf[JMap[_, _]].map { case (key, value) => + (key.toString, fromAvro(value, schema.getValueType)) + } + } + + def unpackFixed(obj: Any, schema: Schema): Array[Byte] = { + unpackBytes(obj.asInstanceOf[GenericFixed].bytes()) + } + + def unpackBytes(obj: Any): Array[Byte] = { + val bytes: Array[Byte] = obj match { + case buf: java.nio.ByteBuffer => buf.array() + case arr: Array[Byte] => arr + case other => throw new SparkException( + s"Unknown BYTES type ${other.getClass.getName}") + } + val bytearray = new Array[Byte](bytes.length) + System.arraycopy(bytes, 0, bytearray, 0, bytes.length) + bytearray + } + + def unpackArray(obj: Any, schema: Schema): JCollection[Any] = obj match { + case c: JCollection[_] => + c.map(fromAvro(_, schema.getElementType)) + case arr: Array[_] if arr.getClass.getComponentType.isPrimitive => + arr.toSeq + case arr: Array[_] => + arr.map(fromAvro(_, schema.getElementType)).toSeq + case other => throw new SparkException( + s"Unknown ARRAY type ${other.getClass.getName}") + } + + def unpackUnion(obj: Any, schema: Schema): Any = { + schema.getTypes.toList match { + case List(s) => fromAvro(obj, s) + case List(n, s) if n.getType == NULL => fromAvro(obj, s) + case List(s, n) if n.getType == NULL => fromAvro(obj, s) + case _ => throw new SparkException( + "Unions may only consist of a concrete type and null") + } + } + + def fromAvro(obj: Any, schema: Schema): Any = { + if (obj == null) { + return null + } + schema.getType match { + case UNION => unpackUnion(obj, schema) + case ARRAY => unpackArray(obj, schema) + case FIXED => unpackFixed(obj, schema) + case MAP => unpackMap(obj, schema) + case BYTES => unpackBytes(obj) + case RECORD => unpackRecord(obj) + case STRING => obj.toString + case ENUM => obj.toString + case NULL => obj + case BOOLEAN => obj + case DOUBLE => obj + case FLOAT => obj + case INT => obj + case LONG => obj + case other => throw new SparkException( + s"Unknown Avro schema type ${other.getName}") + } + } +} diff --git a/examples/src/main/scala/org/apache/spark/examples/pythonconverters/CassandraConverters.scala b/examples/src/main/scala/org/apache/spark/examples/pythonconverters/CassandraConverters.scala index 29a65c7a5f295..83feb5703b908 100644 --- a/examples/src/main/scala/org/apache/spark/examples/pythonconverters/CassandraConverters.scala +++ b/examples/src/main/scala/org/apache/spark/examples/pythonconverters/CassandraConverters.scala @@ -20,7 +20,7 @@ package org.apache.spark.examples.pythonconverters import org.apache.spark.api.python.Converter import java.nio.ByteBuffer import org.apache.cassandra.utils.ByteBufferUtil -import collection.JavaConversions.{mapAsJavaMap, mapAsScalaMap} +import collection.JavaConversions._ /** @@ -44,3 +44,25 @@ class CassandraCQLValueConverter extends Converter[Any, java.util.Map[String, St mapAsJavaMap(result.mapValues(bb => ByteBufferUtil.string(bb))) } } + +/** + * Implementation of [[org.apache.spark.api.python.Converter]] that converts a + * Map[String, Int] to Cassandra key + */ +class ToCassandraCQLKeyConverter extends Converter[Any, java.util.Map[String, ByteBuffer]] { + override def convert(obj: Any): java.util.Map[String, ByteBuffer] = { + val input = obj.asInstanceOf[java.util.Map[String, Int]] + mapAsJavaMap(input.mapValues(i => ByteBufferUtil.bytes(i))) + } +} + +/** + * Implementation of [[org.apache.spark.api.python.Converter]] that converts a + * List[String] to Cassandra value + */ +class ToCassandraCQLValueConverter extends Converter[Any, java.util.List[ByteBuffer]] { + override def convert(obj: Any): java.util.List[ByteBuffer] = { + val input = obj.asInstanceOf[java.util.List[String]] + seqAsJavaList(input.map(s => ByteBufferUtil.bytes(s))) + } +} diff --git a/examples/src/main/scala/org/apache/spark/examples/pythonconverters/HBaseConverters.scala b/examples/src/main/scala/org/apache/spark/examples/pythonconverters/HBaseConverters.scala new file mode 100644 index 0000000000000..273bee0a8b30f --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/pythonconverters/HBaseConverters.scala @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.pythonconverters + +import scala.collection.JavaConversions._ + +import org.apache.spark.api.python.Converter +import org.apache.hadoop.hbase.client.{Put, Result} +import org.apache.hadoop.hbase.io.ImmutableBytesWritable +import org.apache.hadoop.hbase.util.Bytes + +/** + * Implementation of [[org.apache.spark.api.python.Converter]] that converts an + * HBase Result to a String + */ +class HBaseResultToStringConverter extends Converter[Any, String] { + override def convert(obj: Any): String = { + val result = obj.asInstanceOf[Result] + Bytes.toStringBinary(result.value()) + } +} + +/** + * Implementation of [[org.apache.spark.api.python.Converter]] that converts an + * ImmutableBytesWritable to a String + */ +class ImmutableBytesWritableToStringConverter extends Converter[Any, String] { + override def convert(obj: Any): String = { + val key = obj.asInstanceOf[ImmutableBytesWritable] + Bytes.toStringBinary(key.get()) + } +} + +/** + * Implementation of [[org.apache.spark.api.python.Converter]] that converts a + * String to an ImmutableBytesWritable + */ +class StringToImmutableBytesWritableConverter extends Converter[Any, ImmutableBytesWritable] { + override def convert(obj: Any): ImmutableBytesWritable = { + val bytes = Bytes.toBytes(obj.asInstanceOf[String]) + new ImmutableBytesWritable(bytes) + } +} + +/** + * Implementation of [[org.apache.spark.api.python.Converter]] that converts a + * list of Strings to HBase Put + */ +class StringListToPutConverter extends Converter[Any, Put] { + override def convert(obj: Any): Put = { + val output = obj.asInstanceOf[java.util.ArrayList[String]].map(Bytes.toBytes(_)).toArray + val put = new Put(output(0)) + put.add(output(1), output(2), output(3)) + } +} diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala b/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala index 63db688bfb8c0..d56d64c564200 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala @@ -36,7 +36,7 @@ object RDDRelation { val rdd = sc.parallelize((1 to 100).map(i => Record(i, s"val_$i"))) // Any RDD containing case classes can be registered as a table. The schema of the table is // automatically inferred using scala reflection. - rdd.registerAsTable("records") + rdd.registerTempTable("records") // Once tables have been registered, you can run SQL queries over them. println("Result of SELECT *:") @@ -66,7 +66,7 @@ object RDDRelation { parquetFile.where('key === 1).select('value as 'a).collect().foreach(println) // These files can also be registered as tables. - parquetFile.registerAsTable("parquetFile") + parquetFile.registerTempTable("parquetFile") sql("SELECT * FROM parquetFile").collect().foreach(println) } } diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala index b262fabbe0e0d..3423fac0ad303 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala @@ -19,7 +19,7 @@ package org.apache.spark.examples.sql.hive import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.sql._ -import org.apache.spark.sql.hive.LocalHiveContext +import org.apache.spark.sql.hive.HiveContext object HiveFromSpark { case class Record(key: Int, value: String) @@ -31,23 +31,23 @@ object HiveFromSpark { // A local hive context creates an instance of the Hive Metastore in process, storing the // the warehouse data in the current directory. This location can be overridden by // specifying a second parameter to the constructor. - val hiveContext = new LocalHiveContext(sc) + val hiveContext = new HiveContext(sc) import hiveContext._ - hql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") - hql("LOAD DATA LOCAL INPATH 'src/main/resources/kv1.txt' INTO TABLE src") + sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") + sql("LOAD DATA LOCAL INPATH 'src/main/resources/kv1.txt' INTO TABLE src") // Queries are expressed in HiveQL println("Result of 'SELECT *': ") - hql("SELECT * FROM src").collect.foreach(println) + sql("SELECT * FROM src").collect.foreach(println) // Aggregation queries are also supported. - val count = hql("SELECT COUNT(*) FROM src").collect().head.getInt(0) + val count = sql("SELECT COUNT(*) FROM src").collect().head.getLong(0) println(s"COUNT(*): $count") // The results of SQL queries are themselves RDDs and support all normal RDD functions. The // items in the RDD are of type Row, which allows you to access each column by ordinal. - val rddFromSql = hql("SELECT key, value FROM src WHERE key < 10 ORDER BY key") + val rddFromSql = sql("SELECT key, value FROM src WHERE key < 10 ORDER BY key") println("Result of RDD.map:") val rddAsStrings = rddFromSql.map { @@ -56,10 +56,10 @@ object HiveFromSpark { // You can also register RDDs as temporary tables within a HiveContext. val rdd = sc.parallelize((1 to 100).map(i => Record(i, s"val_$i"))) - rdd.registerAsTable("records") + rdd.registerTempTable("records") // Queries can then join RDD data with data stored in Hive. println("Result of SELECT *:") - hql("SELECT * FROM records r JOIN src s ON r.key = s.key").collect().foreach(println) + sql("SELECT * FROM records r JOIN src s ON r.key = s.key").collect().foreach(println) } } diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/FlumePollingEventCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/FlumePollingEventCount.scala new file mode 100644 index 0000000000000..1cc8c8d5c23b6 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/FlumePollingEventCount.scala @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.streaming + +import org.apache.spark.SparkConf +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming._ +import org.apache.spark.streaming.flume._ +import org.apache.spark.util.IntParam +import java.net.InetSocketAddress + +/** + * Produces a count of events received from Flume. + * + * This should be used in conjunction with the Spark Sink running in a Flume agent. See + * the Spark Streaming programming guide for more details. + * + * Usage: FlumePollingEventCount + * `host` is the host on which the Spark Sink is running. + * `port` is the port at which the Spark Sink is listening. + * + * To run this example: + * `$ bin/run-example org.apache.spark.examples.streaming.FlumePollingEventCount [host] [port] ` + */ +object FlumePollingEventCount { + def main(args: Array[String]) { + if (args.length < 2) { + System.err.println( + "Usage: FlumePollingEventCount ") + System.exit(1) + } + + StreamingExamples.setStreamingLogLevels() + + val Array(host, IntParam(port)) = args + + val batchInterval = Milliseconds(2000) + + // Create the context and set the batch size + val sparkConf = new SparkConf().setAppName("FlumePollingEventCount") + val ssc = new StreamingContext(sparkConf, batchInterval) + + // Create a flume stream that polls the Spark Sink running in a Flume agent + val stream = FlumeUtils.createPollingStream(ssc, host, port) + + // Print out the count of events received from this server in each batch + stream.count().map(cnt => "Received " + cnt + " flume events." ).print() + + ssc.start() + ssc.awaitTermination() + } +} diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml new file mode 100644 index 0000000000000..d0bf1cf1ea796 --- /dev/null +++ b/external/flume-sink/pom.xml @@ -0,0 +1,103 @@ + + + + + 4.0.0 + + org.apache.spark + spark-parent + 1.1.0-SNAPSHOT + ../../pom.xml + + + spark-streaming-flume-sink_2.10 + + streaming-flume-sink + + + jar + Spark Project External Flume Sink + http://spark.apache.org/ + + + org.apache.flume + flume-ng-sdk + 1.4.0 + + + io.netty + netty + + + org.apache.thrift + libthrift + + + + + org.apache.flume + flume-ng-core + 1.4.0 + + + io.netty + netty + + + org.apache.thrift + libthrift + + + + + org.scala-lang + scala-library + + + org.scalatest + scalatest_${scala.binary.version} + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + org.scalatest + scalatest-maven-plugin + + + org.apache.avro + avro-maven-plugin + 1.7.3 + + + ${project.basedir}/target/scala-${scala.binary.version}/src_managed/main/compiled_avro + + + + generate-sources + + idl-protocol + + + + + + + diff --git a/external/flume-sink/src/main/avro/sparkflume.avdl b/external/flume-sink/src/main/avro/sparkflume.avdl new file mode 100644 index 0000000000000..8806e863ac7c6 --- /dev/null +++ b/external/flume-sink/src/main/avro/sparkflume.avdl @@ -0,0 +1,40 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +@namespace("org.apache.spark.streaming.flume.sink") + +protocol SparkFlumeProtocol { + + record SparkSinkEvent { + map headers; + bytes body; + } + + record EventBatch { + string errorMsg = ""; // If this is empty it is a valid message, else it represents an error + string sequenceNumber; + array events; + } + + EventBatch getEventBatch (int n); + + void ack (string sequenceNumber); + + void nack (string sequenceNumber); +} diff --git a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/Logging.scala b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/Logging.scala new file mode 100644 index 0000000000000..17cbc6707b5ea --- /dev/null +++ b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/Logging.scala @@ -0,0 +1,125 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.streaming.flume.sink + +import org.slf4j.{Logger, LoggerFactory} + +/** + * Copy of the org.apache.spark.Logging for being used in the Spark Sink. + * The org.apache.spark.Logging is not used so that all of Spark is not brought + * in as a dependency. + */ +private[sink] trait Logging { + // Make the log field transient so that objects with Logging can + // be serialized and used on another machine + @transient private var log_ : Logger = null + + // Method to get or create the logger for this object + protected def log: Logger = { + if (log_ == null) { + initializeIfNecessary() + var className = this.getClass.getName + // Ignore trailing $'s in the class names for Scala objects + if (className.endsWith("$")) { + className = className.substring(0, className.length - 1) + } + log_ = LoggerFactory.getLogger(className) + } + log_ + } + + // Log methods that take only a String + protected def logInfo(msg: => String) { + if (log.isInfoEnabled) log.info(msg) + } + + protected def logDebug(msg: => String) { + if (log.isDebugEnabled) log.debug(msg) + } + + protected def logTrace(msg: => String) { + if (log.isTraceEnabled) log.trace(msg) + } + + protected def logWarning(msg: => String) { + if (log.isWarnEnabled) log.warn(msg) + } + + protected def logError(msg: => String) { + if (log.isErrorEnabled) log.error(msg) + } + + // Log methods that take Throwables (Exceptions/Errors) too + protected def logInfo(msg: => String, throwable: Throwable) { + if (log.isInfoEnabled) log.info(msg, throwable) + } + + protected def logDebug(msg: => String, throwable: Throwable) { + if (log.isDebugEnabled) log.debug(msg, throwable) + } + + protected def logTrace(msg: => String, throwable: Throwable) { + if (log.isTraceEnabled) log.trace(msg, throwable) + } + + protected def logWarning(msg: => String, throwable: Throwable) { + if (log.isWarnEnabled) log.warn(msg, throwable) + } + + protected def logError(msg: => String, throwable: Throwable) { + if (log.isErrorEnabled) log.error(msg, throwable) + } + + protected def isTraceEnabled(): Boolean = { + log.isTraceEnabled + } + + private def initializeIfNecessary() { + if (!Logging.initialized) { + Logging.initLock.synchronized { + if (!Logging.initialized) { + initializeLogging() + } + } + } + } + + private def initializeLogging() { + Logging.initialized = true + + // Force a call into slf4j to initialize it. Avoids this happening from mutliple threads + // and triggering this: http://mailman.qos.ch/pipermail/slf4j-dev/2010-April/002956.html + log + } +} + +private[sink] object Logging { + @volatile private var initialized = false + val initLock = new Object() + try { + // We use reflection here to handle the case where users remove the + // slf4j-to-jul bridge order to route their logs to JUL. + val bridgeClass = Class.forName("org.slf4j.bridge.SLF4JBridgeHandler") + bridgeClass.getMethod("removeHandlersForRootLogger").invoke(null) + val installed = bridgeClass.getMethod("isInstalled").invoke(null).asInstanceOf[Boolean] + if (!installed) { + bridgeClass.getMethod("install").invoke(null) + } + } catch { + case e: ClassNotFoundException => // can't log anything yet so just fail silently + } +} diff --git a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkAvroCallbackHandler.scala b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkAvroCallbackHandler.scala new file mode 100644 index 0000000000000..7da8eb3e35912 --- /dev/null +++ b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkAvroCallbackHandler.scala @@ -0,0 +1,131 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.streaming.flume.sink + +import java.util.concurrent.{ConcurrentHashMap, Executors} +import java.util.concurrent.atomic.AtomicLong + +import org.apache.flume.Channel +import org.apache.commons.lang.RandomStringUtils +import com.google.common.util.concurrent.ThreadFactoryBuilder + +/** + * Class that implements the SparkFlumeProtocol, that is used by the Avro Netty Server to process + * requests. Each getEvents, ack and nack call is forwarded to an instance of this class. + * @param threads Number of threads to use to process requests. + * @param channel The channel that the sink pulls events from + * @param transactionTimeout Timeout in millis after which the transaction if not acked by Spark + * is rolled back. + */ +// Flume forces transactions to be thread-local. So each transaction *must* be committed, or +// rolled back from the thread it was originally created in. So each getEvents call from Spark +// creates a TransactionProcessor which runs in a new thread, in which the transaction is created +// and events are pulled off the channel. Once the events are sent to spark, +// that thread is blocked and the TransactionProcessor is saved in a map, +// until an ACK or NACK comes back or the transaction times out (after the specified timeout). +// When the response comes or a timeout is hit, the TransactionProcessor is retrieved and then +// unblocked, at which point the transaction is committed or rolled back. + +private[flume] class SparkAvroCallbackHandler(val threads: Int, val channel: Channel, + val transactionTimeout: Int, val backOffInterval: Int) extends SparkFlumeProtocol with Logging { + val transactionExecutorOpt = Option(Executors.newFixedThreadPool(threads, + new ThreadFactoryBuilder().setDaemon(true) + .setNameFormat("Spark Sink Processor Thread - %d").build())) + private val processorMap = new ConcurrentHashMap[CharSequence, TransactionProcessor]() + // This sink will not persist sequence numbers and reuses them if it gets restarted. + // So it is possible to commit a transaction which may have been meant for the sink before the + // restart. + // Since the new txn may not have the same sequence number we must guard against accidentally + // committing a new transaction. To reduce the probability of that happening a random string is + // prepended to the sequence number. Does not change for life of sink + private val seqBase = RandomStringUtils.randomAlphanumeric(8) + private val seqCounter = new AtomicLong(0) + + /** + * Returns a bunch of events to Spark over Avro RPC. + * @param n Maximum number of events to return in a batch + * @return [[EventBatch]] instance that has a sequence number and an array of at most n events + */ + override def getEventBatch(n: Int): EventBatch = { + logDebug("Got getEventBatch call from Spark.") + val sequenceNumber = seqBase + seqCounter.incrementAndGet() + val processor = new TransactionProcessor(channel, sequenceNumber, + n, transactionTimeout, backOffInterval, this) + transactionExecutorOpt.foreach(executor => { + executor.submit(processor) + }) + // Wait until a batch is available - will be an error if error message is non-empty + val batch = processor.getEventBatch + if (!SparkSinkUtils.isErrorBatch(batch)) { + processorMap.put(sequenceNumber.toString, processor) + logDebug("Sending event batch with sequence number: " + sequenceNumber) + } + batch + } + + /** + * Called by Spark to indicate successful commit of a batch + * @param sequenceNumber The sequence number of the event batch that was successful + */ + override def ack(sequenceNumber: CharSequence): Void = { + logDebug("Received Ack for batch with sequence number: " + sequenceNumber) + completeTransaction(sequenceNumber, success = true) + null + } + + /** + * Called by Spark to indicate failed commit of a batch + * @param sequenceNumber The sequence number of the event batch that failed + * @return + */ + override def nack(sequenceNumber: CharSequence): Void = { + completeTransaction(sequenceNumber, success = false) + logInfo("Spark failed to commit transaction. Will reattempt events.") + null + } + + /** + * Helper method to commit or rollback a transaction. + * @param sequenceNumber The sequence number of the batch that was completed + * @param success Whether the batch was successful or not. + */ + private def completeTransaction(sequenceNumber: CharSequence, success: Boolean) { + Option(removeAndGetProcessor(sequenceNumber)).foreach(processor => { + processor.batchProcessed(success) + }) + } + + /** + * Helper method to remove the TxnProcessor for a Sequence Number. Can be used to avoid a leak. + * @param sequenceNumber + * @return The transaction processor for the corresponding batch. Note that this instance is no + * longer tracked and the caller is responsible for that txn processor. + */ + private[sink] def removeAndGetProcessor(sequenceNumber: CharSequence): TransactionProcessor = { + processorMap.remove(sequenceNumber.toString) // The toString is required! + } + + /** + * Shuts down the executor used to process transactions. + */ + def shutdown() { + logInfo("Shutting down Spark Avro Callback Handler") + transactionExecutorOpt.foreach(executor => { + executor.shutdownNow() + }) + } +} diff --git a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSink.scala b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSink.scala new file mode 100644 index 0000000000000..948af5947f5e1 --- /dev/null +++ b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSink.scala @@ -0,0 +1,162 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.streaming.flume.sink + +import java.net.InetSocketAddress +import java.util.concurrent._ + +import org.apache.avro.ipc.NettyServer +import org.apache.avro.ipc.specific.SpecificResponder +import org.apache.flume.Context +import org.apache.flume.Sink.Status +import org.apache.flume.conf.{Configurable, ConfigurationException} +import org.apache.flume.sink.AbstractSink + +/** + * A sink that uses Avro RPC to run a server that can be polled by Spark's + * FlumePollingInputDStream. This sink has the following configuration parameters: + * + * hostname - The hostname to bind to. Default: 0.0.0.0 + * port - The port to bind to. (No default - mandatory) + * timeout - Time in seconds after which a transaction is rolled back, + * if an ACK is not received from Spark within that time + * threads - Number of threads to use to receive requests from Spark (Default: 10) + * + * This sink is unlike other Flume sinks in the sense that it does not push data, + * instead the process method in this sink simply blocks the SinkRunner the first time it is + * called. This sink starts up an Avro IPC server that uses the SparkFlumeProtocol. + * + * Each time a getEventBatch call comes, creates a transaction and reads events + * from the channel. When enough events are read, the events are sent to the Spark receiver and + * the thread itself is blocked and a reference to it saved off. + * + * When the ack for that batch is received, + * the thread which created the transaction is is retrieved and it commits the transaction with the + * channel from the same thread it was originally created in (since Flume transactions are + * thread local). If a nack is received instead, the sink rolls back the transaction. If no ack + * is received within the specified timeout, the transaction is rolled back too. If an ack comes + * after that, it is simply ignored and the events get re-sent. + * + */ + +private[flume] +class SparkSink extends AbstractSink with Logging with Configurable { + + // Size of the pool to use for holding transaction processors. + private var poolSize: Integer = SparkSinkConfig.DEFAULT_THREADS + + // Timeout for each transaction. If spark does not respond in this much time, + // rollback the transaction + private var transactionTimeout = SparkSinkConfig.DEFAULT_TRANSACTION_TIMEOUT + + // Address info to bind on + private var hostname: String = SparkSinkConfig.DEFAULT_HOSTNAME + private var port: Int = 0 + + private var backOffInterval: Int = 200 + + // Handle to the server + private var serverOpt: Option[NettyServer] = None + + // The handler that handles the callback from Avro + private var handler: Option[SparkAvroCallbackHandler] = None + + // Latch that blocks off the Flume framework from wasting 1 thread. + private val blockingLatch = new CountDownLatch(1) + + override def start() { + logInfo("Starting Spark Sink: " + getName + " on port: " + port + " and interface: " + + hostname + " with " + "pool size: " + poolSize + " and transaction timeout: " + + transactionTimeout + ".") + handler = Option(new SparkAvroCallbackHandler(poolSize, getChannel, transactionTimeout, + backOffInterval)) + val responder = new SpecificResponder(classOf[SparkFlumeProtocol], handler.get) + // Using the constructor that takes specific thread-pools requires bringing in netty + // dependencies which are being excluded in the build. In practice, + // Netty dependencies are already available on the JVM as Flume would have pulled them in. + serverOpt = Option(new NettyServer(responder, new InetSocketAddress(hostname, port))) + serverOpt.foreach(server => { + logInfo("Starting Avro server for sink: " + getName) + server.start() + }) + super.start() + } + + override def stop() { + logInfo("Stopping Spark Sink: " + getName) + handler.foreach(callbackHandler => { + callbackHandler.shutdown() + }) + serverOpt.foreach(server => { + logInfo("Stopping Avro Server for sink: " + getName) + server.close() + server.join() + }) + blockingLatch.countDown() + super.stop() + } + + override def configure(ctx: Context) { + import SparkSinkConfig._ + hostname = ctx.getString(CONF_HOSTNAME, DEFAULT_HOSTNAME) + port = Option(ctx.getInteger(CONF_PORT)). + getOrElse(throw new ConfigurationException("The port to bind to must be specified")) + poolSize = ctx.getInteger(THREADS, DEFAULT_THREADS) + transactionTimeout = ctx.getInteger(CONF_TRANSACTION_TIMEOUT, DEFAULT_TRANSACTION_TIMEOUT) + backOffInterval = ctx.getInteger(CONF_BACKOFF_INTERVAL, DEFAULT_BACKOFF_INTERVAL) + logInfo("Configured Spark Sink with hostname: " + hostname + ", port: " + port + ", " + + "poolSize: " + poolSize + ", transactionTimeout: " + transactionTimeout + ", " + + "backoffInterval: " + backOffInterval) + } + + override def process(): Status = { + // This method is called in a loop by the Flume framework - block it until the sink is + // stopped to save CPU resources. The sink runner will interrupt this thread when the sink is + // being shut down. + logInfo("Blocking Sink Runner, sink will continue to run..") + blockingLatch.await() + Status.BACKOFF + } + + private[flume] def getPort(): Int = { + serverOpt + .map(_.getPort) + .getOrElse( + throw new RuntimeException("Server was not started!") + ) + } +} + +/** + * Configuration parameters and their defaults. + */ +private[flume] +object SparkSinkConfig { + val THREADS = "threads" + val DEFAULT_THREADS = 10 + + val CONF_TRANSACTION_TIMEOUT = "timeout" + val DEFAULT_TRANSACTION_TIMEOUT = 60 + + val CONF_HOSTNAME = "hostname" + val DEFAULT_HOSTNAME = "0.0.0.0" + + val CONF_PORT = "port" + + val CONF_BACKOFF_INTERVAL = "backoffInterval" + val DEFAULT_BACKOFF_INTERVAL = 200 +} diff --git a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSinkUtils.scala b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSinkUtils.scala new file mode 100644 index 0000000000000..47c0e294d6b52 --- /dev/null +++ b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSinkUtils.scala @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.streaming.flume.sink + +private[flume] object SparkSinkUtils { + /** + * This method determines if this batch represents an error or not. + * @param batch - The batch to check + * @return - true if the batch represents an error + */ + def isErrorBatch(batch: EventBatch): Boolean = { + !batch.getErrorMsg.toString.equals("") // If there is an error message, it is an error batch. + } +} diff --git a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/TransactionProcessor.scala b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/TransactionProcessor.scala new file mode 100644 index 0000000000000..b9e3c786ebb3b --- /dev/null +++ b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/TransactionProcessor.scala @@ -0,0 +1,228 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.streaming.flume.sink + +import java.nio.ByteBuffer +import java.util +import java.util.concurrent.{Callable, CountDownLatch, TimeUnit} + +import scala.util.control.Breaks + +import org.apache.flume.{Transaction, Channel} + +// Flume forces transactions to be thread-local (horrible, I know!) +// So the sink basically spawns a new thread to pull the events out within a transaction. +// The thread fills in the event batch object that is set before the thread is scheduled. +// After filling it in, the thread waits on a condition - which is released only +// when the success message comes back for the specific sequence number for that event batch. +/** + * This class represents a transaction on the Flume channel. This class runs a separate thread + * which owns the transaction. The thread is blocked until the success call for that transaction + * comes back with an ACK or NACK. + * @param channel The channel from which to pull events + * @param seqNum The sequence number to use for the transaction. Must be unique + * @param maxBatchSize The maximum number of events to process per batch + * @param transactionTimeout Time in seconds after which a transaction must be rolled back + * without waiting for an ACK from Spark + * @param parent The parent [[SparkAvroCallbackHandler]] instance, for reporting timeouts + */ +private class TransactionProcessor(val channel: Channel, val seqNum: String, + var maxBatchSize: Int, val transactionTimeout: Int, val backOffInterval: Int, + val parent: SparkAvroCallbackHandler) extends Callable[Void] with Logging { + + // If a real batch is not returned, we always have to return an error batch. + @volatile private var eventBatch: EventBatch = new EventBatch("Unknown Error", "", + util.Collections.emptyList()) + + // Synchronization primitives + val batchGeneratedLatch = new CountDownLatch(1) + val batchAckLatch = new CountDownLatch(1) + + // Sanity check to ensure we don't loop like crazy + val totalAttemptsToRemoveFromChannel = Int.MaxValue / 2 + + // OK to use volatile, since the change would only make this true (otherwise it will be + // changed to false - we never apply a negation operation to this) - which means the transaction + // succeeded. + @volatile private var batchSuccess = false + + // The transaction that this processor would handle + var txOpt: Option[Transaction] = None + + /** + * Get an event batch from the channel. This method will block until a batch of events is + * available from the channel. If no events are available after a large number of attempts of + * polling the channel, this method will return an [[EventBatch]] with a non-empty error message + * + * @return An [[EventBatch]] instance with sequence number set to seqNum, filled with a + * maximum of maxBatchSize events + */ + def getEventBatch: EventBatch = { + batchGeneratedLatch.await() + eventBatch + } + + /** + * This method is to be called by the sink when it receives an ACK or NACK from Spark. This + * method is a no-op if it is called after transactionTimeout has expired since + * getEventBatch returned a batch of events. + * @param success True if an ACK was received and the transaction should be committed, else false. + */ + def batchProcessed(success: Boolean) { + logDebug("Batch processed for sequence number: " + seqNum) + batchSuccess = success + batchAckLatch.countDown() + } + + /** + * Populates events into the event batch. If the batch cannot be populated, + * this method will not set the events into the event batch, but it sets an error message. + */ + private def populateEvents() { + try { + txOpt = Option(channel.getTransaction) + if(txOpt.isEmpty) { + eventBatch.setErrorMsg("Something went wrong. Channel was " + + "unable to create a transaction!") + } + txOpt.foreach(tx => { + tx.begin() + val events = new util.ArrayList[SparkSinkEvent](maxBatchSize) + val loop = new Breaks + var gotEventsInThisTxn = false + var loopCounter: Int = 0 + loop.breakable { + while (events.size() < maxBatchSize + && loopCounter < totalAttemptsToRemoveFromChannel) { + loopCounter += 1 + Option(channel.take()) match { + case Some(event) => + events.add(new SparkSinkEvent(toCharSequenceMap(event.getHeaders), + ByteBuffer.wrap(event.getBody))) + gotEventsInThisTxn = true + case None => + if (!gotEventsInThisTxn) { + logDebug("Sleeping for " + backOffInterval + " millis as no events were read in" + + " the current transaction") + TimeUnit.MILLISECONDS.sleep(backOffInterval) + } else { + loop.break() + } + } + } + } + if (!gotEventsInThisTxn) { + val msg = "Tried several times, " + + "but did not get any events from the channel!" + logWarning(msg) + eventBatch.setErrorMsg(msg) + } else { + // At this point, the events are available, so fill them into the event batch + eventBatch = new EventBatch("",seqNum, events) + } + }) + } catch { + case e: Exception => + logWarning("Error while processing transaction.", e) + eventBatch.setErrorMsg(e.getMessage) + try { + txOpt.foreach(tx => { + rollbackAndClose(tx, close = true) + }) + } finally { + txOpt = None + } + } finally { + batchGeneratedLatch.countDown() + } + } + + /** + * Waits for upto transactionTimeout seconds for an ACK. If an ACK comes in + * this method commits the transaction with the channel. If the ACK does not come in within + * that time or a NACK comes in, this method rolls back the transaction. + */ + private def processAckOrNack() { + batchAckLatch.await(transactionTimeout, TimeUnit.SECONDS) + txOpt.foreach(tx => { + if (batchSuccess) { + try { + logDebug("Committing transaction") + tx.commit() + } catch { + case e: Exception => + logWarning("Error while attempting to commit transaction. Transaction will be rolled " + + "back", e) + rollbackAndClose(tx, close = false) // tx will be closed later anyway + } finally { + tx.close() + } + } else { + logWarning("Spark could not commit transaction, NACK received. Rolling back transaction.") + rollbackAndClose(tx, close = true) + // This might have been due to timeout or a NACK. Either way the following call does not + // cause issues. This is required to ensure the TransactionProcessor instance is not leaked + parent.removeAndGetProcessor(seqNum) + } + }) + } + + /** + * Helper method to rollback and optionally close a transaction + * @param tx The transaction to rollback + * @param close Whether the transaction should be closed or not after rolling back + */ + private def rollbackAndClose(tx: Transaction, close: Boolean) { + try { + logWarning("Spark was unable to successfully process the events. Transaction is being " + + "rolled back.") + tx.rollback() + } catch { + case e: Exception => + logError("Error rolling back transaction. Rollback may have failed!", e) + } finally { + if (close) { + tx.close() + } + } + } + + /** + * Helper method to convert a Map[String, String] to Map[CharSequence, CharSequence] + * @param inMap The map to be converted + * @return The converted map + */ + private def toCharSequenceMap(inMap: java.util.Map[String, String]): java.util.Map[CharSequence, + CharSequence] = { + val charSeqMap = new util.HashMap[CharSequence, CharSequence](inMap.size()) + charSeqMap.putAll(inMap) + charSeqMap + } + + /** + * When the thread is started it sets as many events as the batch size or less (if enough + * events aren't available) into the eventBatch and object and lets any threads waiting on the + * [[getEventBatch]] method to proceed. Then this thread waits for acks or nacks to come in, + * or for a specified timeout and commits or rolls back the transaction. + * @return + */ + override def call(): Void = { + populateEvents() + processAckOrNack() + null + } +} diff --git a/external/flume/pom.xml b/external/flume/pom.xml index c1f581967777b..c532705f3950c 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -27,6 +27,9 @@ org.apache.spark spark-streaming-flume_2.10 + + streaming-flume + jar Spark Project External Flume http://spark.apache.org/ @@ -69,11 +72,21 @@ scalacheck_${scala.binary.version} test + + junit + junit + test + com.novocode junit-interface test + + org.apache.spark + spark-streaming-flume-sink_2.10 + ${project.version} + target/scala-${scala.binary.version}/classes diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/EventTransformer.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/EventTransformer.scala new file mode 100644 index 0000000000000..dc629df4f4ac2 --- /dev/null +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/EventTransformer.scala @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.flume + +import java.io.{ObjectOutput, ObjectInput} + +import scala.collection.JavaConversions._ + +import org.apache.spark.util.Utils +import org.apache.spark.Logging + +/** + * A simple object that provides the implementation of readExternal and writeExternal for both + * the wrapper classes for Flume-style Events. + */ +private[streaming] object EventTransformer extends Logging { + def readExternal(in: ObjectInput): (java.util.HashMap[CharSequence, CharSequence], + Array[Byte]) = { + val bodyLength = in.readInt() + val bodyBuff = new Array[Byte](bodyLength) + in.readFully(bodyBuff) + + val numHeaders = in.readInt() + val headers = new java.util.HashMap[CharSequence, CharSequence] + + for (i <- 0 until numHeaders) { + val keyLength = in.readInt() + val keyBuff = new Array[Byte](keyLength) + in.readFully(keyBuff) + val key: String = Utils.deserialize(keyBuff) + + val valLength = in.readInt() + val valBuff = new Array[Byte](valLength) + in.readFully(valBuff) + val value: String = Utils.deserialize(valBuff) + + headers.put(key, value) + } + (headers, bodyBuff) + } + + def writeExternal(out: ObjectOutput, headers: java.util.Map[CharSequence, CharSequence], + body: Array[Byte]) { + out.writeInt(body.length) + out.write(body) + val numHeaders = headers.size() + out.writeInt(numHeaders) + for ((k,v) <- headers) { + val keyBuff = Utils.serialize(k.toString) + out.writeInt(keyBuff.length) + out.write(keyBuff) + val valBuff = Utils.serialize(v.toString) + out.writeInt(valBuff.length) + out.write(valBuff) + } + } +} diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala index ed35e34ad45ab..4b2ea45fb81d0 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala @@ -20,6 +20,7 @@ package org.apache.spark.streaming.flume import java.net.InetSocketAddress import java.io.{ObjectInput, ObjectOutput, Externalizable} import java.nio.ByteBuffer +import java.util.concurrent.Executors import scala.collection.JavaConversions._ import scala.reflect.ClassTag @@ -29,24 +30,29 @@ import org.apache.flume.source.avro.AvroFlumeEvent import org.apache.flume.source.avro.Status import org.apache.avro.ipc.specific.SpecificResponder import org.apache.avro.ipc.NettyServer - +import org.apache.spark.Logging import org.apache.spark.util.Utils import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.StreamingContext import org.apache.spark.streaming.dstream._ -import org.apache.spark.Logging +import org.apache.spark.streaming.StreamingContext import org.apache.spark.streaming.receiver.Receiver +import org.jboss.netty.channel.ChannelPipelineFactory +import org.jboss.netty.channel.Channels +import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory +import org.jboss.netty.handler.codec.compression._ + private[streaming] class FlumeInputDStream[T: ClassTag]( @transient ssc_ : StreamingContext, host: String, port: Int, - storageLevel: StorageLevel + storageLevel: StorageLevel, + enableDecompression: Boolean ) extends ReceiverInputDStream[SparkFlumeEvent](ssc_) { override def getReceiver(): Receiver[SparkFlumeEvent] = { - new FlumeReceiver(host, port, storageLevel) + new FlumeReceiver(host, port, storageLevel, enableDecompression) } } @@ -134,22 +140,71 @@ private[streaming] class FlumeReceiver( host: String, port: Int, - storageLevel: StorageLevel + storageLevel: StorageLevel, + enableDecompression: Boolean ) extends Receiver[SparkFlumeEvent](storageLevel) with Logging { lazy val responder = new SpecificResponder( classOf[AvroSourceProtocol], new FlumeEventServer(this)) - lazy val server = new NettyServer(responder, new InetSocketAddress(host, port)) + var server: NettyServer = null + + private def initServer() = { + if (enableDecompression) { + val channelFactory = new NioServerSocketChannelFactory(Executors.newCachedThreadPool(), + Executors.newCachedThreadPool()) + val channelPipelineFactory = new CompressionChannelPipelineFactory() + + new NettyServer( + responder, + new InetSocketAddress(host, port), + channelFactory, + channelPipelineFactory, + null) + } else { + new NettyServer(responder, new InetSocketAddress(host, port)) + } + } def onStart() { - server.start() + synchronized { + if (server == null) { + server = initServer() + server.start() + } else { + logWarning("Flume receiver being asked to start more then once with out close") + } + } logInfo("Flume receiver started") } def onStop() { - server.close() + synchronized { + if (server != null) { + server.close() + server = null + } + } logInfo("Flume receiver stopped") } override def preferredLocation = Some(host) + + /** A Netty Pipeline factory that will decompress incoming data from + * and the Netty client and compress data going back to the client. + * + * The compression on the return is required because Flume requires + * a successful response to indicate it can remove the event/batch + * from the configured channel + */ + private[streaming] + class CompressionChannelPipelineFactory extends ChannelPipelineFactory { + + def getPipeline() = { + val pipeline = Channels.pipeline() + val encoder = new ZlibEncoder(6) + pipeline.addFirst("deflater", encoder) + pipeline.addFirst("inflater", new ZlibDecoder()) + pipeline + } +} } diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala new file mode 100644 index 0000000000000..148262bb6771e --- /dev/null +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala @@ -0,0 +1,178 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.streaming.flume + + +import java.net.InetSocketAddress +import java.util.concurrent.{LinkedBlockingQueue, TimeUnit, Executors} + +import scala.collection.JavaConversions._ +import scala.collection.mutable.ArrayBuffer +import scala.reflect.ClassTag + +import com.google.common.util.concurrent.ThreadFactoryBuilder +import org.apache.avro.ipc.NettyTransceiver +import org.apache.avro.ipc.specific.SpecificRequestor +import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory + +import org.apache.spark.Logging +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.StreamingContext +import org.apache.spark.streaming.dstream.ReceiverInputDStream +import org.apache.spark.streaming.receiver.Receiver +import org.apache.spark.streaming.flume.sink._ + +/** + * A [[ReceiverInputDStream]] that can be used to read data from several Flume agents running + * [[org.apache.spark.streaming.flume.sink.SparkSink]]s. + * @param _ssc Streaming context that will execute this input stream + * @param addresses List of addresses at which SparkSinks are listening + * @param maxBatchSize Maximum size of a batch + * @param parallelism Number of parallel connections to open + * @param storageLevel The storage level to use. + * @tparam T Class type of the object of this stream + */ +private[streaming] class FlumePollingInputDStream[T: ClassTag]( + @transient _ssc: StreamingContext, + val addresses: Seq[InetSocketAddress], + val maxBatchSize: Int, + val parallelism: Int, + storageLevel: StorageLevel + ) extends ReceiverInputDStream[SparkFlumeEvent](_ssc) { + + override def getReceiver(): Receiver[SparkFlumeEvent] = { + new FlumePollingReceiver(addresses, maxBatchSize, parallelism, storageLevel) + } +} + +private[streaming] class FlumePollingReceiver( + addresses: Seq[InetSocketAddress], + maxBatchSize: Int, + parallelism: Int, + storageLevel: StorageLevel + ) extends Receiver[SparkFlumeEvent](storageLevel) with Logging { + + lazy val channelFactoryExecutor = + Executors.newCachedThreadPool(new ThreadFactoryBuilder().setDaemon(true). + setNameFormat("Flume Receiver Channel Thread - %d").build()) + + lazy val channelFactory = + new NioClientSocketChannelFactory(channelFactoryExecutor, channelFactoryExecutor) + + lazy val receiverExecutor = Executors.newFixedThreadPool(parallelism, + new ThreadFactoryBuilder().setDaemon(true).setNameFormat("Flume Receiver Thread - %d").build()) + + private lazy val connections = new LinkedBlockingQueue[FlumeConnection]() + + override def onStart(): Unit = { + // Create the connections to each Flume agent. + addresses.foreach(host => { + val transceiver = new NettyTransceiver(host, channelFactory) + val client = SpecificRequestor.getClient(classOf[SparkFlumeProtocol.Callback], transceiver) + connections.add(new FlumeConnection(transceiver, client)) + }) + for (i <- 0 until parallelism) { + logInfo("Starting Flume Polling Receiver worker threads starting..") + // Threads that pull data from Flume. + receiverExecutor.submit(new Runnable { + override def run(): Unit = { + while (true) { + val connection = connections.poll() + val client = connection.client + try { + val eventBatch = client.getEventBatch(maxBatchSize) + if (!SparkSinkUtils.isErrorBatch(eventBatch)) { + // No error, proceed with processing data + val seq = eventBatch.getSequenceNumber + val events: java.util.List[SparkSinkEvent] = eventBatch.getEvents + logDebug( + "Received batch of " + events.size() + " events with sequence number: " + seq) + try { + // Convert each Flume event to a serializable SparkFlumeEvent + val buffer = new ArrayBuffer[SparkFlumeEvent](events.size()) + var j = 0 + while (j < events.size()) { + buffer += toSparkFlumeEvent(events(j)) + j += 1 + } + store(buffer) + logDebug("Sending ack for sequence number: " + seq) + // Send an ack to Flume so that Flume discards the events from its channels. + client.ack(seq) + logDebug("Ack sent for sequence number: " + seq) + } catch { + case e: Exception => + try { + // Let Flume know that the events need to be pushed back into the channel. + logDebug("Sending nack for sequence number: " + seq) + client.nack(seq) // If the agent is down, even this could fail and throw + logDebug("Nack sent for sequence number: " + seq) + } catch { + case e: Exception => logError( + "Sending Nack also failed. A Flume agent is down.") + } + TimeUnit.SECONDS.sleep(2L) // for now just leave this as a fixed 2 seconds. + logWarning("Error while attempting to store events", e) + } + } else { + logWarning("Did not receive events from Flume agent due to error on the Flume " + + "agent: " + eventBatch.getErrorMsg) + } + } catch { + case e: Exception => + logWarning("Error while reading data from Flume", e) + } finally { + connections.add(connection) + } + } + } + }) + } + } + + override def onStop(): Unit = { + logInfo("Shutting down Flume Polling Receiver") + receiverExecutor.shutdownNow() + connections.foreach(connection => { + connection.transceiver.close() + }) + channelFactory.releaseExternalResources() + } + + /** + * Utility method to convert [[SparkSinkEvent]] to [[SparkFlumeEvent]] + * @param event - Event to convert to SparkFlumeEvent + * @return - The SparkFlumeEvent generated from SparkSinkEvent + */ + private def toSparkFlumeEvent(event: SparkSinkEvent): SparkFlumeEvent = { + val sparkFlumeEvent = new SparkFlumeEvent() + sparkFlumeEvent.event.setBody(event.getBody) + sparkFlumeEvent.event.setHeaders(event.getHeaders) + sparkFlumeEvent + } +} + +/** + * A wrapper around the transceiver and the Avro IPC API. + * @param transceiver The transceiver to use for communication with Flume + * @param client The client that the callbacks are received on. + */ +private class FlumeConnection(val transceiver: NettyTransceiver, + val client: SparkFlumeProtocol.Callback) + + + diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala index 499f3560ef768..4b732c1592ab2 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala @@ -17,12 +17,19 @@ package org.apache.spark.streaming.flume +import java.net.InetSocketAddress + +import org.apache.spark.annotation.Experimental import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContext -import org.apache.spark.streaming.api.java.{JavaReceiverInputDStream, JavaInputDStream, JavaStreamingContext, JavaDStream} -import org.apache.spark.streaming.dstream.{ReceiverInputDStream, DStream} +import org.apache.spark.streaming.api.java.{JavaReceiverInputDStream, JavaStreamingContext} +import org.apache.spark.streaming.dstream.ReceiverInputDStream + object FlumeUtils { + private val DEFAULT_POLLING_PARALLELISM = 5 + private val DEFAULT_POLLING_BATCH_SIZE = 1000 + /** * Create a input stream from a Flume source. * @param ssc StreamingContext object @@ -36,7 +43,27 @@ object FlumeUtils { port: Int, storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 ): ReceiverInputDStream[SparkFlumeEvent] = { - val inputStream = new FlumeInputDStream[SparkFlumeEvent](ssc, hostname, port, storageLevel) + createStream(ssc, hostname, port, storageLevel, false) + } + + /** + * Create a input stream from a Flume source. + * @param ssc StreamingContext object + * @param hostname Hostname of the slave machine to which the flume data will be sent + * @param port Port of the slave machine to which the flume data will be sent + * @param storageLevel Storage level to use for storing the received objects + * @param enableDecompression should netty server decompress input stream + */ + def createStream ( + ssc: StreamingContext, + hostname: String, + port: Int, + storageLevel: StorageLevel, + enableDecompression: Boolean + ): ReceiverInputDStream[SparkFlumeEvent] = { + val inputStream = new FlumeInputDStream[SparkFlumeEvent]( + ssc, hostname, port, storageLevel, enableDecompression) + inputStream } @@ -66,6 +93,154 @@ object FlumeUtils { port: Int, storageLevel: StorageLevel ): JavaReceiverInputDStream[SparkFlumeEvent] = { - createStream(jssc.ssc, hostname, port, storageLevel) + createStream(jssc.ssc, hostname, port, storageLevel, false) + } + + /** + * Creates a input stream from a Flume source. + * @param hostname Hostname of the slave machine to which the flume data will be sent + * @param port Port of the slave machine to which the flume data will be sent + * @param storageLevel Storage level to use for storing the received objects + * @param enableDecompression should netty server decompress input stream + */ + def createStream( + jssc: JavaStreamingContext, + hostname: String, + port: Int, + storageLevel: StorageLevel, + enableDecompression: Boolean + ): JavaReceiverInputDStream[SparkFlumeEvent] = { + createStream(jssc.ssc, hostname, port, storageLevel, enableDecompression) + } + + /** + * Creates an input stream that is to be used with the Spark Sink deployed on a Flume agent. + * This stream will poll the sink for data and will pull events as they are available. + * This stream will use a batch size of 1000 events and run 5 threads to pull data. + * @param hostname Address of the host on which the Spark Sink is running + * @param port Port of the host at which the Spark Sink is listening + * @param storageLevel Storage level to use for storing the received objects + */ + @Experimental + def createPollingStream( + ssc: StreamingContext, + hostname: String, + port: Int, + storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 + ): ReceiverInputDStream[SparkFlumeEvent] = { + createPollingStream(ssc, Seq(new InetSocketAddress(hostname, port)), storageLevel) + } + + /** + * Creates an input stream that is to be used with the Spark Sink deployed on a Flume agent. + * This stream will poll the sink for data and will pull events as they are available. + * This stream will use a batch size of 1000 events and run 5 threads to pull data. + * @param addresses List of InetSocketAddresses representing the hosts to connect to. + * @param storageLevel Storage level to use for storing the received objects + */ + @Experimental + def createPollingStream( + ssc: StreamingContext, + addresses: Seq[InetSocketAddress], + storageLevel: StorageLevel + ): ReceiverInputDStream[SparkFlumeEvent] = { + createPollingStream(ssc, addresses, storageLevel, + DEFAULT_POLLING_BATCH_SIZE, DEFAULT_POLLING_PARALLELISM) + } + + /** + * Creates an input stream that is to be used with the Spark Sink deployed on a Flume agent. + * This stream will poll the sink for data and will pull events as they are available. + * @param addresses List of InetSocketAddresses representing the hosts to connect to. + * @param maxBatchSize Maximum number of events to be pulled from the Spark sink in a + * single RPC call + * @param parallelism Number of concurrent requests this stream should send to the sink. Note + * that having a higher number of requests concurrently being pulled will + * result in this stream using more threads + * @param storageLevel Storage level to use for storing the received objects + */ + @Experimental + def createPollingStream( + ssc: StreamingContext, + addresses: Seq[InetSocketAddress], + storageLevel: StorageLevel, + maxBatchSize: Int, + parallelism: Int + ): ReceiverInputDStream[SparkFlumeEvent] = { + new FlumePollingInputDStream[SparkFlumeEvent](ssc, addresses, maxBatchSize, + parallelism, storageLevel) + } + + /** + * Creates an input stream that is to be used with the Spark Sink deployed on a Flume agent. + * This stream will poll the sink for data and will pull events as they are available. + * This stream will use a batch size of 1000 events and run 5 threads to pull data. + * @param hostname Hostname of the host on which the Spark Sink is running + * @param port Port of the host at which the Spark Sink is listening + */ + @Experimental + def createPollingStream( + jssc: JavaStreamingContext, + hostname: String, + port: Int + ): JavaReceiverInputDStream[SparkFlumeEvent] = { + createPollingStream(jssc, hostname, port, StorageLevel.MEMORY_AND_DISK_SER_2) + } + + /** + * Creates an input stream that is to be used with the Spark Sink deployed on a Flume agent. + * This stream will poll the sink for data and will pull events as they are available. + * This stream will use a batch size of 1000 events and run 5 threads to pull data. + * @param hostname Hostname of the host on which the Spark Sink is running + * @param port Port of the host at which the Spark Sink is listening + * @param storageLevel Storage level to use for storing the received objects + */ + @Experimental + def createPollingStream( + jssc: JavaStreamingContext, + hostname: String, + port: Int, + storageLevel: StorageLevel + ): JavaReceiverInputDStream[SparkFlumeEvent] = { + createPollingStream(jssc, Array(new InetSocketAddress(hostname, port)), storageLevel) + } + + /** + * Creates an input stream that is to be used with the Spark Sink deployed on a Flume agent. + * This stream will poll the sink for data and will pull events as they are available. + * This stream will use a batch size of 1000 events and run 5 threads to pull data. + * @param addresses List of InetSocketAddresses on which the Spark Sink is running. + * @param storageLevel Storage level to use for storing the received objects + */ + @Experimental + def createPollingStream( + jssc: JavaStreamingContext, + addresses: Array[InetSocketAddress], + storageLevel: StorageLevel + ): JavaReceiverInputDStream[SparkFlumeEvent] = { + createPollingStream(jssc, addresses, storageLevel, + DEFAULT_POLLING_BATCH_SIZE, DEFAULT_POLLING_PARALLELISM) + } + + /** + * Creates an input stream that is to be used with the Spark Sink deployed on a Flume agent. + * This stream will poll the sink for data and will pull events as they are available. + * @param addresses List of InetSocketAddresses on which the Spark Sink is running + * @param maxBatchSize The maximum number of events to be pulled from the Spark sink in a + * single RPC call + * @param parallelism Number of concurrent requests this stream should send to the sink. Note + * that having a higher number of requests concurrently being pulled will + * result in this stream using more threads + * @param storageLevel Storage level to use for storing the received objects + */ + @Experimental + def createPollingStream( + jssc: JavaStreamingContext, + addresses: Array[InetSocketAddress], + storageLevel: StorageLevel, + maxBatchSize: Int, + parallelism: Int + ): JavaReceiverInputDStream[SparkFlumeEvent] = { + createPollingStream(jssc.ssc, addresses, storageLevel, maxBatchSize, parallelism) } } diff --git a/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumePollingStreamSuite.java b/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumePollingStreamSuite.java new file mode 100644 index 0000000000000..79c5b91654b42 --- /dev/null +++ b/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumePollingStreamSuite.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.flume; + +import java.net.InetSocketAddress; + +import org.apache.spark.storage.StorageLevel; +import org.apache.spark.streaming.LocalJavaStreamingContext; + +import org.apache.spark.streaming.api.java.JavaReceiverInputDStream; +import org.junit.Test; + +public class JavaFlumePollingStreamSuite extends LocalJavaStreamingContext { + @Test + public void testFlumeStream() { + // tests the API, does not actually test data receiving + InetSocketAddress[] addresses = new InetSocketAddress[] { + new InetSocketAddress("localhost", 12345) + }; + JavaReceiverInputDStream test1 = + FlumeUtils.createPollingStream(ssc, "localhost", 12345); + JavaReceiverInputDStream test2 = FlumeUtils.createPollingStream( + ssc, "localhost", 12345, StorageLevel.MEMORY_AND_DISK_SER_2()); + JavaReceiverInputDStream test3 = FlumeUtils.createPollingStream( + ssc, addresses, StorageLevel.MEMORY_AND_DISK_SER_2()); + JavaReceiverInputDStream test4 = FlumeUtils.createPollingStream( + ssc, addresses, StorageLevel.MEMORY_AND_DISK_SER_2(), 100, 5); + } +} diff --git a/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumeStreamSuite.java b/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumeStreamSuite.java index e0ad4f1015205..3b5e0c7746b2c 100644 --- a/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumeStreamSuite.java +++ b/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumeStreamSuite.java @@ -30,5 +30,7 @@ public void testFlumeStream() { JavaReceiverInputDStream test1 = FlumeUtils.createStream(ssc, "localhost", 12345); JavaReceiverInputDStream test2 = FlumeUtils.createStream(ssc, "localhost", 12345, StorageLevel.MEMORY_AND_DISK_SER_2()); + JavaReceiverInputDStream test3 = FlumeUtils.createStream(ssc, "localhost", 12345, + StorageLevel.MEMORY_AND_DISK_SER_2(), false); } } diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala new file mode 100644 index 0000000000000..8a85b0f987e42 --- /dev/null +++ b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala @@ -0,0 +1,227 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.spark.streaming.flume + +import java.net.InetSocketAddress +import java.util.concurrent.{Callable, ExecutorCompletionService, Executors} +import java.util.Random + +import org.apache.spark.TestUtils + +import scala.collection.JavaConversions._ +import scala.collection.mutable.{SynchronizedBuffer, ArrayBuffer} + +import org.apache.flume.Context +import org.apache.flume.channel.MemoryChannel +import org.apache.flume.conf.Configurables +import org.apache.flume.event.EventBuilder + +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.dstream.ReceiverInputDStream +import org.apache.spark.streaming.util.ManualClock +import org.apache.spark.streaming.{TestSuiteBase, TestOutputStream, StreamingContext} +import org.apache.spark.streaming.flume.sink._ +import org.apache.spark.util.Utils + +class FlumePollingStreamSuite extends TestSuiteBase { + + val batchCount = 5 + val eventsPerBatch = 100 + val totalEventsPerChannel = batchCount * eventsPerBatch + val channelCapacity = 5000 + val maxAttempts = 5 + + test("flume polling test") { + testMultipleTimes(testFlumePolling) + } + + test("flume polling test multiple hosts") { + testMultipleTimes(testFlumePollingMultipleHost) + } + + /** + * Run the given test until no more java.net.BindException's are thrown. + * Do this only up to a certain attempt limit. + */ + private def testMultipleTimes(test: () => Unit): Unit = { + var testPassed = false + var attempt = 0 + while (!testPassed && attempt < maxAttempts) { + try { + test() + testPassed = true + } catch { + case e: Exception if Utils.isBindCollision(e) => + logWarning("Exception when running flume polling test: " + e) + attempt += 1 + } + } + assert(testPassed, s"Test failed after $attempt attempts!") + } + + private def testFlumePolling(): Unit = { + // Start the channel and sink. + val context = new Context() + context.put("capacity", channelCapacity.toString) + context.put("transactionCapacity", "1000") + context.put("keep-alive", "0") + val channel = new MemoryChannel() + Configurables.configure(channel, context) + + val sink = new SparkSink() + context.put(SparkSinkConfig.CONF_HOSTNAME, "localhost") + context.put(SparkSinkConfig.CONF_PORT, String.valueOf(0)) + Configurables.configure(sink, context) + sink.setChannel(channel) + sink.start() + // Set up the streaming context and input streams + val ssc = new StreamingContext(conf, batchDuration) + val flumeStream: ReceiverInputDStream[SparkFlumeEvent] = + FlumeUtils.createPollingStream(ssc, Seq(new InetSocketAddress("localhost", sink.getPort())), + StorageLevel.MEMORY_AND_DISK, eventsPerBatch, 1) + val outputBuffer = new ArrayBuffer[Seq[SparkFlumeEvent]] + with SynchronizedBuffer[Seq[SparkFlumeEvent]] + val outputStream = new TestOutputStream(flumeStream, outputBuffer) + outputStream.register() + ssc.start() + + writeAndVerify(Seq(channel), ssc, outputBuffer) + assertChannelIsEmpty(channel) + sink.stop() + channel.stop() + } + + private def testFlumePollingMultipleHost(): Unit = { + // Start the channel and sink. + val context = new Context() + context.put("capacity", channelCapacity.toString) + context.put("transactionCapacity", "1000") + context.put("keep-alive", "0") + val channel = new MemoryChannel() + Configurables.configure(channel, context) + + val channel2 = new MemoryChannel() + Configurables.configure(channel2, context) + + val sink = new SparkSink() + context.put(SparkSinkConfig.CONF_HOSTNAME, "localhost") + context.put(SparkSinkConfig.CONF_PORT, String.valueOf(0)) + Configurables.configure(sink, context) + sink.setChannel(channel) + sink.start() + + val sink2 = new SparkSink() + context.put(SparkSinkConfig.CONF_HOSTNAME, "localhost") + context.put(SparkSinkConfig.CONF_PORT, String.valueOf(0)) + Configurables.configure(sink2, context) + sink2.setChannel(channel2) + sink2.start() + + // Set up the streaming context and input streams + val ssc = new StreamingContext(conf, batchDuration) + val addresses = Seq(sink.getPort(), sink2.getPort()).map(new InetSocketAddress("localhost", _)) + val flumeStream: ReceiverInputDStream[SparkFlumeEvent] = + FlumeUtils.createPollingStream(ssc, addresses, StorageLevel.MEMORY_AND_DISK, + eventsPerBatch, 5) + val outputBuffer = new ArrayBuffer[Seq[SparkFlumeEvent]] + with SynchronizedBuffer[Seq[SparkFlumeEvent]] + val outputStream = new TestOutputStream(flumeStream, outputBuffer) + outputStream.register() + + ssc.start() + writeAndVerify(Seq(channel, channel2), ssc, outputBuffer) + assertChannelIsEmpty(channel) + assertChannelIsEmpty(channel2) + sink.stop() + channel.stop() + } + + def writeAndVerify(channels: Seq[MemoryChannel], ssc: StreamingContext, + outputBuffer: ArrayBuffer[Seq[SparkFlumeEvent]]) { + val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] + val executor = Executors.newCachedThreadPool() + val executorCompletion = new ExecutorCompletionService[Void](executor) + channels.map(channel => { + executorCompletion.submit(new TxnSubmitter(channel, clock)) + }) + for (i <- 0 until channels.size) { + executorCompletion.take() + } + val startTime = System.currentTimeMillis() + while (outputBuffer.size < batchCount * channels.size && + System.currentTimeMillis() - startTime < 15000) { + logInfo("output.size = " + outputBuffer.size) + Thread.sleep(100) + } + val timeTaken = System.currentTimeMillis() - startTime + assert(timeTaken < 15000, "Operation timed out after " + timeTaken + " ms") + logInfo("Stopping context") + ssc.stop() + + val flattenedBuffer = outputBuffer.flatten + assert(flattenedBuffer.size === totalEventsPerChannel * channels.size) + var counter = 0 + for (k <- 0 until channels.size; i <- 0 until totalEventsPerChannel) { + val eventToVerify = EventBuilder.withBody((channels(k).getName + " - " + + String.valueOf(i)).getBytes("utf-8"), + Map[String, String]("test-" + i.toString -> "header")) + var found = false + var j = 0 + while (j < flattenedBuffer.size && !found) { + val strToCompare = new String(flattenedBuffer(j).event.getBody.array(), "utf-8") + if (new String(eventToVerify.getBody, "utf-8") == strToCompare && + eventToVerify.getHeaders.get("test-" + i.toString) + .equals(flattenedBuffer(j).event.getHeaders.get("test-" + i.toString))) { + found = true + counter += 1 + } + j += 1 + } + } + assert(counter === totalEventsPerChannel * channels.size) + } + + def assertChannelIsEmpty(channel: MemoryChannel) = { + val queueRemaining = channel.getClass.getDeclaredField("queueRemaining"); + queueRemaining.setAccessible(true) + val m = queueRemaining.get(channel).getClass.getDeclaredMethod("availablePermits") + assert(m.invoke(queueRemaining.get(channel)).asInstanceOf[Int] === 5000) + } + + private class TxnSubmitter(channel: MemoryChannel, clock: ManualClock) extends Callable[Void] { + override def call(): Void = { + var t = 0 + for (i <- 0 until batchCount) { + val tx = channel.getTransaction + tx.begin() + for (j <- 0 until eventsPerBatch) { + channel.put(EventBuilder.withBody((channel.getName + " - " + String.valueOf(t)).getBytes( + "utf-8"), + Map[String, String]("test-" + t.toString -> "header"))) + t += 1 + } + tx.commit() + tx.close() + Thread.sleep(500) // Allow some time for the events to reach + clock.addToTime(batchDuration.milliseconds) + } + null + } + } +} diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala index dd287d0ef90a0..73dffef953309 100644 --- a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala +++ b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala @@ -33,15 +33,26 @@ import org.apache.spark.streaming.{TestOutputStream, StreamingContext, TestSuite import org.apache.spark.streaming.util.ManualClock import org.apache.spark.streaming.api.java.JavaReceiverInputDStream -class FlumeStreamSuite extends TestSuiteBase { +import org.jboss.netty.channel.ChannelPipeline +import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory +import org.jboss.netty.channel.socket.SocketChannel +import org.jboss.netty.handler.codec.compression._ - val testPort = 9999 +class FlumeStreamSuite extends TestSuiteBase { test("flume input stream") { + runFlumeStreamTest(false, 9998) + } + + test("flume input compressed stream") { + runFlumeStreamTest(true, 9997) + } + + def runFlumeStreamTest(enableDecompression: Boolean, testPort: Int) { // Set up the streaming context and input streams val ssc = new StreamingContext(conf, batchDuration) val flumeStream: JavaReceiverInputDStream[SparkFlumeEvent] = - FlumeUtils.createStream(ssc, "localhost", testPort, StorageLevel.MEMORY_AND_DISK) + FlumeUtils.createStream(ssc, "localhost", testPort, StorageLevel.MEMORY_AND_DISK, enableDecompression) val outputBuffer = new ArrayBuffer[Seq[SparkFlumeEvent]] with SynchronizedBuffer[Seq[SparkFlumeEvent]] val outputStream = new TestOutputStream(flumeStream.receiverInputDStream, outputBuffer) @@ -52,8 +63,17 @@ class FlumeStreamSuite extends TestSuiteBase { val input = Seq(1, 2, 3, 4, 5) Thread.sleep(1000) val transceiver = new NettyTransceiver(new InetSocketAddress("localhost", testPort)) - val client = SpecificRequestor.getClient( - classOf[AvroSourceProtocol], transceiver) + var client: AvroSourceProtocol = null; + + if (enableDecompression) { + client = SpecificRequestor.getClient( + classOf[AvroSourceProtocol], + new NettyTransceiver(new InetSocketAddress("localhost", testPort), + new CompressionChannelFactory(6))); + } else { + client = SpecificRequestor.getClient( + classOf[AvroSourceProtocol], transceiver) + } for (i <- 0 until input.size) { val event = new AvroFlumeEvent @@ -64,6 +84,8 @@ class FlumeStreamSuite extends TestSuiteBase { clock.addToTime(batchDuration.milliseconds) } + Thread.sleep(1000) + val startTime = System.currentTimeMillis() while (outputBuffer.size < input.size && System.currentTimeMillis() - startTime < maxWaitTimeMillis) { logInfo("output.size = " + outputBuffer.size + ", input.size = " + input.size) @@ -85,4 +107,13 @@ class FlumeStreamSuite extends TestSuiteBase { assert(outputBuffer(i).head.event.getHeaders.get("test") === "header") } } + + class CompressionChannelFactory(compressionLevel: Int) extends NioClientSocketChannelFactory { + override def newChannel(pipeline:ChannelPipeline) : SocketChannel = { + var encoder : ZlibEncoder = new ZlibEncoder(compressionLevel); + pipeline.addFirst("deflater", encoder); + pipeline.addFirst("inflater", new ZlibDecoder()); + super.newChannel(pipeline); + } + } } diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml index d014a7aad0fca..4e2275ab238f7 100644 --- a/external/kafka/pom.xml +++ b/external/kafka/pom.xml @@ -27,6 +27,9 @@ org.apache.spark spark-streaming-kafka_2.10 + + streaming-kafka + jar Spark Project External Kafka http://spark.apache.org/ @@ -65,8 +68,18 @@ org.slf4j slf4j-simple + + org.apache.zookeeper + zookeeper + + + net.sf.jopt-simple + jopt-simple + 3.2 + test + org.scalatest scalatest_${scala.binary.version} @@ -77,6 +90,11 @@ scalacheck_${scala.binary.version} test + + junit + junit + test + com.novocode junit-interface diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala index 38095e88dcea9..e20e2c8f26991 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala @@ -18,7 +18,7 @@ package org.apache.spark.streaming.kafka import scala.collection.Map -import scala.reflect.ClassTag +import scala.reflect.{classTag, ClassTag} import java.util.Properties import java.util.concurrent.Executors @@ -48,8 +48,8 @@ private[streaming] class KafkaInputDStream[ K: ClassTag, V: ClassTag, - U <: Decoder[_]: Manifest, - T <: Decoder[_]: Manifest]( + U <: Decoder[_]: ClassTag, + T <: Decoder[_]: ClassTag]( @transient ssc_ : StreamingContext, kafkaParams: Map[String, String], topics: Map[String, Int], @@ -66,8 +66,8 @@ private[streaming] class KafkaReceiver[ K: ClassTag, V: ClassTag, - U <: Decoder[_]: Manifest, - T <: Decoder[_]: Manifest]( + U <: Decoder[_]: ClassTag, + T <: Decoder[_]: ClassTag]( kafkaParams: Map[String, String], topics: Map[String, Int], storageLevel: StorageLevel @@ -103,10 +103,10 @@ class KafkaReceiver[ tryZookeeperConsumerGroupCleanup(zkConnect, kafkaParams("group.id")) } - val keyDecoder = manifest[U].runtimeClass.getConstructor(classOf[VerifiableProperties]) + val keyDecoder = classTag[U].runtimeClass.getConstructor(classOf[VerifiableProperties]) .newInstance(consumerConfig.props) .asInstanceOf[Decoder[K]] - val valueDecoder = manifest[T].runtimeClass.getConstructor(classOf[VerifiableProperties]) + val valueDecoder = classTag[T].runtimeClass.getConstructor(classOf[VerifiableProperties]) .newInstance(consumerConfig.props) .asInstanceOf[Decoder[V]] diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala index 86bb91f362d29..48668f763e41e 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala @@ -65,7 +65,7 @@ object KafkaUtils { * in its own thread. * @param storageLevel Storage level to use for storing the received objects */ - def createStream[K: ClassTag, V: ClassTag, U <: Decoder[_]: Manifest, T <: Decoder[_]: Manifest]( + def createStream[K: ClassTag, V: ClassTag, U <: Decoder[_]: ClassTag, T <: Decoder[_]: ClassTag]( ssc: StreamingContext, kafkaParams: Map[String, String], topics: Map[String, Int], @@ -89,8 +89,6 @@ object KafkaUtils { groupId: String, topics: JMap[String, JInt] ): JavaPairReceiverInputDStream[String, String] = { - implicit val cmt: ClassTag[String] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]] createStream(jssc.ssc, zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*)) } @@ -111,8 +109,6 @@ object KafkaUtils { topics: JMap[String, JInt], storageLevel: StorageLevel ): JavaPairReceiverInputDStream[String, String] = { - implicit val cmt: ClassTag[String] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]] createStream(jssc.ssc, zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*), storageLevel) } @@ -140,13 +136,11 @@ object KafkaUtils { topics: JMap[String, JInt], storageLevel: StorageLevel ): JavaPairReceiverInputDStream[K, V] = { - implicit val keyCmt: ClassTag[K] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]] - implicit val valueCmt: ClassTag[V] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V]] + implicit val keyCmt: ClassTag[K] = ClassTag(keyTypeClass) + implicit val valueCmt: ClassTag[V] = ClassTag(valueTypeClass) - implicit val keyCmd: Manifest[U] = implicitly[Manifest[AnyRef]].asInstanceOf[Manifest[U]] - implicit val valueCmd: Manifest[T] = implicitly[Manifest[AnyRef]].asInstanceOf[Manifest[T]] + implicit val keyCmd: ClassTag[U] = ClassTag(keyDecoderClass) + implicit val valueCmd: ClassTag[T] = ClassTag(valueDecoderClass) createStream[K, V, U, T]( jssc.ssc, kafkaParams.toMap, Map(topics.mapValues(_.intValue()).toSeq: _*), storageLevel) diff --git a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java index 9f8046bf00f8f..0571454c01dae 100644 --- a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java +++ b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java @@ -17,31 +17,118 @@ package org.apache.spark.streaming.kafka; +import java.io.Serializable; import java.util.HashMap; +import java.util.List; + +import scala.Predef; +import scala.Tuple2; +import scala.collection.JavaConverters; + +import junit.framework.Assert; -import org.apache.spark.streaming.api.java.JavaPairReceiverInputDStream; -import org.junit.Test; -import com.google.common.collect.Maps; import kafka.serializer.StringDecoder; + +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.function.Function; import org.apache.spark.storage.StorageLevel; +import org.apache.spark.streaming.Duration; import org.apache.spark.streaming.LocalJavaStreamingContext; +import org.apache.spark.streaming.api.java.JavaDStream; +import org.apache.spark.streaming.api.java.JavaPairDStream; +import org.apache.spark.streaming.api.java.JavaStreamingContext; + +import org.junit.Test; +import org.junit.After; +import org.junit.Before; + +public class JavaKafkaStreamSuite extends LocalJavaStreamingContext implements Serializable { + private transient KafkaStreamSuite testSuite = new KafkaStreamSuite(); + + @Before + @Override + public void setUp() { + testSuite.beforeFunction(); + System.clearProperty("spark.driver.port"); + //System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.SystemClock"); + ssc = new JavaStreamingContext("local[2]", "test", new Duration(1000)); + } + + @After + @Override + public void tearDown() { + ssc.stop(); + ssc = null; + System.clearProperty("spark.driver.port"); + testSuite.afterFunction(); + } -public class JavaKafkaStreamSuite extends LocalJavaStreamingContext { @Test - public void testKafkaStream() { - HashMap topics = Maps.newHashMap(); - - // tests the API, does not actually test data receiving - JavaPairReceiverInputDStream test1 = - KafkaUtils.createStream(ssc, "localhost:12345", "group", topics); - JavaPairReceiverInputDStream test2 = KafkaUtils.createStream(ssc, "localhost:12345", "group", topics, - StorageLevel.MEMORY_AND_DISK_SER_2()); - - HashMap kafkaParams = Maps.newHashMap(); - kafkaParams.put("zookeeper.connect", "localhost:12345"); - kafkaParams.put("group.id","consumer-group"); - JavaPairReceiverInputDStream test3 = KafkaUtils.createStream(ssc, - String.class, String.class, StringDecoder.class, StringDecoder.class, - kafkaParams, topics, StorageLevel.MEMORY_AND_DISK_SER_2()); + public void testKafkaStream() throws InterruptedException { + String topic = "topic1"; + HashMap topics = new HashMap(); + topics.put(topic, 1); + + HashMap sent = new HashMap(); + sent.put("a", 5); + sent.put("b", 3); + sent.put("c", 10); + + testSuite.createTopic(topic); + HashMap tmp = new HashMap(sent); + testSuite.produceAndSendMessage(topic, + JavaConverters.mapAsScalaMapConverter(tmp).asScala().toMap( + Predef.>conforms())); + + HashMap kafkaParams = new HashMap(); + kafkaParams.put("zookeeper.connect", testSuite.zkConnect()); + kafkaParams.put("group.id", "test-consumer-" + KafkaTestUtils.random().nextInt(10000)); + kafkaParams.put("auto.offset.reset", "smallest"); + + JavaPairDStream stream = KafkaUtils.createStream(ssc, + String.class, + String.class, + StringDecoder.class, + StringDecoder.class, + kafkaParams, + topics, + StorageLevel.MEMORY_ONLY_SER()); + + final HashMap result = new HashMap(); + + JavaDStream words = stream.map( + new Function, String>() { + @Override + public String call(Tuple2 tuple2) throws Exception { + return tuple2._2(); + } + } + ); + + words.countByValue().foreachRDD( + new Function, Void>() { + @Override + public Void call(JavaPairRDD rdd) throws Exception { + List> ret = rdd.collect(); + for (Tuple2 r : ret) { + if (result.containsKey(r._1())) { + result.put(r._1(), result.get(r._1()) + r._2()); + } else { + result.put(r._1(), r._2()); + } + } + + return null; + } + } + ); + + ssc.start(); + ssc.awaitTermination(3000); + + Assert.assertEquals(sent.size(), result.size()); + for (String k : sent.keySet()) { + Assert.assertEquals(sent.get(k).intValue(), result.get(k).intValue()); + } } } diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala index e6f2c4a5cf5d1..c0b55e9340253 100644 --- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala +++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala @@ -17,28 +17,193 @@ package org.apache.spark.streaming.kafka -import kafka.serializer.StringDecoder +import java.io.File +import java.net.InetSocketAddress +import java.util.{Properties, Random} + +import scala.collection.mutable + +import kafka.admin.CreateTopicCommand +import kafka.common.TopicAndPartition +import kafka.producer.{KeyedMessage, ProducerConfig, Producer} +import kafka.utils.ZKStringSerializer +import kafka.serializer.{StringDecoder, StringEncoder} +import kafka.server.{KafkaConfig, KafkaServer} + +import org.I0Itec.zkclient.ZkClient + +import org.apache.zookeeper.server.ZooKeeperServer +import org.apache.zookeeper.server.NIOServerCnxnFactory + import org.apache.spark.streaming.{StreamingContext, TestSuiteBase} import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.dstream.ReceiverInputDStream +import org.apache.spark.util.Utils class KafkaStreamSuite extends TestSuiteBase { + import KafkaTestUtils._ + + val zkConnect = "localhost:2181" + val zkConnectionTimeout = 6000 + val zkSessionTimeout = 6000 + + val brokerPort = 9092 + val brokerProps = getBrokerConfig(brokerPort, zkConnect) + val brokerConf = new KafkaConfig(brokerProps) + + protected var zookeeper: EmbeddedZookeeper = _ + protected var zkClient: ZkClient = _ + protected var server: KafkaServer = _ + protected var producer: Producer[String, String] = _ + + override def useManualClock = false + + override def beforeFunction() { + // Zookeeper server startup + zookeeper = new EmbeddedZookeeper(zkConnect) + logInfo("==================== 0 ====================") + zkClient = new ZkClient(zkConnect, zkSessionTimeout, zkConnectionTimeout, ZKStringSerializer) + logInfo("==================== 1 ====================") - test("kafka input stream") { + // Kafka broker startup + server = new KafkaServer(brokerConf) + logInfo("==================== 2 ====================") + server.startup() + logInfo("==================== 3 ====================") + Thread.sleep(2000) + logInfo("==================== 4 ====================") + super.beforeFunction() + } + + override def afterFunction() { + producer.close() + server.shutdown() + brokerConf.logDirs.foreach { f => Utils.deleteRecursively(new File(f)) } + + zkClient.close() + zookeeper.shutdown() + + super.afterFunction() + } + + test("Kafka input stream") { val ssc = new StreamingContext(master, framework, batchDuration) - val topics = Map("my-topic" -> 1) - - // tests the API, does not actually test data receiving - val test1: ReceiverInputDStream[(String, String)] = - KafkaUtils.createStream(ssc, "localhost:1234", "group", topics) - val test2: ReceiverInputDStream[(String, String)] = - KafkaUtils.createStream(ssc, "localhost:12345", "group", topics, StorageLevel.MEMORY_AND_DISK_SER_2) - val kafkaParams = Map("zookeeper.connect"->"localhost:12345","group.id"->"consumer-group") - val test3: ReceiverInputDStream[(String, String)] = - KafkaUtils.createStream[String, String, StringDecoder, StringDecoder]( - ssc, kafkaParams, topics, StorageLevel.MEMORY_AND_DISK_SER_2) - - // TODO: Actually test receiving data + val topic = "topic1" + val sent = Map("a" -> 5, "b" -> 3, "c" -> 10) + createTopic(topic) + produceAndSendMessage(topic, sent) + + val kafkaParams = Map("zookeeper.connect" -> zkConnect, + "group.id" -> s"test-consumer-${random.nextInt(10000)}", + "auto.offset.reset" -> "smallest") + + val stream = KafkaUtils.createStream[String, String, StringDecoder, StringDecoder]( + ssc, + kafkaParams, + Map(topic -> 1), + StorageLevel.MEMORY_ONLY) + val result = new mutable.HashMap[String, Long]() + stream.map { case (k, v) => v } + .countByValue() + .foreachRDD { r => + val ret = r.collect() + ret.toMap.foreach { kv => + val count = result.getOrElseUpdate(kv._1, 0) + kv._2 + result.put(kv._1, count) + } + } + ssc.start() + ssc.awaitTermination(3000) + + assert(sent.size === result.size) + sent.keys.foreach { k => assert(sent(k) === result(k).toInt) } + ssc.stop() } + + private def createTestMessage(topic: String, sent: Map[String, Int]) + : Seq[KeyedMessage[String, String]] = { + val messages = for ((s, freq) <- sent; i <- 0 until freq) yield { + new KeyedMessage[String, String](topic, s) + } + messages.toSeq + } + + def createTopic(topic: String) { + CreateTopicCommand.createTopic(zkClient, topic, 1, 1, "0") + logInfo("==================== 5 ====================") + // wait until metadata is propagated + waitUntilMetadataIsPropagated(Seq(server), topic, 0, 1000) + } + + def produceAndSendMessage(topic: String, sent: Map[String, Int]) { + val brokerAddr = brokerConf.hostName + ":" + brokerConf.port + producer = new Producer[String, String](new ProducerConfig(getProducerConfig(brokerAddr))) + producer.send(createTestMessage(topic, sent): _*) + logInfo("==================== 6 ====================") + } +} + +object KafkaTestUtils { + val random = new Random() + + def getBrokerConfig(port: Int, zkConnect: String): Properties = { + val props = new Properties() + props.put("broker.id", "0") + props.put("host.name", "localhost") + props.put("port", port.toString) + props.put("log.dir", Utils.createTempDir().getAbsolutePath) + props.put("zookeeper.connect", zkConnect) + props.put("log.flush.interval.messages", "1") + props.put("replica.socket.timeout.ms", "1500") + props + } + + def getProducerConfig(brokerList: String): Properties = { + val props = new Properties() + props.put("metadata.broker.list", brokerList) + props.put("serializer.class", classOf[StringEncoder].getName) + props + } + + def waitUntilTrue(condition: () => Boolean, waitTime: Long): Boolean = { + val startTime = System.currentTimeMillis() + while (true) { + if (condition()) + return true + if (System.currentTimeMillis() > startTime + waitTime) + return false + Thread.sleep(waitTime.min(100L)) + } + // Should never go to here + throw new RuntimeException("unexpected error") + } + + def waitUntilMetadataIsPropagated(servers: Seq[KafkaServer], topic: String, partition: Int, + timeout: Long) { + assert(waitUntilTrue(() => + servers.foldLeft(true)(_ && _.apis.leaderCache.keySet.contains( + TopicAndPartition(topic, partition))), timeout), + s"Partition [$topic, $partition] metadata not propagated after timeout") + } + + class EmbeddedZookeeper(val zkConnect: String) { + val random = new Random() + val snapshotDir = Utils.createTempDir() + val logDir = Utils.createTempDir() + + val zookeeper = new ZooKeeperServer(snapshotDir, logDir, 500) + val (ip, port) = { + val splits = zkConnect.split(":") + (splits(0), splits(1).toInt) + } + val factory = new NIOServerCnxnFactory() + factory.configure(new InetSocketAddress(ip, port), 16) + factory.startup(zookeeper) + + def shutdown() { + factory.shutdown() + Utils.deleteRecursively(snapshotDir) + Utils.deleteRecursively(logDir) + } + } } diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index 4980208cba3b0..dc48a08c93de2 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -27,6 +27,9 @@ org.apache.spark spark-streaming-mqtt_2.10 + + streaming-mqtt + jar Spark Project External MQTT http://spark.apache.org/ @@ -64,6 +67,11 @@ scalacheck_${scala.binary.version} test + + junit + junit + test + com.novocode junit-interface diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml index 7073bd4404d9c..b93ad016f84f0 100644 --- a/external/twitter/pom.xml +++ b/external/twitter/pom.xml @@ -27,6 +27,9 @@ org.apache.spark spark-streaming-twitter_2.10 + + streaming-twitter + jar Spark Project External Twitter http://spark.apache.org/ @@ -59,6 +62,11 @@ scalacheck_${scala.binary.version} test + + junit + junit + test + com.novocode junit-interface diff --git a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala index 5ea2e5549d7df..4eacc47da5699 100644 --- a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala +++ b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala @@ -63,7 +63,8 @@ class TwitterReceiver( storageLevel: StorageLevel ) extends Receiver[Status](storageLevel) with Logging { - private var twitterStream: TwitterStream = _ + @volatile private var twitterStream: TwitterStream = _ + @volatile private var stopped = false def onStart() { try { @@ -78,7 +79,9 @@ class TwitterReceiver( def onScrubGeo(l: Long, l1: Long) {} def onStallWarning(stallWarning: StallWarning) {} def onException(e: Exception) { - restart("Error receiving tweets", e) + if (!stopped) { + restart("Error receiving tweets", e) + } } }) @@ -91,12 +94,14 @@ class TwitterReceiver( } setTwitterStream(newTwitterStream) logInfo("Twitter receiver started") + stopped = false } catch { case e: Exception => restart("Error starting Twitter stream", e) } } def onStop() { + stopped = true setTwitterStream(null) logInfo("Twitter receiver stopped") } diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml index cf306e0dca8bd..22c1fff23d9a2 100644 --- a/external/zeromq/pom.xml +++ b/external/zeromq/pom.xml @@ -27,6 +27,9 @@ org.apache.spark spark-streaming-zeromq_2.10 + + streaming-zeromq + jar Spark Project External ZeroMQ http://spark.apache.org/ @@ -59,6 +62,11 @@ scalacheck_${scala.binary.version} test + + junit + junit + test + com.novocode junit-interface diff --git a/extras/java8-tests/pom.xml b/extras/java8-tests/pom.xml index 955ec1a8c3033..5308bb4e440ea 100644 --- a/extras/java8-tests/pom.xml +++ b/extras/java8-tests/pom.xml @@ -28,7 +28,11 @@ java8-tests_2.10 pom Spark Project Java8 Tests POM - + + + java8-tests + + org.apache.spark @@ -46,6 +50,11 @@ ${project.version} test-jar + + junit + junit + test + com.novocode junit-interface diff --git a/extras/kinesis-asl/pom.xml b/extras/kinesis-asl/pom.xml new file mode 100644 index 0000000000000..a54b34235dfb4 --- /dev/null +++ b/extras/kinesis-asl/pom.xml @@ -0,0 +1,96 @@ + + + + 4.0.0 + + org.apache.spark + spark-parent + 1.1.0-SNAPSHOT + ../../pom.xml + + + + org.apache.spark + spark-streaming-kinesis-asl_2.10 + jar + Spark Kinesis Integration + + + kinesis-asl + + + + + org.apache.spark + spark-streaming_${scala.binary.version} + ${project.version} + + + org.apache.spark + spark-streaming_${scala.binary.version} + ${project.version} + test-jar + test + + + com.amazonaws + amazon-kinesis-client + ${aws.kinesis.client.version} + + + com.amazonaws + aws-java-sdk + ${aws.java.sdk.version} + + + org.scalatest + scalatest_${scala.binary.version} + test + + + org.mockito + mockito-all + test + + + org.scalacheck + scalacheck_${scala.binary.version} + test + + + org.easymock + easymockclassextension + test + + + com.novocode + junit-interface + test + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + org.scalatest + scalatest-maven-plugin + + + + diff --git a/extras/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java b/extras/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java new file mode 100644 index 0000000000000..a8b907b241893 --- /dev/null +++ b/extras/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java @@ -0,0 +1,180 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.examples.streaming; + +import java.util.ArrayList; +import java.util.List; +import java.util.regex.Pattern; + +import org.apache.log4j.Logger; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.function.FlatMapFunction; +import org.apache.spark.api.java.function.Function2; +import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.storage.StorageLevel; +import org.apache.spark.streaming.Duration; +import org.apache.spark.streaming.api.java.JavaDStream; +import org.apache.spark.streaming.api.java.JavaPairDStream; +import org.apache.spark.streaming.api.java.JavaStreamingContext; +import org.apache.spark.streaming.kinesis.KinesisUtils; + +import scala.Tuple2; + +import com.amazonaws.auth.DefaultAWSCredentialsProviderChain; +import com.amazonaws.services.kinesis.AmazonKinesisClient; +import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream; +import com.google.common.collect.Lists; + +/** + * Java-friendly Kinesis Spark Streaming WordCount example + * + * See http://spark.apache.org/docs/latest/streaming-kinesis.html for more details + * on the Kinesis Spark Streaming integration. + * + * This example spins up 1 Kinesis Worker (Spark Streaming Receiver) per shard + * for the given stream. + * It then starts pulling from the last checkpointed sequence number of the given + * and . + * + * Valid endpoint urls: http://docs.aws.amazon.com/general/latest/gr/rande.html#ak_region + * + * This code uses the DefaultAWSCredentialsProviderChain and searches for credentials + * in the following order of precedence: + * Environment Variables - AWS_ACCESS_KEY_ID and AWS_SECRET_KEY + * Java System Properties - aws.accessKeyId and aws.secretKey + * Credential profiles file - default location (~/.aws/credentials) shared by all AWS SDKs + * Instance profile credentials - delivered through the Amazon EC2 metadata service + * + * Usage: JavaKinesisWordCountASL + * is the name of the Kinesis stream (ie. mySparkStream) + * is the endpoint of the Kinesis service + * (ie. https://kinesis.us-east-1.amazonaws.com) + * + * Example: + * $ export AWS_ACCESS_KEY_ID= + * $ export AWS_SECRET_KEY= + * $ $SPARK_HOME/bin/run-example \ + * org.apache.spark.examples.streaming.JavaKinesisWordCountASL mySparkStream \ + * https://kinesis.us-east-1.amazonaws.com + * + * There is a companion helper class called KinesisWordCountProducerASL which puts dummy data + * onto the Kinesis stream. + * Usage instructions for KinesisWordCountProducerASL are provided in the class definition. + */ +public final class JavaKinesisWordCountASL { + private static final Pattern WORD_SEPARATOR = Pattern.compile(" "); + private static final Logger logger = Logger.getLogger(JavaKinesisWordCountASL.class); + + /* Make the constructor private to enforce singleton */ + private JavaKinesisWordCountASL() { + } + + public static void main(String[] args) { + /* Check that all required args were passed in. */ + if (args.length < 2) { + System.err.println( + "|Usage: KinesisWordCount \n" + + "| is the name of the Kinesis stream\n" + + "| is the endpoint of the Kinesis service\n" + + "| (e.g. https://kinesis.us-east-1.amazonaws.com)\n"); + System.exit(1); + } + + StreamingExamples.setStreamingLogLevels(); + + /* Populate the appropriate variables from the given args */ + String streamName = args[0]; + String endpointUrl = args[1]; + /* Set the batch interval to a fixed 2000 millis (2 seconds) */ + Duration batchInterval = new Duration(2000); + + /* Create a Kinesis client in order to determine the number of shards for the given stream */ + AmazonKinesisClient kinesisClient = new AmazonKinesisClient( + new DefaultAWSCredentialsProviderChain()); + kinesisClient.setEndpoint(endpointUrl); + + /* Determine the number of shards from the stream */ + int numShards = kinesisClient.describeStream(streamName) + .getStreamDescription().getShards().size(); + + /* In this example, we're going to create 1 Kinesis Worker/Receiver/DStream for each shard */ + int numStreams = numShards; + + /* Must add 1 more thread than the number of receivers or the output won't show properly from the driver */ + int numSparkThreads = numStreams + 1; + + /* Setup the Spark config. */ + SparkConf sparkConfig = new SparkConf().setAppName("KinesisWordCount").setMaster( + "local[" + numSparkThreads + "]"); + + /* Kinesis checkpoint interval. Same as batchInterval for this example. */ + Duration checkpointInterval = batchInterval; + + /* Setup the StreamingContext */ + JavaStreamingContext jssc = new JavaStreamingContext(sparkConfig, batchInterval); + + /* Create the same number of Kinesis DStreams/Receivers as Kinesis stream's shards */ + List> streamsList = new ArrayList>(numStreams); + for (int i = 0; i < numStreams; i++) { + streamsList.add( + KinesisUtils.createStream(jssc, streamName, endpointUrl, checkpointInterval, + InitialPositionInStream.LATEST, StorageLevel.MEMORY_AND_DISK_2()) + ); + } + + /* Union all the streams if there is more than 1 stream */ + JavaDStream unionStreams; + if (streamsList.size() > 1) { + unionStreams = jssc.union(streamsList.get(0), streamsList.subList(1, streamsList.size())); + } else { + /* Otherwise, just use the 1 stream */ + unionStreams = streamsList.get(0); + } + + /* + * Split each line of the union'd DStreams into multiple words using flatMap to produce the collection. + * Convert lines of byte[] to multiple Strings by first converting to String, then splitting on WORD_SEPARATOR. + */ + JavaDStream words = unionStreams.flatMap(new FlatMapFunction() { + @Override + public Iterable call(byte[] line) { + return Lists.newArrayList(WORD_SEPARATOR.split(new String(line))); + } + }); + + /* Map each word to a (word, 1) tuple, then reduce/aggregate by word. */ + JavaPairDStream wordCounts = words.mapToPair( + new PairFunction() { + @Override + public Tuple2 call(String s) { + return new Tuple2(s, 1); + } + }).reduceByKey(new Function2() { + @Override + public Integer call(Integer i1, Integer i2) { + return i1 + i2; + } + }); + + /* Print the first 10 wordCounts */ + wordCounts.print(); + + /* Start the streaming context and await termination */ + jssc.start(); + jssc.awaitTermination(); + } +} diff --git a/extras/kinesis-asl/src/main/resources/log4j.properties b/extras/kinesis-asl/src/main/resources/log4j.properties new file mode 100644 index 0000000000000..97348fb5b6123 --- /dev/null +++ b/extras/kinesis-asl/src/main/resources/log4j.properties @@ -0,0 +1,37 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +log4j.rootCategory=WARN, console + +# File appender +log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file.append=false +log4j.appender.file.file=target/unit-tests.log +log4j.appender.file.layout=org.apache.log4j.PatternLayout +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n + +# Console appender +log4j.appender.console=org.apache.log4j.ConsoleAppender +log4j.appender.console.target=System.out +log4j.appender.console.layout=org.apache.log4j.PatternLayout +log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n + +# Settings to quiet third party logs that are too verbose +log4j.logger.org.eclipse.jetty=WARN +log4j.logger.org.eclipse.jetty.util.component.AbstractLifeCycle=ERROR +log4j.logger.org.apache.spark.repl.SparkIMain$exprTyper=INFO +log4j.logger.org.apache.spark.repl.SparkILoop$SparkILoopInterpreter=INFO \ No newline at end of file diff --git a/extras/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala b/extras/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala new file mode 100644 index 0000000000000..d03edf8b30a9f --- /dev/null +++ b/extras/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala @@ -0,0 +1,251 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.streaming + +import java.nio.ByteBuffer +import scala.util.Random +import org.apache.spark.Logging +import org.apache.spark.SparkConf +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.Milliseconds +import org.apache.spark.streaming.StreamingContext +import org.apache.spark.streaming.StreamingContext.toPairDStreamFunctions +import org.apache.spark.streaming.kinesis.KinesisUtils +import com.amazonaws.auth.DefaultAWSCredentialsProviderChain +import com.amazonaws.services.kinesis.AmazonKinesisClient +import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream +import com.amazonaws.services.kinesis.model.PutRecordRequest +import org.apache.log4j.Logger +import org.apache.log4j.Level + +/** + * Kinesis Spark Streaming WordCount example. + * + * See http://spark.apache.org/docs/latest/streaming-kinesis.html for more details on + * the Kinesis Spark Streaming integration. + * + * This example spins up 1 Kinesis Worker (Spark Streaming Receiver) per shard + * for the given stream. + * It then starts pulling from the last checkpointed sequence number of the given + * and . + * + * Valid endpoint urls: http://docs.aws.amazon.com/general/latest/gr/rande.html#ak_region + * + * This code uses the DefaultAWSCredentialsProviderChain and searches for credentials + * in the following order of precedence: + * Environment Variables - AWS_ACCESS_KEY_ID and AWS_SECRET_KEY + * Java System Properties - aws.accessKeyId and aws.secretKey + * Credential profiles file - default location (~/.aws/credentials) shared by all AWS SDKs + * Instance profile credentials - delivered through the Amazon EC2 metadata service + * + * Usage: KinesisWordCountASL + * is the name of the Kinesis stream (ie. mySparkStream) + * is the endpoint of the Kinesis service + * (ie. https://kinesis.us-east-1.amazonaws.com) + * + * Example: + * $ export AWS_ACCESS_KEY_ID= + * $ export AWS_SECRET_KEY= + * $ $SPARK_HOME/bin/run-example \ + * org.apache.spark.examples.streaming.KinesisWordCountASL mySparkStream \ + * https://kinesis.us-east-1.amazonaws.com + * + * There is a companion helper class below called KinesisWordCountProducerASL which puts + * dummy data onto the Kinesis stream. + * Usage instructions for KinesisWordCountProducerASL are provided in that class definition. + */ +object KinesisWordCountASL extends Logging { + def main(args: Array[String]) { + /* Check that all required args were passed in. */ + if (args.length < 2) { + System.err.println( + """ + |Usage: KinesisWordCount + | is the name of the Kinesis stream + | is the endpoint of the Kinesis service + | (e.g. https://kinesis.us-east-1.amazonaws.com) + """.stripMargin) + System.exit(1) + } + + StreamingExamples.setStreamingLogLevels() + + /* Populate the appropriate variables from the given args */ + val Array(streamName, endpointUrl) = args + + /* Determine the number of shards from the stream */ + val kinesisClient = new AmazonKinesisClient(new DefaultAWSCredentialsProviderChain()) + kinesisClient.setEndpoint(endpointUrl) + val numShards = kinesisClient.describeStream(streamName).getStreamDescription().getShards() + .size() + + /* In this example, we're going to create 1 Kinesis Worker/Receiver/DStream for each shard. */ + val numStreams = numShards + + /* + * numSparkThreads should be 1 more thread than the number of receivers. + * This leaves one thread available for actually processing the data. + */ + val numSparkThreads = numStreams + 1 + + /* Setup the and SparkConfig and StreamingContext */ + /* Spark Streaming batch interval */ + val batchInterval = Milliseconds(2000) + val sparkConfig = new SparkConf().setAppName("KinesisWordCount") + .setMaster(s"local[$numSparkThreads]") + val ssc = new StreamingContext(sparkConfig, batchInterval) + + /* Kinesis checkpoint interval. Same as batchInterval for this example. */ + val kinesisCheckpointInterval = batchInterval + + /* Create the same number of Kinesis DStreams/Receivers as Kinesis stream's shards */ + val kinesisStreams = (0 until numStreams).map { i => + KinesisUtils.createStream(ssc, streamName, endpointUrl, kinesisCheckpointInterval, + InitialPositionInStream.LATEST, StorageLevel.MEMORY_AND_DISK_2) + } + + /* Union all the streams */ + val unionStreams = ssc.union(kinesisStreams) + + /* Convert each line of Array[Byte] to String, split into words, and count them */ + val words = unionStreams.flatMap(byteArray => new String(byteArray) + .split(" ")) + + /* Map each word to a (word, 1) tuple so we can reduce/aggregate by key. */ + val wordCounts = words.map(word => (word, 1)).reduceByKey(_ + _) + + /* Print the first 10 wordCounts */ + wordCounts.print() + + /* Start the streaming context and await termination */ + ssc.start() + ssc.awaitTermination() + } +} + +/** + * Usage: KinesisWordCountProducerASL + * + * is the name of the Kinesis stream (ie. mySparkStream) + * is the endpoint of the Kinesis service + * (ie. https://kinesis.us-east-1.amazonaws.com) + * is the rate of records per second to put onto the stream + * is the rate of records per second to put onto the stream + * + * Example: + * $ export AWS_ACCESS_KEY_ID= + * $ export AWS_SECRET_KEY= + * $ $SPARK_HOME/bin/run-example \ + * org.apache.spark.examples.streaming.KinesisWordCountProducerASL mySparkStream \ + * https://kinesis.us-east-1.amazonaws.com 10 5 + */ +object KinesisWordCountProducerASL { + def main(args: Array[String]) { + if (args.length < 4) { + System.err.println("Usage: KinesisWordCountProducerASL " + + " ") + System.exit(1) + } + + StreamingExamples.setStreamingLogLevels() + + /* Populate the appropriate variables from the given args */ + val Array(stream, endpoint, recordsPerSecond, wordsPerRecord) = args + + /* Generate the records and return the totals */ + val totals = generate(stream, endpoint, recordsPerSecond.toInt, wordsPerRecord.toInt) + + /* Print the array of (index, total) tuples */ + println("Totals") + totals.foreach(total => println(total.toString())) + } + + def generate(stream: String, + endpoint: String, + recordsPerSecond: Int, + wordsPerRecord: Int): Seq[(Int, Int)] = { + + val MaxRandomInts = 10 + + /* Create the Kinesis client */ + val kinesisClient = new AmazonKinesisClient(new DefaultAWSCredentialsProviderChain()) + kinesisClient.setEndpoint(endpoint) + + println(s"Putting records onto stream $stream and endpoint $endpoint at a rate of" + + s" $recordsPerSecond records per second and $wordsPerRecord words per record"); + + val totals = new Array[Int](MaxRandomInts) + /* Put String records onto the stream per the given recordPerSec and wordsPerRecord */ + for (i <- 1 to 5) { + + /* Generate recordsPerSec records to put onto the stream */ + val records = (1 to recordsPerSecond.toInt).map { recordNum => + /* + * Randomly generate each wordsPerRec words between 0 (inclusive) + * and MAX_RANDOM_INTS (exclusive) + */ + val data = (1 to wordsPerRecord.toInt).map(x => { + /* Generate the random int */ + val randomInt = Random.nextInt(MaxRandomInts) + + /* Keep track of the totals */ + totals(randomInt) += 1 + + randomInt.toString() + }).mkString(" ") + + /* Create a partitionKey based on recordNum */ + val partitionKey = s"partitionKey-$recordNum" + + /* Create a PutRecordRequest with an Array[Byte] version of the data */ + val putRecordRequest = new PutRecordRequest().withStreamName(stream) + .withPartitionKey(partitionKey) + .withData(ByteBuffer.wrap(data.getBytes())); + + /* Put the record onto the stream and capture the PutRecordResult */ + val putRecordResult = kinesisClient.putRecord(putRecordRequest); + } + + /* Sleep for a second */ + Thread.sleep(1000) + println("Sent " + recordsPerSecond + " records") + } + + /* Convert the totals to (index, total) tuple */ + (0 to (MaxRandomInts - 1)).zip(totals) + } +} + +/** + * Utility functions for Spark Streaming examples. + * This has been lifted from the examples/ project to remove the circular dependency. + */ +object StreamingExamples extends Logging { + + /** Set reasonable logging levels for streaming if the user has not configured log4j. */ + def setStreamingLogLevels() { + val log4jInitialized = Logger.getRootLogger.getAllAppenders.hasMoreElements + if (!log4jInitialized) { + // We first log something to initialize Spark's default logging, then we override the + // logging level. + logInfo("Setting log level to [WARN] for streaming example." + + " To override add a custom log4j.properties to the classpath.") + Logger.getRootLogger.setLevel(Level.WARN) + } + } +} diff --git a/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisCheckpointState.scala b/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisCheckpointState.scala new file mode 100644 index 0000000000000..0b80b611cdce7 --- /dev/null +++ b/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisCheckpointState.scala @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.streaming.kinesis + +import org.apache.spark.Logging +import org.apache.spark.streaming.Duration +import org.apache.spark.streaming.util.Clock +import org.apache.spark.streaming.util.ManualClock +import org.apache.spark.streaming.util.SystemClock + +/** + * This is a helper class for managing checkpoint clocks. + * + * @param checkpointInterval + * @param currentClock. Default to current SystemClock if none is passed in (mocking purposes) + */ +private[kinesis] class KinesisCheckpointState( + checkpointInterval: Duration, + currentClock: Clock = new SystemClock()) + extends Logging { + + /* Initialize the checkpoint clock using the given currentClock + checkpointInterval millis */ + val checkpointClock = new ManualClock() + checkpointClock.setTime(currentClock.currentTime() + checkpointInterval.milliseconds) + + /** + * Check if it's time to checkpoint based on the current time and the derived time + * for the next checkpoint + * + * @return true if it's time to checkpoint + */ + def shouldCheckpoint(): Boolean = { + new SystemClock().currentTime() > checkpointClock.currentTime() + } + + /** + * Advance the checkpoint clock by the checkpoint interval. + */ + def advanceCheckpoint() = { + checkpointClock.addToTime(checkpointInterval.milliseconds) + } +} diff --git a/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisReceiver.scala b/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisReceiver.scala new file mode 100644 index 0000000000000..1bd1f324298e7 --- /dev/null +++ b/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisReceiver.scala @@ -0,0 +1,149 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.streaming.kinesis + +import java.net.InetAddress +import java.util.UUID + +import org.apache.spark.Logging +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.Duration +import org.apache.spark.streaming.receiver.Receiver + +import com.amazonaws.auth.AWSCredentialsProvider +import com.amazonaws.auth.DefaultAWSCredentialsProviderChain +import com.amazonaws.services.kinesis.clientlibrary.interfaces.IRecordProcessor +import com.amazonaws.services.kinesis.clientlibrary.interfaces.IRecordProcessorFactory +import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream +import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration +import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker + +/** + * Custom AWS Kinesis-specific implementation of Spark Streaming's Receiver. + * This implementation relies on the Kinesis Client Library (KCL) Worker as described here: + * https://github.com/awslabs/amazon-kinesis-client + * This is a custom receiver used with StreamingContext.receiverStream(Receiver) + * as described here: + * http://spark.apache.org/docs/latest/streaming-custom-receivers.html + * Instances of this class will get shipped to the Spark Streaming Workers + * to run within a Spark Executor. + * + * @param appName Kinesis application name. Kinesis Apps are mapped to Kinesis Streams + * by the Kinesis Client Library. If you change the App name or Stream name, + * the KCL will throw errors. This usually requires deleting the backing + * DynamoDB table with the same name this Kinesis application. + * @param streamName Kinesis stream name + * @param endpointUrl Url of Kinesis service (e.g., https://kinesis.us-east-1.amazonaws.com) + * @param checkpointInterval Checkpoint interval for Kinesis checkpointing. + * See the Kinesis Spark Streaming documentation for more + * details on the different types of checkpoints. + * @param initialPositionInStream In the absence of Kinesis checkpoint info, this is the + * worker's initial starting position in the stream. + * The values are either the beginning of the stream + * per Kinesis' limit of 24 hours + * (InitialPositionInStream.TRIM_HORIZON) or + * the tip of the stream (InitialPositionInStream.LATEST). + * @param storageLevel Storage level to use for storing the received objects + * + * @return ReceiverInputDStream[Array[Byte]] + */ +private[kinesis] class KinesisReceiver( + appName: String, + streamName: String, + endpointUrl: String, + checkpointInterval: Duration, + initialPositionInStream: InitialPositionInStream, + storageLevel: StorageLevel) + extends Receiver[Array[Byte]](storageLevel) with Logging { receiver => + + /* + * The following vars are built in the onStart() method which executes in the Spark Worker after + * this code is serialized and shipped remotely. + */ + + /* + * workerId should be based on the ip address of the actual Spark Worker where this code runs + * (not the Driver's ip address.) + */ + var workerId: String = null + + /* + * This impl uses the DefaultAWSCredentialsProviderChain and searches for credentials + * in the following order of precedence: + * Environment Variables - AWS_ACCESS_KEY_ID and AWS_SECRET_KEY + * Java System Properties - aws.accessKeyId and aws.secretKey + * Credential profiles file at the default location (~/.aws/credentials) shared by all + * AWS SDKs and the AWS CLI + * Instance profile credentials delivered through the Amazon EC2 metadata service + */ + var credentialsProvider: AWSCredentialsProvider = null + + /* KCL config instance. */ + var kinesisClientLibConfiguration: KinesisClientLibConfiguration = null + + /* + * RecordProcessorFactory creates impls of IRecordProcessor. + * IRecordProcessor adapts the KCL to our Spark KinesisReceiver via the + * IRecordProcessor.processRecords() method. + * We're using our custom KinesisRecordProcessor in this case. + */ + var recordProcessorFactory: IRecordProcessorFactory = null + + /* + * Create a Kinesis Worker. + * This is the core client abstraction from the Kinesis Client Library (KCL). + * We pass the RecordProcessorFactory from above as well as the KCL config instance. + * A Kinesis Worker can process 1..* shards from the given stream - each with its + * own RecordProcessor. + */ + var worker: Worker = null + + /** + * This is called when the KinesisReceiver starts and must be non-blocking. + * The KCL creates and manages the receiving/processing thread pool through the Worker.run() + * method. + */ + override def onStart() { + workerId = InetAddress.getLocalHost.getHostAddress() + ":" + UUID.randomUUID() + credentialsProvider = new DefaultAWSCredentialsProviderChain() + kinesisClientLibConfiguration = new KinesisClientLibConfiguration(appName, streamName, + credentialsProvider, workerId).withKinesisEndpoint(endpointUrl) + .withInitialPositionInStream(initialPositionInStream).withTaskBackoffTimeMillis(500) + recordProcessorFactory = new IRecordProcessorFactory { + override def createProcessor: IRecordProcessor = new KinesisRecordProcessor(receiver, + workerId, new KinesisCheckpointState(checkpointInterval)) + } + worker = new Worker(recordProcessorFactory, kinesisClientLibConfiguration) + worker.run() + logInfo(s"Started receiver with workerId $workerId") + } + + /** + * This is called when the KinesisReceiver stops. + * The KCL worker.shutdown() method stops the receiving/processing threads. + * The KCL will do its best to drain and checkpoint any in-flight records upon shutdown. + */ + override def onStop() { + worker.shutdown() + logInfo(s"Shut down receiver with workerId $workerId") + workerId = null + credentialsProvider = null + kinesisClientLibConfiguration = null + recordProcessorFactory = null + worker = null + } +} diff --git a/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisRecordProcessor.scala b/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisRecordProcessor.scala new file mode 100644 index 0000000000000..8ecc2d90160b1 --- /dev/null +++ b/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisRecordProcessor.scala @@ -0,0 +1,212 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.streaming.kinesis + +import java.util.List + +import scala.collection.JavaConversions.asScalaBuffer +import scala.util.Random + +import org.apache.spark.Logging + +import com.amazonaws.services.kinesis.clientlibrary.exceptions.InvalidStateException +import com.amazonaws.services.kinesis.clientlibrary.exceptions.KinesisClientLibDependencyException +import com.amazonaws.services.kinesis.clientlibrary.exceptions.ShutdownException +import com.amazonaws.services.kinesis.clientlibrary.exceptions.ThrottlingException +import com.amazonaws.services.kinesis.clientlibrary.interfaces.IRecordProcessor +import com.amazonaws.services.kinesis.clientlibrary.interfaces.IRecordProcessorCheckpointer +import com.amazonaws.services.kinesis.clientlibrary.types.ShutdownReason +import com.amazonaws.services.kinesis.model.Record + +/** + * Kinesis-specific implementation of the Kinesis Client Library (KCL) IRecordProcessor. + * This implementation operates on the Array[Byte] from the KinesisReceiver. + * The Kinesis Worker creates an instance of this KinesisRecordProcessor upon startup. + * + * @param receiver Kinesis receiver + * @param workerId for logging purposes + * @param checkpointState represents the checkpoint state including the next checkpoint time. + * It's injected here for mocking purposes. + */ +private[kinesis] class KinesisRecordProcessor( + receiver: KinesisReceiver, + workerId: String, + checkpointState: KinesisCheckpointState) extends IRecordProcessor with Logging { + + /* shardId to be populated during initialize() */ + var shardId: String = _ + + /** + * The Kinesis Client Library calls this method during IRecordProcessor initialization. + * + * @param shardId assigned by the KCL to this particular RecordProcessor. + */ + override def initialize(shardId: String) { + logInfo(s"Initialize: Initializing workerId $workerId with shardId $shardId") + this.shardId = shardId + } + + /** + * This method is called by the KCL when a batch of records is pulled from the Kinesis stream. + * This is the record-processing bridge between the KCL's IRecordProcessor.processRecords() + * and Spark Streaming's Receiver.store(). + * + * @param batch list of records from the Kinesis stream shard + * @param checkpointer used to update Kinesis when this batch has been processed/stored + * in the DStream + */ + override def processRecords(batch: List[Record], checkpointer: IRecordProcessorCheckpointer) { + if (!receiver.isStopped()) { + try { + /* + * Note: If we try to store the raw ByteBuffer from record.getData(), the Spark Streaming + * Receiver.store(ByteBuffer) attempts to deserialize the ByteBuffer using the + * internally-configured Spark serializer (kryo, etc). + * This is not desirable, so we instead store a raw Array[Byte] and decouple + * ourselves from Spark's internal serialization strategy. + */ + batch.foreach(record => receiver.store(record.getData().array())) + + logDebug(s"Stored: Worker $workerId stored ${batch.size} records for shardId $shardId") + + /* + * Checkpoint the sequence number of the last record successfully processed/stored + * in the batch. + * In this implementation, we're checkpointing after the given checkpointIntervalMillis. + * Note that this logic requires that processRecords() be called AND that it's time to + * checkpoint. I point this out because there is no background thread running the + * checkpointer. Checkpointing is tested and trigger only when a new batch comes in. + * If the worker is shutdown cleanly, checkpoint will happen (see shutdown() below). + * However, if the worker dies unexpectedly, a checkpoint may not happen. + * This could lead to records being processed more than once. + */ + if (checkpointState.shouldCheckpoint()) { + /* Perform the checkpoint */ + KinesisRecordProcessor.retryRandom(checkpointer.checkpoint(), 4, 100) + + /* Update the next checkpoint time */ + checkpointState.advanceCheckpoint() + + logDebug(s"Checkpoint: WorkerId $workerId completed checkpoint of ${batch.size}" + + s" records for shardId $shardId") + logDebug(s"Checkpoint: Next checkpoint is at " + + s" ${checkpointState.checkpointClock.currentTime()} for shardId $shardId") + } + } catch { + case e: Throwable => { + /* + * If there is a failure within the batch, the batch will not be checkpointed. + * This will potentially cause records since the last checkpoint to be processed + * more than once. + */ + logError(s"Exception: WorkerId $workerId encountered and exception while storing " + + " or checkpointing a batch for workerId $workerId and shardId $shardId.", e) + + /* Rethrow the exception to the Kinesis Worker that is managing this RecordProcessor.*/ + throw e + } + } + } else { + /* RecordProcessor has been stopped. */ + logInfo(s"Stopped: The Spark KinesisReceiver has stopped for workerId $workerId" + + s" and shardId $shardId. No more records will be processed.") + } + } + + /** + * Kinesis Client Library is shutting down this Worker for 1 of 2 reasons: + * 1) the stream is resharding by splitting or merging adjacent shards + * (ShutdownReason.TERMINATE) + * 2) the failed or latent Worker has stopped sending heartbeats for whatever reason + * (ShutdownReason.ZOMBIE) + * + * @param checkpointer used to perform a Kinesis checkpoint for ShutdownReason.TERMINATE + * @param reason for shutdown (ShutdownReason.TERMINATE or ShutdownReason.ZOMBIE) + */ + override def shutdown(checkpointer: IRecordProcessorCheckpointer, reason: ShutdownReason) { + logInfo(s"Shutdown: Shutting down workerId $workerId with reason $reason") + reason match { + /* + * TERMINATE Use Case. Checkpoint. + * Checkpoint to indicate that all records from the shard have been drained and processed. + * It's now OK to read from the new shards that resulted from a resharding event. + */ + case ShutdownReason.TERMINATE => + KinesisRecordProcessor.retryRandom(checkpointer.checkpoint(), 4, 100) + + /* + * ZOMBIE Use Case. NoOp. + * No checkpoint because other workers may have taken over and already started processing + * the same records. + * This may lead to records being processed more than once. + */ + case ShutdownReason.ZOMBIE => + + /* Unknown reason. NoOp */ + case _ => + } + } +} + +private[kinesis] object KinesisRecordProcessor extends Logging { + /** + * Retry the given amount of times with a random backoff time (millis) less than the + * given maxBackOffMillis + * + * @param expression expression to evalute + * @param numRetriesLeft number of retries left + * @param maxBackOffMillis: max millis between retries + * + * @return evaluation of the given expression + * @throws Unretryable exception, unexpected exception, + * or any exception that persists after numRetriesLeft reaches 0 + */ + @annotation.tailrec + def retryRandom[T](expression: => T, numRetriesLeft: Int, maxBackOffMillis: Int): T = { + util.Try { expression } match { + /* If the function succeeded, evaluate to x. */ + case util.Success(x) => x + /* If the function failed, either retry or throw the exception */ + case util.Failure(e) => e match { + /* Retry: Throttling or other Retryable exception has occurred */ + case _: ThrottlingException | _: KinesisClientLibDependencyException if numRetriesLeft > 1 + => { + val backOffMillis = Random.nextInt(maxBackOffMillis) + Thread.sleep(backOffMillis) + logError(s"Retryable Exception: Random backOffMillis=${backOffMillis}", e) + retryRandom(expression, numRetriesLeft - 1, maxBackOffMillis) + } + /* Throw: Shutdown has been requested by the Kinesis Client Library.*/ + case _: ShutdownException => { + logError(s"ShutdownException: Caught shutdown exception, skipping checkpoint.", e) + throw e + } + /* Throw: Non-retryable exception has occurred with the Kinesis Client Library */ + case _: InvalidStateException => { + logError(s"InvalidStateException: Cannot save checkpoint to the DynamoDB table used" + + s" by the Amazon Kinesis Client Library. Table likely doesn't exist.", e) + throw e + } + /* Throw: Unexpected exception has occurred */ + case _ => { + logError(s"Unexpected, non-retryable exception.", e) + throw e + } + } + } + } +} diff --git a/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisUtils.scala b/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisUtils.scala new file mode 100644 index 0000000000000..713cac0e293c0 --- /dev/null +++ b/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisUtils.scala @@ -0,0 +1,96 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.streaming.kinesis + +import org.apache.spark.annotation.Experimental +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.Duration +import org.apache.spark.streaming.StreamingContext +import org.apache.spark.streaming.api.java.JavaReceiverInputDStream +import org.apache.spark.streaming.api.java.JavaStreamingContext +import org.apache.spark.streaming.dstream.ReceiverInputDStream + +import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream + + +/** + * Helper class to create Amazon Kinesis Input Stream + * :: Experimental :: + */ +@Experimental +object KinesisUtils { + /** + * Create an InputDStream that pulls messages from a Kinesis stream. + * + * @param ssc StreamingContext object + * @param streamName Kinesis stream name + * @param endpointUrl Url of Kinesis service (e.g., https://kinesis.us-east-1.amazonaws.com) + * @param checkpointInterval Checkpoint interval for Kinesis checkpointing. + * See the Kinesis Spark Streaming documentation for more + * details on the different types of checkpoints. + * @param initialPositionInStream In the absence of Kinesis checkpoint info, this is the + * worker's initial starting position in the stream. + * The values are either the beginning of the stream + * per Kinesis' limit of 24 hours + * (InitialPositionInStream.TRIM_HORIZON) or + * the tip of the stream (InitialPositionInStream.LATEST). + * @param storageLevel Storage level to use for storing the received objects + * + * @return ReceiverInputDStream[Array[Byte]] + */ + def createStream( + ssc: StreamingContext, + streamName: String, + endpointUrl: String, + checkpointInterval: Duration, + initialPositionInStream: InitialPositionInStream, + storageLevel: StorageLevel): ReceiverInputDStream[Array[Byte]] = { + ssc.receiverStream(new KinesisReceiver(ssc.sc.appName, streamName, endpointUrl, + checkpointInterval, initialPositionInStream, storageLevel)) + } + + /** + * Create a Java-friendly InputDStream that pulls messages from a Kinesis stream. + * + * @param jssc Java StreamingContext object + * @param ssc StreamingContext object + * @param streamName Kinesis stream name + * @param endpointUrl Url of Kinesis service (e.g., https://kinesis.us-east-1.amazonaws.com) + * @param checkpointInterval Checkpoint interval for Kinesis checkpointing. + * See the Kinesis Spark Streaming documentation for more + * details on the different types of checkpoints. + * @param initialPositionInStream In the absence of Kinesis checkpoint info, this is the + * worker's initial starting position in the stream. + * The values are either the beginning of the stream + * per Kinesis' limit of 24 hours + * (InitialPositionInStream.TRIM_HORIZON) or + * the tip of the stream (InitialPositionInStream.LATEST). + * @param storageLevel Storage level to use for storing the received objects + * + * @return JavaReceiverInputDStream[Array[Byte]] + */ + def createStream( + jssc: JavaStreamingContext, + streamName: String, + endpointUrl: String, + checkpointInterval: Duration, + initialPositionInStream: InitialPositionInStream, + storageLevel: StorageLevel): JavaReceiverInputDStream[Array[Byte]] = { + jssc.receiverStream(new KinesisReceiver(jssc.ssc.sc.appName, streamName, + endpointUrl, checkpointInterval, initialPositionInStream, storageLevel)) + } +} diff --git a/extras/kinesis-asl/src/test/java/org/apache/spark/streaming/kinesis/JavaKinesisStreamSuite.java b/extras/kinesis-asl/src/test/java/org/apache/spark/streaming/kinesis/JavaKinesisStreamSuite.java new file mode 100644 index 0000000000000..87954a31f60ce --- /dev/null +++ b/extras/kinesis-asl/src/test/java/org/apache/spark/streaming/kinesis/JavaKinesisStreamSuite.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.kinesis; + +import org.apache.spark.storage.StorageLevel; +import org.apache.spark.streaming.Duration; +import org.apache.spark.streaming.LocalJavaStreamingContext; +import org.apache.spark.streaming.api.java.JavaDStream; +import org.junit.Test; + +import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream; + +/** + * Demonstrate the use of the KinesisUtils Java API + */ +public class JavaKinesisStreamSuite extends LocalJavaStreamingContext { + @Test + public void testKinesisStream() { + // Tests the API, does not actually test data receiving + JavaDStream kinesisStream = KinesisUtils.createStream(ssc, "mySparkStream", + "https://kinesis.us-west-2.amazonaws.com", new Duration(2000), + InitialPositionInStream.LATEST, StorageLevel.MEMORY_AND_DISK_2()); + + ssc.stop(); + } +} diff --git a/extras/kinesis-asl/src/test/resources/log4j.properties b/extras/kinesis-asl/src/test/resources/log4j.properties new file mode 100644 index 0000000000000..e01e049595475 --- /dev/null +++ b/extras/kinesis-asl/src/test/resources/log4j.properties @@ -0,0 +1,26 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +log4j.rootCategory=INFO, file +# log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file.append=false +log4j.appender.file.file=target/unit-tests.log +log4j.appender.file.layout=org.apache.log4j.PatternLayout +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n + +# Ignore messages below warning level from Jetty, because it's a bit verbose +log4j.logger.org.eclipse.jetty=WARN diff --git a/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisReceiverSuite.scala b/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisReceiverSuite.scala new file mode 100644 index 0000000000000..41dbd64c2b1fa --- /dev/null +++ b/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisReceiverSuite.scala @@ -0,0 +1,275 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.streaming.kinesis + +import java.nio.ByteBuffer + +import scala.collection.JavaConversions.seqAsJavaList + +import org.apache.spark.annotation.Experimental +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.Milliseconds +import org.apache.spark.streaming.Seconds +import org.apache.spark.streaming.StreamingContext +import org.apache.spark.streaming.TestSuiteBase +import org.apache.spark.streaming.util.Clock +import org.apache.spark.streaming.util.ManualClock +import org.scalatest.BeforeAndAfter +import org.scalatest.Matchers +import org.scalatest.mock.EasyMockSugar + +import com.amazonaws.services.kinesis.clientlibrary.exceptions.InvalidStateException +import com.amazonaws.services.kinesis.clientlibrary.exceptions.KinesisClientLibDependencyException +import com.amazonaws.services.kinesis.clientlibrary.exceptions.ShutdownException +import com.amazonaws.services.kinesis.clientlibrary.exceptions.ThrottlingException +import com.amazonaws.services.kinesis.clientlibrary.interfaces.IRecordProcessorCheckpointer +import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream +import com.amazonaws.services.kinesis.clientlibrary.types.ShutdownReason +import com.amazonaws.services.kinesis.model.Record + +/** + * Suite of Kinesis streaming receiver tests focusing mostly on the KinesisRecordProcessor + */ +class KinesisReceiverSuite extends TestSuiteBase with Matchers with BeforeAndAfter + with EasyMockSugar { + + val app = "TestKinesisReceiver" + val stream = "mySparkStream" + val endpoint = "endpoint-url" + val workerId = "dummyWorkerId" + val shardId = "dummyShardId" + + val record1 = new Record() + record1.setData(ByteBuffer.wrap("Spark In Action".getBytes())) + val record2 = new Record() + record2.setData(ByteBuffer.wrap("Learning Spark".getBytes())) + val batch = List[Record](record1, record2) + + var receiverMock: KinesisReceiver = _ + var checkpointerMock: IRecordProcessorCheckpointer = _ + var checkpointClockMock: ManualClock = _ + var checkpointStateMock: KinesisCheckpointState = _ + var currentClockMock: Clock = _ + + override def beforeFunction() = { + receiverMock = mock[KinesisReceiver] + checkpointerMock = mock[IRecordProcessorCheckpointer] + checkpointClockMock = mock[ManualClock] + checkpointStateMock = mock[KinesisCheckpointState] + currentClockMock = mock[Clock] + } + + test("kinesis utils api") { + val ssc = new StreamingContext(master, framework, batchDuration) + // Tests the API, does not actually test data receiving + val kinesisStream = KinesisUtils.createStream(ssc, "mySparkStream", + "https://kinesis.us-west-2.amazonaws.com", Seconds(2), + InitialPositionInStream.LATEST, StorageLevel.MEMORY_AND_DISK_2); + ssc.stop() + } + + test("process records including store and checkpoint") { + val expectedCheckpointIntervalMillis = 10 + expecting { + receiverMock.isStopped().andReturn(false).once() + receiverMock.store(record1.getData().array()).once() + receiverMock.store(record2.getData().array()).once() + checkpointStateMock.shouldCheckpoint().andReturn(true).once() + checkpointerMock.checkpoint().once() + checkpointStateMock.advanceCheckpoint().once() + } + whenExecuting(receiverMock, checkpointerMock, checkpointStateMock) { + val recordProcessor = new KinesisRecordProcessor(receiverMock, workerId, + checkpointStateMock) + recordProcessor.processRecords(batch, checkpointerMock) + } + } + + test("shouldn't store and checkpoint when receiver is stopped") { + expecting { + receiverMock.isStopped().andReturn(true).once() + } + whenExecuting(receiverMock, checkpointerMock, checkpointStateMock) { + val recordProcessor = new KinesisRecordProcessor(receiverMock, workerId, + checkpointStateMock) + recordProcessor.processRecords(batch, checkpointerMock) + } + } + + test("shouldn't checkpoint when exception occurs during store") { + expecting { + receiverMock.isStopped().andReturn(false).once() + receiverMock.store(record1.getData().array()).andThrow(new RuntimeException()).once() + } + whenExecuting(receiverMock, checkpointerMock, checkpointStateMock) { + intercept[RuntimeException] { + val recordProcessor = new KinesisRecordProcessor(receiverMock, workerId, + checkpointStateMock) + recordProcessor.processRecords(batch, checkpointerMock) + } + } + } + + test("should set checkpoint time to currentTime + checkpoint interval upon instantiation") { + expecting { + currentClockMock.currentTime().andReturn(0).once() + } + whenExecuting(currentClockMock) { + val checkpointIntervalMillis = 10 + val checkpointState = new KinesisCheckpointState(Milliseconds(checkpointIntervalMillis), currentClockMock) + assert(checkpointState.checkpointClock.currentTime() == checkpointIntervalMillis) + } + } + + test("should checkpoint if we have exceeded the checkpoint interval") { + expecting { + currentClockMock.currentTime().andReturn(0).once() + } + whenExecuting(currentClockMock) { + val checkpointState = new KinesisCheckpointState(Milliseconds(Long.MinValue), currentClockMock) + assert(checkpointState.shouldCheckpoint()) + } + } + + test("shouldn't checkpoint if we have not exceeded the checkpoint interval") { + expecting { + currentClockMock.currentTime().andReturn(0).once() + } + whenExecuting(currentClockMock) { + val checkpointState = new KinesisCheckpointState(Milliseconds(Long.MaxValue), currentClockMock) + assert(!checkpointState.shouldCheckpoint()) + } + } + + test("should add to time when advancing checkpoint") { + expecting { + currentClockMock.currentTime().andReturn(0).once() + } + whenExecuting(currentClockMock) { + val checkpointIntervalMillis = 10 + val checkpointState = new KinesisCheckpointState(Milliseconds(checkpointIntervalMillis), currentClockMock) + assert(checkpointState.checkpointClock.currentTime() == checkpointIntervalMillis) + checkpointState.advanceCheckpoint() + assert(checkpointState.checkpointClock.currentTime() == (2 * checkpointIntervalMillis)) + } + } + + test("shutdown should checkpoint if the reason is TERMINATE") { + expecting { + checkpointerMock.checkpoint().once() + } + whenExecuting(checkpointerMock, checkpointStateMock) { + val recordProcessor = new KinesisRecordProcessor(receiverMock, workerId, + checkpointStateMock) + val reason = ShutdownReason.TERMINATE + recordProcessor.shutdown(checkpointerMock, reason) + } + } + + test("shutdown should not checkpoint if the reason is something other than TERMINATE") { + expecting { + } + whenExecuting(checkpointerMock, checkpointStateMock) { + val recordProcessor = new KinesisRecordProcessor(receiverMock, workerId, + checkpointStateMock) + recordProcessor.shutdown(checkpointerMock, ShutdownReason.ZOMBIE) + recordProcessor.shutdown(checkpointerMock, null) + } + } + + test("retry success on first attempt") { + val expectedIsStopped = false + expecting { + receiverMock.isStopped().andReturn(expectedIsStopped).once() + } + whenExecuting(receiverMock) { + val actualVal = KinesisRecordProcessor.retryRandom(receiverMock.isStopped(), 2, 100) + assert(actualVal == expectedIsStopped) + } + } + + test("retry success on second attempt after a Kinesis throttling exception") { + val expectedIsStopped = false + expecting { + receiverMock.isStopped().andThrow(new ThrottlingException("error message")) + .andReturn(expectedIsStopped).once() + } + whenExecuting(receiverMock) { + val actualVal = KinesisRecordProcessor.retryRandom(receiverMock.isStopped(), 2, 100) + assert(actualVal == expectedIsStopped) + } + } + + test("retry success on second attempt after a Kinesis dependency exception") { + val expectedIsStopped = false + expecting { + receiverMock.isStopped().andThrow(new KinesisClientLibDependencyException("error message")) + .andReturn(expectedIsStopped).once() + } + whenExecuting(receiverMock) { + val actualVal = KinesisRecordProcessor.retryRandom(receiverMock.isStopped(), 2, 100) + assert(actualVal == expectedIsStopped) + } + } + + test("retry failed after a shutdown exception") { + expecting { + checkpointerMock.checkpoint().andThrow(new ShutdownException("error message")).once() + } + whenExecuting(checkpointerMock) { + intercept[ShutdownException] { + KinesisRecordProcessor.retryRandom(checkpointerMock.checkpoint(), 2, 100) + } + } + } + + test("retry failed after an invalid state exception") { + expecting { + checkpointerMock.checkpoint().andThrow(new InvalidStateException("error message")).once() + } + whenExecuting(checkpointerMock) { + intercept[InvalidStateException] { + KinesisRecordProcessor.retryRandom(checkpointerMock.checkpoint(), 2, 100) + } + } + } + + test("retry failed after unexpected exception") { + expecting { + checkpointerMock.checkpoint().andThrow(new RuntimeException("error message")).once() + } + whenExecuting(checkpointerMock) { + intercept[RuntimeException] { + KinesisRecordProcessor.retryRandom(checkpointerMock.checkpoint(), 2, 100) + } + } + } + + test("retry failed after exhausing all retries") { + val expectedErrorMessage = "final try error message" + expecting { + checkpointerMock.checkpoint().andThrow(new ThrottlingException("error message")) + .andThrow(new ThrottlingException(expectedErrorMessage)).once() + } + whenExecuting(checkpointerMock) { + val exception = intercept[RuntimeException] { + KinesisRecordProcessor.retryRandom(checkpointerMock.checkpoint(), 2, 100) + } + exception.getMessage().shouldBe(expectedErrorMessage) + } + } +} diff --git a/extras/spark-ganglia-lgpl/pom.xml b/extras/spark-ganglia-lgpl/pom.xml index 22ea330b4374d..a5b162a0482e4 100644 --- a/extras/spark-ganglia-lgpl/pom.xml +++ b/extras/spark-ganglia-lgpl/pom.xml @@ -29,7 +29,11 @@ spark-ganglia-lgpl_2.10 jar Spark Ganglia Integration - + + + ganglia-lgpl + + org.apache.spark diff --git a/extras/spark-ganglia-lgpl/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala b/extras/spark-ganglia-lgpl/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala index d03d7774e8c80..3b1880e143513 100644 --- a/extras/spark-ganglia-lgpl/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala +++ b/extras/spark-ganglia-lgpl/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala @@ -82,5 +82,9 @@ class GangliaSink(val property: Properties, val registry: MetricRegistry, override def stop() { reporter.stop() } + + override def report() { + reporter.report() + } } diff --git a/graphx/pom.xml b/graphx/pom.xml index 7d5d83e7f3bb9..6dd52fc618b1e 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -27,6 +27,9 @@ org.apache.spark spark-graphx_2.10 + + graphx + jar Spark Project GraphX http://spark.apache.org/ diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala index 4db45c9af8fae..fa4b891754c40 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala @@ -107,14 +107,16 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] protected () extends Serializab /** * Repartitions the edges in the graph according to `partitionStrategy`. * - * @param the partitioning strategy to use when partitioning the edges in the graph. + * @param partitionStrategy the partitioning strategy to use when partitioning the edges + * in the graph. */ def partitionBy(partitionStrategy: PartitionStrategy): Graph[VD, ED] /** * Repartitions the edges in the graph according to `partitionStrategy`. * - * @param the partitioning strategy to use when partitioning the edges in the graph. + * @param partitionStrategy the partitioning strategy to use when partitioning the edges + * in the graph. * @param numPartitions the number of edge partitions in the new graph. */ def partitionBy(partitionStrategy: PartitionStrategy, numPartitions: Int): Graph[VD, ED] @@ -342,7 +344,7 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] protected () extends Serializab * * {{{ * val rawGraph: Graph[_, _] = Graph.textFile("webgraph") - * val outDeg: RDD[(VertexId, Int)] = rawGraph.outDegrees() + * val outDeg: RDD[(VertexId, Int)] = rawGraph.outDegrees * val graph = rawGraph.outerJoinVertices(outDeg) { * (vid, data, optDeg) => optDeg.getOrElse(0) * } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala index f97f329c0e832..1948c978c30bf 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala @@ -35,9 +35,6 @@ class GraphKryoRegistrator extends KryoRegistrator { def registerClasses(kryo: Kryo) { kryo.register(classOf[Edge[Object]]) - kryo.register(classOf[MessageToPartition[Object]]) - kryo.register(classOf[VertexBroadcastMsg[Object]]) - kryo.register(classOf[RoutingTableMessage]) kryo.register(classOf[(VertexId, Object)]) kryo.register(classOf[EdgePartition[Object, Object]]) kryo.register(classOf[BitSet]) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala index edd5b79da1522..02afaa987d40d 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala @@ -198,10 +198,10 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Seriali * * {{{ * val rawGraph: Graph[Int, Int] = GraphLoader.edgeListFile(sc, "webgraph") - * .mapVertices(v => 0) - * val outDeg: RDD[(Int, Int)] = rawGraph.outDegrees - * val graph = rawGraph.leftJoinVertices[Int,Int](outDeg, - * (v, deg) => deg ) + * .mapVertices((_, _) => 0) + * val outDeg = rawGraph.outDegrees + * val graph = rawGraph.joinVertices[Int](outDeg) + * ((_, _, outDeg) => outDeg) * }}} * */ diff --git a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala index f1b6df9a3025e..4825d12fc27b3 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala @@ -182,8 +182,8 @@ class VertexRDD[@specialized VD: ClassTag]( /** * Left joins this RDD with another VertexRDD with the same index. This function will fail if * both VertexRDDs do not share the same index. The resulting vertex set contains an entry for - * each - * vertex in `this`. If `other` is missing any vertex in this VertexRDD, `f` is passed `None`. + * each vertex in `this`. + * If `other` is missing any vertex in this VertexRDD, `f` is passed `None`. * * @tparam VD2 the attribute type of the other VertexRDD * @tparam VD3 the attribute type of the resulting VertexRDD diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala index ccdaa82eb9162..33f35cfb69a26 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala @@ -26,7 +26,6 @@ import org.apache.spark.storage.StorageLevel import org.apache.spark.graphx._ import org.apache.spark.graphx.impl.GraphImpl._ -import org.apache.spark.graphx.impl.MsgRDDFunctions._ import org.apache.spark.graphx.util.BytecodeUtils @@ -83,15 +82,13 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( val vdTag = classTag[VD] val newEdges = edges.withPartitionsRDD(edges.map { e => val part: PartitionID = partitionStrategy.getPartition(e.srcId, e.dstId, numPartitions) - - // Should we be using 3-tuple or an optimized class - new MessageToPartition(part, (e.srcId, e.dstId, e.attr)) + (part, (e.srcId, e.dstId, e.attr)) } .partitionBy(new HashPartitioner(numPartitions)) .mapPartitionsWithIndex( { (pid, iter) => val builder = new EdgePartitionBuilder[ED, VD]()(edTag, vdTag) iter.foreach { message => - val data = message.data + val data = message._2 builder.add(data._1, data._2, data._3) } val edgePartition = builder.toEdgePartition diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala index d85afa45b1264..714f3b81c9dad 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala @@ -25,86 +25,10 @@ import org.apache.spark.graphx.{PartitionID, VertexId} import org.apache.spark.rdd.{ShuffledRDD, RDD} -private[graphx] -class VertexBroadcastMsg[@specialized(Int, Long, Double, Boolean) T]( - @transient var partition: PartitionID, - var vid: VertexId, - var data: T) - extends Product2[PartitionID, (VertexId, T)] with Serializable { - - override def _1 = partition - - override def _2 = (vid, data) - - override def canEqual(that: Any): Boolean = that.isInstanceOf[VertexBroadcastMsg[_]] -} - - -/** - * A message used to send a specific value to a partition. - * @param partition index of the target partition. - * @param data value to send - */ -private[graphx] -class MessageToPartition[@specialized(Int, Long, Double, Char, Boolean/* , AnyRef */) T]( - @transient var partition: PartitionID, - var data: T) - extends Product2[PartitionID, T] with Serializable { - - override def _1 = partition - - override def _2 = data - - override def canEqual(that: Any): Boolean = that.isInstanceOf[MessageToPartition[_]] -} - - -private[graphx] -class VertexBroadcastMsgRDDFunctions[T: ClassTag](self: RDD[VertexBroadcastMsg[T]]) { - def partitionBy(partitioner: Partitioner): RDD[VertexBroadcastMsg[T]] = { - val rdd = new ShuffledRDD[PartitionID, (VertexId, T), (VertexId, T), VertexBroadcastMsg[T]]( - self, partitioner) - - // Set a custom serializer if the data is of int or double type. - if (classTag[T] == ClassTag.Int) { - rdd.setSerializer(new IntVertexBroadcastMsgSerializer) - } else if (classTag[T] == ClassTag.Long) { - rdd.setSerializer(new LongVertexBroadcastMsgSerializer) - } else if (classTag[T] == ClassTag.Double) { - rdd.setSerializer(new DoubleVertexBroadcastMsgSerializer) - } - rdd - } -} - - -private[graphx] -class MsgRDDFunctions[T: ClassTag](self: RDD[MessageToPartition[T]]) { - - /** - * Return a copy of the RDD partitioned using the specified partitioner. - */ - def partitionBy(partitioner: Partitioner): RDD[MessageToPartition[T]] = { - new ShuffledRDD[PartitionID, T, T, MessageToPartition[T]](self, partitioner) - } - -} - -private[graphx] -object MsgRDDFunctions { - implicit def rdd2PartitionRDDFunctions[T: ClassTag](rdd: RDD[MessageToPartition[T]]) = { - new MsgRDDFunctions(rdd) - } - - implicit def rdd2vertexMessageRDDFunctions[T: ClassTag](rdd: RDD[VertexBroadcastMsg[T]]) = { - new VertexBroadcastMsgRDDFunctions(rdd) - } -} - private[graphx] class VertexRDDFunctions[VD: ClassTag](self: RDD[(VertexId, VD)]) { def copartitionWithVertices(partitioner: Partitioner): RDD[(VertexId, VD)] = { - val rdd = new ShuffledRDD[VertexId, VD, VD, (VertexId, VD)](self, partitioner) + val rdd = new ShuffledRDD[VertexId, VD, VD](self, partitioner) // Set a custom serializer if the data is of int or double type. if (classTag[VD] == ClassTag.Int) { diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala index 3827ac8d0fd6a..b27485953f719 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala @@ -27,26 +27,13 @@ import org.apache.spark.util.collection.{BitSet, PrimitiveVector} import org.apache.spark.graphx._ import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap -/** - * A message from the edge partition `pid` to the vertex partition containing `vid` specifying that - * the edge partition references `vid` in the specified `position` (src, dst, or both). -*/ -private[graphx] -class RoutingTableMessage( - var vid: VertexId, - var pid: PartitionID, - var position: Byte) - extends Product2[VertexId, (PartitionID, Byte)] with Serializable { - override def _1 = vid - override def _2 = (pid, position) - override def canEqual(that: Any): Boolean = that.isInstanceOf[RoutingTableMessage] -} +import org.apache.spark.graphx.impl.RoutingTablePartition.RoutingTableMessage private[graphx] class RoutingTableMessageRDDFunctions(self: RDD[RoutingTableMessage]) { /** Copartition an `RDD[RoutingTableMessage]` with the vertex RDD with the given `partitioner`. */ def copartitionWithVertices(partitioner: Partitioner): RDD[RoutingTableMessage] = { - new ShuffledRDD[VertexId, (PartitionID, Byte), (PartitionID, Byte), RoutingTableMessage]( + new ShuffledRDD[VertexId, Int, Int]( self, partitioner).setSerializer(new RoutingTableMessageSerializer) } } @@ -62,6 +49,23 @@ object RoutingTableMessageRDDFunctions { private[graphx] object RoutingTablePartition { + /** + * A message from an edge partition to a vertex specifying the position in which the edge + * partition references the vertex (src, dst, or both). The edge partition is encoded in the lower + * 30 bytes of the Int, and the position is encoded in the upper 2 bytes of the Int. + */ + type RoutingTableMessage = (VertexId, Int) + + private def toMessage(vid: VertexId, pid: PartitionID, position: Byte): RoutingTableMessage = { + val positionUpper2 = position << 30 + val pidLower30 = pid & 0x3FFFFFFF + (vid, positionUpper2 | pidLower30) + } + + private def vidFromMessage(msg: RoutingTableMessage): VertexId = msg._1 + private def pidFromMessage(msg: RoutingTableMessage): PartitionID = msg._2 & 0x3FFFFFFF + private def positionFromMessage(msg: RoutingTableMessage): Byte = (msg._2 >> 30).toByte + val empty: RoutingTablePartition = new RoutingTablePartition(Array.empty) /** Generate a `RoutingTableMessage` for each vertex referenced in `edgePartition`. */ @@ -77,7 +81,9 @@ object RoutingTablePartition { map.changeValue(dstId, 0x2, (b: Byte) => (b | 0x2).toByte) } map.iterator.map { vidAndPosition => - new RoutingTableMessage(vidAndPosition._1, pid, vidAndPosition._2) + val vid = vidAndPosition._1 + val position = vidAndPosition._2 + toMessage(vid, pid, position) } } @@ -88,9 +94,12 @@ object RoutingTablePartition { val srcFlags = Array.fill(numEdgePartitions)(new PrimitiveVector[Boolean]) val dstFlags = Array.fill(numEdgePartitions)(new PrimitiveVector[Boolean]) for (msg <- iter) { - pid2vid(msg.pid) += msg.vid - srcFlags(msg.pid) += (msg.position & 0x1) != 0 - dstFlags(msg.pid) += (msg.position & 0x2) != 0 + val vid = vidFromMessage(msg) + val pid = pidFromMessage(msg) + val position = positionFromMessage(msg) + pid2vid(pid) += vid + srcFlags(pid) += (position & 0x1) != 0 + dstFlags(pid) += (position & 0x2) != 0 } new RoutingTablePartition(pid2vid.zipWithIndex.map { @@ -119,7 +128,7 @@ object RoutingTablePartition { */ private[graphx] class RoutingTablePartition( - private val routingTable: Array[(Array[VertexId], BitSet, BitSet)]) { + private val routingTable: Array[(Array[VertexId], BitSet, BitSet)]) extends Serializable { /** The maximum number of edge partitions this `RoutingTablePartition` is built to join with. */ val numEdgePartitions: Int = routingTable.size diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala index 033237f597216..3909efcdfc993 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala @@ -24,9 +24,11 @@ import java.nio.ByteBuffer import scala.reflect.ClassTag -import org.apache.spark.graphx._ import org.apache.spark.serializer._ +import org.apache.spark.graphx._ +import org.apache.spark.graphx.impl.RoutingTablePartition.RoutingTableMessage + private[graphx] class RoutingTableMessageSerializer extends Serializer with Serializable { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { @@ -35,10 +37,8 @@ class RoutingTableMessageSerializer extends Serializer with Serializable { new ShuffleSerializationStream(s) { def writeObject[T: ClassTag](t: T): SerializationStream = { val msg = t.asInstanceOf[RoutingTableMessage] - writeVarLong(msg.vid, optimizePositive = false) - writeUnsignedVarInt(msg.pid) - // TODO: Write only the bottom two bits of msg.position - s.write(msg.position) + writeVarLong(msg._1, optimizePositive = false) + writeInt(msg._2) this } } @@ -47,10 +47,8 @@ class RoutingTableMessageSerializer extends Serializer with Serializable { new ShuffleDeserializationStream(s) { override def readObject[T: ClassTag](): T = { val a = readVarLong(optimizePositive = false) - val b = readUnsignedVarInt() - val c = s.read() - if (c == -1) throw new EOFException - new RoutingTableMessage(a, b, c.toByte).asInstanceOf[T] + val b = readInt() + (a, b).asInstanceOf[T] } } } @@ -76,78 +74,6 @@ class VertexIdMsgSerializer extends Serializer with Serializable { } } -/** A special shuffle serializer for VertexBroadcastMessage[Int]. */ -private[graphx] -class IntVertexBroadcastMsgSerializer extends Serializer with Serializable { - override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { - - override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { - def writeObject[T: ClassTag](t: T) = { - val msg = t.asInstanceOf[VertexBroadcastMsg[Int]] - writeVarLong(msg.vid, optimizePositive = false) - writeInt(msg.data) - this - } - } - - override def deserializeStream(s: InputStream) = new ShuffleDeserializationStream(s) { - override def readObject[T: ClassTag](): T = { - val a = readVarLong(optimizePositive = false) - val b = readInt() - new VertexBroadcastMsg[Int](0, a, b).asInstanceOf[T] - } - } - } -} - -/** A special shuffle serializer for VertexBroadcastMessage[Long]. */ -private[graphx] -class LongVertexBroadcastMsgSerializer extends Serializer with Serializable { - override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { - - override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { - def writeObject[T: ClassTag](t: T) = { - val msg = t.asInstanceOf[VertexBroadcastMsg[Long]] - writeVarLong(msg.vid, optimizePositive = false) - writeLong(msg.data) - this - } - } - - override def deserializeStream(s: InputStream) = new ShuffleDeserializationStream(s) { - override def readObject[T: ClassTag](): T = { - val a = readVarLong(optimizePositive = false) - val b = readLong() - new VertexBroadcastMsg[Long](0, a, b).asInstanceOf[T] - } - } - } -} - -/** A special shuffle serializer for VertexBroadcastMessage[Double]. */ -private[graphx] -class DoubleVertexBroadcastMsgSerializer extends Serializer with Serializable { - override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { - - override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { - def writeObject[T: ClassTag](t: T) = { - val msg = t.asInstanceOf[VertexBroadcastMsg[Double]] - writeVarLong(msg.vid, optimizePositive = false) - writeDouble(msg.data) - this - } - } - - override def deserializeStream(s: InputStream) = new ShuffleDeserializationStream(s) { - def readObject[T: ClassTag](): T = { - val a = readVarLong(optimizePositive = false) - val b = readDouble() - new VertexBroadcastMsg[Double](0, a, b).asInstanceOf[T] - } - } - } -} - /** A special shuffle serializer for AggregationMessage[Int]. */ private[graphx] class IntAggMsgSerializer extends Serializer with Serializable { diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala index 34939b24440aa..5ad6390a56c4f 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBase.scala @@ -60,7 +60,8 @@ private[graphx] object VertexPartitionBase { * `VertexPartitionBaseOpsConstructor` typeclass (for example, * [[VertexPartition.VertexPartitionOpsConstructor]]). */ -private[graphx] abstract class VertexPartitionBase[@specialized(Long, Int, Double) VD: ClassTag] { +private[graphx] abstract class VertexPartitionBase[@specialized(Long, Int, Double) VD: ClassTag] + extends Serializable { def index: VertexIdToIndexMap def values: Array[VD] diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala index a4f769b294010..b40aa1b417a0f 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala @@ -35,7 +35,7 @@ import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap private[graphx] abstract class VertexPartitionBaseOps [VD: ClassTag, Self[X] <: VertexPartitionBase[X] : VertexPartitionBaseOpsConstructor] (self: Self[VD]) - extends Logging { + extends Serializable with Logging { def withIndex(index: VertexIdToIndexMap): Self[VD] def withValues[VD2: ClassTag](values: Array[VD2]): Self[VD2] diff --git a/graphx/src/main/scala/org/apache/spark/graphx/package.scala b/graphx/src/main/scala/org/apache/spark/graphx/package.scala index ff17edeaf8f16..6aab28ff05355 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/package.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/package.scala @@ -30,7 +30,7 @@ package object graphx { */ type VertexId = Long - /** Integer identifer of a graph partition. */ + /** Integer identifer of a graph partition. Must be less than 2^30. */ // TODO: Consider using Char. type PartitionID = Int diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala index 635514f09ece0..60149548ab852 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala @@ -100,8 +100,10 @@ object GraphGenerators { */ private def sampleLogNormal(mu: Double, sigma: Double, maxVal: Int): Int = { val rand = new Random() - val m = math.exp(mu + (sigma * sigma) / 2.0) - val s = math.sqrt((math.exp(sigma*sigma) - 1) * math.exp(2*mu + sigma*sigma)) + val sigmaSq = sigma * sigma + val m = math.exp(mu + sigmaSq / 2.0) + // expm1 is exp(m)-1 with better accuracy for tiny m + val s = math.sqrt(math.expm1(sigmaSq) * math.exp(2*mu + sigmaSq)) // Z ~ N(0, 1) var X: Double = maxVal diff --git a/graphx/src/test/scala/org/apache/spark/graphx/SerializerSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/SerializerSuite.scala index 91caa6b605a1e..864cb1fdf0022 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/SerializerSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/SerializerSuite.scala @@ -26,75 +26,11 @@ import org.scalatest.FunSuite import org.apache.spark._ import org.apache.spark.graphx.impl._ -import org.apache.spark.graphx.impl.MsgRDDFunctions._ import org.apache.spark.serializer.SerializationStream class SerializerSuite extends FunSuite with LocalSparkContext { - test("IntVertexBroadcastMsgSerializer") { - val outMsg = new VertexBroadcastMsg[Int](3, 4, 5) - val bout = new ByteArrayOutputStream - val outStrm = new IntVertexBroadcastMsgSerializer().newInstance().serializeStream(bout) - outStrm.writeObject(outMsg) - outStrm.writeObject(outMsg) - bout.flush() - val bin = new ByteArrayInputStream(bout.toByteArray) - val inStrm = new IntVertexBroadcastMsgSerializer().newInstance().deserializeStream(bin) - val inMsg1: VertexBroadcastMsg[Int] = inStrm.readObject() - val inMsg2: VertexBroadcastMsg[Int] = inStrm.readObject() - assert(outMsg.vid === inMsg1.vid) - assert(outMsg.vid === inMsg2.vid) - assert(outMsg.data === inMsg1.data) - assert(outMsg.data === inMsg2.data) - - intercept[EOFException] { - inStrm.readObject() - } - } - - test("LongVertexBroadcastMsgSerializer") { - val outMsg = new VertexBroadcastMsg[Long](3, 4, 5) - val bout = new ByteArrayOutputStream - val outStrm = new LongVertexBroadcastMsgSerializer().newInstance().serializeStream(bout) - outStrm.writeObject(outMsg) - outStrm.writeObject(outMsg) - bout.flush() - val bin = new ByteArrayInputStream(bout.toByteArray) - val inStrm = new LongVertexBroadcastMsgSerializer().newInstance().deserializeStream(bin) - val inMsg1: VertexBroadcastMsg[Long] = inStrm.readObject() - val inMsg2: VertexBroadcastMsg[Long] = inStrm.readObject() - assert(outMsg.vid === inMsg1.vid) - assert(outMsg.vid === inMsg2.vid) - assert(outMsg.data === inMsg1.data) - assert(outMsg.data === inMsg2.data) - - intercept[EOFException] { - inStrm.readObject() - } - } - - test("DoubleVertexBroadcastMsgSerializer") { - val outMsg = new VertexBroadcastMsg[Double](3, 4, 5.0) - val bout = new ByteArrayOutputStream - val outStrm = new DoubleVertexBroadcastMsgSerializer().newInstance().serializeStream(bout) - outStrm.writeObject(outMsg) - outStrm.writeObject(outMsg) - bout.flush() - val bin = new ByteArrayInputStream(bout.toByteArray) - val inStrm = new DoubleVertexBroadcastMsgSerializer().newInstance().deserializeStream(bin) - val inMsg1: VertexBroadcastMsg[Double] = inStrm.readObject() - val inMsg2: VertexBroadcastMsg[Double] = inStrm.readObject() - assert(outMsg.vid === inMsg1.vid) - assert(outMsg.vid === inMsg2.vid) - assert(outMsg.data === inMsg1.data) - assert(outMsg.data === inMsg2.data) - - intercept[EOFException] { - inStrm.readObject() - } - } - test("IntAggMsgSerializer") { val outMsg = (4: VertexId, 5) val bout = new ByteArrayOutputStream @@ -152,15 +88,6 @@ class SerializerSuite extends FunSuite with LocalSparkContext { } } - test("TestShuffleVertexBroadcastMsg") { - withSpark { sc => - val bmsgs = sc.parallelize(0 until 100, 10).map { pid => - new VertexBroadcastMsg[Int](pid, pid, pid) - } - bmsgs.partitionBy(new HashPartitioner(3)).collect() - } - } - test("variable long encoding") { def testVarLongEncoding(v: Long, optimizePositive: Boolean) { val bout = new ByteArrayOutputStream diff --git a/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala index 28fd112f2b124..9d00f76327e4c 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala @@ -23,6 +23,7 @@ import scala.util.Random import org.scalatest.FunSuite import org.apache.spark.SparkConf +import org.apache.spark.serializer.JavaSerializer import org.apache.spark.serializer.KryoSerializer import org.apache.spark.graphx._ @@ -124,18 +125,21 @@ class EdgePartitionSuite extends FunSuite { assert(ep.numActives == Some(2)) } - test("Kryo serialization") { + test("serialization") { val aList = List((0, 1, 0), (1, 0, 0), (1, 2, 0), (5, 4, 0), (5, 5, 0)) val a: EdgePartition[Int, Int] = makeEdgePartition(aList) - val conf = new SparkConf() + val javaSer = new JavaSerializer(new SparkConf()) + val kryoSer = new KryoSerializer(new SparkConf() .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - .set("spark.kryo.registrator", "org.apache.spark.graphx.GraphKryoRegistrator") - val s = new KryoSerializer(conf).newInstance() - val aSer: EdgePartition[Int, Int] = s.deserialize(s.serialize(a)) - assert(aSer.srcIds.toList === a.srcIds.toList) - assert(aSer.dstIds.toList === a.dstIds.toList) - assert(aSer.data.toList === a.data.toList) - assert(aSer.index != null) - assert(aSer.vertices.iterator.toSet === a.vertices.iterator.toSet) + .set("spark.kryo.registrator", "org.apache.spark.graphx.GraphKryoRegistrator")) + + for (ser <- List(javaSer, kryoSer); s = ser.newInstance()) { + val aSer: EdgePartition[Int, Int] = s.deserialize(s.serialize(a)) + assert(aSer.srcIds.toList === a.srcIds.toList) + assert(aSer.dstIds.toList === a.dstIds.toList) + assert(aSer.data.toList === a.data.toList) + assert(aSer.index != null) + assert(aSer.vertices.iterator.toSet === a.vertices.iterator.toSet) + } } } diff --git a/graphx/src/test/scala/org/apache/spark/graphx/impl/VertexPartitionSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/impl/VertexPartitionSuite.scala index 8bf1384d514c1..f9e771a900013 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/impl/VertexPartitionSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/impl/VertexPartitionSuite.scala @@ -17,9 +17,14 @@ package org.apache.spark.graphx.impl -import org.apache.spark.graphx._ import org.scalatest.FunSuite +import org.apache.spark.SparkConf +import org.apache.spark.serializer.JavaSerializer +import org.apache.spark.serializer.KryoSerializer + +import org.apache.spark.graphx._ + class VertexPartitionSuite extends FunSuite { test("isDefined, filter") { @@ -116,4 +121,17 @@ class VertexPartitionSuite extends FunSuite { assert(vp3.index.getPos(2) === -1) } + test("serialization") { + val verts = Set((0L, 1), (1L, 1), (2L, 1)) + val vp = VertexPartition(verts.iterator) + val javaSer = new JavaSerializer(new SparkConf()) + val kryoSer = new KryoSerializer(new SparkConf() + .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + .set("spark.kryo.registrator", "org.apache.spark.graphx.GraphKryoRegistrator")) + + for (ser <- List(javaSer, kryoSer); s = ser.newInstance()) { + val vpSer: VertexPartition[Int] = s.deserialize(s.serialize(vp)) + assert(vpSer.iterator.toSet === verts) + } + } } diff --git a/make-distribution.sh b/make-distribution.sh index 94b473bf91cd3..f7a6a9d838bb6 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -23,21 +23,6 @@ # The distribution contains fat (assembly) jars that include the Scala library, # so it is completely self contained. # It does not contain source or *.class files. -# -# Optional Arguments -# --tgz: Additionally creates spark-$VERSION-bin.tar.gz -# --hadoop VERSION: Builds against specified version of Hadoop. -# --with-yarn: Enables support for Hadoop YARN. -# --with-hive: Enable support for reading Hive tables. -# --name: A moniker for the release target. Defaults to the Hadoop verison. -# -# Recommended deploy/testing procedure (standalone mode): -# 1) Rsync / deploy the dist/ dir to one host -# 2) cd to deploy dir; ./sbin/start-master.sh -# 3) Verify master is up by visiting web page, ie http://master-ip:8080. Note the spark:// URL. -# 4) ./sbin/start-slave.sh 1 <> -# 5) ./bin/spark-shell --master spark://my-master-ip:7077 -# set -o pipefail set -e @@ -46,26 +31,35 @@ set -e FWDIR="$(cd `dirname $0`; pwd)" DISTDIR="$FWDIR/dist" -# Initialize defaults -SPARK_HADOOP_VERSION=1.0.4 -SPARK_YARN=false -SPARK_HIVE=false SPARK_TACHYON=false MAKE_TGZ=false NAME=none +function exit_with_usage { + echo "make-distribution.sh - tool for making binary distributions of Spark" + echo "" + echo "usage:" + echo "./make-distribution.sh [--name] [--tgz] [--with-tachyon] " + echo "See Spark's \"Building with Maven\" doc for correct Maven options." + echo "" + exit 1 +} + # Parse arguments while (( "$#" )); do case $1 in --hadoop) - SPARK_HADOOP_VERSION="$2" - shift + echo "Error: '--hadoop' is no longer supported:" + echo "Error: use Maven options -Phadoop.version and -Pyarn.version" + exit_with_usage ;; --with-yarn) - SPARK_YARN=true + echo "Error: '--with-yarn' is no longer supported, use Maven option -Pyarn" + exit_with_usage ;; --with-hive) - SPARK_HIVE=true + echo "Error: '--with-hive' is no longer supported, use Maven option -Phive" + exit_with_usage ;; --skip-java-test) SKIP_JAVA_TEST=true @@ -80,6 +74,12 @@ while (( "$#" )); do NAME="$2" shift ;; + --help) + exit_with_usage + ;; + *) + break + ;; esac shift done @@ -128,7 +128,7 @@ if [[ ! "$JAVA_VERSION" =~ "1.6" && -z "$SKIP_JAVA_TEST" ]]; then if [[ ! $REPLY =~ ^[Yy]$ ]]; then echo "Okay, exiting." exit 1 - fi + fi fi if [ "$NAME" == "none" ]; then @@ -143,14 +143,6 @@ else echo "Making distribution for Spark $VERSION in $DISTDIR..." fi -echo "Hadoop version set to $SPARK_HADOOP_VERSION" -echo "Release name set to $NAME" -if [ "$SPARK_YARN" == "true" ]; then - echo "YARN enabled" -else - echo "YARN disabled" -fi - if [ "$SPARK_TACHYON" == "true" ]; then echo "Tachyon Enabled" else @@ -158,37 +150,16 @@ else fi # Build uber fat JAR -cd $FWDIR +cd "$FWDIR" export MAVEN_OPTS="-Xmx2g -XX:MaxPermSize=512M -XX:ReservedCodeCacheSize=512m" -BUILD_COMMAND="mvn clean package" - -# Use special profiles for hadoop versions 0.23.x, 2.2.x, 2.3.x, 2.4.x -if [[ "$SPARK_HADOOP_VERSION" =~ ^0\.23\. ]]; then BUILD_COMMAND="$BUILD_COMMAND -Phadoop-0.23"; fi -if [[ "$SPARK_HADOOP_VERSION" =~ ^2\.2\. ]]; then BUILD_COMMAND="$BUILD_COMMAND -Phadoop-2.2"; fi -if [[ "$SPARK_HADOOP_VERSION" =~ ^2\.3\. ]]; then BUILD_COMMAND="$BUILD_COMMAND -Phadoop-2.3"; fi -if [[ "$SPARK_HADOOP_VERSION" =~ ^2\.4\. ]]; then BUILD_COMMAND="$BUILD_COMMAND -Phadoop-2.4"; fi -if [[ "$SPARK_HIVE" == "true" ]]; then BUILD_COMMAND="$BUILD_COMMAND -Phive"; fi -if [[ "$SPARK_YARN" == "true" ]]; then - # For hadoop versions 0.23.x to 2.1.x, use the yarn-alpha profile - if [[ "$SPARK_HADOOP_VERSION" =~ ^0\.2[3-9]\. ]] || - [[ "$SPARK_HADOOP_VERSION" =~ ^0\.[3-9][0-9]\. ]] || - [[ "$SPARK_HADOOP_VERSION" =~ ^1\.[0-9]\. ]] || - [[ "$SPARK_HADOOP_VERSION" =~ ^2\.[0-1]\. ]]; then - BUILD_COMMAND="$BUILD_COMMAND -Pyarn-alpha" - # For hadoop versions 2.2+, use the yarn profile - elif [[ "$SPARK_HADOOP_VERSION" =~ ^2.[2-9]. ]]; then - BUILD_COMMAND="$BUILD_COMMAND -Pyarn" - fi - BUILD_COMMAND="$BUILD_COMMAND -Dyarn.version=$SPARK_HADOOP_VERSION" -fi -BUILD_COMMAND="$BUILD_COMMAND -Dhadoop.version=$SPARK_HADOOP_VERSION" -BUILD_COMMAND="$BUILD_COMMAND -DskipTests" +BUILD_COMMAND="mvn clean package -DskipTests $@" # Actually build the jar echo -e "\nBuilding with..." echo -e "\$ $BUILD_COMMAND\n" + ${BUILD_COMMAND} # Make directories @@ -197,22 +168,22 @@ mkdir -p "$DISTDIR/lib" echo "Spark $VERSION$GITREVSTRING built for Hadoop $SPARK_HADOOP_VERSION" > "$DISTDIR/RELEASE" # Copy jars -cp $FWDIR/assembly/target/scala*/*assembly*hadoop*.jar "$DISTDIR/lib/" -cp $FWDIR/examples/target/scala*/spark-examples*.jar "$DISTDIR/lib/" +cp "$FWDIR"/assembly/target/scala*/*assembly*hadoop*.jar "$DISTDIR/lib/" +cp "$FWDIR"/examples/target/scala*/spark-examples*.jar "$DISTDIR/lib/" # Copy example sources (needed for python and SQL) mkdir -p "$DISTDIR/examples/src/main" -cp -r $FWDIR/examples/src/main "$DISTDIR/examples/src/" +cp -r "$FWDIR"/examples/src/main "$DISTDIR/examples/src/" if [ "$SPARK_HIVE" == "true" ]; then - cp $FWDIR/lib_managed/jars/datanucleus*.jar "$DISTDIR/lib/" + cp "$FWDIR"/lib_managed/jars/datanucleus*.jar "$DISTDIR/lib/" fi # Copy license and ASF files cp "$FWDIR/LICENSE" "$DISTDIR" cp "$FWDIR/NOTICE" "$DISTDIR" -if [ -e $FWDIR/CHANGES.txt ]; then +if [ -e "$FWDIR"/CHANGES.txt ]; then cp "$FWDIR/CHANGES.txt" "$DISTDIR" fi @@ -228,7 +199,7 @@ cp -r "$FWDIR/ec2" "$DISTDIR" # Download and copy in tachyon, if requested if [ "$SPARK_TACHYON" == "true" ]; then - TACHYON_VERSION="0.4.1" + TACHYON_VERSION="0.5.0" TACHYON_URL="https://github.com/amplab/tachyon/releases/download/v${TACHYON_VERSION}/tachyon-${TACHYON_VERSION}-bin.tar.gz" TMPD=`mktemp -d 2>/dev/null || mktemp -d -t 'disttmp'` diff --git a/mllib/pom.xml b/mllib/pom.xml index b622f96dd7901..fc1ecfbea708f 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -27,6 +27,9 @@ org.apache.spark spark-mllib_2.10 + + mllib + jar Spark Project ML Library http://spark.apache.org/ @@ -37,6 +40,11 @@ spark-core_${scala.binary.version} ${project.version} + + org.apache.spark + spark-streaming_${scala.binary.version} + ${project.version} + org.eclipse.jetty jetty-server @@ -49,7 +57,7 @@ org.scalanlp breeze_${scala.binary.version} - 0.7 + 0.9 @@ -57,6 +65,10 @@ junit junit + + org.apache.commons + commons-math3 + @@ -69,12 +81,30 @@ scalacheck_${scala.binary.version} test + + junit + junit + test + com.novocode junit-interface test + + + netlib-lgpl + + + com.github.fommil.netlib + all + 1.1.2 + pom + + + + target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index c44173793b39a..4343124f102a0 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -19,15 +19,26 @@ package org.apache.spark.mllib.api.python import java.nio.{ByteBuffer, ByteOrder} +import scala.collection.JavaConverters._ + import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.api.java.{JavaSparkContext, JavaRDD} +import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} import org.apache.spark.mllib.classification._ import org.apache.spark.mllib.clustering._ -import org.apache.spark.mllib.linalg.{SparseVector, Vector, Vectors} +import org.apache.spark.mllib.optimization._ +import org.apache.spark.mllib.linalg.{Matrix, SparseVector, Vector, Vectors} +import org.apache.spark.mllib.random.{RandomRDDs => RG} import org.apache.spark.mllib.recommendation._ import org.apache.spark.mllib.regression._ +import org.apache.spark.mllib.tree.configuration.{Algo, Strategy} +import org.apache.spark.mllib.tree.DecisionTree +import org.apache.spark.mllib.tree.impurity._ +import org.apache.spark.mllib.tree.model.DecisionTreeModel +import org.apache.spark.mllib.stat.{MultivariateStatisticalSummary, Statistics} +import org.apache.spark.mllib.stat.correlation.CorrelationNames import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD +import org.apache.spark.util.Utils /** * :: DeveloperApi :: @@ -37,6 +48,463 @@ import org.apache.spark.rdd.RDD */ @DeveloperApi class PythonMLLibAPI extends Serializable { + + + /** + * Loads and serializes labeled points saved with `RDD#saveAsTextFile`. + * @param jsc Java SparkContext + * @param path file or directory path in any Hadoop-supported file system URI + * @param minPartitions min number of partitions + * @return serialized labeled points stored in a JavaRDD of byte array + */ + def loadLabeledPoints( + jsc: JavaSparkContext, + path: String, + minPartitions: Int): JavaRDD[Array[Byte]] = + MLUtils.loadLabeledPoints(jsc.sc, path, minPartitions).map(SerDe.serializeLabeledPoint) + + private def trainRegressionModel( + trainFunc: (RDD[LabeledPoint], Vector) => GeneralizedLinearModel, + dataBytesJRDD: JavaRDD[Array[Byte]], + initialWeightsBA: Array[Byte]): java.util.LinkedList[java.lang.Object] = { + val data = dataBytesJRDD.rdd.map(SerDe.deserializeLabeledPoint) + val initialWeights = SerDe.deserializeDoubleVector(initialWeightsBA) + val model = trainFunc(data, initialWeights) + val ret = new java.util.LinkedList[java.lang.Object]() + ret.add(SerDe.serializeDoubleVector(model.weights)) + ret.add(model.intercept: java.lang.Double) + ret + } + + /** + * Java stub for Python mllib LinearRegressionWithSGD.train() + */ + def trainLinearRegressionModelWithSGD( + dataBytesJRDD: JavaRDD[Array[Byte]], + numIterations: Int, + stepSize: Double, + miniBatchFraction: Double, + initialWeightsBA: Array[Byte], + regParam: Double, + regType: String, + intercept: Boolean): java.util.List[java.lang.Object] = { + val lrAlg = new LinearRegressionWithSGD() + lrAlg.setIntercept(intercept) + lrAlg.optimizer + .setNumIterations(numIterations) + .setRegParam(regParam) + .setStepSize(stepSize) + .setMiniBatchFraction(miniBatchFraction) + if (regType == "l2") { + lrAlg.optimizer.setUpdater(new SquaredL2Updater) + } else if (regType == "l1") { + lrAlg.optimizer.setUpdater(new L1Updater) + } else if (regType != "none") { + throw new java.lang.IllegalArgumentException("Invalid value for 'regType' parameter." + + " Can only be initialized using the following string values: [l1, l2, none].") + } + trainRegressionModel( + (data, initialWeights) => + lrAlg.run(data, initialWeights), + dataBytesJRDD, + initialWeightsBA) + } + + /** + * Java stub for Python mllib LassoWithSGD.train() + */ + def trainLassoModelWithSGD( + dataBytesJRDD: JavaRDD[Array[Byte]], + numIterations: Int, + stepSize: Double, + regParam: Double, + miniBatchFraction: Double, + initialWeightsBA: Array[Byte]): java.util.List[java.lang.Object] = { + trainRegressionModel( + (data, initialWeights) => + LassoWithSGD.train( + data, + numIterations, + stepSize, + regParam, + miniBatchFraction, + initialWeights), + dataBytesJRDD, + initialWeightsBA) + } + + /** + * Java stub for Python mllib RidgeRegressionWithSGD.train() + */ + def trainRidgeModelWithSGD( + dataBytesJRDD: JavaRDD[Array[Byte]], + numIterations: Int, + stepSize: Double, + regParam: Double, + miniBatchFraction: Double, + initialWeightsBA: Array[Byte]): java.util.List[java.lang.Object] = { + trainRegressionModel( + (data, initialWeights) => + RidgeRegressionWithSGD.train( + data, + numIterations, + stepSize, + regParam, + miniBatchFraction, + initialWeights), + dataBytesJRDD, + initialWeightsBA) + } + + /** + * Java stub for Python mllib SVMWithSGD.train() + */ + def trainSVMModelWithSGD( + dataBytesJRDD: JavaRDD[Array[Byte]], + numIterations: Int, + stepSize: Double, + regParam: Double, + miniBatchFraction: Double, + initialWeightsBA: Array[Byte], + regType: String, + intercept: Boolean): java.util.List[java.lang.Object] = { + val SVMAlg = new SVMWithSGD() + SVMAlg.setIntercept(intercept) + SVMAlg.optimizer + .setNumIterations(numIterations) + .setRegParam(regParam) + .setStepSize(stepSize) + .setMiniBatchFraction(miniBatchFraction) + if (regType == "l2") { + SVMAlg.optimizer.setUpdater(new SquaredL2Updater) + } else if (regType == "l1") { + SVMAlg.optimizer.setUpdater(new L1Updater) + } else if (regType != "none") { + throw new java.lang.IllegalArgumentException("Invalid value for 'regType' parameter." + + " Can only be initialized using the following string values: [l1, l2, none].") + } + trainRegressionModel( + (data, initialWeights) => + SVMAlg.run(data, initialWeights), + dataBytesJRDD, + initialWeightsBA) + } + + /** + * Java stub for Python mllib LogisticRegressionWithSGD.train() + */ + def trainLogisticRegressionModelWithSGD( + dataBytesJRDD: JavaRDD[Array[Byte]], + numIterations: Int, + stepSize: Double, + miniBatchFraction: Double, + initialWeightsBA: Array[Byte], + regParam: Double, + regType: String, + intercept: Boolean): java.util.List[java.lang.Object] = { + val LogRegAlg = new LogisticRegressionWithSGD() + LogRegAlg.setIntercept(intercept) + LogRegAlg.optimizer + .setNumIterations(numIterations) + .setRegParam(regParam) + .setStepSize(stepSize) + .setMiniBatchFraction(miniBatchFraction) + if (regType == "l2") { + LogRegAlg.optimizer.setUpdater(new SquaredL2Updater) + } else if (regType == "l1") { + LogRegAlg.optimizer.setUpdater(new L1Updater) + } else if (regType != "none") { + throw new java.lang.IllegalArgumentException("Invalid value for 'regType' parameter." + + " Can only be initialized using the following string values: [l1, l2, none].") + } + trainRegressionModel( + (data, initialWeights) => + LogRegAlg.run(data, initialWeights), + dataBytesJRDD, + initialWeightsBA) + } + + /** + * Java stub for NaiveBayes.train() + */ + def trainNaiveBayes( + dataBytesJRDD: JavaRDD[Array[Byte]], + lambda: Double): java.util.List[java.lang.Object] = { + val data = dataBytesJRDD.rdd.map(SerDe.deserializeLabeledPoint) + val model = NaiveBayes.train(data, lambda) + val ret = new java.util.LinkedList[java.lang.Object]() + ret.add(SerDe.serializeDoubleVector(Vectors.dense(model.labels))) + ret.add(SerDe.serializeDoubleVector(Vectors.dense(model.pi))) + ret.add(SerDe.serializeDoubleMatrix(model.theta)) + ret + } + + /** + * Java stub for Python mllib KMeans.train() + */ + def trainKMeansModel( + dataBytesJRDD: JavaRDD[Array[Byte]], + k: Int, + maxIterations: Int, + runs: Int, + initializationMode: String): java.util.List[java.lang.Object] = { + val data = dataBytesJRDD.rdd.map(bytes => SerDe.deserializeDoubleVector(bytes)) + val model = KMeans.train(data, k, maxIterations, runs, initializationMode) + val ret = new java.util.LinkedList[java.lang.Object]() + ret.add(SerDe.serializeDoubleMatrix(model.clusterCenters.map(_.toArray))) + ret + } + + /** + * Java stub for Python mllib ALS.train(). This stub returns a handle + * to the Java object instead of the content of the Java object. Extra care + * needs to be taken in the Python code to ensure it gets freed on exit; see + * the Py4J documentation. + */ + def trainALSModel( + ratingsBytesJRDD: JavaRDD[Array[Byte]], + rank: Int, + iterations: Int, + lambda: Double, + blocks: Int): MatrixFactorizationModel = { + val ratings = ratingsBytesJRDD.rdd.map(SerDe.unpackRating) + ALS.train(ratings, rank, iterations, lambda, blocks) + } + + /** + * Java stub for Python mllib ALS.trainImplicit(). This stub returns a + * handle to the Java object instead of the content of the Java object. + * Extra care needs to be taken in the Python code to ensure it gets freed on + * exit; see the Py4J documentation. + */ + def trainImplicitALSModel( + ratingsBytesJRDD: JavaRDD[Array[Byte]], + rank: Int, + iterations: Int, + lambda: Double, + blocks: Int, + alpha: Double): MatrixFactorizationModel = { + val ratings = ratingsBytesJRDD.rdd.map(SerDe.unpackRating) + ALS.trainImplicit(ratings, rank, iterations, lambda, blocks, alpha) + } + + /** + * Java stub for Python mllib DecisionTree.train(). + * This stub returns a handle to the Java object instead of the content of the Java object. + * Extra care needs to be taken in the Python code to ensure it gets freed on exit; + * see the Py4J documentation. + * @param dataBytesJRDD Training data + * @param categoricalFeaturesInfoJMap Categorical features info, as Java map + */ + def trainDecisionTreeModel( + dataBytesJRDD: JavaRDD[Array[Byte]], + algoStr: String, + numClasses: Int, + categoricalFeaturesInfoJMap: java.util.Map[Int, Int], + impurityStr: String, + maxDepth: Int, + maxBins: Int): DecisionTreeModel = { + + val data = dataBytesJRDD.rdd.map(SerDe.deserializeLabeledPoint) + + val algo = Algo.fromString(algoStr) + val impurity = Impurities.fromString(impurityStr) + + val strategy = new Strategy( + algo = algo, + impurity = impurity, + maxDepth = maxDepth, + numClassesForClassification = numClasses, + maxBins = maxBins, + categoricalFeaturesInfo = categoricalFeaturesInfoJMap.asScala.toMap) + + DecisionTree.train(data, strategy) + } + + /** + * Predict the label of the given data point. + * This is a Java stub for python DecisionTreeModel.predict() + * + * @param featuresBytes Serialized feature vector for data point + * @return predicted label + */ + def predictDecisionTreeModel( + model: DecisionTreeModel, + featuresBytes: Array[Byte]): Double = { + val features: Vector = SerDe.deserializeDoubleVector(featuresBytes) + model.predict(features) + } + + /** + * Predict the labels of the given data points. + * This is a Java stub for python DecisionTreeModel.predict() + * + * @param dataJRDD A JavaRDD with serialized feature vectors + * @return JavaRDD of serialized predictions + */ + def predictDecisionTreeModel( + model: DecisionTreeModel, + dataJRDD: JavaRDD[Array[Byte]]): JavaRDD[Array[Byte]] = { + val data = dataJRDD.rdd.map(xBytes => SerDe.deserializeDoubleVector(xBytes)) + model.predict(data).map(SerDe.serializeDouble) + } + + /** + * Java stub for mllib Statistics.colStats(X: RDD[Vector]). + * TODO figure out return type. + */ + def colStats(X: JavaRDD[Array[Byte]]): MultivariateStatisticalSummarySerialized = { + val cStats = Statistics.colStats(X.rdd.map(SerDe.deserializeDoubleVector(_))) + new MultivariateStatisticalSummarySerialized(cStats) + } + + /** + * Java stub for mllib Statistics.corr(X: RDD[Vector], method: String). + * Returns the correlation matrix serialized into a byte array understood by deserializers in + * pyspark. + */ + def corr(X: JavaRDD[Array[Byte]], method: String): Array[Byte] = { + val inputMatrix = X.rdd.map(SerDe.deserializeDoubleVector(_)) + val result = Statistics.corr(inputMatrix, getCorrNameOrDefault(method)) + SerDe.serializeDoubleMatrix(SerDe.to2dArray(result)) + } + + /** + * Java stub for mllib Statistics.corr(x: RDD[Double], y: RDD[Double], method: String). + */ + def corr(x: JavaRDD[Array[Byte]], y: JavaRDD[Array[Byte]], method: String): Double = { + val xDeser = x.rdd.map(SerDe.deserializeDouble(_)) + val yDeser = y.rdd.map(SerDe.deserializeDouble(_)) + Statistics.corr(xDeser, yDeser, getCorrNameOrDefault(method)) + } + + // used by the corr methods to retrieve the name of the correlation method passed in via pyspark + private def getCorrNameOrDefault(method: String) = { + if (method == null) CorrelationNames.defaultCorrName else method + } + + // Used by the *RDD methods to get default seed if not passed in from pyspark + private def getSeedOrDefault(seed: java.lang.Long): Long = { + if (seed == null) Utils.random.nextLong else seed + } + + // Used by *RDD methods to get default numPartitions if not passed in from pyspark + private def getNumPartitionsOrDefault(numPartitions: java.lang.Integer, + jsc: JavaSparkContext): Int = { + if (numPartitions == null) { + jsc.sc.defaultParallelism + } else { + numPartitions + } + } + + // Note: for the following methods, numPartitions and seed are boxed to allow nulls to be passed + // in for either argument from pyspark + + /** + * Java stub for Python mllib RandomRDDGenerators.uniformRDD() + */ + def uniformRDD(jsc: JavaSparkContext, + size: Long, + numPartitions: java.lang.Integer, + seed: java.lang.Long): JavaRDD[Array[Byte]] = { + val parts = getNumPartitionsOrDefault(numPartitions, jsc) + val s = getSeedOrDefault(seed) + RG.uniformRDD(jsc.sc, size, parts, s).map(SerDe.serializeDouble) + } + + /** + * Java stub for Python mllib RandomRDDGenerators.normalRDD() + */ + def normalRDD(jsc: JavaSparkContext, + size: Long, + numPartitions: java.lang.Integer, + seed: java.lang.Long): JavaRDD[Array[Byte]] = { + val parts = getNumPartitionsOrDefault(numPartitions, jsc) + val s = getSeedOrDefault(seed) + RG.normalRDD(jsc.sc, size, parts, s).map(SerDe.serializeDouble) + } + + /** + * Java stub for Python mllib RandomRDDGenerators.poissonRDD() + */ + def poissonRDD(jsc: JavaSparkContext, + mean: Double, + size: Long, + numPartitions: java.lang.Integer, + seed: java.lang.Long): JavaRDD[Array[Byte]] = { + val parts = getNumPartitionsOrDefault(numPartitions, jsc) + val s = getSeedOrDefault(seed) + RG.poissonRDD(jsc.sc, mean, size, parts, s).map(SerDe.serializeDouble) + } + + /** + * Java stub for Python mllib RandomRDDGenerators.uniformVectorRDD() + */ + def uniformVectorRDD(jsc: JavaSparkContext, + numRows: Long, + numCols: Int, + numPartitions: java.lang.Integer, + seed: java.lang.Long): JavaRDD[Array[Byte]] = { + val parts = getNumPartitionsOrDefault(numPartitions, jsc) + val s = getSeedOrDefault(seed) + RG.uniformVectorRDD(jsc.sc, numRows, numCols, parts, s).map(SerDe.serializeDoubleVector) + } + + /** + * Java stub for Python mllib RandomRDDGenerators.normalVectorRDD() + */ + def normalVectorRDD(jsc: JavaSparkContext, + numRows: Long, + numCols: Int, + numPartitions: java.lang.Integer, + seed: java.lang.Long): JavaRDD[Array[Byte]] = { + val parts = getNumPartitionsOrDefault(numPartitions, jsc) + val s = getSeedOrDefault(seed) + RG.normalVectorRDD(jsc.sc, numRows, numCols, parts, s).map(SerDe.serializeDoubleVector) + } + + /** + * Java stub for Python mllib RandomRDDGenerators.poissonVectorRDD() + */ + def poissonVectorRDD(jsc: JavaSparkContext, + mean: Double, + numRows: Long, + numCols: Int, + numPartitions: java.lang.Integer, + seed: java.lang.Long): JavaRDD[Array[Byte]] = { + val parts = getNumPartitionsOrDefault(numPartitions, jsc) + val s = getSeedOrDefault(seed) + RG.poissonVectorRDD(jsc.sc, mean, numRows, numCols, parts, s).map(SerDe.serializeDoubleVector) + } + +} + +/** + * :: DeveloperApi :: + * MultivariateStatisticalSummary with Vector fields serialized. + */ +@DeveloperApi +class MultivariateStatisticalSummarySerialized(val summary: MultivariateStatisticalSummary) + extends Serializable { + + def mean: Array[Byte] = SerDe.serializeDoubleVector(summary.mean) + + def variance: Array[Byte] = SerDe.serializeDoubleVector(summary.variance) + + def count: Long = summary.count + + def numNonzeros: Array[Byte] = SerDe.serializeDoubleVector(summary.numNonzeros) + + def max: Array[Byte] = SerDe.serializeDoubleVector(summary.max) + + def min: Array[Byte] = SerDe.serializeDoubleVector(summary.min) +} + +/** + * SerDe utility functions for PythonMLLibAPI. + */ +private[spark] object SerDe extends Serializable { private val DENSE_VECTOR_MAGIC: Byte = 1 private val SPARSE_VECTOR_MAGIC: Byte = 2 private val DENSE_MATRIX_MAGIC: Byte = 3 @@ -54,7 +522,14 @@ class PythonMLLibAPI extends Serializable { } } - private def deserializeDenseVector(bytes: Array[Byte], offset: Int = 0): Vector = { + private[python] def deserializeDouble(bytes: Array[Byte], offset: Int = 0): Double = { + require(bytes.length - offset == 8, "Wrong size byte array for Double") + val bb = ByteBuffer.wrap(bytes, offset, bytes.length - offset) + bb.order(ByteOrder.nativeOrder()) + bb.getDouble + } + + private[python] def deserializeDenseVector(bytes: Array[Byte], offset: Int = 0): Vector = { val packetLength = bytes.length - offset require(packetLength >= 5, "Byte array too short") val bb = ByteBuffer.wrap(bytes, offset, bytes.length - offset) @@ -69,7 +544,7 @@ class PythonMLLibAPI extends Serializable { Vectors.dense(ans) } - private def deserializeSparseVector(bytes: Array[Byte], offset: Int = 0): Vector = { + private[python] def deserializeSparseVector(bytes: Array[Byte], offset: Int = 0): Vector = { val packetLength = bytes.length - offset require(packetLength >= 9, "Byte array too short") val bb = ByteBuffer.wrap(bytes, offset, bytes.length - offset) @@ -89,7 +564,23 @@ class PythonMLLibAPI extends Serializable { Vectors.sparse(size, indices, values) } - private def serializeDenseVector(doubles: Array[Double]): Array[Byte] = { + /** + * Returns an 8-byte array for the input Double. + * + * Note: we currently do not use a magic byte for double for storage efficiency. + * This should be reconsidered when we add Ser/De for other 8-byte types (e.g. Long), for safety. + * The corresponding deserializer, deserializeDouble, needs to be modified as well if the + * serialization scheme changes. + */ + private[python] def serializeDouble(double: Double): Array[Byte] = { + val bytes = new Array[Byte](8) + val bb = ByteBuffer.wrap(bytes) + bb.order(ByteOrder.nativeOrder()) + bb.putDouble(double) + bytes + } + + private[python] def serializeDenseVector(doubles: Array[Double]): Array[Byte] = { val len = doubles.length val bytes = new Array[Byte](5 + 8 * len) val bb = ByteBuffer.wrap(bytes) @@ -101,7 +592,7 @@ class PythonMLLibAPI extends Serializable { bytes } - private def serializeSparseVector(vector: SparseVector): Array[Byte] = { + private[python] def serializeSparseVector(vector: SparseVector): Array[Byte] = { val nonZeros = vector.indices.length val bytes = new Array[Byte](9 + 12 * nonZeros) val bb = ByteBuffer.wrap(bytes) @@ -124,7 +615,7 @@ class PythonMLLibAPI extends Serializable { serializeDenseVector(vector.toArray) } - private def deserializeDoubleMatrix(bytes: Array[Byte]): Array[Array[Double]] = { + private[python] def deserializeDoubleMatrix(bytes: Array[Byte]): Array[Array[Double]] = { val packetLength = bytes.length if (packetLength < 9) { throw new IllegalArgumentException("Byte array too short.") @@ -149,7 +640,7 @@ class PythonMLLibAPI extends Serializable { ans } - private def serializeDoubleMatrix(doubles: Array[Array[Double]]): Array[Byte] = { + private[python] def serializeDoubleMatrix(doubles: Array[Array[Double]]): Array[Byte] = { val rows = doubles.length var cols = 0 if (rows > 0) { @@ -191,176 +682,15 @@ class PythonMLLibAPI extends Serializable { LabeledPoint(label, deserializeDoubleVector(bytes, 9)) } - /** - * Loads and serializes labeled points saved with `RDD#saveAsTextFile`. - * @param jsc Java SparkContext - * @param path file or directory path in any Hadoop-supported file system URI - * @param minPartitions min number of partitions - * @return serialized labeled points stored in a JavaRDD of byte array - */ - def loadLabeledPoints( - jsc: JavaSparkContext, - path: String, - minPartitions: Int): JavaRDD[Array[Byte]] = - MLUtils.loadLabeledPoints(jsc.sc, path, minPartitions).map(serializeLabeledPoint).toJavaRDD() - - private def trainRegressionModel( - trainFunc: (RDD[LabeledPoint], Vector) => GeneralizedLinearModel, - dataBytesJRDD: JavaRDD[Array[Byte]], - initialWeightsBA: Array[Byte]): java.util.LinkedList[java.lang.Object] = { - val data = dataBytesJRDD.rdd.map(deserializeLabeledPoint) - val initialWeights = deserializeDoubleVector(initialWeightsBA) - val model = trainFunc(data, initialWeights) - val ret = new java.util.LinkedList[java.lang.Object]() - ret.add(serializeDoubleVector(model.weights)) - ret.add(model.intercept: java.lang.Double) - ret - } - - /** - * Java stub for Python mllib LinearRegressionWithSGD.train() - */ - def trainLinearRegressionModelWithSGD( - dataBytesJRDD: JavaRDD[Array[Byte]], - numIterations: Int, - stepSize: Double, - miniBatchFraction: Double, - initialWeightsBA: Array[Byte]): java.util.List[java.lang.Object] = { - trainRegressionModel( - (data, initialWeights) => - LinearRegressionWithSGD.train( - data, - numIterations, - stepSize, - miniBatchFraction, - initialWeights), - dataBytesJRDD, - initialWeightsBA) - } - - /** - * Java stub for Python mllib LassoWithSGD.train() - */ - def trainLassoModelWithSGD( - dataBytesJRDD: JavaRDD[Array[Byte]], - numIterations: Int, - stepSize: Double, - regParam: Double, - miniBatchFraction: Double, - initialWeightsBA: Array[Byte]): java.util.List[java.lang.Object] = { - trainRegressionModel( - (data, initialWeights) => - LassoWithSGD.train( - data, - numIterations, - stepSize, - regParam, - miniBatchFraction, - initialWeights), - dataBytesJRDD, - initialWeightsBA) - } - - /** - * Java stub for Python mllib RidgeRegressionWithSGD.train() - */ - def trainRidgeModelWithSGD( - dataBytesJRDD: JavaRDD[Array[Byte]], - numIterations: Int, - stepSize: Double, - regParam: Double, - miniBatchFraction: Double, - initialWeightsBA: Array[Byte]): java.util.List[java.lang.Object] = { - trainRegressionModel( - (data, initialWeights) => - RidgeRegressionWithSGD.train( - data, - numIterations, - stepSize, - regParam, - miniBatchFraction, - initialWeights), - dataBytesJRDD, - initialWeightsBA) - } - - /** - * Java stub for Python mllib SVMWithSGD.train() - */ - def trainSVMModelWithSGD( - dataBytesJRDD: JavaRDD[Array[Byte]], - numIterations: Int, - stepSize: Double, - regParam: Double, - miniBatchFraction: Double, - initialWeightsBA: Array[Byte]): java.util.List[java.lang.Object] = { - trainRegressionModel( - (data, initialWeights) => - SVMWithSGD.train( - data, - numIterations, - stepSize, - regParam, - miniBatchFraction, - initialWeights), - dataBytesJRDD, - initialWeightsBA) - } - - /** - * Java stub for Python mllib LogisticRegressionWithSGD.train() - */ - def trainLogisticRegressionModelWithSGD( - dataBytesJRDD: JavaRDD[Array[Byte]], - numIterations: Int, - stepSize: Double, - miniBatchFraction: Double, - initialWeightsBA: Array[Byte]): java.util.List[java.lang.Object] = { - trainRegressionModel( - (data, initialWeights) => - LogisticRegressionWithSGD.train( - data, - numIterations, - stepSize, - miniBatchFraction, - initialWeights), - dataBytesJRDD, - initialWeightsBA) - } - - /** - * Java stub for NaiveBayes.train() - */ - def trainNaiveBayes( - dataBytesJRDD: JavaRDD[Array[Byte]], - lambda: Double): java.util.List[java.lang.Object] = { - val data = dataBytesJRDD.rdd.map(deserializeLabeledPoint) - val model = NaiveBayes.train(data, lambda) - val ret = new java.util.LinkedList[java.lang.Object]() - ret.add(serializeDoubleVector(Vectors.dense(model.labels))) - ret.add(serializeDoubleVector(Vectors.dense(model.pi))) - ret.add(serializeDoubleMatrix(model.theta)) - ret + // Reformat a Matrix into Array[Array[Double]] for serialization + private[python] def to2dArray(matrix: Matrix): Array[Array[Double]] = { + val values = matrix.toArray + Array.tabulate(matrix.numRows, matrix.numCols)((i, j) => values(i + j * matrix.numRows)) } - /** - * Java stub for Python mllib KMeans.train() - */ - def trainKMeansModel( - dataBytesJRDD: JavaRDD[Array[Byte]], - k: Int, - maxIterations: Int, - runs: Int, - initializationMode: String): java.util.List[java.lang.Object] = { - val data = dataBytesJRDD.rdd.map(bytes => deserializeDoubleVector(bytes)) - val model = KMeans.train(data, k, maxIterations, runs, initializationMode) - val ret = new java.util.LinkedList[java.lang.Object]() - ret.add(serializeDoubleMatrix(model.clusterCenters.map(_.toArray))) - ret - } /** Unpack a Rating object from an array of bytes */ - private def unpackRating(ratingBytes: Array[Byte]): Rating = { + private[python] def unpackRating(ratingBytes: Array[Byte]): Rating = { val bb = ByteBuffer.wrap(ratingBytes) bb.order(ByteOrder.nativeOrder()) val user = bb.getInt() @@ -370,7 +700,7 @@ class PythonMLLibAPI extends Serializable { } /** Unpack a tuple of Ints from an array of bytes */ - private[spark] def unpackTuple(tupleBytes: Array[Byte]): (Int, Int) = { + def unpackTuple(tupleBytes: Array[Byte]): (Int, Int) = { val bb = ByteBuffer.wrap(tupleBytes) bb.order(ByteOrder.nativeOrder()) val v1 = bb.getInt() @@ -379,13 +709,13 @@ class PythonMLLibAPI extends Serializable { } /** - * Serialize a Rating object into an array of bytes. - * It can be deserialized using RatingDeserializer(). - * - * @param rate the Rating object to serialize - * @return - */ - private[spark] def serializeRating(rate: Rating): Array[Byte] = { + * Serialize a Rating object into an array of bytes. + * It can be deserialized using RatingDeserializer(). + * + * @param rate the Rating object to serialize + * @return + */ + def serializeRating(rate: Rating): Array[Byte] = { val len = 3 val bytes = new Array[Byte](4 + 8 * len) val bb = ByteBuffer.wrap(bytes) @@ -397,37 +727,4 @@ class PythonMLLibAPI extends Serializable { db.put(rate.rating) bytes } - - /** - * Java stub for Python mllib ALS.train(). This stub returns a handle - * to the Java object instead of the content of the Java object. Extra care - * needs to be taken in the Python code to ensure it gets freed on exit; see - * the Py4J documentation. - */ - def trainALSModel( - ratingsBytesJRDD: JavaRDD[Array[Byte]], - rank: Int, - iterations: Int, - lambda: Double, - blocks: Int): MatrixFactorizationModel = { - val ratings = ratingsBytesJRDD.rdd.map(unpackRating) - ALS.train(ratings, rank, iterations, lambda, blocks) - } - - /** - * Java stub for Python mllib ALS.trainImplicit(). This stub returns a - * handle to the Java object instead of the content of the Java object. - * Extra care needs to be taken in the Python code to ensure it gets freed on - * exit; see the Py4J documentation. - */ - def trainImplicitALSModel( - ratingsBytesJRDD: JavaRDD[Array[Byte]], - rank: Int, - iterations: Int, - lambda: Double, - blocks: Int, - alpha: Double): MatrixFactorizationModel = { - val ratings = ratingsBytesJRDD.rdd.map(unpackRating) - ALS.trainImplicit(ratings, rank, iterations, lambda, blocks, alpha) - } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala index 90aa8ac998ba9..486bdbfa9cb47 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala @@ -30,7 +30,7 @@ import org.apache.spark.rdd.RDD * @param weights Weights computed for every feature. * @param intercept Intercept computed for this model. */ -class LogisticRegressionModel private[mllib] ( +class LogisticRegressionModel ( override val weights: Vector, override val intercept: Double) extends GeneralizedLinearModel(weights, intercept) with ClassificationModel with Serializable { @@ -62,7 +62,7 @@ class LogisticRegressionModel private[mllib] ( override protected def predictPoint(dataMatrix: Vector, weightMatrix: Vector, intercept: Double) = { val margin = weightMatrix.toBreeze.dot(dataMatrix.toBreeze) + intercept - val score = 1.0/ (1.0 + math.exp(-margin)) + val score = 1.0 / (1.0 + math.exp(-margin)) threshold match { case Some(t) => if (score < t) 0.0 else 1.0 case None => score @@ -73,6 +73,8 @@ class LogisticRegressionModel private[mllib] ( /** * Train a classification model for Logistic Regression using Stochastic Gradient Descent. * NOTE: Labels used in Logistic Regression should be {0, 1} + * + * Using [[LogisticRegressionWithLBFGS]] is recommended over this. */ class LogisticRegressionWithSGD private ( private var stepSize: Double, @@ -101,7 +103,7 @@ class LogisticRegressionWithSGD private ( } /** - * Top-level methods for calling Logistic Regression. + * Top-level methods for calling Logistic Regression using Stochastic Gradient Descent. * NOTE: Labels used in Logistic Regression should be {0, 1} */ object LogisticRegressionWithSGD { @@ -188,3 +190,22 @@ object LogisticRegressionWithSGD { train(input, numIterations, 1.0, 1.0) } } + +/** + * Train a classification model for Logistic Regression using Limited-memory BFGS. + * Standard feature scaling and L2 regularization are used by default. + * NOTE: Labels used in Logistic Regression should be {0, 1} + */ +class LogisticRegressionWithLBFGS + extends GeneralizedLinearAlgorithm[LogisticRegressionModel] with Serializable { + + this.setFeatureScaling(true) + + override val optimizer = new LBFGS(new LogisticGradient, new SquaredL2Updater) + + override protected val validators = List(DataValidators.binaryLabelValidator) + + override protected def createModel(weights: Vector, intercept: Double) = { + new LogisticRegressionModel(weights, intercept) + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala index b6e0c4a80e27b..6c7be0a4f1dcb 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala @@ -54,7 +54,13 @@ class NaiveBayesModel private[mllib] ( } } - override def predict(testData: RDD[Vector]): RDD[Double] = testData.map(predict) + override def predict(testData: RDD[Vector]): RDD[Double] = { + val bcModel = testData.context.broadcast(this) + testData.mapPartitions { iter => + val model = bcModel.value + iter.map(model.predict) + } + } override def predict(testData: Vector): Double = { labels(brzArgmax(brzPi + brzTheta * testData.toBreeze)) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala index 316ecd713b715..80f8a1b2f1e84 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala @@ -30,7 +30,7 @@ import org.apache.spark.rdd.RDD * @param weights Weights computed for every feature. * @param intercept Intercept computed for this model. */ -class SVMModel private[mllib] ( +class SVMModel ( override val weights: Vector, override val intercept: Double) extends GeneralizedLinearModel(weights, intercept) with ClassificationModel with Serializable { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala index de22fbb6ffc10..fce8fe29f6e40 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala @@ -52,13 +52,13 @@ class KMeans private ( def this() = this(2, 20, 1, KMeans.K_MEANS_PARALLEL, 5, 1e-4) /** Set the number of clusters to create (k). Default: 2. */ - def setK(k: Int): KMeans = { + def setK(k: Int): this.type = { this.k = k this } /** Set maximum number of iterations to run. Default: 20. */ - def setMaxIterations(maxIterations: Int): KMeans = { + def setMaxIterations(maxIterations: Int): this.type = { this.maxIterations = maxIterations this } @@ -68,7 +68,7 @@ class KMeans private ( * initial cluster centers, or "k-means||" to use a parallel variant of k-means++ * (Bahmani et al., Scalable K-Means++, VLDB 2012). Default: k-means||. */ - def setInitializationMode(initializationMode: String): KMeans = { + def setInitializationMode(initializationMode: String): this.type = { if (initializationMode != KMeans.RANDOM && initializationMode != KMeans.K_MEANS_PARALLEL) { throw new IllegalArgumentException("Invalid initialization mode: " + initializationMode) } @@ -83,7 +83,7 @@ class KMeans private ( * return the best clustering found over any run. Default: 1. */ @Experimental - def setRuns(runs: Int): KMeans = { + def setRuns(runs: Int): this.type = { if (runs <= 0) { throw new IllegalArgumentException("Number of runs must be positive") } @@ -95,7 +95,7 @@ class KMeans private ( * Set the number of steps for the k-means|| initialization mode. This is an advanced * setting -- the default of 5 is almost always enough. Default: 5. */ - def setInitializationSteps(initializationSteps: Int): KMeans = { + def setInitializationSteps(initializationSteps: Int): this.type = { if (initializationSteps <= 0) { throw new IllegalArgumentException("Number of initialization steps must be positive") } @@ -107,7 +107,7 @@ class KMeans private ( * Set the distance threshold within which we've consider centers to have converged. * If all centers move less than this Euclidean distance, we stop iterating one run. */ - def setEpsilon(epsilon: Double): KMeans = { + def setEpsilon(epsilon: Double): this.type = { this.epsilon = epsilon this } @@ -165,18 +165,21 @@ class KMeans private ( val activeCenters = activeRuns.map(r => centers(r)).toArray val costAccums = activeRuns.map(_ => sc.accumulator(0.0)) + val bcActiveCenters = sc.broadcast(activeCenters) + // Find the sum and count of points mapping to each center val totalContribs = data.mapPartitions { points => - val runs = activeCenters.length - val k = activeCenters(0).length - val dims = activeCenters(0)(0).vector.length + val thisActiveCenters = bcActiveCenters.value + val runs = thisActiveCenters.length + val k = thisActiveCenters(0).length + val dims = thisActiveCenters(0)(0).vector.length val sums = Array.fill(runs, k)(BDV.zeros[Double](dims).asInstanceOf[BV[Double]]) val counts = Array.fill(runs, k)(0L) points.foreach { point => (0 until runs).foreach { i => - val (bestCenter, cost) = KMeans.findClosest(activeCenters(i), point) + val (bestCenter, cost) = KMeans.findClosest(thisActiveCenters(i), point) costAccums(i) += cost sums(i)(bestCenter) += point.vector counts(i)(bestCenter) += 1 @@ -264,16 +267,17 @@ class KMeans private ( // to their squared distance from that run's current centers var step = 0 while (step < initializationSteps) { + val bcCenters = data.context.broadcast(centers) val sumCosts = data.flatMap { point => (0 until runs).map { r => - (r, KMeans.pointCost(centers(r), point)) + (r, KMeans.pointCost(bcCenters.value(r), point)) } }.reduceByKey(_ + _).collectAsMap() val chosen = data.mapPartitionsWithIndex { (index, points) => val rand = new XORShiftRandom(seed ^ (step << 16) ^ index) points.flatMap { p => (0 until runs).filter { r => - rand.nextDouble() < 2.0 * KMeans.pointCost(centers(r), p) * k / sumCosts(r) + rand.nextDouble() < 2.0 * KMeans.pointCost(bcCenters.value(r), p) * k / sumCosts(r) }.map((_, p)) } }.collect() @@ -286,9 +290,10 @@ class KMeans private ( // Finally, we might have a set of more than k candidate centers for each run; weigh each // candidate by the number of points in the dataset mapping to it and run a local k-means++ // on the weighted centers to pick just k of them + val bcCenters = data.context.broadcast(centers) val weightMap = data.flatMap { p => (0 until runs).map { r => - ((r, KMeans.findClosest(centers(r), p)._1), 1.0) + ((r, KMeans.findClosest(bcCenters.value(r), p)._1), 1.0) } }.reduceByKey(_ + _).collectAsMap() val finalCenters = (0 until runs).map { r => diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala index fba21aefaaacd..5823cb6e52e7f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala @@ -38,7 +38,8 @@ class KMeansModel private[mllib] (val clusterCenters: Array[Vector]) extends Ser /** Maps given points to their cluster indices. */ def predict(points: RDD[Vector]): RDD[Int] = { val centersWithNorm = clusterCentersWithNorm - points.map(p => KMeans.findClosest(centersWithNorm, new BreezeVectorWithNorm(p))._1) + val bcCentersWithNorm = points.context.broadcast(centersWithNorm) + points.map(p => KMeans.findClosest(bcCentersWithNorm.value, new BreezeVectorWithNorm(p))._1) } /** Maps given points to their cluster indices. */ @@ -51,7 +52,8 @@ class KMeansModel private[mllib] (val clusterCenters: Array[Vector]) extends Ser */ def computeCost(data: RDD[Vector]): Double = { val centersWithNorm = clusterCentersWithNorm - data.map(p => KMeans.pointCost(centersWithNorm, new BreezeVectorWithNorm(p))).sum() + val bcCentersWithNorm = data.context.broadcast(centersWithNorm) + data.map(p => KMeans.pointCost(bcCentersWithNorm.value, new BreezeVectorWithNorm(p))).sum() } private def clusterCentersWithNorm: Iterable[BreezeVectorWithNorm] = diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LocalKMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LocalKMeans.scala index 2e3a4ce783de7..f0722d7c14a46 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LocalKMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LocalKMeans.scala @@ -59,7 +59,13 @@ private[mllib] object LocalKMeans extends Logging { cumulativeScore += weights(j) * KMeans.pointCost(curCenters, points(j)) j += 1 } - centers(i) = points(j-1).toDense + if (j == 0) { + logWarning("kMeansPlusPlus initialization ran out of distinct points for centers." + + s" Using duplicate point for center k = $i.") + centers(i) = points(0).toDense + } else { + centers(i) = points(j - 1).toDense + } } // Run up to maxIterations iterations of Lloyd's algorithm diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetrics.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetrics.scala index 079743742d86d..1af40de2c7fcf 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetrics.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetrics.scala @@ -103,11 +103,11 @@ class BinaryClassificationMetrics(scoreAndLabels: RDD[(Double, Double)]) extends mergeValue = (c: BinaryLabelCounter, label: Double) => c += label, mergeCombiners = (c1: BinaryLabelCounter, c2: BinaryLabelCounter) => c1 += c2 ).sortByKey(ascending = false) - val agg = counts.values.mapPartitions({ iter => + val agg = counts.values.mapPartitions { iter => val agg = new BinaryLabelCounter() iter.foreach(agg += _) Iterator(agg) - }, preservesPartitioning = true).collect() + }.collect() val partitionwiseCumulativeCounts = agg.scanLeft(new BinaryLabelCounter())( (agg: BinaryLabelCounter, c: BinaryLabelCounter) => agg.clone() += c) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala new file mode 100644 index 0000000000000..666362ae6739a --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala @@ -0,0 +1,190 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.evaluation + +import scala.collection.Map + +import org.apache.spark.SparkContext._ +import org.apache.spark.annotation.Experimental +import org.apache.spark.mllib.linalg.{Matrices, Matrix} +import org.apache.spark.rdd.RDD + +/** + * ::Experimental:: + * Evaluator for multiclass classification. + * + * @param predictionAndLabels an RDD of (prediction, label) pairs. + */ +@Experimental +class MulticlassMetrics(predictionAndLabels: RDD[(Double, Double)]) { + + private lazy val labelCountByClass: Map[Double, Long] = predictionAndLabels.values.countByValue() + private lazy val labelCount: Long = labelCountByClass.values.sum + private lazy val tpByClass: Map[Double, Int] = predictionAndLabels + .map { case (prediction, label) => + (label, if (label == prediction) 1 else 0) + }.reduceByKey(_ + _) + .collectAsMap() + private lazy val fpByClass: Map[Double, Int] = predictionAndLabels + .map { case (prediction, label) => + (prediction, if (prediction != label) 1 else 0) + }.reduceByKey(_ + _) + .collectAsMap() + private lazy val confusions = predictionAndLabels + .map { case (prediction, label) => + ((label, prediction), 1) + }.reduceByKey(_ + _) + .collectAsMap() + + /** + * Returns confusion matrix: + * predicted classes are in columns, + * they are ordered by class label ascending, + * as in "labels" + */ + def confusionMatrix: Matrix = { + val n = labels.size + val values = Array.ofDim[Double](n * n) + var i = 0 + while (i < n) { + var j = 0 + while (j < n) { + values(i + j * n) = confusions.getOrElse((labels(i), labels(j)), 0).toDouble + j += 1 + } + i += 1 + } + Matrices.dense(n, n, values) + } + + /** + * Returns true positive rate for a given label (category) + * @param label the label. + */ + def truePositiveRate(label: Double): Double = recall(label) + + /** + * Returns false positive rate for a given label (category) + * @param label the label. + */ + def falsePositiveRate(label: Double): Double = { + val fp = fpByClass.getOrElse(label, 0) + fp.toDouble / (labelCount - labelCountByClass(label)) + } + + /** + * Returns precision for a given label (category) + * @param label the label. + */ + def precision(label: Double): Double = { + val tp = tpByClass(label) + val fp = fpByClass.getOrElse(label, 0) + if (tp + fp == 0) 0 else tp.toDouble / (tp + fp) + } + + /** + * Returns recall for a given label (category) + * @param label the label. + */ + def recall(label: Double): Double = tpByClass(label).toDouble / labelCountByClass(label) + + /** + * Returns f-measure for a given label (category) + * @param label the label. + * @param beta the beta parameter. + */ + def fMeasure(label: Double, beta: Double): Double = { + val p = precision(label) + val r = recall(label) + val betaSqrd = beta * beta + if (p + r == 0) 0 else (1 + betaSqrd) * p * r / (betaSqrd * p + r) + } + + /** + * Returns f1-measure for a given label (category) + * @param label the label. + */ + def fMeasure(label: Double): Double = fMeasure(label, 1.0) + + /** + * Returns precision + */ + lazy val precision: Double = tpByClass.values.sum.toDouble / labelCount + + /** + * Returns recall + * (equals to precision for multiclass classifier + * because sum of all false positives is equal to sum + * of all false negatives) + */ + lazy val recall: Double = precision + + /** + * Returns f-measure + * (equals to precision and recall because precision equals recall) + */ + lazy val fMeasure: Double = precision + + /** + * Returns weighted true positive rate + * (equals to precision, recall and f-measure) + */ + lazy val weightedTruePositiveRate: Double = weightedRecall + + /** + * Returns weighted false positive rate + */ + lazy val weightedFalsePositiveRate: Double = labelCountByClass.map { case (category, count) => + falsePositiveRate(category) * count.toDouble / labelCount + }.sum + + /** + * Returns weighted averaged recall + * (equals to precision, recall and f-measure) + */ + lazy val weightedRecall: Double = labelCountByClass.map { case (category, count) => + recall(category) * count.toDouble / labelCount + }.sum + + /** + * Returns weighted averaged precision + */ + lazy val weightedPrecision: Double = labelCountByClass.map { case (category, count) => + precision(category) * count.toDouble / labelCount + }.sum + + /** + * Returns weighted averaged f-measure + * @param beta the beta parameter. + */ + def weightedFMeasure(beta: Double): Double = labelCountByClass.map { case (category, count) => + fMeasure(category, beta) * count.toDouble / labelCount + }.sum + + /** + * Returns weighted averaged f1-measure + */ + lazy val weightedFMeasure: Double = labelCountByClass.map { case (category, count) => + fMeasure(category, 1.0) * count.toDouble / labelCount + }.sum + + /** + * Returns the sequence of labels in ascending order + */ + lazy val labels: Array[Double] = tpByClass.keys.toArray.sorted +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/HashingTF.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/HashingTF.scala new file mode 100644 index 0000000000000..c53475818395f --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/HashingTF.scala @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.feature + +import java.lang.{Iterable => JavaIterable} + +import scala.collection.JavaConverters._ +import scala.collection.mutable + +import org.apache.spark.annotation.Experimental +import org.apache.spark.api.java.JavaRDD +import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.rdd.RDD +import org.apache.spark.util.Utils + +/** + * :: Experimental :: + * Maps a sequence of terms to their term frequencies using the hashing trick. + * + * @param numFeatures number of features (default: 2^20^) + */ +@Experimental +class HashingTF(val numFeatures: Int) extends Serializable { + + def this() = this(1 << 20) + + /** + * Returns the index of the input term. + */ + def indexOf(term: Any): Int = Utils.nonNegativeMod(term.##, numFeatures) + + /** + * Transforms the input document into a sparse term frequency vector. + */ + def transform(document: Iterable[_]): Vector = { + val termFrequencies = mutable.HashMap.empty[Int, Double] + document.foreach { term => + val i = indexOf(term) + termFrequencies.put(i, termFrequencies.getOrElse(i, 0.0) + 1.0) + } + Vectors.sparse(numFeatures, termFrequencies.toSeq) + } + + /** + * Transforms the input document into a sparse term frequency vector (Java version). + */ + def transform(document: JavaIterable[_]): Vector = { + transform(document.asScala) + } + + /** + * Transforms the input document to term frequency vectors. + */ + def transform[D <: Iterable[_]](dataset: RDD[D]): RDD[Vector] = { + dataset.map(this.transform) + } + + /** + * Transforms the input document to term frequency vectors (Java version). + */ + def transform[D <: JavaIterable[_]](dataset: JavaRDD[D]): JavaRDD[Vector] = { + dataset.rdd.map(this.transform).toJavaRDD() + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala new file mode 100644 index 0000000000000..d40d5553c1d21 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala @@ -0,0 +1,186 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.feature + +import breeze.linalg.{DenseVector => BDV} + +import org.apache.spark.annotation.Experimental +import org.apache.spark.api.java.JavaRDD +import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors} +import org.apache.spark.mllib.rdd.RDDFunctions._ +import org.apache.spark.rdd.RDD + +/** + * :: Experimental :: + * Inverse document frequency (IDF). + * The standard formulation is used: `idf = log((m + 1) / (d(t) + 1))`, where `m` is the total + * number of documents and `d(t)` is the number of documents that contain term `t`. + */ +@Experimental +class IDF { + + // TODO: Allow different IDF formulations. + + /** + * Computes the inverse document frequency. + * @param dataset an RDD of term frequency vectors + */ + def fit(dataset: RDD[Vector]): IDFModel = { + val idf = dataset.treeAggregate(new IDF.DocumentFrequencyAggregator)( + seqOp = (df, v) => df.add(v), + combOp = (df1, df2) => df1.merge(df2) + ).idf() + new IDFModel(idf) + } + + /** + * Computes the inverse document frequency. + * @param dataset a JavaRDD of term frequency vectors + */ + def fit(dataset: JavaRDD[Vector]): IDFModel = { + fit(dataset.rdd) + } +} + +private object IDF { + + /** Document frequency aggregator. */ + class DocumentFrequencyAggregator extends Serializable { + + /** number of documents */ + private var m = 0L + /** document frequency vector */ + private var df: BDV[Long] = _ + + /** Adds a new document. */ + def add(doc: Vector): this.type = { + if (isEmpty) { + df = BDV.zeros(doc.size) + } + doc match { + case sv: SparseVector => + val nnz = sv.indices.size + var k = 0 + while (k < nnz) { + if (sv.values(k) > 0) { + df(sv.indices(k)) += 1L + } + k += 1 + } + case dv: DenseVector => + val n = dv.size + var j = 0 + while (j < n) { + if (dv.values(j) > 0.0) { + df(j) += 1L + } + j += 1 + } + case other => + throw new UnsupportedOperationException( + s"Only sparse and dense vectors are supported but got ${other.getClass}.") + } + m += 1L + this + } + + /** Merges another. */ + def merge(other: DocumentFrequencyAggregator): this.type = { + if (!other.isEmpty) { + m += other.m + if (df == null) { + df = other.df.copy + } else { + df += other.df + } + } + this + } + + private def isEmpty: Boolean = m == 0L + + /** Returns the current IDF vector. */ + def idf(): Vector = { + if (isEmpty) { + throw new IllegalStateException("Haven't seen any document yet.") + } + val n = df.length + val inv = new Array[Double](n) + var j = 0 + while (j < n) { + inv(j) = math.log((m + 1.0)/ (df(j) + 1.0)) + j += 1 + } + Vectors.dense(inv) + } + } +} + +/** + * :: Experimental :: + * Represents an IDF model that can transform term frequency vectors. + */ +@Experimental +class IDFModel private[mllib] (val idf: Vector) extends Serializable { + + /** + * Transforms term frequency (TF) vectors to TF-IDF vectors. + * @param dataset an RDD of term frequency vectors + * @return an RDD of TF-IDF vectors + */ + def transform(dataset: RDD[Vector]): RDD[Vector] = { + val bcIdf = dataset.context.broadcast(idf) + dataset.mapPartitions { iter => + val thisIdf = bcIdf.value + iter.map { v => + val n = v.size + v match { + case sv: SparseVector => + val nnz = sv.indices.size + val newValues = new Array[Double](nnz) + var k = 0 + while (k < nnz) { + newValues(k) = sv.values(k) * thisIdf(sv.indices(k)) + k += 1 + } + Vectors.sparse(n, sv.indices, newValues) + case dv: DenseVector => + val newValues = new Array[Double](n) + var j = 0 + while (j < n) { + newValues(j) = dv.values(j) * thisIdf(j) + j += 1 + } + Vectors.dense(newValues) + case other => + throw new UnsupportedOperationException( + s"Only sparse and dense vectors are supported but got ${other.getClass}.") + } + } + } + } + + /** + * Transforms term frequency (TF) vectors to TF-IDF vectors (Java version). + * @param dataset a JavaRDD of term frequency vectors + * @return a JavaRDD of TF-IDF vectors + */ + def transform(dataset: JavaRDD[Vector]): JavaRDD[Vector] = { + transform(dataset.rdd).toJavaRDD() + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala new file mode 100644 index 0000000000000..3afb47767281c --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.feature + +import breeze.linalg.{DenseVector => BDV, SparseVector => BSV} + +import org.apache.spark.annotation.Experimental +import org.apache.spark.mllib.linalg.{Vector, Vectors} + +/** + * :: Experimental :: + * Normalizes samples individually to unit L^p^ norm + * + * For any 1 <= p < Double.PositiveInfinity, normalizes samples using + * sum(abs(vector).^p^)^(1/p)^ as norm. + * + * For p = Double.PositiveInfinity, max(abs(vector)) will be used as norm for normalization. + * + * @param p Normalization in L^p^ space, p = 2 by default. + */ +@Experimental +class Normalizer(p: Double) extends VectorTransformer { + + def this() = this(2) + + require(p >= 1.0) + + /** + * Applies unit length normalization on a vector. + * + * @param vector vector to be normalized. + * @return normalized vector. If the norm of the input is zero, it will return the input vector. + */ + override def transform(vector: Vector): Vector = { + var norm = vector.toBreeze.norm(p) + + if (norm != 0.0) { + // For dense vector, we've to allocate new memory for new output vector. + // However, for sparse vector, the `index` array will not be changed, + // so we can re-use it to save memory. + vector.toBreeze match { + case dv: BDV[Double] => Vectors.fromBreeze(dv :/ norm) + case sv: BSV[Double] => + val output = new BSV[Double](sv.index, sv.data.clone(), sv.length) + var i = 0 + while (i < output.data.length) { + output.data(i) /= norm + i += 1 + } + Vectors.fromBreeze(output) + case v => throw new IllegalArgumentException("Do not support vector type " + v.getClass) + } + } else { + // Since the norm is zero, return the input vector object itself. + // Note that it's safe since we always assume that the data in RDD + // should be immutable. + vector + } + } + +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala new file mode 100644 index 0000000000000..4dfd1f0ab8134 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala @@ -0,0 +1,131 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.feature + +import breeze.linalg.{DenseVector => BDV, SparseVector => BSV} + +import org.apache.spark.Logging +import org.apache.spark.annotation.Experimental +import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.mllib.rdd.RDDFunctions._ +import org.apache.spark.mllib.stat.MultivariateOnlineSummarizer +import org.apache.spark.rdd.RDD + +/** + * :: Experimental :: + * Standardizes features by removing the mean and scaling to unit variance using column summary + * statistics on the samples in the training set. + * + * @param withMean False by default. Centers the data with mean before scaling. It will build a + * dense output, so this does not work on sparse input and will raise an exception. + * @param withStd True by default. Scales the data to unit standard deviation. + */ +@Experimental +class StandardScaler(withMean: Boolean, withStd: Boolean) extends Logging { + + def this() = this(false, true) + + if (!(withMean || withStd)) { + logWarning("Both withMean and withStd are false. The model does nothing.") + } + + /** + * Computes the mean and variance and stores as a model to be used for later scaling. + * + * @param data The data used to compute the mean and variance to build the transformation model. + * @return a StandardScalarModel + */ + def fit(data: RDD[Vector]): StandardScalerModel = { + // TODO: skip computation if both withMean and withStd are false + val summary = data.treeAggregate(new MultivariateOnlineSummarizer)( + (aggregator, data) => aggregator.add(data), + (aggregator1, aggregator2) => aggregator1.merge(aggregator2)) + new StandardScalerModel(withMean, withStd, summary.mean, summary.variance) + } +} + +/** + * :: Experimental :: + * Represents a StandardScaler model that can transform vectors. + * + * @param withMean whether to center the data before scaling + * @param withStd whether to scale the data to have unit standard deviation + * @param mean column mean values + * @param variance column variance values + */ +@Experimental +class StandardScalerModel private[mllib] ( + val withMean: Boolean, + val withStd: Boolean, + val mean: Vector, + val variance: Vector) extends VectorTransformer { + + require(mean.size == variance.size) + + private lazy val factor: BDV[Double] = { + val f = BDV.zeros[Double](variance.size) + var i = 0 + while (i < f.size) { + f(i) = if (variance(i) != 0.0) 1.0 / math.sqrt(variance(i)) else 0.0 + i += 1 + } + f + } + + /** + * Applies standardization transformation on a vector. + * + * @param vector Vector to be standardized. + * @return Standardized vector. If the variance of a column is zero, it will return default `0.0` + * for the column with zero variance. + */ + override def transform(vector: Vector): Vector = { + require(mean.size == vector.size) + if (withMean) { + vector.toBreeze match { + case dv: BDV[Double] => + val output = vector.toBreeze.copy + var i = 0 + while (i < output.length) { + output(i) = (output(i) - mean(i)) * (if (withStd) factor(i) else 1.0) + i += 1 + } + Vectors.fromBreeze(output) + case v => throw new IllegalArgumentException("Do not support vector type " + v.getClass) + } + } else if (withStd) { + vector.toBreeze match { + case dv: BDV[Double] => Vectors.fromBreeze(dv :* factor) + case sv: BSV[Double] => + // For sparse vector, the `index` array inside sparse vector object will not be changed, + // so we can re-use it to save memory. + val output = new BSV[Double](sv.index, sv.data.clone(), sv.length) + var i = 0 + while (i < output.data.length) { + output.data(i) *= factor(output.index(i)) + i += 1 + } + Vectors.fromBreeze(output) + case v => throw new IllegalArgumentException("Do not support vector type " + v.getClass) + } + } else { + // Note that it's safe since we always assume that the data in RDD should be immutable. + vector + } + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/VectorTransformer.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/VectorTransformer.scala new file mode 100644 index 0000000000000..415a845332d45 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/VectorTransformer.scala @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.feature + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.rdd.RDD + +/** + * :: DeveloperApi :: + * Trait for transformation of a vector + */ +@DeveloperApi +trait VectorTransformer extends Serializable { + + /** + * Applies transformation on a vector. + * + * @param vector vector to be transformed. + * @return transformed vector. + */ + def transform(vector: Vector): Vector + + /** + * Applies transformation on an RDD[Vector]. + * + * @param data RDD[Vector] to be transformed. + * @return transformed RDD[Vector]. + */ + def transform(data: RDD[Vector]): RDD[Vector] = { + // Later in #1498 , all RDD objects are sent via broadcasting instead of akka. + // So it should be no longer necessary to explicitly broadcast `this` object. + data.map(x => this.transform(x)) + } + +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala new file mode 100644 index 0000000000000..d2ae62b482aff --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala @@ -0,0 +1,474 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.feature + +import java.lang.{Iterable => JavaIterable} + +import scala.collection.JavaConverters._ +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer + +import com.github.fommil.netlib.BLAS.{getInstance => blas} + +import org.apache.spark.Logging +import org.apache.spark.SparkContext._ +import org.apache.spark.annotation.Experimental +import org.apache.spark.api.java.JavaRDD +import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.mllib.rdd.RDDFunctions._ +import org.apache.spark.rdd._ +import org.apache.spark.util.Utils +import org.apache.spark.util.random.XORShiftRandom +import org.apache.spark.util.collection.PrimitiveKeyOpenHashMap + +/** + * Entry in vocabulary + */ +private case class VocabWord( + var word: String, + var cn: Int, + var point: Array[Int], + var code: Array[Int], + var codeLen:Int +) + +/** + * :: Experimental :: + * Word2Vec creates vector representation of words in a text corpus. + * The algorithm first constructs a vocabulary from the corpus + * and then learns vector representation of words in the vocabulary. + * The vector representation can be used as features in + * natural language processing and machine learning algorithms. + * + * We used skip-gram model in our implementation and hierarchical softmax + * method to train the model. The variable names in the implementation + * matches the original C implementation. + * + * For original C implementation, see https://code.google.com/p/word2vec/ + * For research papers, see + * Efficient Estimation of Word Representations in Vector Space + * and + * Distributed Representations of Words and Phrases and their Compositionality. + */ +@Experimental +class Word2Vec extends Serializable with Logging { + + private var vectorSize = 100 + private var startingAlpha = 0.025 + private var numPartitions = 1 + private var numIterations = 1 + private var seed = Utils.random.nextLong() + + /** + * Sets vector size (default: 100). + */ + def setVectorSize(vectorSize: Int): this.type = { + this.vectorSize = vectorSize + this + } + + /** + * Sets initial learning rate (default: 0.025). + */ + def setLearningRate(learningRate: Double): this.type = { + this.startingAlpha = learningRate + this + } + + /** + * Sets number of partitions (default: 1). Use a small number for accuracy. + */ + def setNumPartitions(numPartitions: Int): this.type = { + require(numPartitions > 0, s"numPartitions must be greater than 0 but got $numPartitions") + this.numPartitions = numPartitions + this + } + + /** + * Sets number of iterations (default: 1), which should be smaller than or equal to number of + * partitions. + */ + def setNumIterations(numIterations: Int): this.type = { + this.numIterations = numIterations + this + } + + /** + * Sets random seed (default: a random long integer). + */ + def setSeed(seed: Long): this.type = { + this.seed = seed + this + } + + private val EXP_TABLE_SIZE = 1000 + private val MAX_EXP = 6 + private val MAX_CODE_LENGTH = 40 + private val MAX_SENTENCE_LENGTH = 1000 + + /** context words from [-window, window] */ + private val window = 5 + + /** minimum frequency to consider a vocabulary word */ + private val minCount = 5 + + private var trainWordsCount = 0 + private var vocabSize = 0 + private var vocab: Array[VocabWord] = null + private var vocabHash = mutable.HashMap.empty[String, Int] + + private def learnVocab(words: RDD[String]): Unit = { + vocab = words.map(w => (w, 1)) + .reduceByKey(_ + _) + .map(x => VocabWord( + x._1, + x._2, + new Array[Int](MAX_CODE_LENGTH), + new Array[Int](MAX_CODE_LENGTH), + 0)) + .filter(_.cn >= minCount) + .collect() + .sortWith((a, b) => a.cn > b.cn) + + vocabSize = vocab.length + var a = 0 + while (a < vocabSize) { + vocabHash += vocab(a).word -> a + trainWordsCount += vocab(a).cn + a += 1 + } + logInfo("trainWordsCount = " + trainWordsCount) + } + + private def createExpTable(): Array[Float] = { + val expTable = new Array[Float](EXP_TABLE_SIZE) + var i = 0 + while (i < EXP_TABLE_SIZE) { + val tmp = math.exp((2.0 * i / EXP_TABLE_SIZE - 1.0) * MAX_EXP) + expTable(i) = (tmp / (tmp + 1.0)).toFloat + i += 1 + } + expTable + } + + private def createBinaryTree(): Unit = { + val count = new Array[Long](vocabSize * 2 + 1) + val binary = new Array[Int](vocabSize * 2 + 1) + val parentNode = new Array[Int](vocabSize * 2 + 1) + val code = new Array[Int](MAX_CODE_LENGTH) + val point = new Array[Int](MAX_CODE_LENGTH) + var a = 0 + while (a < vocabSize) { + count(a) = vocab(a).cn + a += 1 + } + while (a < 2 * vocabSize) { + count(a) = 1e9.toInt + a += 1 + } + var pos1 = vocabSize - 1 + var pos2 = vocabSize + + var min1i = 0 + var min2i = 0 + + a = 0 + while (a < vocabSize - 1) { + if (pos1 >= 0) { + if (count(pos1) < count(pos2)) { + min1i = pos1 + pos1 -= 1 + } else { + min1i = pos2 + pos2 += 1 + } + } else { + min1i = pos2 + pos2 += 1 + } + if (pos1 >= 0) { + if (count(pos1) < count(pos2)) { + min2i = pos1 + pos1 -= 1 + } else { + min2i = pos2 + pos2 += 1 + } + } else { + min2i = pos2 + pos2 += 1 + } + count(vocabSize + a) = count(min1i) + count(min2i) + parentNode(min1i) = vocabSize + a + parentNode(min2i) = vocabSize + a + binary(min2i) = 1 + a += 1 + } + // Now assign binary code to each vocabulary word + var i = 0 + a = 0 + while (a < vocabSize) { + var b = a + i = 0 + while (b != vocabSize * 2 - 2) { + code(i) = binary(b) + point(i) = b + i += 1 + b = parentNode(b) + } + vocab(a).codeLen = i + vocab(a).point(0) = vocabSize - 2 + b = 0 + while (b < i) { + vocab(a).code(i - b - 1) = code(b) + vocab(a).point(i - b) = point(b) - vocabSize + b += 1 + } + a += 1 + } + } + + /** + * Computes the vector representation of each word in vocabulary. + * @param dataset an RDD of words + * @return a Word2VecModel + */ + def fit[S <: Iterable[String]](dataset: RDD[S]): Word2VecModel = { + + val words = dataset.flatMap(x => x) + + learnVocab(words) + + createBinaryTree() + + val sc = dataset.context + + val expTable = sc.broadcast(createExpTable()) + val bcVocab = sc.broadcast(vocab) + val bcVocabHash = sc.broadcast(vocabHash) + + val sentences: RDD[Array[Int]] = words.mapPartitions { iter => + new Iterator[Array[Int]] { + def hasNext: Boolean = iter.hasNext + + def next(): Array[Int] = { + var sentence = new ArrayBuffer[Int] + var sentenceLength = 0 + while (iter.hasNext && sentenceLength < MAX_SENTENCE_LENGTH) { + val word = bcVocabHash.value.get(iter.next()) + word match { + case Some(w) => + sentence += w + sentenceLength += 1 + case None => + } + } + sentence.toArray + } + } + } + + val newSentences = sentences.repartition(numPartitions).cache() + val initRandom = new XORShiftRandom(seed) + var syn0Global = + Array.fill[Float](vocabSize * vectorSize)((initRandom.nextFloat() - 0.5f) / vectorSize) + var syn1Global = new Array[Float](vocabSize * vectorSize) + var alpha = startingAlpha + for (k <- 1 to numIterations) { + val partial = newSentences.mapPartitionsWithIndex { case (idx, iter) => + val random = new XORShiftRandom(seed ^ ((idx + 1) << 16) ^ ((-k - 1) << 8)) + val syn0Modify = new Array[Int](vocabSize) + val syn1Modify = new Array[Int](vocabSize) + val model = iter.foldLeft((syn0Global, syn1Global, 0, 0)) { + case ((syn0, syn1, lastWordCount, wordCount), sentence) => + var lwc = lastWordCount + var wc = wordCount + if (wordCount - lastWordCount > 10000) { + lwc = wordCount + // TODO: discount by iteration? + alpha = + startingAlpha * (1 - numPartitions * wordCount.toDouble / (trainWordsCount + 1)) + if (alpha < startingAlpha * 0.0001) alpha = startingAlpha * 0.0001 + logInfo("wordCount = " + wordCount + ", alpha = " + alpha) + } + wc += sentence.size + var pos = 0 + while (pos < sentence.size) { + val word = sentence(pos) + val b = random.nextInt(window) + // Train Skip-gram + var a = b + while (a < window * 2 + 1 - b) { + if (a != window) { + val c = pos - window + a + if (c >= 0 && c < sentence.size) { + val lastWord = sentence(c) + val l1 = lastWord * vectorSize + val neu1e = new Array[Float](vectorSize) + // Hierarchical softmax + var d = 0 + while (d < bcVocab.value(word).codeLen) { + val inner = bcVocab.value(word).point(d) + val l2 = inner * vectorSize + // Propagate hidden -> output + var f = blas.sdot(vectorSize, syn0, l1, 1, syn1, l2, 1) + if (f > -MAX_EXP && f < MAX_EXP) { + val ind = ((f + MAX_EXP) * (EXP_TABLE_SIZE / MAX_EXP / 2.0)).toInt + f = expTable.value(ind) + val g = ((1 - bcVocab.value(word).code(d) - f) * alpha).toFloat + blas.saxpy(vectorSize, g, syn1, l2, 1, neu1e, 0, 1) + blas.saxpy(vectorSize, g, syn0, l1, 1, syn1, l2, 1) + syn1Modify(inner) += 1 + } + d += 1 + } + blas.saxpy(vectorSize, 1.0f, neu1e, 0, 1, syn0, l1, 1) + syn0Modify(lastWord) += 1 + } + } + a += 1 + } + pos += 1 + } + (syn0, syn1, lwc, wc) + } + val syn0Local = model._1 + val syn1Local = model._2 + val synOut = new PrimitiveKeyOpenHashMap[Int, Array[Float]](vocabSize * 2) + var index = 0 + while(index < vocabSize) { + if (syn0Modify(index) != 0) { + synOut.update(index, syn0Local.slice(index * vectorSize, (index + 1) * vectorSize)) + } + if (syn1Modify(index) != 0) { + synOut.update(index + vocabSize, + syn1Local.slice(index * vectorSize, (index + 1) * vectorSize)) + } + index += 1 + } + Iterator(synOut) + } + val synAgg = partial.flatMap(x => x).reduceByKey { case (v1, v2) => + blas.saxpy(vectorSize, 1.0f, v2, 1, v1, 1) + v1 + }.collect() + var i = 0 + while (i < synAgg.length) { + val index = synAgg(i)._1 + if (index < vocabSize) { + Array.copy(synAgg(i)._2, 0, syn0Global, index * vectorSize, vectorSize) + } else { + Array.copy(synAgg(i)._2, 0, syn1Global, (index - vocabSize) * vectorSize, vectorSize) + } + i += 1 + } + } + newSentences.unpersist() + + val word2VecMap = mutable.HashMap.empty[String, Array[Float]] + var i = 0 + while (i < vocabSize) { + val word = bcVocab.value(i).word + val vector = new Array[Float](vectorSize) + Array.copy(syn0Global, i * vectorSize, vector, 0, vectorSize) + word2VecMap += word -> vector + i += 1 + } + + new Word2VecModel(word2VecMap.toMap) + } + + /** + * Computes the vector representation of each word in vocabulary (Java version). + * @param dataset a JavaRDD of words + * @return a Word2VecModel + */ + def fit[S <: JavaIterable[String]](dataset: JavaRDD[S]): Word2VecModel = { + fit(dataset.rdd.map(_.asScala)) + } +} + +/** + * :: Experimental :: + * Word2Vec model + */ +@Experimental +class Word2VecModel private[mllib] ( + private val model: Map[String, Array[Float]]) extends Serializable { + + private def cosineSimilarity(v1: Array[Float], v2: Array[Float]): Double = { + require(v1.length == v2.length, "Vectors should have the same length") + val n = v1.length + val norm1 = blas.snrm2(n, v1, 1) + val norm2 = blas.snrm2(n, v2, 1) + if (norm1 == 0 || norm2 == 0) return 0.0 + blas.sdot(n, v1, 1, v2,1) / norm1 / norm2 + } + + /** + * Transforms a word to its vector representation + * @param word a word + * @return vector representation of word + */ + def transform(word: String): Vector = { + model.get(word) match { + case Some(vec) => + Vectors.dense(vec.map(_.toDouble)) + case None => + throw new IllegalStateException(s"$word not in vocabulary") + } + } + + /** + * Transforms an RDD to its vector representation + * @param dataset a an RDD of words + * @return RDD of vector representation + */ + def transform(dataset: RDD[String]): RDD[Vector] = { + dataset.map(word => transform(word)) + } + + /** + * Find synonyms of a word + * @param word a word + * @param num number of synonyms to find + * @return array of (word, similarity) + */ + def findSynonyms(word: String, num: Int): Array[(String, Double)] = { + val vector = transform(word) + findSynonyms(vector,num) + } + + /** + * Find synonyms of the vector representation of a word + * @param vector vector representation of a word + * @param num number of synonyms to find + * @return array of (word, cosineSimilarity) + */ + def findSynonyms(vector: Vector, num: Int): Array[(String, Double)] = { + require(num > 0, "Number of similar words should > 0") + // TODO: optimize top-k + val fVector = vector.toArray.map(_.toFloat) + model.mapValues(vec => cosineSimilarity(fVector, vec)) + .toSeq + .sortBy(- _._2) + .take(num + 1) + .tail + .toArray + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala new file mode 100644 index 0000000000000..70e23033c8754 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala @@ -0,0 +1,200 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.linalg + +import com.github.fommil.netlib.{BLAS => NetlibBLAS, F2jBLAS} + +/** + * BLAS routines for MLlib's vectors and matrices. + */ +private[mllib] object BLAS extends Serializable { + + @transient private var _f2jBLAS: NetlibBLAS = _ + + // For level-1 routines, we use Java implementation. + private def f2jBLAS: NetlibBLAS = { + if (_f2jBLAS == null) { + _f2jBLAS = new F2jBLAS + } + _f2jBLAS + } + + /** + * y += a * x + */ + def axpy(a: Double, x: Vector, y: Vector): Unit = { + require(x.size == y.size) + y match { + case dy: DenseVector => + x match { + case sx: SparseVector => + axpy(a, sx, dy) + case dx: DenseVector => + axpy(a, dx, dy) + case _ => + throw new UnsupportedOperationException( + s"axpy doesn't support x type ${x.getClass}.") + } + case _ => + throw new IllegalArgumentException( + s"axpy only supports adding to a dense vector but got type ${y.getClass}.") + } + } + + /** + * y += a * x + */ + private def axpy(a: Double, x: DenseVector, y: DenseVector): Unit = { + val n = x.size + f2jBLAS.daxpy(n, a, x.values, 1, y.values, 1) + } + + /** + * y += a * x + */ + private def axpy(a: Double, x: SparseVector, y: DenseVector): Unit = { + val nnz = x.indices.size + if (a == 1.0) { + var k = 0 + while (k < nnz) { + y.values(x.indices(k)) += x.values(k) + k += 1 + } + } else { + var k = 0 + while (k < nnz) { + y.values(x.indices(k)) += a * x.values(k) + k += 1 + } + } + } + + /** + * dot(x, y) + */ + def dot(x: Vector, y: Vector): Double = { + require(x.size == y.size) + (x, y) match { + case (dx: DenseVector, dy: DenseVector) => + dot(dx, dy) + case (sx: SparseVector, dy: DenseVector) => + dot(sx, dy) + case (dx: DenseVector, sy: SparseVector) => + dot(sy, dx) + case (sx: SparseVector, sy: SparseVector) => + dot(sx, sy) + case _ => + throw new IllegalArgumentException(s"dot doesn't support (${x.getClass}, ${y.getClass}).") + } + } + + /** + * dot(x, y) + */ + private def dot(x: DenseVector, y: DenseVector): Double = { + val n = x.size + f2jBLAS.ddot(n, x.values, 1, y.values, 1) + } + + /** + * dot(x, y) + */ + private def dot(x: SparseVector, y: DenseVector): Double = { + val nnz = x.indices.size + var sum = 0.0 + var k = 0 + while (k < nnz) { + sum += x.values(k) * y.values(x.indices(k)) + k += 1 + } + sum + } + + /** + * dot(x, y) + */ + private def dot(x: SparseVector, y: SparseVector): Double = { + var kx = 0 + val nnzx = x.indices.size + var ky = 0 + val nnzy = y.indices.size + var sum = 0.0 + // y catching x + while (kx < nnzx && ky < nnzy) { + val ix = x.indices(kx) + while (ky < nnzy && y.indices(ky) < ix) { + ky += 1 + } + if (ky < nnzy && y.indices(ky) == ix) { + sum += x.values(kx) * y.values(ky) + ky += 1 + } + kx += 1 + } + sum + } + + /** + * y = x + */ + def copy(x: Vector, y: Vector): Unit = { + val n = y.size + require(x.size == n) + y match { + case dy: DenseVector => + x match { + case sx: SparseVector => + var i = 0 + var k = 0 + val nnz = sx.indices.size + while (k < nnz) { + val j = sx.indices(k) + while (i < j) { + dy.values(i) = 0.0 + i += 1 + } + dy.values(i) = sx.values(k) + i += 1 + k += 1 + } + while (i < n) { + dy.values(i) = 0.0 + i += 1 + } + case dx: DenseVector => + Array.copy(dx.values, 0, dy.values, 0, n) + } + case _ => + throw new IllegalArgumentException(s"y must be dense in copy but got ${y.getClass}") + } + } + + /** + * x = a * x + */ + def scal(a: Double, x: Vector): Unit = { + x match { + case sx: SparseVector => + f2jBLAS.dscal(sx.values.size, a, sx.values, 1) + case dx: DenseVector => + f2jBLAS.dscal(dx.values.size, a, dx.values, 1) + case _ => + throw new IllegalArgumentException(s"scal doesn't support vector type ${x.getClass}.") + } + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/EigenValueDecomposition.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/EigenValueDecomposition.scala new file mode 100644 index 0000000000000..3515461b52493 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/EigenValueDecomposition.scala @@ -0,0 +1,157 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.linalg + +import breeze.linalg.{DenseMatrix => BDM, DenseVector => BDV} +import com.github.fommil.netlib.ARPACK +import org.netlib.util.{intW, doubleW} + +import org.apache.spark.annotation.Experimental + +/** + * :: Experimental :: + * Compute eigen-decomposition. + */ +@Experimental +private[mllib] object EigenValueDecomposition { + /** + * Compute the leading k eigenvalues and eigenvectors on a symmetric square matrix using ARPACK. + * The caller needs to ensure that the input matrix is real symmetric. This function requires + * memory for `n*(4*k+4)` doubles. + * + * @param mul a function that multiplies the symmetric matrix with a DenseVector. + * @param n dimension of the square matrix (maximum Int.MaxValue). + * @param k number of leading eigenvalues required, 0 < k < n. + * @param tol tolerance of the eigs computation. + * @param maxIterations the maximum number of Arnoldi update iterations. + * @return a dense vector of eigenvalues in descending order and a dense matrix of eigenvectors + * (columns of the matrix). + * @note The number of computed eigenvalues might be smaller than k when some Ritz values do not + * satisfy the convergence criterion specified by tol (see ARPACK Users Guide, Chapter 4.6 + * for more details). The maximum number of Arnoldi update iterations is set to 300 in this + * function. + */ + private[mllib] def symmetricEigs( + mul: BDV[Double] => BDV[Double], + n: Int, + k: Int, + tol: Double, + maxIterations: Int): (BDV[Double], BDM[Double]) = { + // TODO: remove this function and use eigs in breeze when switching breeze version + require(n > k, s"Number of required eigenvalues $k must be smaller than matrix dimension $n") + + val arpack = ARPACK.getInstance() + + // tolerance used in stopping criterion + val tolW = new doubleW(tol) + // number of desired eigenvalues, 0 < nev < n + val nev = new intW(k) + // nev Lanczos vectors are generated in the first iteration + // ncv-nev Lanczos vectors are generated in each subsequent iteration + // ncv must be smaller than n + val ncv = math.min(2 * k, n) + + // "I" for standard eigenvalue problem, "G" for generalized eigenvalue problem + val bmat = "I" + // "LM" : compute the NEV largest (in magnitude) eigenvalues + val which = "LM" + + var iparam = new Array[Int](11) + // use exact shift in each iteration + iparam(0) = 1 + // maximum number of Arnoldi update iterations, or the actual number of iterations on output + iparam(2) = maxIterations + // Mode 1: A*x = lambda*x, A symmetric + iparam(6) = 1 + + var ido = new intW(0) + var info = new intW(0) + var resid = new Array[Double](n) + var v = new Array[Double](n * ncv) + var workd = new Array[Double](n * 3) + var workl = new Array[Double](ncv * (ncv + 8)) + var ipntr = new Array[Int](11) + + // call ARPACK's reverse communication, first iteration with ido = 0 + arpack.dsaupd(ido, bmat, n, which, nev.`val`, tolW, resid, ncv, v, n, iparam, ipntr, workd, + workl, workl.length, info) + + val w = BDV(workd) + + // ido = 99 : done flag in reverse communication + while (ido.`val` != 99) { + if (ido.`val` != -1 && ido.`val` != 1) { + throw new IllegalStateException("ARPACK returns ido = " + ido.`val` + + " This flag is not compatible with Mode 1: A*x = lambda*x, A symmetric.") + } + // multiply working vector with the matrix + val inputOffset = ipntr(0) - 1 + val outputOffset = ipntr(1) - 1 + val x = w.slice(inputOffset, inputOffset + n) + val y = w.slice(outputOffset, outputOffset + n) + y := mul(x) + // call ARPACK's reverse communication + arpack.dsaupd(ido, bmat, n, which, nev.`val`, tolW, resid, ncv, v, n, iparam, ipntr, + workd, workl, workl.length, info) + } + + if (info.`val` != 0) { + info.`val` match { + case 1 => throw new IllegalStateException("ARPACK returns non-zero info = " + info.`val` + + " Maximum number of iterations taken. (Refer ARPACK user guide for details)") + case 2 => throw new IllegalStateException("ARPACK returns non-zero info = " + info.`val` + + " No shifts could be applied. Try to increase NCV. " + + "(Refer ARPACK user guide for details)") + case _ => throw new IllegalStateException("ARPACK returns non-zero info = " + info.`val` + + " Please refer ARPACK user guide for error message.") + } + } + + val d = new Array[Double](nev.`val`) + val select = new Array[Boolean](ncv) + // copy the Ritz vectors + val z = java.util.Arrays.copyOfRange(v, 0, nev.`val` * n) + + // call ARPACK's post-processing for eigenvectors + arpack.dseupd(true, "A", select, d, z, n, 0.0, bmat, n, which, nev, tol, resid, ncv, v, n, + iparam, ipntr, workd, workl, workl.length, info) + + // number of computed eigenvalues, might be smaller than k + val computed = iparam(4) + + val eigenPairs = java.util.Arrays.copyOfRange(d, 0, computed).zipWithIndex.map { r => + (r._1, java.util.Arrays.copyOfRange(z, r._2 * n, r._2 * n + n)) + } + + // sort the eigen-pairs in descending order + val sortedEigenPairs = eigenPairs.sortBy(- _._1) + + // copy eigenvectors in descending order of eigenvalues + val sortedU = BDM.zeros[Double](n, computed) + sortedEigenPairs.zipWithIndex.foreach { r => + val b = r._2 * n + var i = 0 + while (i < n) { + sortedU.data(b + i) = r._1._2(i) + i += 1 + } + } + + (BDV[Double](sortedEigenPairs.map(_._1)), sortedU) + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala index c818a0b9c3e43..a45781d12e41e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala @@ -18,7 +18,7 @@ package org.apache.spark.mllib.linalg import java.lang.{Double => JavaDouble, Integer => JavaInteger, Iterable => JavaIterable} -import java.util.Arrays +import java.util import scala.annotation.varargs import scala.collection.JavaConverters._ @@ -30,6 +30,8 @@ import org.apache.spark.SparkException /** * Represents a numeric vector, whose index type is Int and value type is Double. + * + * Note: Users should not implement this interface. */ trait Vector extends Serializable { @@ -46,12 +48,12 @@ trait Vector extends Serializable { override def equals(other: Any): Boolean = { other match { case v: Vector => - Arrays.equals(this.toArray, v.toArray) + util.Arrays.equals(this.toArray, v.toArray) case _ => false } } - override def hashCode(): Int = Arrays.hashCode(this.toArray) + override def hashCode(): Int = util.Arrays.hashCode(this.toArray) /** * Converts the instance to a breeze vector. @@ -62,7 +64,14 @@ trait Vector extends Serializable { * Gets the value of the ith element. * @param i index */ - private[mllib] def apply(i: Int): Double = toBreeze(i) + def apply(i: Int): Double = toBreeze(i) + + /** + * Makes a deep copy of this vector. + */ + def copy: Vector = { + throw new NotImplementedError(s"copy is not implemented for ${this.getClass}.") + } } /** @@ -127,6 +136,16 @@ object Vectors { }.toSeq) } + /** + * Creates a dense vector of all zeros. + * + * @param size vector size + * @return a zero vector + */ + def zeros(size: Int): Vector = { + new DenseVector(new Array[Double](size)) + } + /** * Parses a string resulted from `Vector#toString` into * an [[org.apache.spark.mllib.linalg.Vector]]. @@ -142,7 +161,7 @@ object Vectors { case Seq(size: Double, indices: Array[Double], values: Array[Double]) => Vectors.sparse(size.toInt, indices.map(_.toInt), values) case other => - throw new SparkException(s"Cannot parse $other.") + throw new SparkException(s"Cannot parse $other.") } } @@ -183,6 +202,10 @@ class DenseVector(val values: Array[Double]) extends Vector { private[mllib] override def toBreeze: BV[Double] = new BDV[Double](values) override def apply(i: Int) = values(i) + + override def copy: DenseVector = { + new DenseVector(values.clone()) + } } /** @@ -213,5 +236,9 @@ class SparseVector( data } + override def copy: SparseVector = { + new SparseVector(size, indices.clone(), values.clone()) + } + private[mllib] override def toBreeze: BV[Double] = new BSV[Double](indices, values, size) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala index 695e03b736baf..e76bc9fefff01 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala @@ -17,9 +17,10 @@ package org.apache.spark.mllib.linalg.distributed -import java.util +import java.util.Arrays -import breeze.linalg.{Vector => BV, DenseMatrix => BDM, DenseVector => BDV, svd => brzSvd} +import breeze.linalg.{DenseMatrix => BDM, DenseVector => BDV, SparseVector => BSV} +import breeze.linalg.{svd => brzSvd, axpy => brzAxpy} import breeze.numerics.{sqrt => brzSqrt} import com.github.fommil.netlib.BLAS.{getInstance => blas} @@ -27,138 +28,8 @@ import org.apache.spark.annotation.Experimental import org.apache.spark.mllib.linalg._ import org.apache.spark.rdd.RDD import org.apache.spark.Logging -import org.apache.spark.mllib.stat.MultivariateStatisticalSummary - -/** - * Column statistics aggregator implementing - * [[org.apache.spark.mllib.stat.MultivariateStatisticalSummary]] - * together with add() and merge() function. - * A numerically stable algorithm is implemented to compute sample mean and variance: - *[[http://en.wikipedia.org/wiki/Algorithms_for_calculating_variance variance-wiki]]. - * Zero elements (including explicit zero values) are skipped when calling add() and merge(), - * to have time complexity O(nnz) instead of O(n) for each column. - */ -private class ColumnStatisticsAggregator(private val n: Int) - extends MultivariateStatisticalSummary with Serializable { - - private val currMean: BDV[Double] = BDV.zeros[Double](n) - private val currM2n: BDV[Double] = BDV.zeros[Double](n) - private var totalCnt = 0.0 - private val nnz: BDV[Double] = BDV.zeros[Double](n) - private val currMax: BDV[Double] = BDV.fill(n)(Double.MinValue) - private val currMin: BDV[Double] = BDV.fill(n)(Double.MaxValue) - - override def mean: Vector = { - val realMean = BDV.zeros[Double](n) - var i = 0 - while (i < n) { - realMean(i) = currMean(i) * nnz(i) / totalCnt - i += 1 - } - Vectors.fromBreeze(realMean) - } - - override def variance: Vector = { - val realVariance = BDV.zeros[Double](n) - - val denominator = totalCnt - 1.0 - - // Sample variance is computed, if the denominator is less than 0, the variance is just 0. - if (denominator > 0.0) { - val deltaMean = currMean - var i = 0 - while (i < currM2n.size) { - realVariance(i) = - currM2n(i) + deltaMean(i) * deltaMean(i) * nnz(i) * (totalCnt - nnz(i)) / totalCnt - realVariance(i) /= denominator - i += 1 - } - } - - Vectors.fromBreeze(realVariance) - } - - override def count: Long = totalCnt.toLong - - override def numNonzeros: Vector = Vectors.fromBreeze(nnz) - - override def max: Vector = { - var i = 0 - while (i < n) { - if ((nnz(i) < totalCnt) && (currMax(i) < 0.0)) currMax(i) = 0.0 - i += 1 - } - Vectors.fromBreeze(currMax) - } - - override def min: Vector = { - var i = 0 - while (i < n) { - if ((nnz(i) < totalCnt) && (currMin(i) > 0.0)) currMin(i) = 0.0 - i += 1 - } - Vectors.fromBreeze(currMin) - } - - /** - * Aggregates a row. - */ - def add(currData: BV[Double]): this.type = { - currData.activeIterator.foreach { - case (_, 0.0) => // Skip explicit zero elements. - case (i, value) => - if (currMax(i) < value) { - currMax(i) = value - } - if (currMin(i) > value) { - currMin(i) = value - } - - val tmpPrevMean = currMean(i) - currMean(i) = (currMean(i) * nnz(i) + value) / (nnz(i) + 1.0) - currM2n(i) += (value - currMean(i)) * (value - tmpPrevMean) - - nnz(i) += 1.0 - } - - totalCnt += 1.0 - this - } - - /** - * Merges another aggregator. - */ - def merge(other: ColumnStatisticsAggregator): this.type = { - require(n == other.n, s"Dimensions mismatch. Expecting $n but got ${other.n}.") - - totalCnt += other.totalCnt - val deltaMean = currMean - other.currMean - - var i = 0 - while (i < n) { - // merge mean together - if (other.currMean(i) != 0.0) { - currMean(i) = (currMean(i) * nnz(i) + other.currMean(i) * other.nnz(i)) / - (nnz(i) + other.nnz(i)) - } - // merge m2n together - if (nnz(i) + other.nnz(i) != 0.0) { - currM2n(i) += other.currM2n(i) + deltaMean(i) * deltaMean(i) * nnz(i) * other.nnz(i) / - (nnz(i) + other.nnz(i)) - } - if (currMax(i) < other.currMax(i)) { - currMax(i) = other.currMax(i) - } - if (currMin(i) > other.currMin(i)) { - currMin(i) = other.currMin(i) - } - i += 1 - } - - nnz += other.nnz - this - } -} +import org.apache.spark.mllib.rdd.RDDFunctions._ +import org.apache.spark.mllib.stat.{MultivariateOnlineSummarizer, MultivariateStatisticalSummary} /** * :: Experimental :: @@ -200,6 +71,30 @@ class RowMatrix( nRows } + /** + * Multiplies the Gramian matrix `A^T A` by a dense vector on the right without computing `A^T A`. + * + * @param v a dense vector whose length must match the number of columns of this matrix + * @return a dense vector representing the product + */ + private[mllib] def multiplyGramianMatrixBy(v: BDV[Double]): BDV[Double] = { + val n = numCols().toInt + val vbr = rows.context.broadcast(v) + rows.treeAggregate(BDV.zeros[Double](n))( + seqOp = (U, r) => { + val rBrz = r.toBreeze + val a = rBrz.dot(vbr.value) + rBrz match { + // use specialized axpy for better performance + case _: BDV[_] => brzAxpy(a, rBrz.asInstanceOf[BDV[Double]], U) + case _: BSV[_] => brzAxpy(a, rBrz.asInstanceOf[BSV[Double]], U) + case _ => throw new UnsupportedOperationException( + s"Do not support vector operation from type ${rBrz.getClass.getName}.") + } + U + }, combOp = (U1, U2) => U1 += U2) + } + /** * Computes the Gramian matrix `A^T A`. */ @@ -208,62 +103,145 @@ class RowMatrix( val nt: Int = n * (n + 1) / 2 // Compute the upper triangular part of the gram matrix. - val GU = rows.aggregate(new BDV[Double](new Array[Double](nt)))( + val GU = rows.treeAggregate(new BDV[Double](new Array[Double](nt)))( seqOp = (U, v) => { RowMatrix.dspr(1.0, v, U.data) U - }, - combOp = (U1, U2) => U1 += U2 - ) + }, combOp = (U1, U2) => U1 += U2) RowMatrix.triuToFull(n, GU.data) } /** - * Computes the singular value decomposition of this matrix. - * Denote this matrix by A (m x n), this will compute matrices U, S, V such that A = U * S * V'. + * Computes singular value decomposition of this matrix. Denote this matrix by A (m x n). This + * will compute matrices U, S, V such that A ~= U * S * V', where S contains the leading k + * singular values, U and V contain the corresponding singular vectors. * - * There is no restriction on m, but we require `n^2` doubles to fit in memory. - * Further, n should be less than m. - - * The decomposition is computed by first computing A'A = V S^2 V', - * computing svd locally on that (since n x n is small), from which we recover S and V. - * Then we compute U via easy matrix multiplication as U = A * (V * S^-1). - * Note that this approach requires `O(n^3)` time on the master node. + * At most k largest non-zero singular values and associated vectors are returned. If there are k + * such values, then the dimensions of the return will be: + * - U is a RowMatrix of size m x k that satisfies U' * U = eye(k), + * - s is a Vector of size k, holding the singular values in descending order, + * - V is a Matrix of size n x k that satisfies V' * V = eye(k). * - * At most k largest non-zero singular values and associated vectors are returned. - * If there are k such values, then the dimensions of the return will be: + * We assume n is smaller than m. The singular values and the right singular vectors are derived + * from the eigenvalues and the eigenvectors of the Gramian matrix A' * A. U, the matrix + * storing the right singular vectors, is computed via matrix multiplication as + * U = A * (V * S^-1^), if requested by user. The actual method to use is determined + * automatically based on the cost: + * - If n is small (n < 100) or k is large compared with n (k > n / 2), we compute the Gramian + * matrix first and then compute its top eigenvalues and eigenvectors locally on the driver. + * This requires a single pass with O(n^2^) storage on each executor and on the driver, and + * O(n^2^ k) time on the driver. + * - Otherwise, we compute (A' * A) * v in a distributive way and send it to ARPACK's DSAUPD to + * compute (A' * A)'s top eigenvalues and eigenvectors on the driver node. This requires O(k) + * passes, O(n) storage on each executor, and O(n k) storage on the driver. * - * U is a RowMatrix of size m x k that satisfies U'U = eye(k), - * s is a Vector of size k, holding the singular values in descending order, - * and V is a Matrix of size n x k that satisfies V'V = eye(k). + * Several internal parameters are set to default values. The reciprocal condition number rCond + * is set to 1e-9. All singular values smaller than rCond * sigma(0) are treated as zeros, where + * sigma(0) is the largest singular value. The maximum number of Arnoldi update iterations for + * ARPACK is set to 300 or k * 3, whichever is larger. The numerical tolerance for ARPACK's + * eigen-decomposition is set to 1e-10. * - * @param k number of singular values to keep. We might return less than k if there are - * numerically zero singular values. See rCond. + * @note The conditions that decide which method to use internally and the default parameters are + * subject to change. + * + * @param k number of leading singular values to keep (0 < k <= n). It might return less than k if + * there are numerically zero singular values or there are not enough Ritz values + * converged before the maximum number of Arnoldi update iterations is reached (in case + * that matrix A is ill-conditioned). * @param computeU whether to compute U * @param rCond the reciprocal condition number. All singular values smaller than rCond * sigma(0) * are treated as zero, where sigma(0) is the largest singular value. - * @return SingularValueDecomposition(U, s, V) + * @return SingularValueDecomposition(U, s, V). U = null if computeU = false. */ def computeSVD( k: Int, computeU: Boolean = false, rCond: Double = 1e-9): SingularValueDecomposition[RowMatrix, Matrix] = { + // maximum number of Arnoldi update iterations for invoking ARPACK + val maxIter = math.max(300, k * 3) + // numerical tolerance for invoking ARPACK + val tol = 1e-10 + computeSVD(k, computeU, rCond, maxIter, tol, "auto") + } + + /** + * The actual SVD implementation, visible for testing. + * + * @param k number of leading singular values to keep (0 < k <= n) + * @param computeU whether to compute U + * @param rCond the reciprocal condition number + * @param maxIter max number of iterations (if ARPACK is used) + * @param tol termination tolerance (if ARPACK is used) + * @param mode computation mode (auto: determine automatically which mode to use, + * local-svd: compute gram matrix and computes its full SVD locally, + * local-eigs: compute gram matrix and computes its top eigenvalues locally, + * dist-eigs: compute the top eigenvalues of the gram matrix distributively) + * @return SingularValueDecomposition(U, s, V). U = null if computeU = false. + */ + private[mllib] def computeSVD( + k: Int, + computeU: Boolean, + rCond: Double, + maxIter: Int, + tol: Double, + mode: String): SingularValueDecomposition[RowMatrix, Matrix] = { val n = numCols().toInt - require(k > 0 && k <= n, s"Request up to n singular values k=$k n=$n.") + require(k > 0 && k <= n, s"Request up to n singular values but got k=$k and n=$n.") + + object SVDMode extends Enumeration { + val LocalARPACK, LocalLAPACK, DistARPACK = Value + } + + val computeMode = mode match { + case "auto" => + // TODO: The conditions below are not fully tested. + if (n < 100 || k > n / 2) { + // If n is small or k is large compared with n, we better compute the Gramian matrix first + // and then compute its eigenvalues locally, instead of making multiple passes. + if (k < n / 3) { + SVDMode.LocalARPACK + } else { + SVDMode.LocalLAPACK + } + } else { + // If k is small compared with n, we use ARPACK with distributed multiplication. + SVDMode.DistARPACK + } + case "local-svd" => SVDMode.LocalLAPACK + case "local-eigs" => SVDMode.LocalARPACK + case "dist-eigs" => SVDMode.DistARPACK + case _ => throw new IllegalArgumentException(s"Do not support mode $mode.") + } - val G = computeGramianMatrix() + // Compute the eigen-decomposition of A' * A. + val (sigmaSquares: BDV[Double], u: BDM[Double]) = computeMode match { + case SVDMode.LocalARPACK => + require(k < n, s"k must be smaller than n in local-eigs mode but got k=$k and n=$n.") + val G = computeGramianMatrix().toBreeze.asInstanceOf[BDM[Double]] + EigenValueDecomposition.symmetricEigs(v => G * v, n, k, tol, maxIter) + case SVDMode.LocalLAPACK => + val G = computeGramianMatrix().toBreeze.asInstanceOf[BDM[Double]] + val brzSvd.SVD(uFull: BDM[Double], sigmaSquaresFull: BDV[Double], _) = brzSvd(G) + (sigmaSquaresFull, uFull) + case SVDMode.DistARPACK => + require(k < n, s"k must be smaller than n in dist-eigs mode but got k=$k and n=$n.") + EigenValueDecomposition.symmetricEigs(multiplyGramianMatrixBy, n, k, tol, maxIter) + } - // TODO: Use sparse SVD instead. - val (u: BDM[Double], sigmaSquares: BDV[Double], v: BDM[Double]) = - brzSvd(G.toBreeze.asInstanceOf[BDM[Double]]) val sigmas: BDV[Double] = brzSqrt(sigmaSquares) - // Determine effective rank. + // Determine the effective rank. val sigma0 = sigmas(0) val threshold = rCond * sigma0 var i = 0 - while (i < k && sigmas(i) >= threshold) { + // sigmas might have a length smaller than k, if some Ritz values do not satisfy the convergence + // criterion specified by tol after max number of iterations. + // Thus use i < min(k, sigmas.length) instead of i < k. + if (sigmas.length < k) { + logWarning(s"Requested $k singular values but only found ${sigmas.length} converged.") + } + while (i < math.min(k, sigmas.length) && sigmas(i) >= threshold) { i += 1 } val sk = i @@ -272,12 +250,12 @@ class RowMatrix( logWarning(s"Requested $k singular values but only found $sk nonzeros.") } - val s = Vectors.dense(util.Arrays.copyOfRange(sigmas.data, 0, sk)) - val V = Matrices.dense(n, sk, util.Arrays.copyOfRange(u.data, 0, n * sk)) + val s = Vectors.dense(Arrays.copyOfRange(sigmas.data, 0, sk)) + val V = Matrices.dense(n, sk, Arrays.copyOfRange(u.data, 0, n * sk)) if (computeU) { // N = Vk * Sk^{-1} - val N = new BDM[Double](n, sk, util.Arrays.copyOfRange(u.data, 0, n * sk)) + val N = new BDM[Double](n, sk, Arrays.copyOfRange(u.data, 0, n * sk)) var i = 0 var j = 0 while (j < sk) { @@ -309,9 +287,10 @@ class RowMatrix( s"We need at least $mem bytes of memory.") } - val (m, mean) = rows.aggregate[(Long, BDV[Double])]((0L, BDV.zeros[Double](n)))( + val (m, mean) = rows.treeAggregate[(Long, BDV[Double])]((0L, BDV.zeros[Double](n)))( seqOp = (s: (Long, BDV[Double]), v: Vector) => (s._1 + 1L, s._2 += v.toBreeze), - combOp = (s1: (Long, BDV[Double]), s2: (Long, BDV[Double])) => (s1._1 + s2._1, s1._2 += s2._2) + combOp = (s1: (Long, BDV[Double]), s2: (Long, BDV[Double])) => + (s1._1 + s2._1, s1._2 += s2._2) ) updateNumRows(m) @@ -359,12 +338,12 @@ class RowMatrix( val Cov = computeCovariance().toBreeze.asInstanceOf[BDM[Double]] - val (u: BDM[Double], _, _) = brzSvd(Cov) + val brzSvd.SVD(u: BDM[Double], _, _) = brzSvd(Cov) if (k == n) { Matrices.dense(n, k, u.data) } else { - Matrices.dense(n, k, util.Arrays.copyOfRange(u.data, 0, n * k)) + Matrices.dense(n, k, Arrays.copyOfRange(u.data, 0, n * k)) } } @@ -372,11 +351,9 @@ class RowMatrix( * Computes column-wise summary statistics. */ def computeColumnSummaryStatistics(): MultivariateStatisticalSummary = { - val zeroValue = new ColumnStatisticsAggregator(numCols().toInt) - val summary = rows.map(_.toBreeze).aggregate[ColumnStatisticsAggregator](zeroValue)( + val summary = rows.treeAggregate(new MultivariateOnlineSummarizer)( (aggregator, data) => aggregator.add(data), - (aggregator1, aggregator2) => aggregator1.merge(aggregator2) - ) + (aggregator1, aggregator2) => aggregator1.merge(aggregator2)) updateNumRows(summary.count) summary } @@ -390,16 +367,25 @@ class RowMatrix( */ def multiply(B: Matrix): RowMatrix = { val n = numCols().toInt + val k = B.numCols require(n == B.numRows, s"Dimension mismatch: $n vs ${B.numRows}") require(B.isInstanceOf[DenseMatrix], s"Only support dense matrix at this time but found ${B.getClass.getName}.") - val Bb = rows.context.broadcast(B) - val AB = rows.mapPartitions({ iter => - val Bi = Bb.value.toBreeze.asInstanceOf[BDM[Double]] - iter.map(v => Vectors.fromBreeze(Bi.t * v.toBreeze)) - }, preservesPartitioning = true) + val Bb = rows.context.broadcast(B.toBreeze.asInstanceOf[BDM[Double]].toDenseVector.toArray) + val AB = rows.mapPartitions { iter => + val Bi = Bb.value + iter.map { row => + val v = BDV.zeros[Double](k) + var i = 0 + while (i < k) { + v(i) = row.toBreeze.dot(new BDV(Bi, i * n, 1, n)) + i += 1 + } + Vectors.fromBreeze(v) + } + } new RowMatrix(AB, nRows, B.numCols) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala index 679842f831c2a..fdd67160114ca 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala @@ -17,10 +17,9 @@ package org.apache.spark.mllib.optimization -import breeze.linalg.{axpy => brzAxpy} - import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.mllib.linalg.{Vectors, Vector} +import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.mllib.linalg.BLAS.{axpy, dot, scal} /** * :: DeveloperApi :: @@ -61,19 +60,18 @@ abstract class Gradient extends Serializable { @DeveloperApi class LogisticGradient extends Gradient { override def compute(data: Vector, label: Double, weights: Vector): (Vector, Double) = { - val brzData = data.toBreeze - val brzWeights = weights.toBreeze - val margin: Double = -1.0 * brzWeights.dot(brzData) + val margin = -1.0 * dot(data, weights) val gradientMultiplier = (1.0 / (1.0 + math.exp(margin))) - label - val gradient = brzData * gradientMultiplier + val gradient = data.copy + scal(gradientMultiplier, gradient) val loss = if (label > 0) { - math.log(1 + math.exp(margin)) + math.log1p(math.exp(margin)) // log1p is log(1+p) but more accurate for small p } else { - math.log(1 + math.exp(margin)) - margin + math.log1p(math.exp(margin)) - margin } - (Vectors.fromBreeze(gradient), loss) + (gradient, loss) } override def compute( @@ -81,17 +79,13 @@ class LogisticGradient extends Gradient { label: Double, weights: Vector, cumGradient: Vector): Double = { - val brzData = data.toBreeze - val brzWeights = weights.toBreeze - val margin: Double = -1.0 * brzWeights.dot(brzData) + val margin = -1.0 * dot(data, weights) val gradientMultiplier = (1.0 / (1.0 + math.exp(margin))) - label - - brzAxpy(gradientMultiplier, brzData, cumGradient.toBreeze) - + axpy(gradientMultiplier, data, cumGradient) if (label > 0) { - math.log(1 + math.exp(margin)) + math.log1p(math.exp(margin)) } else { - math.log(1 + math.exp(margin)) - margin + math.log1p(math.exp(margin)) - margin } } } @@ -106,13 +100,11 @@ class LogisticGradient extends Gradient { @DeveloperApi class LeastSquaresGradient extends Gradient { override def compute(data: Vector, label: Double, weights: Vector): (Vector, Double) = { - val brzData = data.toBreeze - val brzWeights = weights.toBreeze - val diff = brzWeights.dot(brzData) - label + val diff = dot(data, weights) - label val loss = diff * diff - val gradient = brzData * (2.0 * diff) - - (Vectors.fromBreeze(gradient), loss) + val gradient = data.copy + scal(2.0 * diff, gradient) + (gradient, loss) } override def compute( @@ -120,12 +112,8 @@ class LeastSquaresGradient extends Gradient { label: Double, weights: Vector, cumGradient: Vector): Double = { - val brzData = data.toBreeze - val brzWeights = weights.toBreeze - val diff = brzWeights.dot(brzData) - label - - brzAxpy(2.0 * diff, brzData, cumGradient.toBreeze) - + val diff = dot(data, weights) - label + axpy(2.0 * diff, data, cumGradient) diff * diff } } @@ -139,18 +127,16 @@ class LeastSquaresGradient extends Gradient { @DeveloperApi class HingeGradient extends Gradient { override def compute(data: Vector, label: Double, weights: Vector): (Vector, Double) = { - val brzData = data.toBreeze - val brzWeights = weights.toBreeze - val dotProduct = brzWeights.dot(brzData) - + val dotProduct = dot(data, weights) // Our loss function with {0, 1} labels is max(0, 1 - (2y – 1) (f_w(x))) // Therefore the gradient is -(2y - 1)*x val labelScaled = 2 * label - 1.0 - if (1.0 > labelScaled * dotProduct) { - (Vectors.fromBreeze(brzData * (-labelScaled)), 1.0 - labelScaled * dotProduct) + val gradient = data.copy + scal(-labelScaled, gradient) + (gradient, 1.0 - labelScaled * dotProduct) } else { - (Vectors.dense(new Array[Double](weights.size)), 0.0) + (Vectors.sparse(weights.size, Array.empty, Array.empty), 0.0) } } @@ -159,16 +145,12 @@ class HingeGradient extends Gradient { label: Double, weights: Vector, cumGradient: Vector): Double = { - val brzData = data.toBreeze - val brzWeights = weights.toBreeze - val dotProduct = brzWeights.dot(brzData) - + val dotProduct = dot(data, weights) // Our loss function with {0, 1} labels is max(0, 1 - (2y – 1) (f_w(x))) // Therefore the gradient is -(2y - 1)*x val labelScaled = 2 * label - 1.0 - if (1.0 > labelScaled * dotProduct) { - brzAxpy(-labelScaled, brzData, cumGradient.toBreeze) + axpy(-labelScaled, data, cumGradient) 1.0 - labelScaled * dotProduct } else { 0.0 diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala index 7030eeabe400a..a6912056395d7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala @@ -25,6 +25,7 @@ import org.apache.spark.annotation.{Experimental, DeveloperApi} import org.apache.spark.Logging import org.apache.spark.rdd.RDD import org.apache.spark.mllib.linalg.{Vectors, Vector} +import org.apache.spark.mllib.rdd.RDDFunctions._ /** * Class used to solve an optimization problem using Gradient Descent. @@ -161,8 +162,17 @@ object GradientDescent extends Logging { val numExamples = data.count() val miniBatchSize = numExamples * miniBatchFraction + // if no data, return initial weights to avoid NaNs + if (numExamples == 0) { + + logInfo("GradientDescent.runMiniBatchSGD returning initial weights, no data found") + return (initialWeights, stochasticLossHistory.toArray) + + } + // Initialize weights as a column vector var weights = Vectors.dense(initialWeights.toArray) + val n = weights.size /** * For the first iteration, the regVal will be initialized as sum of weight squares @@ -172,12 +182,13 @@ object GradientDescent extends Logging { weights, Vectors.dense(new Array[Double](weights.size)), 0, 1, regParam)._2 for (i <- 1 to numIterations) { + val bcWeights = data.context.broadcast(weights) // Sample a subset (fraction miniBatchFraction) of the total data // compute and sum up the subgradients on this subset (this is one map-reduce) val (gradientSum, lossSum) = data.sample(false, miniBatchFraction, 42 + i) - .aggregate((BDV.zeros[Double](weights.size), 0.0))( + .treeAggregate((BDV.zeros[Double](n), 0.0))( seqOp = (c, v) => (c, v) match { case ((grad, loss), (label, features)) => - val l = gradient.compute(features, label, weights, Vectors.fromBreeze(grad)) + val l = gradient.compute(features, label, bcWeights.value, Vectors.fromBreeze(grad)) (grad, loss + l) }, combOp = (c1, c2) => (c1, c2) match { case ((grad1, loss1), (grad2, loss2)) => @@ -199,5 +210,6 @@ object GradientDescent extends Logging { stochasticLossHistory.takeRight(10).mkString(", "))) (weights, stochasticLossHistory.toArray) + } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala index 7bbed9c8fdbef..d16d0daf08565 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala @@ -19,13 +19,15 @@ package org.apache.spark.mllib.optimization import scala.collection.mutable.ArrayBuffer -import breeze.linalg.{DenseVector => BDV, axpy} +import breeze.linalg.{DenseVector => BDV} import breeze.optimize.{CachedDiffFunction, DiffFunction, LBFGS => BreezeLBFGS} -import org.apache.spark.annotation.DeveloperApi import org.apache.spark.Logging +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.mllib.linalg.BLAS.axpy +import org.apache.spark.mllib.rdd.RDDFunctions._ import org.apache.spark.rdd.RDD -import org.apache.spark.mllib.linalg.{Vectors, Vector} /** * :: DeveloperApi :: @@ -67,8 +69,17 @@ class LBFGS(private var gradient: Gradient, private var updater: Updater) /** * Set the maximal number of iterations for L-BFGS. Default 100. + * @deprecated use [[LBFGS#setNumIterations]] instead */ + @deprecated("use setNumIterations instead", "1.1.0") def setMaxNumIterations(iters: Int): this.type = { + this.setNumIterations(iters) + } + + /** + * Set the maximal number of iterations for L-BFGS. Default 100. + */ + def setNumIterations(iters: Int): this.type = { this.maxNumIterations = iters this } @@ -191,30 +202,29 @@ object LBFGS extends Logging { regParam: Double, numExamples: Long) extends DiffFunction[BDV[Double]] { - private var i = 0 - - override def calculate(weights: BDV[Double]) = { + override def calculate(weights: BDV[Double]): (Double, BDV[Double]) = { // Have a local copy to avoid the serialization of CostFun object which is not serializable. - val localData = data + val w = Vectors.fromBreeze(weights) + val n = w.size + val bcW = data.context.broadcast(w) val localGradient = gradient - val (gradientSum, lossSum) = localData.aggregate((BDV.zeros[Double](weights.size), 0.0))( + val (gradientSum, lossSum) = data.treeAggregate((Vectors.zeros(n), 0.0))( seqOp = (c, v) => (c, v) match { case ((grad, loss), (label, features)) => val l = localGradient.compute( - features, label, Vectors.fromBreeze(weights), Vectors.fromBreeze(grad)) + features, label, bcW.value, grad) (grad, loss + l) }, combOp = (c1, c2) => (c1, c2) match { case ((grad1, loss1), (grad2, loss2)) => - (grad1 += grad2, loss1 + loss2) + axpy(1.0, grad2, grad1) + (grad1, loss1 + loss2) }) /** * regVal is sum of weight squares if it's L2 updater; * for other updater, the same logic is followed. */ - val regVal = updater.compute( - Vectors.fromBreeze(weights), - Vectors.dense(new Array[Double](weights.size)), 0, 1, regParam)._2 + val regVal = updater.compute(w, Vectors.zeros(n), 0, 1, regParam)._2 val loss = lossSum / numExamples + regVal /** @@ -234,17 +244,13 @@ object LBFGS extends Logging { */ // The following gradientTotal is actually the regularization part of gradient. // Will add the gradientSum computed from the data with weights in the next step. - val gradientTotal = weights - updater.compute( - Vectors.fromBreeze(weights), - Vectors.dense(new Array[Double](weights.size)), 1, 1, regParam)._1.toBreeze + val gradientTotal = w.copy + axpy(-1.0, updater.compute(w, Vectors.zeros(n), 1, 1, regParam)._1, gradientTotal) // gradientTotal = gradientSum / numExamples + gradientTotal axpy(1.0 / numExamples, gradientSum, gradientTotal) - i += 1 - - (loss, gradientTotal) + (loss, gradientTotal.toBreeze.asInstanceOf[BDV[Double]]) } } - } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/random/RandomDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/random/RandomDataGenerator.scala new file mode 100644 index 0000000000000..9cab49f6ed1f0 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/random/RandomDataGenerator.scala @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.random + +import cern.jet.random.Poisson +import cern.jet.random.engine.DRand + +import org.apache.spark.annotation.Experimental +import org.apache.spark.util.random.{XORShiftRandom, Pseudorandom} + +/** + * :: Experimental :: + * Trait for random data generators that generate i.i.d. data. + */ +@Experimental +trait RandomDataGenerator[T] extends Pseudorandom with Serializable { + + /** + * Returns an i.i.d. sample as a generic type from an underlying distribution. + */ + def nextValue(): T + + /** + * Returns a copy of the RandomDataGenerator with a new instance of the rng object used in the + * class when applicable for non-locking concurrent usage. + */ + def copy(): RandomDataGenerator[T] +} + +/** + * :: Experimental :: + * Generates i.i.d. samples from U[0.0, 1.0] + */ +@Experimental +class UniformGenerator extends RandomDataGenerator[Double] { + + // XORShiftRandom for better performance. Thread safety isn't necessary here. + private val random = new XORShiftRandom() + + override def nextValue(): Double = { + random.nextDouble() + } + + override def setSeed(seed: Long) = random.setSeed(seed) + + override def copy(): UniformGenerator = new UniformGenerator() +} + +/** + * :: Experimental :: + * Generates i.i.d. samples from the standard normal distribution. + */ +@Experimental +class StandardNormalGenerator extends RandomDataGenerator[Double] { + + // XORShiftRandom for better performance. Thread safety isn't necessary here. + private val random = new XORShiftRandom() + + override def nextValue(): Double = { + random.nextGaussian() + } + + override def setSeed(seed: Long) = random.setSeed(seed) + + override def copy(): StandardNormalGenerator = new StandardNormalGenerator() +} + +/** + * :: Experimental :: + * Generates i.i.d. samples from the Poisson distribution with the given mean. + * + * @param mean mean for the Poisson distribution. + */ +@Experimental +class PoissonGenerator(val mean: Double) extends RandomDataGenerator[Double] { + + private var rng = new Poisson(mean, new DRand) + + override def nextValue(): Double = rng.nextDouble() + + override def setSeed(seed: Long) { + rng = new Poisson(mean, new DRand(seed.toInt)) + } + + override def copy(): PoissonGenerator = new PoissonGenerator(mean) +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDs.scala b/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDs.scala new file mode 100644 index 0000000000000..36270369526cd --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDs.scala @@ -0,0 +1,493 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.random + +import scala.reflect.ClassTag + +import org.apache.spark.SparkContext +import org.apache.spark.annotation.Experimental +import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.rdd.{RandomVectorRDD, RandomRDD} +import org.apache.spark.rdd.RDD +import org.apache.spark.util.Utils + +/** + * :: Experimental :: + * Generator methods for creating RDDs comprised of i.i.d. samples from some distribution. + */ +@Experimental +object RandomRDDs { + + /** + * :: Experimental :: + * Generates an RDD comprised of i.i.d. samples from the uniform distribution on [0.0, 1.0]. + * + * To transform the distribution in the generated RDD from U[0.0, 1.0] to U[a, b], use + * `RandomRDDGenerators.uniformRDD(sc, n, p, seed).map(v => a + (b - a) * v)`. + * + * @param sc SparkContext used to create the RDD. + * @param size Size of the RDD. + * @param numPartitions Number of partitions in the RDD. + * @param seed Seed for the RNG that generates the seed for the generator in each partition. + * @return RDD[Double] comprised of i.i.d. samples ~ U[0.0, 1.0]. + */ + @Experimental + def uniformRDD(sc: SparkContext, size: Long, numPartitions: Int, seed: Long): RDD[Double] = { + val uniform = new UniformGenerator() + randomRDD(sc, uniform, size, numPartitions, seed) + } + + /** + * :: Experimental :: + * Generates an RDD comprised of i.i.d. samples from the uniform distribution on [0.0, 1.0]. + * + * To transform the distribution in the generated RDD from U[0.0, 1.0] to U[a, b], use + * `RandomRDDGenerators.uniformRDD(sc, n, p).map(v => a + (b - a) * v)`. + * + * @param sc SparkContext used to create the RDD. + * @param size Size of the RDD. + * @param numPartitions Number of partitions in the RDD. + * @return RDD[Double] comprised of i.i.d. samples ~ U[0.0, 1.0]. + */ + @Experimental + def uniformRDD(sc: SparkContext, size: Long, numPartitions: Int): RDD[Double] = { + uniformRDD(sc, size, numPartitions, Utils.random.nextLong) + } + + /** + * :: Experimental :: + * Generates an RDD comprised of i.i.d. samples from the uniform distribution on [0.0, 1.0]. + * sc.defaultParallelism used for the number of partitions in the RDD. + * + * To transform the distribution in the generated RDD from U[0.0, 1.0] to U[a, b], use + * `RandomRDDGenerators.uniformRDD(sc, n).map(v => a + (b - a) * v)`. + * + * @param sc SparkContext used to create the RDD. + * @param size Size of the RDD. + * @return RDD[Double] comprised of i.i.d. samples ~ U[0.0, 1.0]. + */ + @Experimental + def uniformRDD(sc: SparkContext, size: Long): RDD[Double] = { + uniformRDD(sc, size, sc.defaultParallelism, Utils.random.nextLong) + } + + /** + * :: Experimental :: + * Generates an RDD comprised of i.i.d. samples from the standard normal distribution. + * + * To transform the distribution in the generated RDD from standard normal to some other normal + * N(mean, sigma), use `RandomRDDGenerators.normalRDD(sc, n, p, seed).map(v => mean + sigma * v)`. + * + * @param sc SparkContext used to create the RDD. + * @param size Size of the RDD. + * @param numPartitions Number of partitions in the RDD. + * @param seed Seed for the RNG that generates the seed for the generator in each partition. + * @return RDD[Double] comprised of i.i.d. samples ~ N(0.0, 1.0). + */ + @Experimental + def normalRDD(sc: SparkContext, size: Long, numPartitions: Int, seed: Long): RDD[Double] = { + val normal = new StandardNormalGenerator() + randomRDD(sc, normal, size, numPartitions, seed) + } + + /** + * :: Experimental :: + * Generates an RDD comprised of i.i.d. samples from the standard normal distribution. + * + * To transform the distribution in the generated RDD from standard normal to some other normal + * N(mean, sigma), use `RandomRDDGenerators.normalRDD(sc, n, p).map(v => mean + sigma * v)`. + * + * @param sc SparkContext used to create the RDD. + * @param size Size of the RDD. + * @param numPartitions Number of partitions in the RDD. + * @return RDD[Double] comprised of i.i.d. samples ~ N(0.0, 1.0). + */ + @Experimental + def normalRDD(sc: SparkContext, size: Long, numPartitions: Int): RDD[Double] = { + normalRDD(sc, size, numPartitions, Utils.random.nextLong) + } + + /** + * :: Experimental :: + * Generates an RDD comprised of i.i.d. samples from the standard normal distribution. + * sc.defaultParallelism used for the number of partitions in the RDD. + * + * To transform the distribution in the generated RDD from standard normal to some other normal + * N(mean, sigma), use `RandomRDDGenerators.normalRDD(sc, n).map(v => mean + sigma * v)`. + * + * @param sc SparkContext used to create the RDD. + * @param size Size of the RDD. + * @return RDD[Double] comprised of i.i.d. samples ~ N(0.0, 1.0). + */ + @Experimental + def normalRDD(sc: SparkContext, size: Long): RDD[Double] = { + normalRDD(sc, size, sc.defaultParallelism, Utils.random.nextLong) + } + + /** + * :: Experimental :: + * Generates an RDD comprised of i.i.d. samples from the Poisson distribution with the input mean. + * + * @param sc SparkContext used to create the RDD. + * @param mean Mean, or lambda, for the Poisson distribution. + * @param size Size of the RDD. + * @param numPartitions Number of partitions in the RDD. + * @param seed Seed for the RNG that generates the seed for the generator in each partition. + * @return RDD[Double] comprised of i.i.d. samples ~ Pois(mean). + */ + @Experimental + def poissonRDD(sc: SparkContext, + mean: Double, + size: Long, + numPartitions: Int, + seed: Long): RDD[Double] = { + val poisson = new PoissonGenerator(mean) + randomRDD(sc, poisson, size, numPartitions, seed) + } + + /** + * :: Experimental :: + * Generates an RDD comprised of i.i.d. samples from the Poisson distribution with the input mean. + * + * @param sc SparkContext used to create the RDD. + * @param mean Mean, or lambda, for the Poisson distribution. + * @param size Size of the RDD. + * @param numPartitions Number of partitions in the RDD. + * @return RDD[Double] comprised of i.i.d. samples ~ Pois(mean). + */ + @Experimental + def poissonRDD(sc: SparkContext, mean: Double, size: Long, numPartitions: Int): RDD[Double] = { + poissonRDD(sc, mean, size, numPartitions, Utils.random.nextLong) + } + + /** + * :: Experimental :: + * Generates an RDD comprised of i.i.d. samples from the Poisson distribution with the input mean. + * sc.defaultParallelism used for the number of partitions in the RDD. + * + * @param sc SparkContext used to create the RDD. + * @param mean Mean, or lambda, for the Poisson distribution. + * @param size Size of the RDD. + * @return RDD[Double] comprised of i.i.d. samples ~ Pois(mean). + */ + @Experimental + def poissonRDD(sc: SparkContext, mean: Double, size: Long): RDD[Double] = { + poissonRDD(sc, mean, size, sc.defaultParallelism, Utils.random.nextLong) + } + + /** + * :: Experimental :: + * Generates an RDD comprised of i.i.d. samples produced by the input DistributionGenerator. + * + * @param sc SparkContext used to create the RDD. + * @param generator DistributionGenerator used to populate the RDD. + * @param size Size of the RDD. + * @param numPartitions Number of partitions in the RDD. + * @param seed Seed for the RNG that generates the seed for the generator in each partition. + * @return RDD[Double] comprised of i.i.d. samples produced by generator. + */ + @Experimental + def randomRDD[T: ClassTag](sc: SparkContext, + generator: RandomDataGenerator[T], + size: Long, + numPartitions: Int, + seed: Long): RDD[T] = { + new RandomRDD[T](sc, size, numPartitions, generator, seed) + } + + /** + * :: Experimental :: + * Generates an RDD comprised of i.i.d. samples produced by the input DistributionGenerator. + * + * @param sc SparkContext used to create the RDD. + * @param generator DistributionGenerator used to populate the RDD. + * @param size Size of the RDD. + * @param numPartitions Number of partitions in the RDD. + * @return RDD[Double] comprised of i.i.d. samples produced by generator. + */ + @Experimental + def randomRDD[T: ClassTag](sc: SparkContext, + generator: RandomDataGenerator[T], + size: Long, + numPartitions: Int): RDD[T] = { + randomRDD[T](sc, generator, size, numPartitions, Utils.random.nextLong) + } + + /** + * :: Experimental :: + * Generates an RDD comprised of i.i.d. samples produced by the input DistributionGenerator. + * sc.defaultParallelism used for the number of partitions in the RDD. + * + * @param sc SparkContext used to create the RDD. + * @param generator DistributionGenerator used to populate the RDD. + * @param size Size of the RDD. + * @return RDD[Double] comprised of i.i.d. samples produced by generator. + */ + @Experimental + def randomRDD[T: ClassTag](sc: SparkContext, + generator: RandomDataGenerator[T], + size: Long): RDD[T] = { + randomRDD[T](sc, generator, size, sc.defaultParallelism, Utils.random.nextLong) + } + + // TODO Generate RDD[Vector] from multivariate distributions. + + /** + * :: Experimental :: + * Generates an RDD[Vector] with vectors containing i.i.d. samples drawn from the + * uniform distribution on [0.0 1.0]. + * + * @param sc SparkContext used to create the RDD. + * @param numRows Number of Vectors in the RDD. + * @param numCols Number of elements in each Vector. + * @param numPartitions Number of partitions in the RDD. + * @param seed Seed for the RNG that generates the seed for the generator in each partition. + * @return RDD[Vector] with vectors containing i.i.d samples ~ U[0.0, 1.0]. + */ + @Experimental + def uniformVectorRDD(sc: SparkContext, + numRows: Long, + numCols: Int, + numPartitions: Int, + seed: Long): RDD[Vector] = { + val uniform = new UniformGenerator() + randomVectorRDD(sc, uniform, numRows, numCols, numPartitions, seed) + } + + /** + * :: Experimental :: + * Generates an RDD[Vector] with vectors containing i.i.d. samples drawn from the + * uniform distribution on [0.0 1.0]. + * + * @param sc SparkContext used to create the RDD. + * @param numRows Number of Vectors in the RDD. + * @param numCols Number of elements in each Vector. + * @param numPartitions Number of partitions in the RDD. + * @return RDD[Vector] with vectors containing i.i.d. samples ~ U[0.0, 1.0]. + */ + @Experimental + def uniformVectorRDD(sc: SparkContext, + numRows: Long, + numCols: Int, + numPartitions: Int): RDD[Vector] = { + uniformVectorRDD(sc, numRows, numCols, numPartitions, Utils.random.nextLong) + } + + /** + * :: Experimental :: + * Generates an RDD[Vector] with vectors containing i.i.d. samples drawn from the + * uniform distribution on [0.0 1.0]. + * sc.defaultParallelism used for the number of partitions in the RDD. + * + * @param sc SparkContext used to create the RDD. + * @param numRows Number of Vectors in the RDD. + * @param numCols Number of elements in each Vector. + * @return RDD[Vector] with vectors containing i.i.d. samples ~ U[0.0, 1.0]. + */ + @Experimental + def uniformVectorRDD(sc: SparkContext, numRows: Long, numCols: Int): RDD[Vector] = { + uniformVectorRDD(sc, numRows, numCols, sc.defaultParallelism, Utils.random.nextLong) + } + + /** + * :: Experimental :: + * Generates an RDD[Vector] with vectors containing i.i.d. samples drawn from the + * standard normal distribution. + * + * @param sc SparkContext used to create the RDD. + * @param numRows Number of Vectors in the RDD. + * @param numCols Number of elements in each Vector. + * @param numPartitions Number of partitions in the RDD. + * @param seed Seed for the RNG that generates the seed for the generator in each partition. + * @return RDD[Vector] with vectors containing i.i.d. samples ~ N(0.0, 1.0). + */ + @Experimental + def normalVectorRDD(sc: SparkContext, + numRows: Long, + numCols: Int, + numPartitions: Int, + seed: Long): RDD[Vector] = { + val uniform = new StandardNormalGenerator() + randomVectorRDD(sc, uniform, numRows, numCols, numPartitions, seed) + } + + /** + * :: Experimental :: + * Generates an RDD[Vector] with vectors containing i.i.d. samples drawn from the + * standard normal distribution. + * + * @param sc SparkContext used to create the RDD. + * @param numRows Number of Vectors in the RDD. + * @param numCols Number of elements in each Vector. + * @param numPartitions Number of partitions in the RDD. + * @return RDD[Vector] with vectors containing i.i.d. samples ~ N(0.0, 1.0). + */ + @Experimental + def normalVectorRDD(sc: SparkContext, + numRows: Long, + numCols: Int, + numPartitions: Int): RDD[Vector] = { + normalVectorRDD(sc, numRows, numCols, numPartitions, Utils.random.nextLong) + } + + /** + * :: Experimental :: + * Generates an RDD[Vector] with vectors containing i.i.d. samples drawn from the + * standard normal distribution. + * sc.defaultParallelism used for the number of partitions in the RDD. + * + * @param sc SparkContext used to create the RDD. + * @param numRows Number of Vectors in the RDD. + * @param numCols Number of elements in each Vector. + * @return RDD[Vector] with vectors containing i.i.d. samples ~ N(0.0, 1.0). + */ + @Experimental + def normalVectorRDD(sc: SparkContext, numRows: Long, numCols: Int): RDD[Vector] = { + normalVectorRDD(sc, numRows, numCols, sc.defaultParallelism, Utils.random.nextLong) + } + + /** + * :: Experimental :: + * Generates an RDD[Vector] with vectors containing i.i.d. samples drawn from the + * Poisson distribution with the input mean. + * + * @param sc SparkContext used to create the RDD. + * @param mean Mean, or lambda, for the Poisson distribution. + * @param numRows Number of Vectors in the RDD. + * @param numCols Number of elements in each Vector. + * @param numPartitions Number of partitions in the RDD. + * @param seed Seed for the RNG that generates the seed for the generator in each partition. + * @return RDD[Vector] with vectors containing i.i.d. samples ~ Pois(mean). + */ + @Experimental + def poissonVectorRDD(sc: SparkContext, + mean: Double, + numRows: Long, + numCols: Int, + numPartitions: Int, + seed: Long): RDD[Vector] = { + val poisson = new PoissonGenerator(mean) + randomVectorRDD(sc, poisson, numRows, numCols, numPartitions, seed) + } + + /** + * :: Experimental :: + * Generates an RDD[Vector] with vectors containing i.i.d. samples drawn from the + * Poisson distribution with the input mean. + * + * @param sc SparkContext used to create the RDD. + * @param mean Mean, or lambda, for the Poisson distribution. + * @param numRows Number of Vectors in the RDD. + * @param numCols Number of elements in each Vector. + * @param numPartitions Number of partitions in the RDD. + * @return RDD[Vector] with vectors containing i.i.d. samples ~ Pois(mean). + */ + @Experimental + def poissonVectorRDD(sc: SparkContext, + mean: Double, + numRows: Long, + numCols: Int, + numPartitions: Int): RDD[Vector] = { + poissonVectorRDD(sc, mean, numRows, numCols, numPartitions, Utils.random.nextLong) + } + + /** + * :: Experimental :: + * Generates an RDD[Vector] with vectors containing i.i.d. samples drawn from the + * Poisson distribution with the input mean. + * sc.defaultParallelism used for the number of partitions in the RDD. + * + * @param sc SparkContext used to create the RDD. + * @param mean Mean, or lambda, for the Poisson distribution. + * @param numRows Number of Vectors in the RDD. + * @param numCols Number of elements in each Vector. + * @return RDD[Vector] with vectors containing i.i.d. samples ~ Pois(mean). + */ + @Experimental + def poissonVectorRDD(sc: SparkContext, + mean: Double, + numRows: Long, + numCols: Int): RDD[Vector] = { + poissonVectorRDD(sc, mean, numRows, numCols, sc.defaultParallelism, Utils.random.nextLong) + } + + /** + * :: Experimental :: + * Generates an RDD[Vector] with vectors containing i.i.d. samples produced by the + * input DistributionGenerator. + * + * @param sc SparkContext used to create the RDD. + * @param generator DistributionGenerator used to populate the RDD. + * @param numRows Number of Vectors in the RDD. + * @param numCols Number of elements in each Vector. + * @param numPartitions Number of partitions in the RDD. + * @param seed Seed for the RNG that generates the seed for the generator in each partition. + * @return RDD[Vector] with vectors containing i.i.d. samples produced by generator. + */ + @Experimental + def randomVectorRDD(sc: SparkContext, + generator: RandomDataGenerator[Double], + numRows: Long, + numCols: Int, + numPartitions: Int, + seed: Long): RDD[Vector] = { + new RandomVectorRDD(sc, numRows, numCols, numPartitions, generator, seed) + } + + /** + * :: Experimental :: + * Generates an RDD[Vector] with vectors containing i.i.d. samples produced by the + * input DistributionGenerator. + * + * @param sc SparkContext used to create the RDD. + * @param generator DistributionGenerator used to populate the RDD. + * @param numRows Number of Vectors in the RDD. + * @param numCols Number of elements in each Vector. + * @param numPartitions Number of partitions in the RDD. + * @return RDD[Vector] with vectors containing i.i.d. samples produced by generator. + */ + @Experimental + def randomVectorRDD(sc: SparkContext, + generator: RandomDataGenerator[Double], + numRows: Long, + numCols: Int, + numPartitions: Int): RDD[Vector] = { + randomVectorRDD(sc, generator, numRows, numCols, numPartitions, Utils.random.nextLong) + } + + /** + * :: Experimental :: + * Generates an RDD[Vector] with vectors containing i.i.d. samples produced by the + * input DistributionGenerator. + * sc.defaultParallelism used for the number of partitions in the RDD. + * + * @param sc SparkContext used to create the RDD. + * @param generator DistributionGenerator used to populate the RDD. + * @param numRows Number of Vectors in the RDD. + * @param numCols Number of elements in each Vector. + * @return RDD[Vector] with vectors containing i.i.d. samples produced by generator. + */ + @Experimental + def randomVectorRDD(sc: SparkContext, + generator: RandomDataGenerator[Double], + numRows: Long, + numCols: Int): RDD[Vector] = { + randomVectorRDD(sc, generator, numRows, numCols, + sc.defaultParallelism, Utils.random.nextLong) + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/rdd/RDDFunctions.scala b/mllib/src/main/scala/org/apache/spark/mllib/rdd/RDDFunctions.scala index 365b5e75d7f75..b5e403bc8c14d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/rdd/RDDFunctions.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/rdd/RDDFunctions.scala @@ -20,7 +20,10 @@ package org.apache.spark.mllib.rdd import scala.language.implicitConversions import scala.reflect.ClassTag +import org.apache.spark.HashPartitioner +import org.apache.spark.SparkContext._ import org.apache.spark.rdd.RDD +import org.apache.spark.util.Utils /** * Machine learning specific RDD functions. @@ -44,6 +47,69 @@ class RDDFunctions[T: ClassTag](self: RDD[T]) { new SlidingRDD[T](self, windowSize) } } + + /** + * Reduces the elements of this RDD in a multi-level tree pattern. + * + * @param depth suggested depth of the tree (default: 2) + * @see [[org.apache.spark.rdd.RDD#reduce]] + */ + def treeReduce(f: (T, T) => T, depth: Int = 2): T = { + require(depth >= 1, s"Depth must be greater than or equal to 1 but got $depth.") + val cleanF = self.context.clean(f) + val reducePartition: Iterator[T] => Option[T] = iter => { + if (iter.hasNext) { + Some(iter.reduceLeft(cleanF)) + } else { + None + } + } + val partiallyReduced = self.mapPartitions(it => Iterator(reducePartition(it))) + val op: (Option[T], Option[T]) => Option[T] = (c, x) => { + if (c.isDefined && x.isDefined) { + Some(cleanF(c.get, x.get)) + } else if (c.isDefined) { + c + } else if (x.isDefined) { + x + } else { + None + } + } + RDDFunctions.fromRDD(partiallyReduced).treeAggregate(Option.empty[T])(op, op, depth) + .getOrElse(throw new UnsupportedOperationException("empty collection")) + } + + /** + * Aggregates the elements of this RDD in a multi-level tree pattern. + * + * @param depth suggested depth of the tree (default: 2) + * @see [[org.apache.spark.rdd.RDD#aggregate]] + */ + def treeAggregate[U: ClassTag](zeroValue: U)( + seqOp: (U, T) => U, + combOp: (U, U) => U, + depth: Int = 2): U = { + require(depth >= 1, s"Depth must be greater than or equal to 1 but got $depth.") + if (self.partitions.size == 0) { + return Utils.clone(zeroValue, self.context.env.closureSerializer.newInstance()) + } + val cleanSeqOp = self.context.clean(seqOp) + val cleanCombOp = self.context.clean(combOp) + val aggregatePartition = (it: Iterator[T]) => it.aggregate(zeroValue)(cleanSeqOp, cleanCombOp) + var partiallyAggregated = self.mapPartitions(it => Iterator(aggregatePartition(it))) + var numPartitions = partiallyAggregated.partitions.size + val scale = math.max(math.ceil(math.pow(numPartitions, 1.0 / depth)).toInt, 2) + // If creating an extra level doesn't help reduce the wall-clock time, we stop tree aggregation. + while (numPartitions > scale + numPartitions / scale) { + numPartitions /= scale + val curNumPartitions = numPartitions + partiallyAggregated = partiallyAggregated.mapPartitionsWithIndex { (i, iter) => + iter.map((i % curNumPartitions, _)) + }.reduceByKey(new HashPartitioner(curNumPartitions), cleanCombOp).values + } + partiallyAggregated.reduce(cleanCombOp) + } } private[mllib] diff --git a/mllib/src/main/scala/org/apache/spark/mllib/rdd/RandomRDD.scala b/mllib/src/main/scala/org/apache/spark/mllib/rdd/RandomRDD.scala new file mode 100644 index 0000000000000..910eff9540a47 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/rdd/RandomRDD.scala @@ -0,0 +1,120 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.rdd + +import org.apache.spark.{Partition, SparkContext, TaskContext} +import org.apache.spark.mllib.linalg.{DenseVector, Vector} +import org.apache.spark.mllib.random.RandomDataGenerator +import org.apache.spark.rdd.RDD +import org.apache.spark.util.Utils + +import scala.reflect.ClassTag +import scala.util.Random + +private[mllib] class RandomRDDPartition[T](override val index: Int, + val size: Int, + val generator: RandomDataGenerator[T], + val seed: Long) extends Partition { + + require(size >= 0, "Non-negative partition size required.") +} + +// These two classes are necessary since Range objects in Scala cannot have size > Int.MaxValue +private[mllib] class RandomRDD[T: ClassTag](@transient sc: SparkContext, + size: Long, + numPartitions: Int, + @transient rng: RandomDataGenerator[T], + @transient seed: Long = Utils.random.nextLong) extends RDD[T](sc, Nil) { + + require(size > 0, "Positive RDD size required.") + require(numPartitions > 0, "Positive number of partitions required") + require(math.ceil(size.toDouble / numPartitions) <= Int.MaxValue, + "Partition size cannot exceed Int.MaxValue") + + override def compute(splitIn: Partition, context: TaskContext): Iterator[T] = { + val split = splitIn.asInstanceOf[RandomRDDPartition[T]] + RandomRDD.getPointIterator[T](split) + } + + override def getPartitions: Array[Partition] = { + RandomRDD.getPartitions(size, numPartitions, rng, seed) + } +} + +private[mllib] class RandomVectorRDD(@transient sc: SparkContext, + size: Long, + vectorSize: Int, + numPartitions: Int, + @transient rng: RandomDataGenerator[Double], + @transient seed: Long = Utils.random.nextLong) extends RDD[Vector](sc, Nil) { + + require(size > 0, "Positive RDD size required.") + require(numPartitions > 0, "Positive number of partitions required") + require(vectorSize > 0, "Positive vector size required.") + require(math.ceil(size.toDouble / numPartitions) <= Int.MaxValue, + "Partition size cannot exceed Int.MaxValue") + + override def compute(splitIn: Partition, context: TaskContext): Iterator[Vector] = { + val split = splitIn.asInstanceOf[RandomRDDPartition[Double]] + RandomRDD.getVectorIterator(split, vectorSize) + } + + override protected def getPartitions: Array[Partition] = { + RandomRDD.getPartitions(size, numPartitions, rng, seed) + } +} + +private[mllib] object RandomRDD { + + def getPartitions[T](size: Long, + numPartitions: Int, + rng: RandomDataGenerator[T], + seed: Long): Array[Partition] = { + + val partitions = new Array[RandomRDDPartition[T]](numPartitions) + var i = 0 + var start: Long = 0 + var end: Long = 0 + val random = new Random(seed) + while (i < numPartitions) { + end = ((i + 1) * size) / numPartitions + partitions(i) = new RandomRDDPartition(i, (end - start).toInt, rng, random.nextLong()) + start = end + i += 1 + } + partitions.asInstanceOf[Array[Partition]] + } + + // The RNG has to be reset every time the iterator is requested to guarantee same data + // every time the content of the RDD is examined. + def getPointIterator[T: ClassTag](partition: RandomRDDPartition[T]): Iterator[T] = { + val generator = partition.generator.copy() + generator.setSeed(partition.seed) + Iterator.fill(partition.size)(generator.nextValue()) + } + + // The RNG has to be reset every time the iterator is requested to guarantee same data + // every time the content of the RDD is examined. + def getVectorIterator( + partition: RandomRDDPartition[Double], + vectorSize: Int): Iterator[Vector] = { + val generator = partition.generator.copy() + generator.setSeed(partition.seed) + Iterator.fill(partition.size)(new DenseVector(Array.fill(vectorSize)(generator.nextValue()))) + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala index cc56fd6ef28d6..84d192db53e26 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala @@ -17,7 +17,8 @@ package org.apache.spark.mllib.recommendation -import scala.collection.mutable.{ArrayBuffer, BitSet} +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer import scala.math.{abs, sqrt} import scala.util.Random import scala.util.Sorting @@ -25,7 +26,7 @@ import scala.util.hashing.byteswap32 import org.jblas.{DoubleMatrix, SimpleBlas, Solve} -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.broadcast.Broadcast import org.apache.spark.{Logging, HashPartitioner, Partitioner} import org.apache.spark.storage.StorageLevel @@ -39,7 +40,8 @@ import org.apache.spark.mllib.optimization.NNLS * of the elements within this block, and the list of destination blocks that each user or * product will need to send its feature vector to. */ -private[recommendation] case class OutLinkBlock(elementIds: Array[Int], shouldSend: Array[BitSet]) +private[recommendation] +case class OutLinkBlock(elementIds: Array[Int], shouldSend: Array[mutable.BitSet]) /** @@ -109,11 +111,17 @@ class ALS private ( */ def this() = this(-1, -1, 10, 10, 0.01, false, 1.0) + /** If true, do alternating nonnegative least squares. */ + private var nonnegative = false + + /** storage level for user/product in/out links */ + private var intermediateRDDStorageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK + /** * Set the number of blocks for both user blocks and product blocks to parallelize the computation * into; pass -1 for an auto-configured number of blocks. Default: -1. */ - def setBlocks(numBlocks: Int): ALS = { + def setBlocks(numBlocks: Int): this.type = { this.numUserBlocks = numBlocks this.numProductBlocks = numBlocks this @@ -122,7 +130,7 @@ class ALS private ( /** * Set the number of user blocks to parallelize the computation. */ - def setUserBlocks(numUserBlocks: Int): ALS = { + def setUserBlocks(numUserBlocks: Int): this.type = { this.numUserBlocks = numUserBlocks this } @@ -130,31 +138,31 @@ class ALS private ( /** * Set the number of product blocks to parallelize the computation. */ - def setProductBlocks(numProductBlocks: Int): ALS = { + def setProductBlocks(numProductBlocks: Int): this.type = { this.numProductBlocks = numProductBlocks this } /** Set the rank of the feature matrices computed (number of features). Default: 10. */ - def setRank(rank: Int): ALS = { + def setRank(rank: Int): this.type = { this.rank = rank this } /** Set the number of iterations to run. Default: 10. */ - def setIterations(iterations: Int): ALS = { + def setIterations(iterations: Int): this.type = { this.iterations = iterations this } /** Set the regularization parameter, lambda. Default: 0.01. */ - def setLambda(lambda: Double): ALS = { + def setLambda(lambda: Double): this.type = { this.lambda = lambda this } /** Sets whether to use implicit preference. Default: false. */ - def setImplicitPrefs(implicitPrefs: Boolean): ALS = { + def setImplicitPrefs(implicitPrefs: Boolean): this.type = { this.implicitPrefs = implicitPrefs this } @@ -164,29 +172,38 @@ class ALS private ( * Sets the constant used in computing confidence in implicit ALS. Default: 1.0. */ @Experimental - def setAlpha(alpha: Double): ALS = { + def setAlpha(alpha: Double): this.type = { this.alpha = alpha this } /** Sets a random seed to have deterministic results. */ - def setSeed(seed: Long): ALS = { + def setSeed(seed: Long): this.type = { this.seed = seed this } - /** If true, do alternating nonnegative least squares. */ - private var nonnegative = false - /** * Set whether the least-squares problems solved at each iteration should have * nonnegativity constraints. */ - def setNonnegative(b: Boolean): ALS = { + def setNonnegative(b: Boolean): this.type = { this.nonnegative = b this } + /** + * :: DeveloperApi :: + * Sets storage level for intermediate RDDs (user/product in/out links). The default value is + * `MEMORY_AND_DISK`. Users can change it to a serialized storage, e.g., `MEMORY_AND_DISK_SER` and + * set `spark.rdd.compress` to `true` to reduce the space requirement, at the cost of speed. + */ + @DeveloperApi + def setIntermediateRDDStorageLevel(storageLevel: StorageLevel): this.type = { + this.intermediateRDDStorageLevel = storageLevel + this + } + /** * Run ALS with the configured parameters on an input RDD of (user, product, rating) triples. * Returns a MatrixFactorizationModel with feature vectors for each user and product. @@ -252,14 +269,17 @@ class ALS private ( val YtY = Some(sc.broadcast(computeYtY(users))) val previousProducts = products products = updateFeatures(numProductBlocks, users, userOutLinks, productInLinks, - userPartitioner, rank, lambda, alpha, YtY) + rank, lambda, alpha, YtY) previousProducts.unpersist() logInfo("Re-computing U given I (Iteration %d/%d)".format(iter, iterations)) + if (sc.checkpointDir.isDefined && (iter % 3 == 0)) { + products.checkpoint() + } products.setName(s"products-$iter").persist() val XtX = Some(sc.broadcast(computeYtY(products))) val previousUsers = users users = updateFeatures(numUserBlocks, products, productOutLinks, userInLinks, - productPartitioner, rank, lambda, alpha, XtX) + rank, lambda, alpha, XtX) previousUsers.unpersist() } } else { @@ -267,11 +287,14 @@ class ALS private ( // perform ALS update logInfo("Re-computing I given U (Iteration %d/%d)".format(iter, iterations)) products = updateFeatures(numProductBlocks, users, userOutLinks, productInLinks, - userPartitioner, rank, lambda, alpha, YtY = None) + rank, lambda, alpha, YtY = None) + if (sc.checkpointDir.isDefined && (iter % 3 == 0)) { + products.checkpoint() + } products.setName(s"products-$iter") logInfo("Re-computing U given I (Iteration %d/%d)".format(iter, iterations)) users = updateFeatures(numUserBlocks, products, productOutLinks, userInLinks, - productPartitioner, rank, lambda, alpha, YtY = None) + rank, lambda, alpha, YtY = None) users.setName(s"users-$iter") } } @@ -284,8 +307,8 @@ class ALS private ( val usersOut = unblockFactors(users, userOutLinks) val productsOut = unblockFactors(products, productOutLinks) - usersOut.setName("usersOut").persist() - productsOut.setName("productsOut").persist() + usersOut.setName("usersOut").persist(StorageLevel.MEMORY_AND_DISK) + productsOut.setName("productsOut").persist(StorageLevel.MEMORY_AND_DISK) // Materialize usersOut and productsOut. usersOut.count() @@ -376,7 +399,7 @@ class ALS private ( val userIds = ratings.map(_.user).distinct.sorted val numUsers = userIds.length val userIdToPos = userIds.zipWithIndex.toMap - val shouldSend = Array.fill(numUsers)(new BitSet(numProductBlocks)) + val shouldSend = Array.fill(numUsers)(new mutable.BitSet(numProductBlocks)) for (r <- ratings) { shouldSend(userIdToPos(r.user))(productPartitioner.getPartition(r.product)) = true } @@ -430,11 +453,11 @@ class ALS private ( val inLinkBlock = makeInLinkBlock(numProductBlocks, ratings, productPartitioner) val outLinkBlock = makeOutLinkBlock(numProductBlocks, ratings, productPartitioner) Iterator.single((blockId, (inLinkBlock, outLinkBlock))) - }, true) + }, preservesPartitioning = true) val inLinks = links.mapValues(_._1) val outLinks = links.mapValues(_._2) - inLinks.persist(StorageLevel.MEMORY_AND_DISK) - outLinks.persist(StorageLevel.MEMORY_AND_DISK) + inLinks.persist(intermediateRDDStorageLevel) + outLinks.persist(intermediateRDDStorageLevel) (inLinks, outLinks) } @@ -464,7 +487,6 @@ class ALS private ( products: RDD[(Int, Array[Array[Double]])], productOutLinks: RDD[(Int, OutLinkBlock)], userInLinks: RDD[(Int, InLinkBlock)], - productPartitioner: Partitioner, rank: Int, lambda: Double, alpha: Double, @@ -477,7 +499,7 @@ class ALS private ( } } toSend.zipWithIndex.map{ case (buf, idx) => (idx, (bid, buf.toArray)) } - }.groupByKey(productPartitioner) + }.groupByKey(new HashPartitioner(numUserBlocks)) .join(userInLinks) .mapValues{ case (messages, inLinkBlock) => updateBlock(messages, inLinkBlock, rank, lambda, alpha, YtY) @@ -792,4 +814,120 @@ object ALS { : MatrixFactorizationModel = { trainImplicit(ratings, rank, iterations, 0.01, -1, 1.0) } + + /** + * :: DeveloperApi :: + * Statistics of a block in ALS computation. + * + * @param category type of this block, "user" or "product" + * @param index index of this block + * @param count number of users or products inside this block, the same as the number of + * least-squares problems to solve on this block in each iteration + * @param numRatings total number of ratings inside this block, the same as the number of outer + * products we need to make on this block in each iteration + * @param numInLinks total number of incoming links, the same as the number of vectors to retrieve + * before each iteration + * @param numOutLinks total number of outgoing links, the same as the number of vectors to send + * for the next iteration + */ + @DeveloperApi + case class BlockStats( + category: String, + index: Int, + count: Long, + numRatings: Long, + numInLinks: Long, + numOutLinks: Long) + + /** + * :: DeveloperApi :: + * Given an RDD of ratings, number of user blocks, and number of product blocks, computes the + * statistics of each block in ALS computation. This is useful for estimating cost and diagnosing + * load balance. + * + * @param ratings an RDD of ratings + * @param numUserBlocks number of user blocks + * @param numProductBlocks number of product blocks + * @return statistics of user blocks and product blocks + */ + @DeveloperApi + def analyzeBlocks( + ratings: RDD[Rating], + numUserBlocks: Int, + numProductBlocks: Int): Array[BlockStats] = { + + val userPartitioner = new ALSPartitioner(numUserBlocks) + val productPartitioner = new ALSPartitioner(numProductBlocks) + + val ratingsByUserBlock = ratings.map { rating => + (userPartitioner.getPartition(rating.user), rating) + } + val ratingsByProductBlock = ratings.map { rating => + (productPartitioner.getPartition(rating.product), + Rating(rating.product, rating.user, rating.rating)) + } + + val als = new ALS() + val (userIn, userOut) = + als.makeLinkRDDs(numUserBlocks, numProductBlocks, ratingsByUserBlock, userPartitioner) + val (prodIn, prodOut) = + als.makeLinkRDDs(numProductBlocks, numUserBlocks, ratingsByProductBlock, productPartitioner) + + def sendGrid(outLinks: RDD[(Int, OutLinkBlock)]): Map[(Int, Int), Long] = { + outLinks.map { x => + val grid = new mutable.HashMap[(Int, Int), Long]() + val uPartition = x._1 + x._2.shouldSend.foreach { ss => + ss.foreach { pPartition => + val pair = (uPartition, pPartition) + grid.put(pair, grid.getOrElse(pair, 0L) + 1L) + } + } + grid + }.reduce { (grid1, grid2) => + grid2.foreach { x => + grid1.put(x._1, grid1.getOrElse(x._1, 0L) + x._2) + } + grid1 + }.toMap + } + + val userSendGrid = sendGrid(userOut) + val prodSendGrid = sendGrid(prodOut) + + val userInbound = new Array[Long](numUserBlocks) + val prodInbound = new Array[Long](numProductBlocks) + val userOutbound = new Array[Long](numUserBlocks) + val prodOutbound = new Array[Long](numProductBlocks) + + for (u <- 0 until numUserBlocks; p <- 0 until numProductBlocks) { + userOutbound(u) += userSendGrid.getOrElse((u, p), 0L) + prodInbound(p) += userSendGrid.getOrElse((u, p), 0L) + userInbound(u) += prodSendGrid.getOrElse((p, u), 0L) + prodOutbound(p) += prodSendGrid.getOrElse((p, u), 0L) + } + + val userCounts = userOut.mapValues(x => x.elementIds.length).collectAsMap() + val prodCounts = prodOut.mapValues(x => x.elementIds.length).collectAsMap() + + val userRatings = countRatings(userIn) + val prodRatings = countRatings(prodIn) + + val userStats = Array.tabulate(numUserBlocks)( + u => BlockStats("user", u, userCounts(u), userRatings(u), userInbound(u), userOutbound(u))) + val productStatus = Array.tabulate(numProductBlocks)( + p => BlockStats("product", p, prodCounts(p), prodRatings(p), prodInbound(p), prodOutbound(p))) + + (userStats ++ productStatus).toArray + } + + private def countRatings(inLinks: RDD[(Int, InLinkBlock)]): Map[Int, Long] = { + inLinks.mapValues { ilb => + var numRatings = 0L + ilb.ratingsForBlock.foreach { ar => + ar.foreach { p => numRatings += p._1.length } + } + numRatings + }.collectAsMap().toMap + } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala index 899286d235a9d..478c6485052b6 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala @@ -23,7 +23,7 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.api.java.JavaRDD import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext._ -import org.apache.spark.mllib.api.python.PythonMLLibAPI +import org.apache.spark.mllib.api.python.SerDe /** * Model representing the result of matrix factorization. @@ -65,6 +65,48 @@ class MatrixFactorizationModel private[mllib] ( } } + /** + * Recommends products to a user. + * + * @param user the user to recommend products to + * @param num how many products to return. The number returned may be less than this. + * @return [[Rating]] objects, each of which contains the given user ID, a product ID, and a + * "score" in the rating field. Each represents one recommended product, and they are sorted + * by score, decreasing. The first returned is the one predicted to be most strongly + * recommended to the user. The score is an opaque value that indicates how strongly + * recommended the product is. + */ + def recommendProducts(user: Int, num: Int): Array[Rating] = + recommend(userFeatures.lookup(user).head, productFeatures, num) + .map(t => Rating(user, t._1, t._2)) + + /** + * Recommends users to a product. That is, this returns users who are most likely to be + * interested in a product. + * + * @param product the product to recommend users to + * @param num how many users to return. The number returned may be less than this. + * @return [[Rating]] objects, each of which contains a user ID, the given product ID, and a + * "score" in the rating field. Each represents one recommended user, and they are sorted + * by score, decreasing. The first returned is the one predicted to be most strongly + * recommended to the product. The score is an opaque value that indicates how strongly + * recommended the user is. + */ + def recommendUsers(product: Int, num: Int): Array[Rating] = + recommend(productFeatures.lookup(product).head, userFeatures, num) + .map(t => Rating(t._1, product, t._2)) + + private def recommend( + recommendToFeatures: Array[Double], + recommendableFeatures: RDD[(Int, Array[Double])], + num: Int): Array[(Int, Double)] = { + val recommendToVector = new DoubleMatrix(recommendToFeatures) + val scored = recommendableFeatures.map { case (id,features) => + (id, recommendToVector.dot(new DoubleMatrix(features))) + } + scored.top(num)(Ordering.by(_._2)) + } + /** * :: DeveloperApi :: * Predict the rating of many users for many products. @@ -75,11 +117,8 @@ class MatrixFactorizationModel private[mllib] ( */ @DeveloperApi def predict(usersProductsJRDD: JavaRDD[Array[Byte]]): JavaRDD[Array[Byte]] = { - val pythonAPI = new PythonMLLibAPI() - val usersProducts = usersProductsJRDD.rdd.map(xBytes => pythonAPI.unpackTuple(xBytes)) - predict(usersProducts).map(rate => pythonAPI.serializeRating(rate)) + val usersProducts = usersProductsJRDD.rdd.map(xBytes => SerDe.unpackTuple(xBytes)) + predict(usersProducts).map(rate => SerDe.serializeRating(rate)) } - // TODO: Figure out what other good bulk prediction methods would look like. - // Probably want a way to get the top users for a product or vice-versa. } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala index 8cca926f1c92e..20c1fdd2269ce 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala @@ -17,13 +17,13 @@ package org.apache.spark.mllib.regression -import breeze.linalg.{DenseVector => BDV, SparseVector => BSV} - import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.mllib.feature.StandardScaler import org.apache.spark.{Logging, SparkException} import org.apache.spark.rdd.RDD import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.linalg.{Vectors, Vector} +import org.apache.spark.mllib.util.MLUtils._ /** * :: DeveloperApi :: @@ -57,9 +57,12 @@ abstract class GeneralizedLinearModel(val weights: Vector, val intercept: Double // A small optimization to avoid serializing the entire model. Only the weightsMatrix // and intercept is needed. val localWeights = weights + val bcWeights = testData.context.broadcast(localWeights) val localIntercept = intercept - - testData.map(v => predictPoint(v, localWeights, localIntercept)) + testData.mapPartitions { iter => + val w = bcWeights.value + iter.map(v => predictPoint(v, w, localIntercept)) + } } /** @@ -92,6 +95,22 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] protected var validateData: Boolean = true + /** + * Whether to perform feature scaling before model training to reduce the condition numbers + * which can significantly help the optimizer converging faster. The scaling correction will be + * translated back to resulting model weights, so it's transparent to users. + * Note: This technique is used in both libsvm and glmnet packages. Default false. + */ + private var useFeatureScaling = false + + /** + * Set if the algorithm should use feature scaling to improve the convergence during optimization. + */ + private[mllib] def setFeatureScaling(useFeatureScaling: Boolean): this.type = { + this.useFeatureScaling = useFeatureScaling + this + } + /** * Create a model given the weights and intercept */ @@ -124,16 +143,6 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] run(input, initialWeights) } - /** Prepends one to the input vector. */ - private def prependOne(vector: Vector): Vector = { - val vector1 = vector.toBreeze match { - case dv: BDV[Double] => BDV.vertcat(BDV.ones[Double](1), dv) - case sv: BSV[Double] => BSV.vertcat(new BSV[Double](Array(0), Array(1.0), 1), sv) - case v: Any => throw new IllegalArgumentException("Do not support vector type " + v.getClass) - } - Vectors.fromBreeze(vector1) - } - /** * Run the algorithm with the configured parameters on an input RDD * of LabeledPoint entries starting from the initial weights provided. @@ -145,29 +154,75 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] throw new SparkException("Input validation failed.") } + /** + * Scaling columns to unit variance as a heuristic to reduce the condition number: + * + * During the optimization process, the convergence (rate) depends on the condition number of + * the training dataset. Scaling the variables often reduces this condition number + * heuristically, thus improving the convergence rate. Without reducing the condition number, + * some training datasets mixing the columns with different scales may not be able to converge. + * + * GLMNET and LIBSVM packages perform the scaling to reduce the condition number, and return + * the weights in the original scale. + * See page 9 in http://cran.r-project.org/web/packages/glmnet/glmnet.pdf + * + * Here, if useFeatureScaling is enabled, we will standardize the training features by dividing + * the variance of each column (without subtracting the mean), and train the model in the + * scaled space. Then we transform the coefficients from the scaled space to the original scale + * as GLMNET and LIBSVM do. + * + * Currently, it's only enabled in LogisticRegressionWithLBFGS + */ + val scaler = if (useFeatureScaling) { + (new StandardScaler).fit(input.map(x => x.features)) + } else { + null + } + // Prepend an extra variable consisting of all 1.0's for the intercept. val data = if (addIntercept) { - input.map(labeledPoint => (labeledPoint.label, prependOne(labeledPoint.features))) + if(useFeatureScaling) { + input.map(labeledPoint => + (labeledPoint.label, appendBias(scaler.transform(labeledPoint.features)))) + } else { + input.map(labeledPoint => (labeledPoint.label, appendBias(labeledPoint.features))) + } } else { - input.map(labeledPoint => (labeledPoint.label, labeledPoint.features)) + if (useFeatureScaling) { + input.map(labeledPoint => (labeledPoint.label, scaler.transform(labeledPoint.features))) + } else { + input.map(labeledPoint => (labeledPoint.label, labeledPoint.features)) + } } val initialWeightsWithIntercept = if (addIntercept) { - prependOne(initialWeights) + appendBias(initialWeights) } else { initialWeights } val weightsWithIntercept = optimizer.optimize(data, initialWeightsWithIntercept) - val intercept = if (addIntercept) weightsWithIntercept(0) else 0.0 - val weights = + val intercept = if (addIntercept) weightsWithIntercept(weightsWithIntercept.size - 1) else 0.0 + var weights = if (addIntercept) { - Vectors.dense(weightsWithIntercept.toArray.slice(1, weightsWithIntercept.size)) + Vectors.dense(weightsWithIntercept.toArray.slice(0, weightsWithIntercept.size - 1)) } else { weightsWithIntercept } + /** + * The weights and intercept are trained in the scaled space; we're converting them back to + * the original scale. + * + * Math shows that if we only perform standardization without subtracting means, the intercept + * will not be changed. w_i = w_i' / v_i where w_i' is the coefficient in the scaled space, w_i + * is the coefficient in the original space, and v_i is the variance of the column i. + */ + if (useFeatureScaling) { + weights = scaler.transform(weights) + } + createModel(weights, intercept) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala index 62a03af4a9964..17c753c56681f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala @@ -36,7 +36,7 @@ case class LabeledPoint(label: Double, features: Vector) { /** * Parser for [[org.apache.spark.mllib.regression.LabeledPoint]]. */ -private[mllib] object LabeledPointParser { +object LabeledPoint { /** * Parses a string resulted from `LabeledPoint#toString` into * an [[org.apache.spark.mllib.regression.LabeledPoint]]. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala index a05dfc045fb8e..cb0d39e759a9f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala @@ -28,7 +28,7 @@ import org.apache.spark.rdd.RDD * @param weights Weights computed for every feature. * @param intercept Intercept computed for this model. */ -class LassoModel private[mllib] ( +class LassoModel ( override val weights: Vector, override val intercept: Double) extends GeneralizedLinearModel(weights, intercept) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala index 0ebad4eb58d88..81b6598377ff5 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala @@ -27,7 +27,7 @@ import org.apache.spark.mllib.optimization._ * @param weights Weights computed for every feature. * @param intercept Intercept computed for this model. */ -class LinearRegressionModel private[mllib] ( +class LinearRegressionModel ( override val weights: Vector, override val intercept: Double) extends GeneralizedLinearModel(weights, intercept) with RegressionModel with Serializable { @@ -49,7 +49,7 @@ class LinearRegressionModel private[mllib] ( * its corresponding right hand side label y. * See also the documentation for the precise formulation. */ -class LinearRegressionWithSGD private ( +class LinearRegressionWithSGD private[mllib] ( private var stepSize: Double, private var numIterations: Int, private var miniBatchFraction: Double) @@ -68,7 +68,7 @@ class LinearRegressionWithSGD private ( */ def this() = this(1.0, 100, 1.0) - override protected def createModel(weights: Vector, intercept: Double) = { + override protected[mllib] def createModel(weights: Vector, intercept: Double) = { new LinearRegressionModel(weights, intercept) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala index bd983bac001a0..a826deb695ee1 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala @@ -28,7 +28,7 @@ import org.apache.spark.mllib.linalg.Vector * @param weights Weights computed for every feature. * @param intercept Intercept computed for this model. */ -class RidgeRegressionModel private[mllib] ( +class RidgeRegressionModel ( override val weights: Vector, override val intercept: Double) extends GeneralizedLinearModel(weights, intercept) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearAlgorithm.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearAlgorithm.scala new file mode 100644 index 0000000000000..b8b0b42611775 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearAlgorithm.scala @@ -0,0 +1,106 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.regression + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.Logging +import org.apache.spark.streaming.dstream.DStream + +/** + * :: DeveloperApi :: + * StreamingLinearAlgorithm implements methods for continuously + * training a generalized linear model model on streaming data, + * and using it for prediction on (possibly different) streaming data. + * + * This class takes as type parameters a GeneralizedLinearModel, + * and a GeneralizedLinearAlgorithm, making it easy to extend to construct + * streaming versions of any analyses using GLMs. + * Initial weights must be set before calling trainOn or predictOn. + * Only weights will be updated, not an intercept. If the model needs + * an intercept, it should be manually appended to the input data. + * + * For example usage, see `StreamingLinearRegressionWithSGD`. + * + * NOTE(Freeman): In some use cases, the order in which trainOn and predictOn + * are called in an application will affect the results. When called on + * the same DStream, if trainOn is called before predictOn, when new data + * arrive the model will update and the prediction will be based on the new + * model. Whereas if predictOn is called first, the prediction will use the model + * from the previous update. + * + * NOTE(Freeman): It is ok to call predictOn repeatedly on multiple streams; this + * will generate predictions for each one all using the current model. + * It is also ok to call trainOn on different streams; this will update + * the model using each of the different sources, in sequence. + * + */ +@DeveloperApi +abstract class StreamingLinearAlgorithm[ + M <: GeneralizedLinearModel, + A <: GeneralizedLinearAlgorithm[M]] extends Logging { + + /** The model to be updated and used for prediction. */ + protected var model: M + + /** The algorithm to use for updating. */ + protected val algorithm: A + + /** Return the latest model. */ + def latestModel(): M = { + model + } + + /** + * Update the model by training on batches of data from a DStream. + * This operation registers a DStream for training the model, + * and updates the model based on every subsequent + * batch of data from the stream. + * + * @param data DStream containing labeled data + */ + def trainOn(data: DStream[LabeledPoint]) { + if (Option(model.weights) == None) { + logError("Initial weights must be set before starting training") + throw new IllegalArgumentException + } + data.foreachRDD { (rdd, time) => + model = algorithm.run(rdd, model.weights) + logInfo("Model updated at time %s".format(time.toString)) + val display = model.weights.size match { + case x if x > 100 => model.weights.toArray.take(100).mkString("[", ",", "...") + case _ => model.weights.toArray.mkString("[", ",", "]") + } + logInfo("Current model: weights, %s".format (display)) + } + } + + /** + * Use the model to make predictions on batches of data from a DStream + * + * @param data DStream containing labeled data + * @return DStream containing predictions + */ + def predictOn(data: DStream[LabeledPoint]): DStream[Double] = { + if (Option(model.weights) == None) { + logError("Initial weights must be set before starting prediction") + throw new IllegalArgumentException + } + data.map(x => model.predict(x.features)) + } + +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionWithSGD.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionWithSGD.scala new file mode 100644 index 0000000000000..1d11fde24712c --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionWithSGD.scala @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.regression + +import org.apache.spark.annotation.Experimental +import org.apache.spark.mllib.linalg.Vector + +/** + * Train or predict a linear regression model on streaming data. Training uses + * Stochastic Gradient Descent to update the model based on each new batch of + * incoming data from a DStream (see `LinearRegressionWithSGD` for model equation) + * + * Each batch of data is assumed to be an RDD of LabeledPoints. + * The number of data points per batch can vary, but the number + * of features must be constant. An initial weight + * vector must be provided. + * + * Use a builder pattern to construct a streaming linear regression + * analysis in an application, like: + * + * val model = new StreamingLinearRegressionWithSGD() + * .setStepSize(0.5) + * .setNumIterations(10) + * .setInitialWeights(Vectors.dense(...)) + * .trainOn(DStream) + * + */ +@Experimental +class StreamingLinearRegressionWithSGD ( + private var stepSize: Double, + private var numIterations: Int, + private var miniBatchFraction: Double, + private var initialWeights: Vector) + extends StreamingLinearAlgorithm[ + LinearRegressionModel, LinearRegressionWithSGD] with Serializable { + + /** + * Construct a StreamingLinearRegression object with default parameters: + * {stepSize: 0.1, numIterations: 50, miniBatchFraction: 1.0}. + * Initial weights must be set before using trainOn or predictOn + * (see `StreamingLinearAlgorithm`) + */ + def this() = this(0.1, 50, 1.0, null) + + val algorithm = new LinearRegressionWithSGD(stepSize, numIterations, miniBatchFraction) + + var model = algorithm.createModel(initialWeights, 0.0) + + /** Set the step size for gradient descent. Default: 0.1. */ + def setStepSize(stepSize: Double): this.type = { + this.algorithm.optimizer.setStepSize(stepSize) + this + } + + /** Set the number of iterations of gradient descent to run per update. Default: 50. */ + def setNumIterations(numIterations: Int): this.type = { + this.algorithm.optimizer.setNumIterations(numIterations) + this + } + + /** Set the fraction of each batch to use for updates. Default: 1.0. */ + def setMiniBatchFraction(miniBatchFraction: Double): this.type = { + this.algorithm.optimizer.setMiniBatchFraction(miniBatchFraction) + this + } + + /** Set the initial weights. Default: [0.0, 0.0]. */ + def setInitialWeights(initialWeights: Vector): this.type = { + this.model = algorithm.createModel(initialWeights, 0.0) + this + } + +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala new file mode 100644 index 0000000000000..5105b5c37aaaa --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala @@ -0,0 +1,201 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.stat + +import breeze.linalg.{DenseVector => BDV} + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.mllib.linalg.{Vectors, Vector} + +/** + * :: DeveloperApi :: + * MultivariateOnlineSummarizer implements [[MultivariateStatisticalSummary]] to compute the mean, + * variance, minimum, maximum, counts, and nonzero counts for samples in sparse or dense vector + * format in a online fashion. + * + * Two MultivariateOnlineSummarizer can be merged together to have a statistical summary of + * the corresponding joint dataset. + * + * A numerically stable algorithm is implemented to compute sample mean and variance: + * Reference: [[http://en.wikipedia.org/wiki/Algorithms_for_calculating_variance variance-wiki]] + * Zero elements (including explicit zero values) are skipped when calling add(), + * to have time complexity O(nnz) instead of O(n) for each column. + */ +@DeveloperApi +class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with Serializable { + + private var n = 0 + private var currMean: BDV[Double] = _ + private var currM2n: BDV[Double] = _ + private var totalCnt: Long = 0 + private var nnz: BDV[Double] = _ + private var currMax: BDV[Double] = _ + private var currMin: BDV[Double] = _ + + /** + * Add a new sample to this summarizer, and update the statistical summary. + * + * @param sample The sample in dense/sparse vector format to be added into this summarizer. + * @return This MultivariateOnlineSummarizer object. + */ + def add(sample: Vector): this.type = { + if (n == 0) { + require(sample.toBreeze.length > 0, s"Vector should have dimension larger than zero.") + n = sample.toBreeze.length + + currMean = BDV.zeros[Double](n) + currM2n = BDV.zeros[Double](n) + nnz = BDV.zeros[Double](n) + currMax = BDV.fill(n)(Double.MinValue) + currMin = BDV.fill(n)(Double.MaxValue) + } + + require(n == sample.toBreeze.length, s"Dimensions mismatch when adding new sample." + + s" Expecting $n but got ${sample.toBreeze.length}.") + + sample.toBreeze.activeIterator.foreach { + case (_, 0.0) => // Skip explicit zero elements. + case (i, value) => + if (currMax(i) < value) { + currMax(i) = value + } + if (currMin(i) > value) { + currMin(i) = value + } + + val tmpPrevMean = currMean(i) + currMean(i) = (currMean(i) * nnz(i) + value) / (nnz(i) + 1.0) + currM2n(i) += (value - currMean(i)) * (value - tmpPrevMean) + + nnz(i) += 1.0 + } + + totalCnt += 1 + this + } + + /** + * Merge another MultivariateOnlineSummarizer, and update the statistical summary. + * (Note that it's in place merging; as a result, `this` object will be modified.) + * + * @param other The other MultivariateOnlineSummarizer to be merged. + * @return This MultivariateOnlineSummarizer object. + */ + def merge(other: MultivariateOnlineSummarizer): this.type = { + if (this.totalCnt != 0 && other.totalCnt != 0) { + require(n == other.n, s"Dimensions mismatch when merging with another summarizer. " + + s"Expecting $n but got ${other.n}.") + totalCnt += other.totalCnt + val deltaMean: BDV[Double] = currMean - other.currMean + var i = 0 + while (i < n) { + // merge mean together + if (other.currMean(i) != 0.0) { + currMean(i) = (currMean(i) * nnz(i) + other.currMean(i) * other.nnz(i)) / + (nnz(i) + other.nnz(i)) + } + // merge m2n together + if (nnz(i) + other.nnz(i) != 0.0) { + currM2n(i) += other.currM2n(i) + deltaMean(i) * deltaMean(i) * nnz(i) * other.nnz(i) / + (nnz(i) + other.nnz(i)) + } + if (currMax(i) < other.currMax(i)) { + currMax(i) = other.currMax(i) + } + if (currMin(i) > other.currMin(i)) { + currMin(i) = other.currMin(i) + } + i += 1 + } + nnz += other.nnz + } else if (totalCnt == 0 && other.totalCnt != 0) { + this.n = other.n + this.currMean = other.currMean.copy + this.currM2n = other.currM2n.copy + this.totalCnt = other.totalCnt + this.nnz = other.nnz.copy + this.currMax = other.currMax.copy + this.currMin = other.currMin.copy + } + this + } + + override def mean: Vector = { + require(totalCnt > 0, s"Nothing has been added to this summarizer.") + + val realMean = BDV.zeros[Double](n) + var i = 0 + while (i < n) { + realMean(i) = currMean(i) * (nnz(i) / totalCnt) + i += 1 + } + Vectors.fromBreeze(realMean) + } + + override def variance: Vector = { + require(totalCnt > 0, s"Nothing has been added to this summarizer.") + + val realVariance = BDV.zeros[Double](n) + + val denominator = totalCnt - 1.0 + + // Sample variance is computed, if the denominator is less than 0, the variance is just 0. + if (denominator > 0.0) { + val deltaMean = currMean + var i = 0 + while (i < currM2n.size) { + realVariance(i) = + currM2n(i) + deltaMean(i) * deltaMean(i) * nnz(i) * (totalCnt - nnz(i)) / totalCnt + realVariance(i) /= denominator + i += 1 + } + } + + Vectors.fromBreeze(realVariance) + } + + override def count: Long = totalCnt + + override def numNonzeros: Vector = { + require(totalCnt > 0, s"Nothing has been added to this summarizer.") + + Vectors.fromBreeze(nnz) + } + + override def max: Vector = { + require(totalCnt > 0, s"Nothing has been added to this summarizer.") + + var i = 0 + while (i < n) { + if ((nnz(i) < totalCnt) && (currMax(i) < 0.0)) currMax(i) = 0.0 + i += 1 + } + Vectors.fromBreeze(currMax) + } + + override def min: Vector = { + require(totalCnt > 0, s"Nothing has been added to this summarizer.") + + var i = 0 + while (i < n) { + if ((nnz(i) < totalCnt) && (currMin(i) > 0.0)) currMin(i) = 0.0 + i += 1 + } + Vectors.fromBreeze(currMin) + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/Statistics.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/Statistics.scala new file mode 100644 index 0000000000000..3cf4e807b4cf7 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/Statistics.scala @@ -0,0 +1,169 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.stat + +import org.apache.spark.annotation.Experimental +import org.apache.spark.mllib.linalg.distributed.RowMatrix +import org.apache.spark.mllib.linalg.{Matrix, Vector} +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.stat.correlation.Correlations +import org.apache.spark.mllib.stat.test.{ChiSqTest, ChiSqTestResult} +import org.apache.spark.rdd.RDD + +/** + * API for statistical functions in MLlib. + */ +@Experimental +object Statistics { + + /** + * :: Experimental :: + * Computes column-wise summary statistics for the input RDD[Vector]. + * + * @param X an RDD[Vector] for which column-wise summary statistics are to be computed. + * @return [[MultivariateStatisticalSummary]] object containing column-wise summary statistics. + */ + @Experimental + def colStats(X: RDD[Vector]): MultivariateStatisticalSummary = { + new RowMatrix(X).computeColumnSummaryStatistics() + } + + /** + * :: Experimental :: + * Compute the Pearson correlation matrix for the input RDD of Vectors. + * Columns with 0 covariance produce NaN entries in the correlation matrix. + * + * @param X an RDD[Vector] for which the correlation matrix is to be computed. + * @return Pearson correlation matrix comparing columns in X. + */ + @Experimental + def corr(X: RDD[Vector]): Matrix = Correlations.corrMatrix(X) + + /** + * :: Experimental :: + * Compute the correlation matrix for the input RDD of Vectors using the specified method. + * Methods currently supported: `pearson` (default), `spearman`. + * + * Note that for Spearman, a rank correlation, we need to create an RDD[Double] for each column + * and sort it in order to retrieve the ranks and then join the columns back into an RDD[Vector], + * which is fairly costly. Cache the input RDD before calling corr with `method = "spearman"` to + * avoid recomputing the common lineage. + * + * @param X an RDD[Vector] for which the correlation matrix is to be computed. + * @param method String specifying the method to use for computing correlation. + * Supported: `pearson` (default), `spearman` + * @return Correlation matrix comparing columns in X. + */ + @Experimental + def corr(X: RDD[Vector], method: String): Matrix = Correlations.corrMatrix(X, method) + + /** + * :: Experimental :: + * Compute the Pearson correlation for the input RDDs. + * Returns NaN if either vector has 0 variance. + * + * Note: the two input RDDs need to have the same number of partitions and the same number of + * elements in each partition. + * + * @param x RDD[Double] of the same cardinality as y. + * @param y RDD[Double] of the same cardinality as x. + * @return A Double containing the Pearson correlation between the two input RDD[Double]s + */ + @Experimental + def corr(x: RDD[Double], y: RDD[Double]): Double = Correlations.corr(x, y) + + /** + * :: Experimental :: + * Compute the correlation for the input RDDs using the specified method. + * Methods currently supported: `pearson` (default), `spearman`. + * + * Note: the two input RDDs need to have the same number of partitions and the same number of + * elements in each partition. + * + * @param x RDD[Double] of the same cardinality as y. + * @param y RDD[Double] of the same cardinality as x. + * @param method String specifying the method to use for computing correlation. + * Supported: `pearson` (default), `spearman` + *@return A Double containing the correlation between the two input RDD[Double]s using the + * specified method. + */ + @Experimental + def corr(x: RDD[Double], y: RDD[Double], method: String): Double = Correlations.corr(x, y, method) + + /** + * :: Experimental :: + * Conduct Pearson's chi-squared goodness of fit test of the observed data against the + * expected distribution. + * + * Note: the two input Vectors need to have the same size. + * `observed` cannot contain negative values. + * `expected` cannot contain nonpositive values. + * + * @param observed Vector containing the observed categorical counts/relative frequencies. + * @param expected Vector containing the expected categorical counts/relative frequencies. + * `expected` is rescaled if the `expected` sum differs from the `observed` sum. + * @return ChiSquaredTest object containing the test statistic, degrees of freedom, p-value, + * the method used, and the null hypothesis. + */ + @Experimental + def chiSqTest(observed: Vector, expected: Vector): ChiSqTestResult = { + ChiSqTest.chiSquared(observed, expected) + } + + /** + * :: Experimental :: + * Conduct Pearson's chi-squared goodness of fit test of the observed data against the uniform + * distribution, with each category having an expected frequency of `1 / observed.size`. + * + * Note: `observed` cannot contain negative values. + * + * @param observed Vector containing the observed categorical counts/relative frequencies. + * @return ChiSquaredTest object containing the test statistic, degrees of freedom, p-value, + * the method used, and the null hypothesis. + */ + @Experimental + def chiSqTest(observed: Vector): ChiSqTestResult = ChiSqTest.chiSquared(observed) + + /** + * :: Experimental :: + * Conduct Pearson's independence test on the input contingency matrix, which cannot contain + * negative entries or columns or rows that sum up to 0. + * + * @param observed The contingency matrix (containing either counts or relative frequencies). + * @return ChiSquaredTest object containing the test statistic, degrees of freedom, p-value, + * the method used, and the null hypothesis. + */ + @Experimental + def chiSqTest(observed: Matrix): ChiSqTestResult = ChiSqTest.chiSquaredMatrix(observed) + + /** + * :: Experimental :: + * Conduct Pearson's independence test for every feature against the label across the input RDD. + * For each feature, the (feature, label) pairs are converted into a contingency matrix for which + * the chi-squared statistic is computed. All label and feature values must be categorical. + * + * @param data an `RDD[LabeledPoint]` containing the labeled dataset with categorical features. + * Real-valued features will be treated as categorical for each distinct value. + * @return an array containing the ChiSquaredTestResult for every feature against the label. + * The order of the elements in the returned array reflects the order of input features. + */ + @Experimental + def chiSqTest(data: RDD[LabeledPoint]): Array[ChiSqTestResult] = { + ChiSqTest.chiSquaredFeatures(data) + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/Correlation.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/Correlation.scala new file mode 100644 index 0000000000000..1fb8d7b3d4f32 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/Correlation.scala @@ -0,0 +1,96 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.stat.correlation + +import org.apache.spark.mllib.linalg.{DenseVector, Matrix, Vector} +import org.apache.spark.rdd.RDD + +/** + * Trait for correlation algorithms. + */ +private[stat] trait Correlation { + + /** + * Compute correlation for two datasets. + */ + def computeCorrelation(x: RDD[Double], y: RDD[Double]): Double + + /** + * Compute the correlation matrix S, for the input matrix, where S(i, j) is the correlation + * between column i and j. S(i, j) can be NaN if the correlation is undefined for column i and j. + */ + def computeCorrelationMatrix(X: RDD[Vector]): Matrix + + /** + * Combine the two input RDD[Double]s into an RDD[Vector] and compute the correlation using the + * correlation implementation for RDD[Vector]. Can be NaN if correlation is undefined for the + * input vectors. + */ + def computeCorrelationWithMatrixImpl(x: RDD[Double], y: RDD[Double]): Double = { + val mat: RDD[Vector] = x.zip(y).map { case (xi, yi) => new DenseVector(Array(xi, yi)) } + computeCorrelationMatrix(mat)(0, 1) + } + +} + +/** + * Delegates computation to the specific correlation object based on the input method name. + */ +private[stat] object Correlations { + + def corr(x: RDD[Double], + y: RDD[Double], + method: String = CorrelationNames.defaultCorrName): Double = { + val correlation = getCorrelationFromName(method) + correlation.computeCorrelation(x, y) + } + + def corrMatrix(X: RDD[Vector], + method: String = CorrelationNames.defaultCorrName): Matrix = { + val correlation = getCorrelationFromName(method) + correlation.computeCorrelationMatrix(X) + } + + // Match input correlation name with a known name via simple string matching. + def getCorrelationFromName(method: String): Correlation = { + try { + CorrelationNames.nameToObjectMap(method) + } catch { + case nse: NoSuchElementException => + throw new IllegalArgumentException("Unrecognized method name. Supported correlations: " + + CorrelationNames.nameToObjectMap.keys.mkString(", ")) + } + } +} + +/** + * Maintains supported and default correlation names. + * + * Currently supported correlations: `pearson`, `spearman`. + * Current default correlation: `pearson`. + * + * After new correlation algorithms are added, please update the documentation here and in + * Statistics.scala for the correlation APIs. + */ +private[mllib] object CorrelationNames { + + // Note: after new types of correlations are implemented, please update this map. + val nameToObjectMap = Map(("pearson", PearsonCorrelation), ("spearman", SpearmanCorrelation)) + val defaultCorrName: String = "pearson" + +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/PearsonCorrelation.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/PearsonCorrelation.scala new file mode 100644 index 0000000000000..23b291eee070b --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/PearsonCorrelation.scala @@ -0,0 +1,107 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.stat.correlation + +import breeze.linalg.{DenseMatrix => BDM} + +import org.apache.spark.Logging +import org.apache.spark.mllib.linalg.{Matrices, Matrix, Vector} +import org.apache.spark.mllib.linalg.distributed.RowMatrix +import org.apache.spark.rdd.RDD + +/** + * Compute Pearson correlation for two RDDs of the type RDD[Double] or the correlation matrix + * for an RDD of the type RDD[Vector]. + * + * Definition of Pearson correlation can be found at + * http://en.wikipedia.org/wiki/Pearson_product-moment_correlation_coefficient + */ +private[stat] object PearsonCorrelation extends Correlation with Logging { + + /** + * Compute the Pearson correlation for two datasets. NaN if either vector has 0 variance. + */ + override def computeCorrelation(x: RDD[Double], y: RDD[Double]): Double = { + computeCorrelationWithMatrixImpl(x, y) + } + + /** + * Compute the Pearson correlation matrix S, for the input matrix, where S(i, j) is the + * correlation between column i and j. 0 covariance results in a correlation value of Double.NaN. + */ + override def computeCorrelationMatrix(X: RDD[Vector]): Matrix = { + val rowMatrix = new RowMatrix(X) + val cov = rowMatrix.computeCovariance() + computeCorrelationMatrixFromCovariance(cov) + } + + /** + * Compute the Pearson correlation matrix from the covariance matrix. + * 0 covariance results in a correlation value of Double.NaN. + */ + def computeCorrelationMatrixFromCovariance(covarianceMatrix: Matrix): Matrix = { + val cov = covarianceMatrix.toBreeze.asInstanceOf[BDM[Double]] + val n = cov.cols + + // Compute the standard deviation on the diagonals first + var i = 0 + while (i < n) { + // TODO remove once covariance numerical issue resolved. + cov(i, i) = if (closeToZero(cov(i, i))) 0.0 else math.sqrt(cov(i, i)) + i +=1 + } + + // Loop through columns since cov is column major + var j = 0 + var sigma = 0.0 + var containNaN = false + while (j < n) { + sigma = cov(j, j) + i = 0 + while (i < j) { + val corr = if (sigma == 0.0 || cov(i, i) == 0.0) { + containNaN = true + Double.NaN + } else { + cov(i, j) / (sigma * cov(i, i)) + } + cov(i, j) = corr + cov(j, i) = corr + i += 1 + } + j += 1 + } + + // put 1.0 on the diagonals + i = 0 + while (i < n) { + cov(i, i) = 1.0 + i +=1 + } + + if (containNaN) { + logWarning("Pearson correlation matrix contains NaN values.") + } + + Matrices.fromBreeze(cov) + } + + private def closeToZero(value: Double, threshhold: Double = 1e-12): Boolean = { + math.abs(value) <= threshhold + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/SpearmanCorrelation.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/SpearmanCorrelation.scala new file mode 100644 index 0000000000000..4a6c677f06d28 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/SpearmanCorrelation.scala @@ -0,0 +1,93 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.stat.correlation + +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.Logging +import org.apache.spark.SparkContext._ +import org.apache.spark.mllib.linalg.{Matrix, Vector, Vectors} +import org.apache.spark.rdd.RDD + +/** + * Compute Spearman's correlation for two RDDs of the type RDD[Double] or the correlation matrix + * for an RDD of the type RDD[Vector]. + * + * Definition of Spearman's correlation can be found at + * http://en.wikipedia.org/wiki/Spearman's_rank_correlation_coefficient + */ +private[stat] object SpearmanCorrelation extends Correlation with Logging { + + /** + * Compute Spearman's correlation for two datasets. + */ + override def computeCorrelation(x: RDD[Double], y: RDD[Double]): Double = { + computeCorrelationWithMatrixImpl(x, y) + } + + /** + * Compute Spearman's correlation matrix S, for the input matrix, where S(i, j) is the + * correlation between column i and j. + */ + override def computeCorrelationMatrix(X: RDD[Vector]): Matrix = { + // ((columnIndex, value), rowUid) + val colBased = X.zipWithUniqueId().flatMap { case (vec, uid) => + vec.toArray.view.zipWithIndex.map { case (v, j) => + ((j, v), uid) + } + } + // global sort by (columnIndex, value) + val sorted = colBased.sortByKey() + // assign global ranks (using average ranks for tied values) + val globalRanks = sorted.zipWithIndex().mapPartitions { iter => + var preCol = -1 + var preVal = Double.NaN + var startRank = -1.0 + var cachedUids = ArrayBuffer.empty[Long] + val flush: () => Iterable[(Long, (Int, Double))] = () => { + val averageRank = startRank + (cachedUids.size - 1) / 2.0 + val output = cachedUids.map { uid => + (uid, (preCol, averageRank)) + } + cachedUids.clear() + output + } + iter.flatMap { case (((j, v), uid), rank) => + // If we see a new value or cachedUids is too big, we flush ids with their average rank. + if (j != preCol || v != preVal || cachedUids.size >= 10000000) { + val output = flush() + preCol = j + preVal = v + startRank = rank + cachedUids += uid + output + } else { + cachedUids += uid + Iterator.empty + } + } ++ flush() + } + // Replace values in the input matrix by their ranks compared with values in the same column. + // Note that shifting all ranks in a column by a constant value doesn't affect result. + val groupedRanks = globalRanks.groupByKey().map { case (uid, iter) => + // sort by column index and then convert values to a vector + Vectors.dense(iter.toSeq.sortBy(_._1).map(_._2).toArray) + } + PearsonCorrelation.computeCorrelationMatrix(groupedRanks) + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/ChiSqTest.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/ChiSqTest.scala new file mode 100644 index 0000000000000..0089419c2c5d4 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/ChiSqTest.scala @@ -0,0 +1,244 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.stat.test + +import breeze.linalg.{DenseMatrix => BDM} +import cern.jet.stat.Probability.chiSquareComplemented + +import org.apache.spark.{SparkException, Logging} +import org.apache.spark.mllib.linalg.{Matrices, Matrix, Vector, Vectors} +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.rdd.RDD + +import scala.collection.mutable + +/** + * Conduct the chi-squared test for the input RDDs using the specified method. + * Goodness-of-fit test is conducted on two `Vectors`, whereas test of independence is conducted + * on an input of type `Matrix` in which independence between columns is assessed. + * We also provide a method for computing the chi-squared statistic between each feature and the + * label for an input `RDD[LabeledPoint]`, return an `Array[ChiSquaredTestResult]` of size = + * number of features in the inpuy RDD. + * + * Supported methods for goodness of fit: `pearson` (default) + * Supported methods for independence: `pearson` (default) + * + * More information on Chi-squared test: http://en.wikipedia.org/wiki/Chi-squared_test + */ +private[stat] object ChiSqTest extends Logging { + + /** + * @param name String name for the method. + * @param chiSqFunc Function for computing the statistic given the observed and expected counts. + */ + case class Method(name: String, chiSqFunc: (Double, Double) => Double) + + // Pearson's chi-squared test: http://en.wikipedia.org/wiki/Pearson%27s_chi-squared_test + val PEARSON = new Method("pearson", (observed: Double, expected: Double) => { + val dev = observed - expected + dev * dev / expected + }) + + // Null hypothesis for the two different types of chi-squared tests to be included in the result. + object NullHypothesis extends Enumeration { + type NullHypothesis = Value + val goodnessOfFit = Value("observed follows the same distribution as expected.") + val independence = Value("the occurrence of the outcomes is statistically independent.") + } + + // Method identification based on input methodName string + private def methodFromString(methodName: String): Method = { + methodName match { + case PEARSON.name => PEARSON + case _ => throw new IllegalArgumentException("Unrecognized method for Chi squared test.") + } + } + + /** + * Conduct Pearson's independence test for each feature against the label across the input RDD. + * The contingency table is constructed from the raw (feature, label) pairs and used to conduct + * the independence test. + * Returns an array containing the ChiSquaredTestResult for every feature against the label. + */ + def chiSquaredFeatures(data: RDD[LabeledPoint], + methodName: String = PEARSON.name): Array[ChiSqTestResult] = { + val maxCategories = 10000 + val numCols = data.first().features.size + val results = new Array[ChiSqTestResult](numCols) + var labels: Map[Double, Int] = null + // at most 1000 columns at a time + val batchSize = 1000 + var batch = 0 + while (batch * batchSize < numCols) { + // The following block of code can be cleaned up and made public as + // chiSquared(data: RDD[(V1, V2)]) + val startCol = batch * batchSize + val endCol = startCol + math.min(batchSize, numCols - startCol) + val pairCounts = data.mapPartitions { iter => + val distinctLabels = mutable.HashSet.empty[Double] + val allDistinctFeatures: Map[Int, mutable.HashSet[Double]] = + Map((startCol until endCol).map(col => (col, mutable.HashSet.empty[Double])): _*) + var i = 1 + iter.flatMap { case LabeledPoint(label, features) => + if (i % 1000 == 0) { + if (distinctLabels.size > maxCategories) { + throw new SparkException(s"Chi-square test expect factors (categorical values) but " + + s"found more than $maxCategories distinct label values.") + } + allDistinctFeatures.foreach { case (col, distinctFeatures) => + if (distinctFeatures.size > maxCategories) { + throw new SparkException(s"Chi-square test expect factors (categorical values) but " + + s"found more than $maxCategories distinct values in column $col.") + } + } + } + i += 1 + distinctLabels += label + features.toArray.view.zipWithIndex.slice(startCol, endCol).map { case (feature, col) => + allDistinctFeatures(col) += feature + (col, feature, label) + } + } + }.countByValue() + + if (labels == null) { + // Do this only once for the first column since labels are invariant across features. + labels = + pairCounts.keys.filter(_._1 == startCol).map(_._3).toArray.distinct.zipWithIndex.toMap + } + val numLabels = labels.size + pairCounts.keys.groupBy(_._1).map { case (col, keys) => + val features = keys.map(_._2).toArray.distinct.zipWithIndex.toMap + val numRows = features.size + val contingency = new BDM(numRows, numLabels, new Array[Double](numRows * numLabels)) + keys.foreach { case (_, feature, label) => + val i = features(feature) + val j = labels(label) + contingency(i, j) += pairCounts((col, feature, label)) + } + results(col) = chiSquaredMatrix(Matrices.fromBreeze(contingency), methodName) + } + batch += 1 + } + results + } + + /* + * Pearon's goodness of fit test on the input observed and expected counts/relative frequencies. + * Uniform distribution is assumed when `expected` is not passed in. + */ + def chiSquared(observed: Vector, + expected: Vector = Vectors.dense(Array[Double]()), + methodName: String = PEARSON.name): ChiSqTestResult = { + + // Validate input arguments + val method = methodFromString(methodName) + if (expected.size != 0 && observed.size != expected.size) { + throw new IllegalArgumentException("observed and expected must be of the same size.") + } + val size = observed.size + if (size > 1000) { + logWarning("Chi-squared approximation may not be accurate due to low expected frequencies " + + s" as a result of a large number of categories: $size.") + } + val obsArr = observed.toArray + val expArr = if (expected.size == 0) Array.tabulate(size)(_ => 1.0 / size) else expected.toArray + if (!obsArr.forall(_ >= 0.0)) { + throw new IllegalArgumentException("Negative entries disallowed in the observed vector.") + } + if (expected.size != 0 && ! expArr.forall(_ >= 0.0)) { + throw new IllegalArgumentException("Negative entries disallowed in the expected vector.") + } + + // Determine the scaling factor for expected + val obsSum = obsArr.sum + val expSum = if (expected.size == 0.0) 1.0 else expArr.sum + val scale = if (math.abs(obsSum - expSum) < 1e-7) 1.0 else obsSum / expSum + + // compute chi-squared statistic + val statistic = obsArr.zip(expArr).foldLeft(0.0) { case (stat, (obs, exp)) => + if (exp == 0.0) { + if (obs == 0.0) { + throw new IllegalArgumentException("Chi-squared statistic undefined for input vectors due" + + " to 0.0 values in both observed and expected.") + } else { + return new ChiSqTestResult(0.0, size - 1, Double.PositiveInfinity, PEARSON.name, + NullHypothesis.goodnessOfFit.toString) + } + } + if (scale == 1.0) { + stat + method.chiSqFunc(obs, exp) + } else { + stat + method.chiSqFunc(obs, exp * scale) + } + } + val df = size - 1 + val pValue = chiSquareComplemented(df, statistic) + new ChiSqTestResult(pValue, df, statistic, PEARSON.name, NullHypothesis.goodnessOfFit.toString) + } + + /* + * Pearon's independence test on the input contingency matrix. + * TODO: optimize for SparseMatrix when it becomes supported. + */ + def chiSquaredMatrix(counts: Matrix, methodName:String = PEARSON.name): ChiSqTestResult = { + val method = methodFromString(methodName) + val numRows = counts.numRows + val numCols = counts.numCols + + // get row and column sums + val colSums = new Array[Double](numCols) + val rowSums = new Array[Double](numRows) + val colMajorArr = counts.toArray + var i = 0 + while (i < colMajorArr.size) { + val elem = colMajorArr(i) + if (elem < 0.0) { + throw new IllegalArgumentException("Contingency table cannot contain negative entries.") + } + colSums(i / numRows) += elem + rowSums(i % numRows) += elem + i += 1 + } + val total = colSums.sum + + // second pass to collect statistic + var statistic = 0.0 + var j = 0 + while (j < colMajorArr.size) { + val col = j / numRows + val colSum = colSums(col) + if (colSum == 0.0) { + throw new IllegalArgumentException("Chi-squared statistic undefined for input matrix due to" + + s"0 sum in column [$col].") + } + val row = j % numRows + val rowSum = rowSums(row) + if (rowSum == 0.0) { + throw new IllegalArgumentException("Chi-squared statistic undefined for input matrix due to" + + s"0 sum in row [$row].") + } + val expected = colSum * rowSum / total + statistic += method.chiSqFunc(colMajorArr(j), expected) + j += 1 + } + val df = (numCols - 1) * (numRows - 1) + val pValue = chiSquareComplemented(df, statistic) + new ChiSqTestResult(pValue, df, statistic, methodName, NullHypothesis.independence.toString) + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/TestResult.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/TestResult.scala new file mode 100644 index 0000000000000..4784f9e947908 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/TestResult.scala @@ -0,0 +1,92 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.stat.test + +import org.apache.spark.annotation.Experimental + +/** + * :: Experimental :: + * Trait for hypothesis test results. + * @tparam DF Return type of `degreesOfFreedom`. + */ +@Experimental +trait TestResult[DF] { + + /** + * The probability of obtaining a test statistic result at least as extreme as the one that was + * actually observed, assuming that the null hypothesis is true. + */ + def pValue: Double + + /** + * Returns the degree(s) of freedom of the hypothesis test. + * Return type should be Number(e.g. Int, Double) or tuples of Numbers for toString compatibility. + */ + def degreesOfFreedom: DF + + /** + * Test statistic. + */ + def statistic: Double + + /** + * Null hypothesis of the test. + */ + def nullHypothesis: String + + /** + * String explaining the hypothesis test result. + * Specific classes implementing this trait should override this method to output test-specific + * information. + */ + override def toString: String = { + + // String explaining what the p-value indicates. + val pValueExplain = if (pValue <= 0.01) { + s"Very strong presumption against null hypothesis: $nullHypothesis." + } else if (0.01 < pValue && pValue <= 0.05) { + s"Strong presumption against null hypothesis: $nullHypothesis." + } else if (0.05 < pValue && pValue <= 0.1) { + s"Low presumption against null hypothesis: $nullHypothesis." + } else { + s"No presumption against null hypothesis: $nullHypothesis." + } + + s"degrees of freedom = ${degreesOfFreedom.toString} \n" + + s"statistic = $statistic \n" + + s"pValue = $pValue \n" + pValueExplain + } +} + +/** + * :: Experimental :: + * Object containing the test results for the chi-squared hypothesis test. + */ +@Experimental +class ChiSqTestResult private[stat] (override val pValue: Double, + override val degreesOfFreedom: Int, + override val statistic: Double, + val method: String, + override val nullHypothesis: String) extends TestResult[Int] { + + override def toString: String = { + "Chi squared test summary:\n" + + s"method: $method\n" + + super.toString + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala index 74d5d7ba10960..6b9a8f72c244e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala @@ -17,22 +17,28 @@ package org.apache.spark.mllib.tree +import scala.collection.JavaConverters._ + import org.apache.spark.annotation.Experimental +import org.apache.spark.api.java.JavaRDD import org.apache.spark.Logging import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.Strategy import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.configuration.FeatureType._ import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ -import org.apache.spark.mllib.tree.impurity.Impurity +import org.apache.spark.mllib.tree.impl.{DecisionTreeMetadata, TimeTracker, TreePoint} +import org.apache.spark.mllib.tree.impurity.{Impurities, Impurity} import org.apache.spark.mllib.tree.model._ import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel import org.apache.spark.util.random.XORShiftRandom + /** * :: Experimental :: - * A class that implements a decision tree algorithm for classification and regression. It - * supports both continuous and categorical features. + * A class which implements a decision tree learning algorithm for classification and regression. + * It supports both continuous and categorical features. * @param strategy The configuration parameters for the tree algorithm which specify the type * of algorithm (classification, regression, etc.), feature type (continuous, * categorical), depth of the tree, quantile calculation strategy, etc. @@ -40,48 +46,54 @@ import org.apache.spark.util.random.XORShiftRandom @Experimental class DecisionTree (private val strategy: Strategy) extends Serializable with Logging { + strategy.assertValid() + /** * Method to train a decision tree model over an RDD - * @param input RDD of [[org.apache.spark.mllib.regression.LabeledPoint]] used as training data - * @return a DecisionTreeModel that can be used for prediction + * @param input Training data: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]] + * @return DecisionTreeModel that can be used for prediction */ def train(input: RDD[LabeledPoint]): DecisionTreeModel = { - // Cache input RDD for speedup during multiple passes. - input.cache() + val timer = new TimeTracker() + + timer.start("total") + + timer.start("init") + + val retaggedInput = input.retag(classOf[LabeledPoint]) + val metadata = DecisionTreeMetadata.buildMetadata(retaggedInput, strategy) logDebug("algo = " + strategy.algo) // Find the splits and the corresponding bins (interval between the splits) using a sample // of the input data. - val (splits, bins) = DecisionTree.findSplitsBins(input, strategy) + timer.start("findSplitsBins") + val (splits, bins) = DecisionTree.findSplitsBins(retaggedInput, metadata) val numBins = bins(0).length + timer.stop("findSplitsBins") logDebug("numBins = " + numBins) + // Bin feature values (TreePoint representation). + // Cache input RDD for speedup during multiple passes. + val treeInput = TreePoint.convertToTreeRDD(retaggedInput, bins, metadata) + .persist(StorageLevel.MEMORY_AND_DISK) + + val numFeatures = metadata.numFeatures // depth of the decision tree val maxDepth = strategy.maxDepth // the max number of nodes possible given the depth of the tree - val maxNumNodes = math.pow(2, maxDepth).toInt - 1 - // Initialize an array to hold filters applied to points for each node. - val filters = new Array[List[Filter]](maxNumNodes) - // The filter at the top node is an empty list. - filters(0) = List() + val maxNumNodes = (2 << maxDepth) - 1 // Initialize an array to hold parent impurity calculations for each node. val parentImpurities = new Array[Double](maxNumNodes) // dummy value for top node (updated during first split calculation) val nodes = new Array[Node](maxNumNodes) - // num features - val numFeatures = input.take(1)(0).features.size // Calculate level for single group construction // Max memory usage for aggregates val maxMemoryUsage = strategy.maxMemoryInMB * 1024 * 1024 logDebug("max memory usage for aggregates = " + maxMemoryUsage + " bytes.") - val numElementsPerNode = - strategy.algo match { - case Classification => 2 * numBins * numFeatures - case Regression => 3 * numBins * numFeatures - } + val numElementsPerNode = DecisionTree.getElementsPerNode(metadata, numBins) logDebug("numElementsPerNode = " + numElementsPerNode) val arraySizePerNode = 8 * numElementsPerNode // approx. memory usage for bin aggregate array @@ -92,35 +104,57 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo (math.log(maxNumberOfNodesPerGroup) / math.log(2)).floor.toInt, 0) logDebug("max level for single group = " + maxLevelForSingleGroup) + timer.stop("init") + /* * The main idea here is to perform level-wise training of the decision tree nodes thus * reducing the passes over the data from l to log2(l) where l is the total number of nodes. - * Each data sample is checked for validity w.r.t to each node at a given level -- i.e., - * the sample is only used for the split calculation at the node if the sampled would have - * still survived the filters of the parent nodes. + * Each data sample is handled by a particular node at that level (or it reaches a leaf + * beforehand and is not used in later levels. */ var level = 0 var break = false - while (level < maxDepth && !break) { + while (level <= maxDepth && !break) { logDebug("#####################################") logDebug("level = " + level) logDebug("#####################################") // Find best split for all nodes at a level. - val splitsStatsForLevel = DecisionTree.findBestSplits(input, parentImpurities, strategy, - level, filters, splits, bins, maxLevelForSingleGroup) + timer.start("findBestSplits") + val splitsStatsForLevel = DecisionTree.findBestSplits(treeInput, parentImpurities, + metadata, level, nodes, splits, bins, maxLevelForSingleGroup, timer) + timer.stop("findBestSplits") + val levelNodeIndexOffset = (1 << level) - 1 for ((nodeSplitStats, index) <- splitsStatsForLevel.view.zipWithIndex) { - // Extract info for nodes at the current level. + val nodeIndex = levelNodeIndexOffset + index + val isLeftChild = level != 0 && nodeIndex % 2 == 1 + val parentNodeIndex = if (isLeftChild) { // -1 for root node + (nodeIndex - 1) / 2 + } else { + (nodeIndex - 2) / 2 + } + // Extract info for this node (index) at the current level. + timer.start("extractNodeInfo") extractNodeInfo(nodeSplitStats, level, index, nodes) + timer.stop("extractNodeInfo") + if (level != 0) { + // Set parent. + if (isLeftChild) { + nodes(parentNodeIndex).leftNode = Some(nodes(nodeIndex)) + } else { + nodes(parentNodeIndex).rightNode = Some(nodes(nodeIndex)) + } + } // Extract info for nodes at the next lower level. - extractInfoForLowerLevels(level, index, maxDepth, nodeSplitStats, parentImpurities, - filters) + timer.start("extractInfoForLowerLevels") + extractInfoForLowerLevels(level, index, maxDepth, nodeSplitStats, parentImpurities) + timer.stop("extractInfoForLowerLevels") logDebug("final best split = " + nodeSplitStats._1) } - require(math.pow(2, level) == splitsStatsForLevel.length) + require((1 << level) == splitsStatsForLevel.length) // Check whether all the nodes at the current level at leaves. val allLeaf = splitsStatsForLevel.forall(_._2.gain <= 0) logDebug("all leaf = " + allLeaf) @@ -140,6 +174,11 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo // Build the full tree using the node info calculated in the level-wise best split calculations. topNode.build(nodes) + timer.stop("total") + + logInfo("Internal timing for DecisionTree:") + logInfo(s"$timer") + new DecisionTreeModel(topNode, strategy.algo) } @@ -153,8 +192,8 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo nodes: Array[Node]): Unit = { val split = nodeSplitStats._1 val stats = nodeSplitStats._2 - val nodeIndex = math.pow(2, level).toInt - 1 + index - val isLeaf = (stats.gain <= 0) || (level == strategy.maxDepth - 1) + val nodeIndex = (1 << level) - 1 + index + val isLeaf = (stats.gain <= 0) || (level == strategy.maxDepth) val node = new Node(nodeIndex, stats.predict, isLeaf, Some(split), None, None, Some(stats)) logDebug("Node = " + node) nodes(nodeIndex) = node @@ -168,189 +207,299 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo index: Int, maxDepth: Int, nodeSplitStats: (Split, InformationGainStats), - parentImpurities: Array[Double], - filters: Array[List[Filter]]): Unit = { - // 0 corresponds to the left child node and 1 corresponds to the right child node. - var i = 0 - while (i <= 1) { - // Calculate the index of the node from the node level and the index at the current level. - val nodeIndex = math.pow(2, level + 1).toInt - 1 + 2 * index + i - if (level < maxDepth - 1) { - val impurity = if (i == 0) { - nodeSplitStats._2.leftImpurity - } else { - nodeSplitStats._2.rightImpurity - } - logDebug("nodeIndex = " + nodeIndex + ", impurity = " + impurity) - // noting the parent impurities - parentImpurities(nodeIndex) = impurity - // noting the parents filters for the child nodes - val childFilter = new Filter(nodeSplitStats._1, if (i == 0) -1 else 1) - filters(nodeIndex) = childFilter :: filters((nodeIndex - 1) / 2) - for (filter <- filters(nodeIndex)) { - logDebug("Filter = " + filter) - } - } - i += 1 + parentImpurities: Array[Double]): Unit = { + + if (level >= maxDepth) { + return } + + val leftNodeIndex = (2 << level) - 1 + 2 * index + val leftImpurity = nodeSplitStats._2.leftImpurity + logDebug("leftNodeIndex = " + leftNodeIndex + ", impurity = " + leftImpurity) + parentImpurities(leftNodeIndex) = leftImpurity + + val rightNodeIndex = leftNodeIndex + 1 + val rightImpurity = nodeSplitStats._2.rightImpurity + logDebug("rightNodeIndex = " + rightNodeIndex + ", impurity = " + rightImpurity) + parentImpurities(rightNodeIndex) = rightImpurity } } object DecisionTree extends Serializable with Logging { /** - * Method to train a decision tree model where the instances are represented as an RDD of - * (label, features) pairs. The method supports binary classification and regression. For the - * binary classification, the label for each instance should either be 0 or 1 to denote the two - * classes. The parameters for the algorithm are specified using the strategy parameter. + * Method to train a decision tree model. + * The method supports binary and multiclass classification and regression. * - * @param input RDD of [[org.apache.spark.mllib.regression.LabeledPoint]] used as training data - * for DecisionTree + * Note: Using [[org.apache.spark.mllib.tree.DecisionTree$#trainClassifier]] + * and [[org.apache.spark.mllib.tree.DecisionTree$#trainRegressor]] + * is recommended to clearly separate classification and regression. + * + * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. + * For classification, labels should take values {0, 1, ..., numClasses-1}. + * For regression, labels are real numbers. * @param strategy The configuration parameters for the tree algorithm which specify the type * of algorithm (classification, regression, etc.), feature type (continuous, * categorical), depth of the tree, quantile calculation strategy, etc. - * @return a DecisionTreeModel that can be used for prediction + * @return DecisionTreeModel that can be used for prediction */ def train(input: RDD[LabeledPoint], strategy: Strategy): DecisionTreeModel = { - new DecisionTree(strategy).train(input: RDD[LabeledPoint]) + new DecisionTree(strategy).train(input) } /** - * Method to train a decision tree model where the instances are represented as an RDD of - * (label, features) pairs. The method supports binary classification and regression. For the - * binary classification, the label for each instance should either be 0 or 1 to denote the two - * classes. + * Method to train a decision tree model. + * The method supports binary and multiclass classification and regression. + * + * Note: Using [[org.apache.spark.mllib.tree.DecisionTree$#trainClassifier]] + * and [[org.apache.spark.mllib.tree.DecisionTree$#trainRegressor]] + * is recommended to clearly separate classification and regression. * - * @param input input RDD of [[org.apache.spark.mllib.regression.LabeledPoint]] used as - * training data + * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. + * For classification, labels should take values {0, 1, ..., numClasses-1}. + * For regression, labels are real numbers. * @param algo algorithm, classification or regression * @param impurity impurity criterion used for information gain calculation - * @param maxDepth maxDepth maximum depth of the tree - * @return a DecisionTreeModel that can be used for prediction + * @param maxDepth Maximum depth of the tree. + * E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes. + * @return DecisionTreeModel that can be used for prediction */ def train( input: RDD[LabeledPoint], algo: Algo, impurity: Impurity, maxDepth: Int): DecisionTreeModel = { - val strategy = new Strategy(algo,impurity,maxDepth) - new DecisionTree(strategy).train(input: RDD[LabeledPoint]) + val strategy = new Strategy(algo, impurity, maxDepth) + new DecisionTree(strategy).train(input) } + /** + * Method to train a decision tree model. + * The method supports binary and multiclass classification and regression. + * + * Note: Using [[org.apache.spark.mllib.tree.DecisionTree$#trainClassifier]] + * and [[org.apache.spark.mllib.tree.DecisionTree$#trainRegressor]] + * is recommended to clearly separate classification and regression. + * + * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. + * For classification, labels should take values {0, 1, ..., numClasses-1}. + * For regression, labels are real numbers. + * @param algo algorithm, classification or regression + * @param impurity impurity criterion used for information gain calculation + * @param maxDepth Maximum depth of the tree. + * E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes. + * @param numClassesForClassification number of classes for classification. Default value of 2. + * @return DecisionTreeModel that can be used for prediction + */ + def train( + input: RDD[LabeledPoint], + algo: Algo, + impurity: Impurity, + maxDepth: Int, + numClassesForClassification: Int): DecisionTreeModel = { + val strategy = new Strategy(algo, impurity, maxDepth, numClassesForClassification) + new DecisionTree(strategy).train(input) + } /** - * Method to train a decision tree model where the instances are represented as an RDD of - * (label, features) pairs. The decision tree method supports binary classification and - * regression. For the binary classification, the label for each instance should either be 0 or - * 1 to denote the two classes. The method also supports categorical features inputs where the - * number of categories can specified using the categoricalFeaturesInfo option. + * Method to train a decision tree model. + * The method supports binary and multiclass classification and regression. + * + * Note: Using [[org.apache.spark.mllib.tree.DecisionTree$#trainClassifier]] + * and [[org.apache.spark.mllib.tree.DecisionTree$#trainRegressor]] + * is recommended to clearly separate classification and regression. * - * @param input input RDD of [[org.apache.spark.mllib.regression.LabeledPoint]] used as - * training data for DecisionTree + * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. + * For classification, labels should take values {0, 1, ..., numClasses-1}. + * For regression, labels are real numbers. * @param algo classification or regression * @param impurity criterion used for information gain calculation - * @param maxDepth maximum depth of the tree + * @param maxDepth Maximum depth of the tree. + * E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes. + * @param numClassesForClassification number of classes for classification. Default value of 2. * @param maxBins maximum number of bins used for splitting features * @param quantileCalculationStrategy algorithm for calculating quantiles - * @param categoricalFeaturesInfo A map storing information about the categorical variables and - * the number of discrete values they take. For example, - * an entry (n -> k) implies the feature n is categorical with k - * categories 0, 1, 2, ... , k-1. It's important to note that - * features are zero-indexed. - * @return a DecisionTreeModel that can be used for prediction + * @param categoricalFeaturesInfo Map storing arity of categorical features. + * E.g., an entry (n -> k) indicates that feature n is categorical + * with k categories indexed from 0: {0, 1, ..., k-1}. + * @return DecisionTreeModel that can be used for prediction */ def train( input: RDD[LabeledPoint], algo: Algo, impurity: Impurity, maxDepth: Int, + numClassesForClassification: Int, maxBins: Int, quantileCalculationStrategy: QuantileStrategy, categoricalFeaturesInfo: Map[Int,Int]): DecisionTreeModel = { - val strategy = new Strategy(algo, impurity, maxDepth, maxBins, quantileCalculationStrategy, + val strategy = new Strategy(algo, impurity, maxDepth, numClassesForClassification, maxBins, + quantileCalculationStrategy, categoricalFeaturesInfo) + new DecisionTree(strategy).train(input) + } + + /** + * Method to train a decision tree model for binary or multiclass classification. + * + * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. + * Labels should take values {0, 1, ..., numClasses-1}. + * @param numClassesForClassification number of classes for classification. + * @param categoricalFeaturesInfo Map storing arity of categorical features. + * E.g., an entry (n -> k) indicates that feature n is categorical + * with k categories indexed from 0: {0, 1, ..., k-1}. + * @param impurity Criterion used for information gain calculation. + * Supported values: "gini" (recommended) or "entropy". + * @param maxDepth Maximum depth of the tree. + * E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes. + * (suggested value: 4) + * @param maxBins maximum number of bins used for splitting features + * (suggested value: 100) + * @return DecisionTreeModel that can be used for prediction + */ + def trainClassifier( + input: RDD[LabeledPoint], + numClassesForClassification: Int, + categoricalFeaturesInfo: Map[Int, Int], + impurity: String, + maxDepth: Int, + maxBins: Int): DecisionTreeModel = { + val impurityType = Impurities.fromString(impurity) + train(input, Classification, impurityType, maxDepth, numClassesForClassification, maxBins, Sort, categoricalFeaturesInfo) - new DecisionTree(strategy).train(input: RDD[LabeledPoint]) } + /** + * Java-friendly API for [[org.apache.spark.mllib.tree.DecisionTree$#trainClassifier]] + */ + def trainClassifier( + input: JavaRDD[LabeledPoint], + numClassesForClassification: Int, + categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer], + impurity: String, + maxDepth: Int, + maxBins: Int): DecisionTreeModel = { + trainClassifier(input.rdd, numClassesForClassification, + categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap, + impurity, maxDepth, maxBins) + } + + /** + * Method to train a decision tree model for regression. + * + * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. + * Labels are real numbers. + * @param categoricalFeaturesInfo Map storing arity of categorical features. + * E.g., an entry (n -> k) indicates that feature n is categorical + * with k categories indexed from 0: {0, 1, ..., k-1}. + * @param impurity Criterion used for information gain calculation. + * Supported values: "variance". + * @param maxDepth Maximum depth of the tree. + * E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes. + * (suggested value: 4) + * @param maxBins maximum number of bins used for splitting features + * (suggested value: 100) + * @return DecisionTreeModel that can be used for prediction + */ + def trainRegressor( + input: RDD[LabeledPoint], + categoricalFeaturesInfo: Map[Int, Int], + impurity: String, + maxDepth: Int, + maxBins: Int): DecisionTreeModel = { + val impurityType = Impurities.fromString(impurity) + train(input, Regression, impurityType, maxDepth, 0, maxBins, Sort, categoricalFeaturesInfo) + } + + /** + * Java-friendly API for [[org.apache.spark.mllib.tree.DecisionTree$#trainRegressor]] + */ + def trainRegressor( + input: JavaRDD[LabeledPoint], + categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer], + impurity: String, + maxDepth: Int, + maxBins: Int): DecisionTreeModel = { + trainRegressor(input.rdd, + categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap, + impurity, maxDepth, maxBins) + } + + private val InvalidBinIndex = -1 /** * Returns an array of optimal splits for all nodes at a given level. Splits the task into * multiple groups if the level-wise training task could lead to memory overflow. * - * @param input RDD of [[org.apache.spark.mllib.regression.LabeledPoint]] used as training data - * for DecisionTree + * @param input Training data: RDD of [[org.apache.spark.mllib.tree.impl.TreePoint]] * @param parentImpurities Impurities for all parent nodes for the current level - * @param strategy [[org.apache.spark.mllib.tree.configuration.Strategy]] instance containing - * parameters for construction the DecisionTree + * @param metadata Learning and dataset metadata * @param level Level of the tree - * @param filters Filters for all nodes at a given level * @param splits possible splits for all features * @param bins possible bins for all features * @param maxLevelForSingleGroup the deepest level for single-group level-wise computation. - * @return array of splits with best splits for all nodes at a given level. + * @return array (over nodes) of splits with best split for each node at a given level. */ protected[tree] def findBestSplits( - input: RDD[LabeledPoint], + input: RDD[TreePoint], parentImpurities: Array[Double], - strategy: Strategy, + metadata: DecisionTreeMetadata, level: Int, - filters: Array[List[Filter]], + nodes: Array[Node], splits: Array[Array[Split]], bins: Array[Array[Bin]], - maxLevelForSingleGroup: Int): Array[(Split, InformationGainStats)] = { + maxLevelForSingleGroup: Int, + timer: TimeTracker = new TimeTracker): Array[(Split, InformationGainStats)] = { // split into groups to avoid memory overflow during aggregation if (level > maxLevelForSingleGroup) { // When information for all nodes at a given level cannot be stored in memory, // the nodes are divided into multiple groups at each level with the number of groups // increasing exponentially per level. For example, if maxLevelForSingleGroup is 10, // numGroups is equal to 2 at level 11 and 4 at level 12, respectively. - val numGroups = math.pow(2, (level - maxLevelForSingleGroup)).toInt + val numGroups = 1 << level - maxLevelForSingleGroup logDebug("numGroups = " + numGroups) var bestSplits = new Array[(Split, InformationGainStats)](0) // Iterate over each group of nodes at a level. var groupIndex = 0 while (groupIndex < numGroups) { - val bestSplitsForGroup = findBestSplitsPerGroup(input, parentImpurities, strategy, level, - filters, splits, bins, numGroups, groupIndex) + val bestSplitsForGroup = findBestSplitsPerGroup(input, parentImpurities, metadata, level, + nodes, splits, bins, timer, numGroups, groupIndex) bestSplits = Array.concat(bestSplits, bestSplitsForGroup) groupIndex += 1 } bestSplits } else { - findBestSplitsPerGroup(input, parentImpurities, strategy, level, filters, splits, bins) + findBestSplitsPerGroup(input, parentImpurities, metadata, level, nodes, splits, bins, timer) } } - /** + /** * Returns an array of optimal splits for a group of nodes at a given level * - * @param input RDD of [[org.apache.spark.mllib.regression.LabeledPoint]] used as training data - * for DecisionTree + * @param input Training data: RDD of [[org.apache.spark.mllib.tree.impl.TreePoint]] * @param parentImpurities Impurities for all parent nodes for the current level - * @param strategy [[org.apache.spark.mllib.tree.configuration.Strategy]] instance containing - * parameters for construction the DecisionTree + * @param metadata Learning and dataset metadata * @param level Level of the tree - * @param filters Filters for all nodes at a given level * @param splits possible splits for all features - * @param bins possible bins for all features + * @param bins possible bins for all features, indexed as (numFeatures)(numBins) * @param numGroups total number of node groups at the current level. Default value is set to 1. * @param groupIndex index of the node group being processed. Default value is set to 0. * @return array of splits with best splits for all nodes at a given level. */ private def findBestSplitsPerGroup( - input: RDD[LabeledPoint], + input: RDD[TreePoint], parentImpurities: Array[Double], - strategy: Strategy, + metadata: DecisionTreeMetadata, level: Int, - filters: Array[List[Filter]], + nodes: Array[Node], splits: Array[Array[Split]], bins: Array[Array[Bin]], + timer: TimeTracker, numGroups: Int = 1, groupIndex: Int = 0): Array[(Split, InformationGainStats)] = { /* - * The high-level description for the best split optimizations are noted here. + * The high-level descriptions of the best split optimizations are noted here. * * *Level-wise training* * We perform bin calculations for all nodes at the given level to avoid making multiple @@ -361,7 +510,7 @@ object DecisionTree extends Serializable with Logging { * We use a bin-wise best split computation strategy instead of a straightforward best split * computation strategy. Instead of analyzing each sample for contribution to the left/right * child node impurity of every split, we first categorize each feature of a sample into a - * bin. Each bin is an interval between a low and high split. Since each splits, and thus bin, + * bin. Each bin is an interval between a low and high split. Since each split, and thus bin, * is ordered (read ordering for categorical variables in the findSplitsBins method), * we exploit this structure to calculate aggregates for bins and then use these aggregates * to calculate information gain for each split. @@ -373,264 +522,289 @@ object DecisionTree extends Serializable with Logging { * drastically reduce the communication overhead. */ - // common calculations for multiple nested methods - val numNodes = math.pow(2, level).toInt / numGroups + // Common calculations for multiple nested methods: + + // numNodes: Number of nodes in this (level of tree, group), + // where nodes at deeper (larger) levels may be divided into groups. + val numNodes = (1 << level) / numGroups logDebug("numNodes = " + numNodes) + // Find the number of features by looking at the first sample. - val numFeatures = input.first().features.size + val numFeatures = metadata.numFeatures logDebug("numFeatures = " + numFeatures) + + // numBins: Number of bins = 1 + number of possible splits val numBins = bins(0).length logDebug("numBins = " + numBins) + val numClasses = metadata.numClasses + logDebug("numClasses = " + numClasses) + + val isMulticlass = metadata.isMulticlass + logDebug("isMulticlass = " + isMulticlass) + + val isMulticlassWithCategoricalFeatures = metadata.isMulticlassWithCategoricalFeatures + logDebug("isMultiClassWithCategoricalFeatures = " + isMulticlassWithCategoricalFeatures) + // shift when more than one group is used at deep tree level val groupShift = numNodes * groupIndex - /** Find the filters used before reaching the current code. */ - def findParentFilters(nodeIndex: Int): List[Filter] = { - if (level == 0) { - List[Filter]() - } else { - val nodeFilterIndex = math.pow(2, level).toInt - 1 + nodeIndex + groupShift - filters(nodeFilterIndex) - } - } - /** - * Find whether the sample is valid input for the current node, i.e., whether it passes through - * all the filters for the current node. + * Get the node index corresponding to this data point. + * This function mimics prediction, passing an example from the root node down to a node + * at the current level being trained; that node's index is returned. + * + * @return Leaf index if the data point reaches a leaf. + * Otherwise, last node reachable in tree matching this example. */ - def isSampleValid(parentFilters: List[Filter], labeledPoint: LabeledPoint): Boolean = { - // leaf - if ((level > 0) && (parentFilters.length == 0)) { - return false - } - - // Apply each filter and check sample validity. Return false when invalid condition found. - for (filter <- parentFilters) { - val features = labeledPoint.features - val featureIndex = filter.split.feature - val threshold = filter.split.threshold - val comparison = filter.comparison - val categories = filter.split.categories - val isFeatureContinuous = filter.split.featureType == Continuous - val feature = features(featureIndex) - if (isFeatureContinuous) { - comparison match { - case -1 => if (feature > threshold) return false - case 1 => if (feature <= threshold) return false + def predictNodeIndex(node: Node, binnedFeatures: Array[Int]): Int = { + if (node.isLeaf) { + node.id + } else { + val featureIndex = node.split.get.feature + val splitLeft = node.split.get.featureType match { + case Continuous => { + val binIndex = binnedFeatures(featureIndex) + val featureValueUpperBound = bins(featureIndex)(binIndex).highSplit.threshold + // bin binIndex has range (bin.lowSplit.threshold, bin.highSplit.threshold] + // We do not need to check lowSplit since bins are separated by splits. + featureValueUpperBound <= node.split.get.threshold + } + case Categorical => { + val featureValue = if (metadata.isUnordered(featureIndex)) { + binnedFeatures(featureIndex) + } else { + val binIndex = binnedFeatures(featureIndex) + bins(featureIndex)(binIndex).category + } + node.split.get.categories.contains(featureValue) + } + case _ => throw new RuntimeException(s"predictNodeIndex failed for unknown reason.") + } + if (node.leftNode.isEmpty || node.rightNode.isEmpty) { + // Return index from next layer of nodes to train + if (splitLeft) { + node.id * 2 + 1 // left + } else { + node.id * 2 + 2 // right } } else { - val containsFeature = categories.contains(feature) - comparison match { - case -1 => if (!containsFeature) return false - case 1 => if (containsFeature) return false + if (splitLeft) { + predictNodeIndex(node.leftNode.get, binnedFeatures) + } else { + predictNodeIndex(node.rightNode.get, binnedFeatures) } - } } + } - // Return true when the sample is valid for all filters. - true + def nodeIndexToLevel(idx: Int): Int = { + if (idx == 0) { + 0 + } else { + math.floor(math.log(idx) / math.log(2)).toInt + } } + // Used for treePointToNodeIndex + val levelOffset = (1 << level) - 1 + /** - * Find bin for one feature. + * Find the node index for the given example. + * Nodes are indexed from 0 at the start of this (level, group). + * If the example does not reach this level, returns a value < 0. */ - def findBin( - featureIndex: Int, - labeledPoint: LabeledPoint, - isFeatureContinuous: Boolean): Int = { - val binForFeatures = bins(featureIndex) - val feature = labeledPoint.features(featureIndex) - - /** - * Binary search helper method for continuous feature. - */ - def binarySearchForBins(): Int = { - var left = 0 - var right = binForFeatures.length - 1 - while (left <= right) { - val mid = left + (right - left) / 2 - val bin = binForFeatures(mid) - val lowThreshold = bin.lowSplit.threshold - val highThreshold = bin.highSplit.threshold - if ((lowThreshold < feature) && (highThreshold >= feature)){ - return mid - } - else if (lowThreshold >= feature) { - right = mid - 1 - } - else { - left = mid + 1 - } - } - -1 + def treePointToNodeIndex(treePoint: TreePoint): Int = { + if (level == 0) { + 0 + } else { + val globalNodeIndex = predictNodeIndex(nodes(0), treePoint.binnedFeatures) + // Get index for this (level, group). + globalNodeIndex - levelOffset - groupShift } + } - /** - * Sequential search helper method to find bin for categorical feature. - */ - def sequentialBinSearchForCategoricalFeature(): Int = { - val numCategoricalBins = strategy.categoricalFeaturesInfo(featureIndex) - var binIndex = 0 - while (binIndex < numCategoricalBins) { - val bin = bins(featureIndex)(binIndex) - val category = bin.category - val features = labeledPoint.features - if (category == features(featureIndex)) { - return binIndex - } - binIndex += 1 - } - -1 - } + /** + * Increment aggregate in location for (node, feature, bin, label). + * + * @param treePoint Data point being aggregated. + * @param agg Array storing aggregate calculation, of size: + * numClasses * numBins * numFeatures * numNodes. + * Indexed by (node, feature, bin, label) where label is the least significant bit. + * @param nodeIndex Node corresponding to treePoint. Indexed from 0 at start of (level, group). + */ + def updateBinForOrderedFeature( + treePoint: TreePoint, + agg: Array[Double], + nodeIndex: Int, + featureIndex: Int): Unit = { + // Update the left or right count for one bin. + val aggIndex = + numClasses * numBins * numFeatures * nodeIndex + + numClasses * numBins * featureIndex + + numClasses * treePoint.binnedFeatures(featureIndex) + + treePoint.label.toInt + agg(aggIndex) += 1 + } - if (isFeatureContinuous) { - // Perform binary search for finding bin for continuous features. - val binIndex = binarySearchForBins() - if (binIndex == -1){ - throw new UnknownError("no bin was found for continuous variable.") - } - binIndex - } else { - // Perform sequential search to find bin for categorical features. - val binIndex = sequentialBinSearchForCategoricalFeature() - if (binIndex == -1){ - throw new UnknownError("no bin was found for categorical variable.") + /** + * Increment aggregate in location for (nodeIndex, featureIndex, [bins], label), + * where [bins] ranges over all bins. + * Updates left or right side of aggregate depending on split. + * + * @param nodeIndex Node corresponding to treePoint. Indexed from 0 at start of (level, group). + * @param treePoint Data point being aggregated. + * @param agg Indexed by (left/right, node, feature, bin, label) + * where label is the least significant bit. + * The left/right specifier is a 0/1 index indicating left/right child info. + * @param rightChildShift Offset for right side of agg. + */ + def updateBinForUnorderedFeature( + nodeIndex: Int, + featureIndex: Int, + treePoint: TreePoint, + agg: Array[Double], + rightChildShift: Int): Unit = { + val featureValue = treePoint.binnedFeatures(featureIndex) + // Update the left or right count for one bin. + val aggShift = + numClasses * numBins * numFeatures * nodeIndex + + numClasses * numBins * featureIndex + + treePoint.label.toInt + // Find all matching bins and increment their values + val featureCategories = metadata.featureArity(featureIndex) + val numCategoricalBins = (1 << featureCategories - 1) - 1 + var binIndex = 0 + while (binIndex < numCategoricalBins) { + val aggIndex = aggShift + binIndex * numClasses + if (bins(featureIndex)(binIndex).highSplit.categories.contains(featureValue)) { + agg(aggIndex) += 1 + } else { + agg(rightChildShift + aggIndex) += 1 } - binIndex + binIndex += 1 } } /** - * Finds bins for all nodes (and all features) at a given level. - * For l nodes, k features the storage is as follows: - * label, b_11, b_12, .. , b_1k, b_21, b_22, .. , b_2k, b_l1, b_l2, .. , b_lk, - * where b_ij is an integer between 0 and numBins - 1. - * Invalid sample is denoted by noting bin for feature 1 as -1. + * Helper for binSeqOp. + * + * @param agg Array storing aggregate calculation, of size: + * numClasses * numBins * numFeatures * numNodes. + * Indexed by (node, feature, bin, label) where label is the least significant bit. + * @param treePoint Data point being aggregated. + * @param nodeIndex Node corresponding to treePoint. Indexed from 0 at start of (level, group). */ - def findBinsForLevel(labeledPoint: LabeledPoint): Array[Double] = { - // Calculate bin index and label per feature per node. - val arr = new Array[Double](1 + (numFeatures * numNodes)) - arr(0) = labeledPoint.label - var nodeIndex = 0 - while (nodeIndex < numNodes) { - val parentFilters = findParentFilters(nodeIndex) - // Find out whether the sample qualifies for the particular node. - val sampleValid = isSampleValid(parentFilters, labeledPoint) - val shift = 1 + numFeatures * nodeIndex - if (!sampleValid) { - // Mark one bin as -1 is sufficient. - arr(shift) = InvalidBinIndex - } else { - var featureIndex = 0 - while (featureIndex < numFeatures) { - val isFeatureContinuous = strategy.categoricalFeaturesInfo.get(featureIndex).isEmpty - arr(shift + featureIndex) = findBin(featureIndex, labeledPoint,isFeatureContinuous) - featureIndex += 1 - } - } - nodeIndex += 1 + def binaryOrNotCategoricalBinSeqOp( + agg: Array[Double], + treePoint: TreePoint, + nodeIndex: Int): Unit = { + // Iterate over all features. + var featureIndex = 0 + while (featureIndex < numFeatures) { + updateBinForOrderedFeature(treePoint, agg, nodeIndex, featureIndex) + featureIndex += 1 } - arr } + val rightChildShift = numClasses * numBins * numFeatures * numNodes + /** - * Performs a sequential aggregation over a partition for classification. For l nodes, - * k features, either the left count or the right count of one of the p bins is - * incremented based upon whether the feature is classified as 0 or 1. + * Helper for binSeqOp. * - * @param agg Array[Double] storing aggregate calculation of size - * 2 * numSplits * numFeatures*numNodes for classification - * @param arr Array[Double] of size 1 + (numFeatures * numNodes) - * @return Array[Double] storing aggregate calculation of size - * 2 * numSplits * numFeatures * numNodes for classification + * @param agg Array storing aggregate calculation. + * For ordered features, this is of size: + * numClasses * numBins * numFeatures * numNodes. + * For unordered features, this is of size: + * 2 * numClasses * numBins * numFeatures * numNodes. + * @param treePoint Data point being aggregated. + * @param nodeIndex Node corresponding to treePoint. Indexed from 0 at start of (level, group). */ - def classificationBinSeqOp(arr: Array[Double], agg: Array[Double]) { - // Iterate over all nodes. - var nodeIndex = 0 - while (nodeIndex < numNodes) { - // Check whether the instance was valid for this nodeIndex. - val validSignalIndex = 1 + numFeatures * nodeIndex - val isSampleValidForNode = arr(validSignalIndex) != InvalidBinIndex - if (isSampleValidForNode) { - // actual class label - val label = arr(0) - // Iterate over all features. - var featureIndex = 0 - while (featureIndex < numFeatures) { - // Find the bin index for this feature. - val arrShift = 1 + numFeatures * nodeIndex - val arrIndex = arrShift + featureIndex - // Update the left or right count for one bin. - val aggShift = 2 * numBins * numFeatures * nodeIndex - val aggIndex = aggShift + 2 * featureIndex * numBins + arr(arrIndex).toInt * 2 - label match { - case 0.0 => agg(aggIndex) = agg(aggIndex) + 1 - case 1.0 => agg(aggIndex + 1) = agg(aggIndex + 1) + 1 - } - featureIndex += 1 - } + def multiclassWithCategoricalBinSeqOp( + agg: Array[Double], + treePoint: TreePoint, + nodeIndex: Int): Unit = { + val label = treePoint.label + // Iterate over all features. + var featureIndex = 0 + while (featureIndex < numFeatures) { + if (metadata.isUnordered(featureIndex)) { + updateBinForUnorderedFeature(nodeIndex, featureIndex, treePoint, agg, rightChildShift) + } else { + updateBinForOrderedFeature(treePoint, agg, nodeIndex, featureIndex) } - nodeIndex += 1 + featureIndex += 1 } } /** - * Performs a sequential aggregation over a partition for regression. For l nodes, k features, + * Performs a sequential aggregation over a partition for regression. + * For l nodes, k features, * the count, sum, sum of squares of one of the p bins is incremented. * - * @param agg Array[Double] storing aggregate calculation of size - * 3 * numSplits * numFeatures * numNodes for classification - * @param arr Array[Double] of size 1 + (numFeatures * numNodes) - * @return Array[Double] storing aggregate calculation of size - * 3 * numSplits * numFeatures * numNodes for regression + * @param agg Array storing aggregate calculation, updated by this function. + * Size: 3 * numBins * numFeatures * numNodes + * @param treePoint Data point being aggregated. + * @param nodeIndex Node corresponding to treePoint. Indexed from 0 at start of (level, group). + * @return agg */ - def regressionBinSeqOp(arr: Array[Double], agg: Array[Double]) { - // Iterate over all nodes. - var nodeIndex = 0 - while (nodeIndex < numNodes) { - // Check whether the instance was valid for this nodeIndex. - val validSignalIndex = 1 + numFeatures * nodeIndex - val isSampleValidForNode = arr(validSignalIndex) != InvalidBinIndex - if (isSampleValidForNode) { - // actual class label - val label = arr(0) - // Iterate over all features. - var featureIndex = 0 - while (featureIndex < numFeatures) { - // Find the bin index for this feature. - val arrShift = 1 + numFeatures * nodeIndex - val arrIndex = arrShift + featureIndex - // Update count, sum, and sum^2 for one bin. - val aggShift = 3 * numBins * numFeatures * nodeIndex - val aggIndex = aggShift + 3 * featureIndex * numBins + arr(arrIndex).toInt * 3 - agg(aggIndex) = agg(aggIndex) + 1 - agg(aggIndex + 1) = agg(aggIndex + 1) + label - agg(aggIndex + 2) = agg(aggIndex + 2) + label*label - featureIndex += 1 - } - } - nodeIndex += 1 + def regressionBinSeqOp(agg: Array[Double], treePoint: TreePoint, nodeIndex: Int): Unit = { + val label = treePoint.label + // Iterate over all features. + var featureIndex = 0 + while (featureIndex < numFeatures) { + // Update count, sum, and sum^2 for one bin. + val binIndex = treePoint.binnedFeatures(featureIndex) + val aggIndex = + 3 * numBins * numFeatures * nodeIndex + + 3 * numBins * featureIndex + + 3 * binIndex + agg(aggIndex) += 1 + agg(aggIndex + 1) += label + agg(aggIndex + 2) += label * label + featureIndex += 1 } } /** * Performs a sequential aggregation over a partition. + * For l nodes, k features, + * For classification: + * Either the left count or the right count of one of the bins is + * incremented based upon whether the feature is classified as 0 or 1. + * For regression: + * The count, sum, sum of squares of one of the bins is incremented. + * + * @param agg Array storing aggregate calculation, updated by this function. + * Size for classification: + * numClasses * numBins * numFeatures * numNodes for ordered features, or + * 2 * numClasses * numBins * numFeatures * numNodes for unordered features. + * Size for regression: + * 3 * numBins * numFeatures * numNodes. + * @param treePoint Data point being aggregated. + * @return agg */ - def binSeqOp(agg: Array[Double], arr: Array[Double]): Array[Double] = { - strategy.algo match { - case Classification => classificationBinSeqOp(arr, agg) - case Regression => regressionBinSeqOp(arr, agg) + def binSeqOp(agg: Array[Double], treePoint: TreePoint): Array[Double] = { + val nodeIndex = treePointToNodeIndex(treePoint) + // If the example does not reach this level, then nodeIndex < 0. + // If the example reaches this level but is handled in a different group, + // then either nodeIndex < 0 (previous group) or nodeIndex >= numNodes (later group). + if (nodeIndex >= 0 && nodeIndex < numNodes) { + if (metadata.isClassification) { + if (isMulticlassWithCategoricalFeatures) { + multiclassWithCategoricalBinSeqOp(agg, treePoint, nodeIndex) + } else { + binaryOrNotCategoricalBinSeqOp(agg, treePoint, nodeIndex) + } + } else { + regressionBinSeqOp(agg, treePoint, nodeIndex) + } } agg } // Calculate bin aggregate length for classification or regression. - val binAggregateLength = strategy.algo match { - case Classification => 2 * numBins * numFeatures * numNodes - case Regression => 3 * numBins * numFeatures * numNodes - } + val binAggregateLength = numNodes * getElementsPerNode(metadata, numBins) logDebug("binAggregateLength = " + binAggregateLength) /** @@ -649,226 +823,300 @@ object DecisionTree extends Serializable with Logging { combinedAggregate } - // Find feature bins for all nodes at a level. - val binMappedRDD = input.map(x => findBinsForLevel(x)) - // Calculate bin aggregates. + timer.start("aggregation") val binAggregates = { - binMappedRDD.aggregate(Array.fill[Double](binAggregateLength)(0))(binSeqOp,binCombOp) + input.aggregate(Array.fill[Double](binAggregateLength)(0))(binSeqOp, binCombOp) } + timer.stop("aggregation") logDebug("binAggregates.length = " + binAggregates.length) /** - * Calculates the information gain for all splits based upon left/right split aggregates. - * @param leftNodeAgg left node aggregates - * @param featureIndex feature index - * @param splitIndex split index - * @param rightNodeAgg right node aggregate + * Calculate the information gain for a given (feature, split) based upon left/right aggregates. + * @param leftNodeAgg left node aggregates for this (feature, split) + * @param rightNodeAgg right node aggregate for this (feature, split) * @param topImpurity impurity of the parent node * @return information gain and statistics for all splits */ def calculateGainForSplit( - leftNodeAgg: Array[Array[Double]], - featureIndex: Int, - splitIndex: Int, - rightNodeAgg: Array[Array[Double]], + leftNodeAgg: Array[Double], + rightNodeAgg: Array[Double], topImpurity: Double): InformationGainStats = { - strategy.algo match { - case Classification => - val left0Count = leftNodeAgg(featureIndex)(2 * splitIndex) - val left1Count = leftNodeAgg(featureIndex)(2 * splitIndex + 1) - val leftCount = left0Count + left1Count - - val right0Count = rightNodeAgg(featureIndex)(2 * splitIndex) - val right1Count = rightNodeAgg(featureIndex)(2 * splitIndex + 1) - val rightCount = right0Count + right1Count - - val impurity = { - if (level > 0) { - topImpurity - } else { - // Calculate impurity for root node. - strategy.impurity.calculate(left0Count + right0Count, left1Count + right1Count) + if (metadata.isClassification) { + val leftTotalCount = leftNodeAgg.sum + val rightTotalCount = rightNodeAgg.sum + + val impurity = { + if (level > 0) { + topImpurity + } else { + // Calculate impurity for root node. + val rootNodeCounts = new Array[Double](numClasses) + var classIndex = 0 + while (classIndex < numClasses) { + rootNodeCounts(classIndex) = leftNodeAgg(classIndex) + rightNodeAgg(classIndex) + classIndex += 1 } + metadata.impurity.calculate(rootNodeCounts, leftTotalCount + rightTotalCount) + } + } + + val totalCount = leftTotalCount + rightTotalCount + if (totalCount == 0) { + // Return arbitrary prediction. + return new InformationGainStats(0, topImpurity, topImpurity, topImpurity, 0) + } + + // Sum of count for each label + val leftrightNodeAgg: Array[Double] = + leftNodeAgg.zip(rightNodeAgg).map { case (leftCount, rightCount) => + leftCount + rightCount } - if (leftCount == 0) { - return new InformationGainStats(0, topImpurity, Double.MinValue, topImpurity,1) + def indexOfLargestArrayElement(array: Array[Double]): Int = { + val result = array.foldLeft(-1, Double.MinValue, 0) { + case ((maxIndex, maxValue, currentIndex), currentValue) => + if (currentValue > maxValue) { + (currentIndex, currentValue, currentIndex + 1) + } else { + (maxIndex, maxValue, currentIndex + 1) + } } - if (rightCount == 0) { - return new InformationGainStats(0, topImpurity, topImpurity, Double.MinValue,0) + if (result._1 < 0) { + throw new RuntimeException("DecisionTree internal error:" + + " calculateGainForSplit failed in indexOfLargestArrayElement") } + result._1 + } - val leftImpurity = strategy.impurity.calculate(left0Count, left1Count) - val rightImpurity = strategy.impurity.calculate(right0Count, right1Count) + val predict = indexOfLargestArrayElement(leftrightNodeAgg) + val prob = leftrightNodeAgg(predict) / totalCount - val leftWeight = leftCount.toDouble / (leftCount + rightCount) - val rightWeight = rightCount.toDouble / (leftCount + rightCount) + val leftImpurity = if (leftTotalCount == 0) { + topImpurity + } else { + metadata.impurity.calculate(leftNodeAgg, leftTotalCount) + } + val rightImpurity = if (rightTotalCount == 0) { + topImpurity + } else { + metadata.impurity.calculate(rightNodeAgg, rightTotalCount) + } - val gain = { - if (level > 0) { - impurity - leftWeight * leftImpurity - rightWeight * rightImpurity - } else { - impurity - leftWeight * leftImpurity - rightWeight * rightImpurity - } - } + val leftWeight = leftTotalCount / totalCount + val rightWeight = rightTotalCount / totalCount - val predict = (left1Count + right1Count) / (leftCount + rightCount) - - new InformationGainStats(gain, impurity, leftImpurity, rightImpurity, predict) - case Regression => - val leftCount = leftNodeAgg(featureIndex)(3 * splitIndex) - val leftSum = leftNodeAgg(featureIndex)(3 * splitIndex + 1) - val leftSumSquares = leftNodeAgg(featureIndex)(3 * splitIndex + 2) - - val rightCount = rightNodeAgg(featureIndex)(3 * splitIndex) - val rightSum = rightNodeAgg(featureIndex)(3 * splitIndex + 1) - val rightSumSquares = rightNodeAgg(featureIndex)(3 * splitIndex + 2) - - val impurity = { - if (level > 0) { - topImpurity - } else { - // Calculate impurity for root node. - val count = leftCount + rightCount - val sum = leftSum + rightSum - val sumSquares = leftSumSquares + rightSumSquares - strategy.impurity.calculate(count, sum, sumSquares) - } - } + val gain = impurity - leftWeight * leftImpurity - rightWeight * rightImpurity - if (leftCount == 0) { - return new InformationGainStats(0, topImpurity, Double.MinValue, topImpurity, - rightSum / rightCount) - } - if (rightCount == 0) { - return new InformationGainStats(0, topImpurity ,topImpurity, - Double.MinValue, leftSum / leftCount) - } + new InformationGainStats(gain, impurity, leftImpurity, rightImpurity, predict, prob) - val leftImpurity = strategy.impurity.calculate(leftCount, leftSum, leftSumSquares) - val rightImpurity = strategy.impurity.calculate(rightCount, rightSum, rightSumSquares) + } else { + // Regression - val leftWeight = leftCount.toDouble / (leftCount + rightCount) - val rightWeight = rightCount.toDouble / (leftCount + rightCount) + val leftCount = leftNodeAgg(0) + val leftSum = leftNodeAgg(1) + val leftSumSquares = leftNodeAgg(2) - val gain = { - if (level > 0) { - impurity - leftWeight * leftImpurity - rightWeight * rightImpurity - } else { - impurity - leftWeight * leftImpurity - rightWeight * rightImpurity - } + val rightCount = rightNodeAgg(0) + val rightSum = rightNodeAgg(1) + val rightSumSquares = rightNodeAgg(2) + + val impurity = { + if (level > 0) { + topImpurity + } else { + // Calculate impurity for root node. + val count = leftCount + rightCount + val sum = leftSum + rightSum + val sumSquares = leftSumSquares + rightSumSquares + metadata.impurity.calculate(count, sum, sumSquares) } + } - val predict = (leftSum + rightSum) / (leftCount + rightCount) - new InformationGainStats(gain, impurity, leftImpurity, rightImpurity, predict) + if (leftCount == 0) { + return new InformationGainStats(0, topImpurity, Double.MinValue, topImpurity, + rightSum / rightCount) + } + if (rightCount == 0) { + return new InformationGainStats(0, topImpurity, topImpurity, + Double.MinValue, leftSum / leftCount) + } + + val leftImpurity = metadata.impurity.calculate(leftCount, leftSum, leftSumSquares) + val rightImpurity = metadata.impurity.calculate(rightCount, rightSum, rightSumSquares) + + val leftWeight = leftCount.toDouble / (leftCount + rightCount) + val rightWeight = rightCount.toDouble / (leftCount + rightCount) + + val gain = impurity - leftWeight * leftImpurity - rightWeight * rightImpurity + + val predict = (leftSum + rightSum) / (leftCount + rightCount) + new InformationGainStats(gain, impurity, leftImpurity, rightImpurity, predict) } } /** * Extracts left and right split aggregates. - * @param binData Array[Double] of size 2*numFeatures*numSplits - * @return (leftNodeAgg, rightNodeAgg) tuple of type (Array[Double], - * Array[Double]) where each array is of size(numFeature,2*(numSplits-1)) + * @param binData Aggregate array slice from getBinDataForNode. + * For classification: + * For unordered features, this is leftChildData ++ rightChildData, + * each of which is indexed by (feature, split/bin, class), + * with class being the least significant bit. + * For ordered features, this is of size numClasses * numBins * numFeatures. + * For regression: + * This is of size 2 * numFeatures * numBins. + * @return (leftNodeAgg, rightNodeAgg) pair of arrays. + * For classification, each array is of size (numFeatures, (numBins - 1), numClasses). + * For regression, each array is of size (numFeatures, (numBins - 1), 3). + * */ def extractLeftRightNodeAggregates( - binData: Array[Double]): (Array[Array[Double]], Array[Array[Double]]) = { - strategy.algo match { - case Classification => - // Initialize left and right split aggregates. - val leftNodeAgg = Array.ofDim[Double](numFeatures, 2 * (numBins - 1)) - val rightNodeAgg = Array.ofDim[Double](numFeatures, 2 * (numBins - 1)) - // Iterate over all features. - var featureIndex = 0 - while (featureIndex < numFeatures) { - // shift for this featureIndex - val shift = 2 * featureIndex * numBins - - // left node aggregate for the lowest split - leftNodeAgg(featureIndex)(0) = binData(shift + 0) - leftNodeAgg(featureIndex)(1) = binData(shift + 1) - - // right node aggregate for the highest split - rightNodeAgg(featureIndex)(2 * (numBins - 2)) - = binData(shift + (2 * (numBins - 1))) - rightNodeAgg(featureIndex)(2 * (numBins - 2) + 1) - = binData(shift + (2 * (numBins - 1)) + 1) - - // Iterate over all splits. - var splitIndex = 1 - while (splitIndex < numBins - 1) { - // calculating left node aggregate for a split as a sum of left node aggregate of a - // lower split and the left bin aggregate of a bin where the split is a high split - leftNodeAgg(featureIndex)(2 * splitIndex) = binData(shift + 2 * splitIndex) + - leftNodeAgg(featureIndex)(2 * splitIndex - 2) - leftNodeAgg(featureIndex)(2 * splitIndex + 1) = binData(shift + 2 * splitIndex + 1) + - leftNodeAgg(featureIndex)(2 * splitIndex - 2 + 1) - - // calculating right node aggregate for a split as a sum of right node aggregate of a - // higher split and the right bin aggregate of a bin where the split is a low split - rightNodeAgg(featureIndex)(2 * (numBins - 2 - splitIndex)) = - binData(shift + (2 *(numBins - 1 - splitIndex))) + - rightNodeAgg(featureIndex)(2 * (numBins - 1 - splitIndex)) - rightNodeAgg(featureIndex)(2 * (numBins - 2 - splitIndex) + 1) = - binData(shift + (2* (numBins - 1 - splitIndex) + 1)) + - rightNodeAgg(featureIndex)(2 * (numBins - 1 - splitIndex) + 1) - - splitIndex += 1 - } - featureIndex += 1 + binData: Array[Double]): (Array[Array[Array[Double]]], Array[Array[Array[Double]]]) = { + + + /** + * The input binData is indexed as (feature, bin, class). + * This computes cumulative sums over splits. + * Each (feature, class) pair is handled separately. + * Note: numSplits = numBins - 1. + * @param leftNodeAgg Each (feature, class) slice is an array over splits. + * Element i (i = 0, ..., numSplits - 2) is set to be + * the cumulative sum (from left) over binData for bins 0, ..., i. + * @param rightNodeAgg Each (feature, class) slice is an array over splits. + * Element i (i = 1, ..., numSplits - 1) is set to be + * the cumulative sum (from right) over binData for bins + * numBins - 1, ..., numBins - 1 - i. + */ + def findAggForOrderedFeatureClassification( + leftNodeAgg: Array[Array[Array[Double]]], + rightNodeAgg: Array[Array[Array[Double]]], + featureIndex: Int) { + + // shift for this featureIndex + val shift = numClasses * featureIndex * numBins + + var classIndex = 0 + while (classIndex < numClasses) { + // left node aggregate for the lowest split + leftNodeAgg(featureIndex)(0)(classIndex) = binData(shift + classIndex) + // right node aggregate for the highest split + rightNodeAgg(featureIndex)(numBins - 2)(classIndex) + = binData(shift + (numClasses * (numBins - 1)) + classIndex) + classIndex += 1 + } + + // Iterate over all splits. + var splitIndex = 1 + while (splitIndex < numBins - 1) { + // calculating left node aggregate for a split as a sum of left node aggregate of a + // lower split and the left bin aggregate of a bin where the split is a high split + var innerClassIndex = 0 + while (innerClassIndex < numClasses) { + leftNodeAgg(featureIndex)(splitIndex)(innerClassIndex) + = binData(shift + numClasses * splitIndex + innerClassIndex) + + leftNodeAgg(featureIndex)(splitIndex - 1)(innerClassIndex) + rightNodeAgg(featureIndex)(numBins - 2 - splitIndex)(innerClassIndex) = + binData(shift + (numClasses * (numBins - 1 - splitIndex) + innerClassIndex)) + + rightNodeAgg(featureIndex)(numBins - 1 - splitIndex)(innerClassIndex) + innerClassIndex += 1 } - (leftNodeAgg, rightNodeAgg) - case Regression => - // Initialize left and right split aggregates. - val leftNodeAgg = Array.ofDim[Double](numFeatures, 3 * (numBins - 1)) - val rightNodeAgg = Array.ofDim[Double](numFeatures, 3 * (numBins - 1)) - // Iterate over all features. - var featureIndex = 0 - while (featureIndex < numFeatures) { + splitIndex += 1 + } + } + + /** + * Reshape binData for this feature. + * Indexes binData as (feature, split, class) with class as the least significant bit. + * @param leftNodeAgg leftNodeAgg(featureIndex)(splitIndex)(classIndex) = aggregate value + */ + def findAggForUnorderedFeatureClassification( + leftNodeAgg: Array[Array[Array[Double]]], + rightNodeAgg: Array[Array[Array[Double]]], + featureIndex: Int) { + + val rightChildShift = numClasses * numBins * numFeatures + var splitIndex = 0 + while (splitIndex < numBins - 1) { + var classIndex = 0 + while (classIndex < numClasses) { // shift for this featureIndex - val shift = 3 * featureIndex * numBins - // left node aggregate for the lowest split - leftNodeAgg(featureIndex)(0) = binData(shift + 0) - leftNodeAgg(featureIndex)(1) = binData(shift + 1) - leftNodeAgg(featureIndex)(2) = binData(shift + 2) - - // right node aggregate for the highest split - rightNodeAgg(featureIndex)(3 * (numBins - 2)) = - binData(shift + (3 * (numBins - 1))) - rightNodeAgg(featureIndex)(3 * (numBins - 2) + 1) = - binData(shift + (3 * (numBins - 1)) + 1) - rightNodeAgg(featureIndex)(3 * (numBins - 2) + 2) = - binData(shift + (3 * (numBins - 1)) + 2) - - // Iterate over all splits. - var splitIndex = 1 - while (splitIndex < numBins - 1) { - // calculating left node aggregate for a split as a sum of left node aggregate of a - // lower split and the left bin aggregate of a bin where the split is a high split - leftNodeAgg(featureIndex)(3 * splitIndex) = binData(shift + 3 * splitIndex) + - leftNodeAgg(featureIndex)(3 * splitIndex - 3) - leftNodeAgg(featureIndex)(3 * splitIndex + 1) = binData(shift + 3 * splitIndex + 1) + - leftNodeAgg(featureIndex)(3 * splitIndex - 3 + 1) - leftNodeAgg(featureIndex)(3 * splitIndex + 2) = binData(shift + 3 * splitIndex + 2) + - leftNodeAgg(featureIndex)(3 * splitIndex - 3 + 2) - - // calculating right node aggregate for a split as a sum of right node aggregate of a - // higher split and the right bin aggregate of a bin where the split is a low split - rightNodeAgg(featureIndex)(3 * (numBins - 2 - splitIndex)) = - binData(shift + (3 * (numBins - 1 - splitIndex))) + - rightNodeAgg(featureIndex)(3 * (numBins - 1 - splitIndex)) - rightNodeAgg(featureIndex)(3 * (numBins - 2 - splitIndex) + 1) = - binData(shift + (3 * (numBins - 1 - splitIndex) + 1)) + - rightNodeAgg(featureIndex)(3 * (numBins - 1 - splitIndex) + 1) - rightNodeAgg(featureIndex)(3 * (numBins - 2 - splitIndex) + 2) = - binData(shift + (3 * (numBins - 1 - splitIndex) + 2)) + - rightNodeAgg(featureIndex)(3 * (numBins - 1 - splitIndex) + 2) - - splitIndex += 1 - } - featureIndex += 1 + val shift = numClasses * featureIndex * numBins + splitIndex * numClasses + val leftBinValue = binData(shift + classIndex) + val rightBinValue = binData(rightChildShift + shift + classIndex) + leftNodeAgg(featureIndex)(splitIndex)(classIndex) = leftBinValue + rightNodeAgg(featureIndex)(splitIndex)(classIndex) = rightBinValue + classIndex += 1 + } + splitIndex += 1 + } + } + + def findAggForRegression( + leftNodeAgg: Array[Array[Array[Double]]], + rightNodeAgg: Array[Array[Array[Double]]], + featureIndex: Int) { + + // shift for this featureIndex + val shift = 3 * featureIndex * numBins + // left node aggregate for the lowest split + leftNodeAgg(featureIndex)(0)(0) = binData(shift + 0) + leftNodeAgg(featureIndex)(0)(1) = binData(shift + 1) + leftNodeAgg(featureIndex)(0)(2) = binData(shift + 2) + + // right node aggregate for the highest split + rightNodeAgg(featureIndex)(numBins - 2)(0) = + binData(shift + (3 * (numBins - 1))) + rightNodeAgg(featureIndex)(numBins - 2)(1) = + binData(shift + (3 * (numBins - 1)) + 1) + rightNodeAgg(featureIndex)(numBins - 2)(2) = + binData(shift + (3 * (numBins - 1)) + 2) + + // Iterate over all splits. + var splitIndex = 1 + while (splitIndex < numBins - 1) { + var i = 0 // index for regression histograms + while (i < 3) { // count, sum, sum^2 + // calculating left node aggregate for a split as a sum of left node aggregate of a + // lower split and the left bin aggregate of a bin where the split is a high split + leftNodeAgg(featureIndex)(splitIndex)(i) = binData(shift + 3 * splitIndex + i) + + leftNodeAgg(featureIndex)(splitIndex - 1)(i) + // calculating right node aggregate for a split as a sum of right node aggregate of a + // higher split and the right bin aggregate of a bin where the split is a low split + rightNodeAgg(featureIndex)(numBins - 2 - splitIndex)(i) = + binData(shift + (3 * (numBins - 1 - splitIndex) + i)) + + rightNodeAgg(featureIndex)(numBins - 1 - splitIndex)(i) + i += 1 } - (leftNodeAgg, rightNodeAgg) + splitIndex += 1 + } + } + + if (metadata.isClassification) { + // Initialize left and right split aggregates. + val leftNodeAgg = Array.ofDim[Double](numFeatures, numBins - 1, numClasses) + val rightNodeAgg = Array.ofDim[Double](numFeatures, numBins - 1, numClasses) + var featureIndex = 0 + while (featureIndex < numFeatures) { + if (metadata.isUnordered(featureIndex)) { + findAggForUnorderedFeatureClassification(leftNodeAgg, rightNodeAgg, featureIndex) + } else { + findAggForOrderedFeatureClassification(leftNodeAgg, rightNodeAgg, featureIndex) + } + featureIndex += 1 + } + (leftNodeAgg, rightNodeAgg) + } else { + // Regression + // Initialize left and right split aggregates. + val leftNodeAgg = Array.ofDim[Double](numFeatures, numBins - 1, 3) + val rightNodeAgg = Array.ofDim[Double](numFeatures, numBins - 1, 3) + // Iterate over all features. + var featureIndex = 0 + while (featureIndex < numFeatures) { + findAggForRegression(leftNodeAgg, rightNodeAgg, featureIndex) + featureIndex += 1 + } + (leftNodeAgg, rightNodeAgg) } } @@ -876,23 +1124,46 @@ object DecisionTree extends Serializable with Logging { * Calculates information gain for all nodes splits. */ def calculateGainsForAllNodeSplits( - leftNodeAgg: Array[Array[Double]], - rightNodeAgg: Array[Array[Double]], + leftNodeAgg: Array[Array[Array[Double]]], + rightNodeAgg: Array[Array[Array[Double]]], nodeImpurity: Double): Array[Array[InformationGainStats]] = { val gains = Array.ofDim[InformationGainStats](numFeatures, numBins - 1) - for (featureIndex <- 0 until numFeatures) { - for (splitIndex <- 0 until numBins - 1) { - gains(featureIndex)(splitIndex) = calculateGainForSplit(leftNodeAgg, featureIndex, - splitIndex, rightNodeAgg, nodeImpurity) + var featureIndex = 0 + while (featureIndex < numFeatures) { + val numSplitsForFeature = getNumSplitsForFeature(featureIndex) + var splitIndex = 0 + while (splitIndex < numSplitsForFeature) { + gains(featureIndex)(splitIndex) = + calculateGainForSplit(leftNodeAgg(featureIndex)(splitIndex), + rightNodeAgg(featureIndex)(splitIndex), nodeImpurity) + splitIndex += 1 } + featureIndex += 1 } gains } + /** + * Get the number of splits for a feature. + */ + def getNumSplitsForFeature(featureIndex: Int): Int = { + if (metadata.isContinuous(featureIndex)) { + numBins - 1 + } else { + // Categorical feature + val featureCategories = metadata.featureArity(featureIndex) + if (metadata.isUnordered(featureIndex)) { + (1 << featureCategories - 1) - 1 + } else { + featureCategories + } + } + } + /** * Find the best split for a node. - * @param binData Array[Double] of size 2 * numSplits * numFeatures + * @param binData Bin data slice for this node, given by getBinDataForNode. * @param nodeImpurity impurity of the top node * @return tuple of split and information gain */ @@ -908,7 +1179,7 @@ object DecisionTree extends Serializable with Logging { // Calculate gains for all splits. val gains = calculateGainsForAllNodeSplits(leftNodeAgg, rightNodeAgg, nodeImpurity) - val (bestFeatureIndex,bestSplitIndex, gainStats) = { + val (bestFeatureIndex, bestSplitIndex, gainStats) = { // Initialize with infeasible values. var bestFeatureIndex = Int.MinValue var bestSplitIndex = Int.MinValue @@ -918,7 +1189,8 @@ object DecisionTree extends Serializable with Logging { while (featureIndex < numFeatures) { // Iterate over all splits. var splitIndex = 0 - while (splitIndex < numBins - 1) { + val numSplitsForFeature = getNumSplitsForFeature(featureIndex) + while (splitIndex < numSplitsForFeature) { val gainStats = gains(featureIndex)(splitIndex) if (gainStats.gain > bestGainStats.gain) { bestGainStats = gainStats @@ -932,8 +1204,8 @@ object DecisionTree extends Serializable with Logging { (bestFeatureIndex, bestSplitIndex, bestGainStats) } + logDebug("best split = " + splits(bestFeatureIndex)(bestSplitIndex)) logDebug("best split bin = " + bins(bestFeatureIndex)(bestSplitIndex)) - logDebug("best split bin = " + splits(bestFeatureIndex)(bestSplitIndex)) (splits(bestFeatureIndex)(bestSplitIndex), gainStats) } @@ -942,65 +1214,126 @@ object DecisionTree extends Serializable with Logging { * Get bin data for one node. */ def getBinDataForNode(node: Int): Array[Double] = { - strategy.algo match { - case Classification => - val shift = 2 * node * numBins * numFeatures - val binsForNode = binAggregates.slice(shift, shift + 2 * numBins * numFeatures) + if (metadata.isClassification) { + if (isMulticlassWithCategoricalFeatures) { + val shift = numClasses * node * numBins * numFeatures + val rightChildShift = numClasses * numBins * numFeatures * numNodes + val binsForNode = { + val leftChildData + = binAggregates.slice(shift, shift + numClasses * numBins * numFeatures) + val rightChildData + = binAggregates.slice(rightChildShift + shift, + rightChildShift + shift + numClasses * numBins * numFeatures) + leftChildData ++ rightChildData + } binsForNode - case Regression => - val shift = 3 * node * numBins * numFeatures - val binsForNode = binAggregates.slice(shift, shift + 3 * numBins * numFeatures) + } else { + val shift = numClasses * node * numBins * numFeatures + val binsForNode = binAggregates.slice(shift, shift + numClasses * numBins * numFeatures) binsForNode + } + } else { + // Regression + val shift = 3 * node * numBins * numFeatures + val binsForNode = binAggregates.slice(shift, shift + 3 * numBins * numFeatures) + binsForNode } } // Calculate best splits for all nodes at a given level + timer.start("chooseSplits") val bestSplits = new Array[(Split, InformationGainStats)](numNodes) // Iterating over all nodes at this level var node = 0 while (node < numNodes) { - val nodeImpurityIndex = math.pow(2, level).toInt - 1 + node + groupShift + val nodeImpurityIndex = (1 << level) - 1 + node + groupShift val binsForNode: Array[Double] = getBinDataForNode(node) logDebug("nodeImpurityIndex = " + nodeImpurityIndex) val parentNodeImpurity = parentImpurities(nodeImpurityIndex) - logDebug("node impurity = " + parentNodeImpurity) + logDebug("parent node impurity = " + parentNodeImpurity) bestSplits(node) = binsToBestSplit(binsForNode, parentNodeImpurity) node += 1 } + timer.stop("chooseSplits") bestSplits } /** - * Returns split and bins for decision tree calculation. - * @param input RDD of [[org.apache.spark.mllib.regression.LabeledPoint]] used as training data - * for DecisionTree - * @param strategy [[org.apache.spark.mllib.tree.configuration.Strategy]] instance containing - * parameters for construction the DecisionTree - * @return a tuple of (splits,bins) where splits is an Array of [org.apache.spark.mllib.tree - * .model.Split] of size (numFeatures, numSplits-1) and bins is an Array of [org.apache - * .spark.mllib.tree.model.Bin] of size (numFeatures, numSplits1) + * Get the number of values to be stored per node in the bin aggregates. + * + * @param numBins Number of bins = 1 + number of possible splits. + */ + private def getElementsPerNode(metadata: DecisionTreeMetadata, numBins: Int): Int = { + if (metadata.isClassification) { + if (metadata.isMulticlassWithCategoricalFeatures) { + 2 * metadata.numClasses * numBins * metadata.numFeatures + } else { + metadata.numClasses * numBins * metadata.numFeatures + } + } else { + 3 * numBins * metadata.numFeatures + } + } + + /** + * Returns splits and bins for decision tree calculation. + * Continuous and categorical features are handled differently. + * + * Continuous features: + * For each feature, there are numBins - 1 possible splits representing the possible binary + * decisions at each node in the tree. + * + * Categorical features: + * For each feature, there is 1 bin per split. + * Splits and bins are handled in 2 ways: + * (a) "unordered features" + * For multiclass classification with a low-arity feature + * (i.e., if isMulticlass && isSpaceSufficientForAllCategoricalSplits), + * the feature is split based on subsets of categories. + * There are (1 << maxFeatureValue - 1) - 1 splits. + * (b) "ordered features" + * For regression and binary classification, + * and for multiclass classification with a high-arity feature, + * there is one bin per category. + * + * @param input Training data: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]] + * @param metadata Learning and dataset metadata + * @return A tuple of (splits, bins). + * Splits is an Array of [[org.apache.spark.mllib.tree.model.Split]] + * of size (numFeatures, numBins - 1). + * Bins is an Array of [[org.apache.spark.mllib.tree.model.Bin]] + * of size (numFeatures, numBins). */ protected[tree] def findSplitsBins( input: RDD[LabeledPoint], - strategy: Strategy): (Array[Array[Split]], Array[Array[Bin]]) = { + metadata: DecisionTreeMetadata): (Array[Array[Split]], Array[Array[Bin]]) = { + val count = input.count() // Find the number of features by looking at the first sample val numFeatures = input.take(1)(0).features.size - val maxBins = strategy.maxBins + val maxBins = metadata.maxBins val numBins = if (maxBins <= count) maxBins else count.toInt logDebug("numBins = " + numBins) + val isMulticlass = metadata.isMulticlass + logDebug("isMulticlass = " + isMulticlass) /* - * TODO: Add a require statement ensuring #bins is always greater than the categories. + * Ensure numBins is always greater than the categories. For multiclass classification, + * numBins should be greater than 2^(maxCategories - 1) - 1. * It's a limitation of the current implementation but a reasonable trade-off since features * with large number of categories get favored over continuous features. + * + * This needs to be checked here instead of in Strategy since numBins can be determined + * by the number of training examples. + * TODO: Allow this case, where we simply will know nothing about some categories. */ - if (strategy.categoricalFeaturesInfo.size > 0) { - val maxCategoriesForFeatures = strategy.categoricalFeaturesInfo.maxBy(_._2)._2 - require(numBins >= maxCategoriesForFeatures) + if (metadata.featureArity.size > 0) { + val maxCategoriesForFeatures = metadata.featureArity.maxBy(_._2)._2 + require(numBins > maxCategoriesForFeatures, "numBins should be greater than max categories " + + "in categorical features") } // Calculate the number of sample for approximate quantile calculation. @@ -1009,13 +1342,14 @@ object DecisionTree extends Serializable with Logging { logDebug("fraction of data used for calculating quantiles = " + fraction) // sampled input for RDD calculation - val sampledInput = input.sample(false, fraction, new XORShiftRandom().nextInt()).collect() + val sampledInput = + input.sample(withReplacement = false, fraction, new XORShiftRandom().nextInt()).collect() val numSamples = sampledInput.length val stride: Double = numSamples.toDouble / numBins logDebug("stride = " + stride) - strategy.quantileCalculationStrategy match { + metadata.quantileStrategy match { case Sort => val splits = Array.ofDim[Split](numFeatures, numBins - 1) val bins = Array.ofDim[Bin](numFeatures, numBins) @@ -1024,60 +1358,110 @@ object DecisionTree extends Serializable with Logging { // Iterate over all features. var featureIndex = 0 - while (featureIndex < numFeatures){ + while (featureIndex < numFeatures) { // Check whether the feature is continuous. - val isFeatureContinuous = strategy.categoricalFeaturesInfo.get(featureIndex).isEmpty + val isFeatureContinuous = metadata.isContinuous(featureIndex) if (isFeatureContinuous) { val featureSamples = sampledInput.map(lp => lp.features(featureIndex)).sorted val stride: Double = numSamples.toDouble / numBins logDebug("stride = " + stride) for (index <- 0 until numBins - 1) { - val sampleIndex = (index + 1) * stride.toInt - val split = new Split(featureIndex, featureSamples(sampleIndex), Continuous, List()) + val sampleIndex = index * stride.toInt + // Set threshold halfway in between 2 samples. + val threshold = (featureSamples(sampleIndex) + featureSamples(sampleIndex + 1)) / 2.0 + val split = new Split(featureIndex, threshold, Continuous, List()) splits(featureIndex)(index) = split } - } else { - val maxFeatureValue = strategy.categoricalFeaturesInfo(featureIndex) - require(maxFeatureValue < numBins, "number of categories should be less than number " + - "of bins") - - // For categorical variables, each bin is a category. The bins are sorted and they - // are ordered by calculating the centroid of their corresponding labels. - val centroidForCategories = - sampledInput.map(lp => (lp.features(featureIndex),lp.label)) - .groupBy(_._1) - .mapValues(x => x.map(_._2).sum / x.map(_._1).length) - - // Check for missing categorical variables and putting them last in the sorted list. - val fullCentroidForCategories = scala.collection.mutable.Map[Double,Double]() - for (i <- 0 until maxFeatureValue) { - if (centroidForCategories.contains(i)) { - fullCentroidForCategories(i) = centroidForCategories(i) - } else { - fullCentroidForCategories(i) = Double.MaxValue - } - } - - // bins sorted by centroids - val categoriesSortedByCentroid = fullCentroidForCategories.toList.sortBy(_._2) - - logDebug("centriod for categorical variable = " + categoriesSortedByCentroid) - - var categoriesForSplit = List[Double]() - categoriesSortedByCentroid.iterator.zipWithIndex.foreach { - case ((key, value), index) => - categoriesForSplit = key :: categoriesForSplit - splits(featureIndex)(index) = new Split(featureIndex, Double.MinValue, Categorical, - categoriesForSplit) + } else { // Categorical feature + val featureCategories = metadata.featureArity(featureIndex) + + // Use different bin/split calculation strategy for categorical features in multiclass + // classification that satisfy the space constraint. + if (metadata.isUnordered(featureIndex)) { + // 2^(maxFeatureValue- 1) - 1 combinations + var index = 0 + while (index < (1 << featureCategories - 1) - 1) { + val categories: List[Double] + = extractMultiClassCategories(index + 1, featureCategories) + splits(featureIndex)(index) + = new Split(featureIndex, Double.MinValue, Categorical, categories) bins(featureIndex)(index) = { if (index == 0) { - new Bin(new DummyCategoricalSplit(featureIndex, Categorical), - splits(featureIndex)(0), Categorical, key) + new Bin( + new DummyCategoricalSplit(featureIndex, Categorical), + splits(featureIndex)(0), + Categorical, + Double.MinValue) } else { - new Bin(splits(featureIndex)(index-1), splits(featureIndex)(index), - Categorical, key) + new Bin( + splits(featureIndex)(index - 1), + splits(featureIndex)(index), + Categorical, + Double.MinValue) } } + index += 1 + } + } else { // ordered feature + /* For a given categorical feature, use a subsample of the data + * to choose how to arrange possible splits. + * This examines each category and computes a centroid. + * These centroids are later used to sort the possible splits. + * centroidForCategories is a mapping: category (for the given feature) --> centroid + */ + val centroidForCategories = { + if (isMulticlass) { + // For categorical variables in multiclass classification, + // each bin is a category. The bins are sorted and they + // are ordered by calculating the impurity of their corresponding labels. + sampledInput.map(lp => (lp.features(featureIndex), lp.label)) + .groupBy(_._1) + .mapValues(x => x.groupBy(_._2).mapValues(x => x.size.toDouble)) + .map(x => (x._1, x._2.values.toArray)) + .map(x => (x._1, metadata.impurity.calculate(x._2, x._2.sum))) + } else { // regression or binary classification + // For categorical variables in regression and binary classification, + // each bin is a category. The bins are sorted and they + // are ordered by calculating the centroid of their corresponding labels. + sampledInput.map(lp => (lp.features(featureIndex), lp.label)) + .groupBy(_._1) + .mapValues(x => x.map(_._2).sum / x.map(_._1).length) + } + } + + logDebug("centroid for categories = " + centroidForCategories.mkString(",")) + + // Check for missing categorical variables and putting them last in the sorted list. + val fullCentroidForCategories = scala.collection.mutable.Map[Double,Double]() + for (i <- 0 until featureCategories) { + if (centroidForCategories.contains(i)) { + fullCentroidForCategories(i) = centroidForCategories(i) + } else { + fullCentroidForCategories(i) = Double.MaxValue + } + } + + // bins sorted by centroids + val categoriesSortedByCentroid = fullCentroidForCategories.toList.sortBy(_._2) + + logDebug("centroid for categorical variable = " + categoriesSortedByCentroid) + + var categoriesForSplit = List[Double]() + categoriesSortedByCentroid.iterator.zipWithIndex.foreach { + case ((key, value), index) => + categoriesForSplit = key :: categoriesForSplit + splits(featureIndex)(index) = new Split(featureIndex, Double.MinValue, + Categorical, categoriesForSplit) + bins(featureIndex)(index) = { + if (index == 0) { + new Bin(new DummyCategoricalSplit(featureIndex, Categorical), + splits(featureIndex)(0), Categorical, key) + } else { + new Bin(splits(featureIndex)(index-1), splits(featureIndex)(index), + Categorical, key) + } + } + } } } featureIndex += 1 @@ -1086,11 +1470,11 @@ object DecisionTree extends Serializable with Logging { // Find all bins. featureIndex = 0 while (featureIndex < numFeatures) { - val isFeatureContinuous = strategy.categoricalFeaturesInfo.get(featureIndex).isEmpty + val isFeatureContinuous = metadata.isContinuous(featureIndex) if (isFeatureContinuous) { // Bins for categorical variables are already assigned. bins(featureIndex)(0) = new Bin(new DummyLowSplit(featureIndex, Continuous), splits(featureIndex)(0), Continuous, Double.MinValue) - for (index <- 1 until numBins - 1){ + for (index <- 1 until numBins - 1) { val bin = new Bin(splits(featureIndex)(index-1), splits(featureIndex)(index), Continuous, Double.MinValue) bins(featureIndex)(index) = bin @@ -1100,11 +1484,36 @@ object DecisionTree extends Serializable with Logging { } featureIndex += 1 } - (splits,bins) + (splits, bins) case MinMax => throw new UnsupportedOperationException("minmax not supported yet.") case ApproxHist => throw new UnsupportedOperationException("approximate histogram not supported yet.") } } + + /** + * Nested method to extract list of eligible categories given an index. It extracts the + * position of ones in a binary representation of the input. If binary + * representation of an number is 01101 (13), the output list should (3.0, 2.0, + * 0.0). The maxFeatureValue depict the number of rightmost digits that will be tested for ones. + */ + private[tree] def extractMultiClassCategories( + input: Int, + maxFeatureValue: Int): List[Double] = { + var categories = List[Double]() + var j = 0 + var bitShiftedInput = input + while (j < maxFeatureValue) { + if (bitShiftedInput % 2 != 0) { + // updating the list of categories. + categories = j.toDouble :: categories + } + // Right shift by one + bitShiftedInput = bitShiftedInput >> 1 + j += 1 + } + categories + } + } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Algo.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Algo.scala index 79a01f58319e8..0ef9c6181a0a0 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Algo.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Algo.scala @@ -27,4 +27,10 @@ import org.apache.spark.annotation.Experimental object Algo extends Enumeration { type Algo = Value val Classification, Regression = Value + + private[mllib] def fromString(name: String): Algo = name match { + case "classification" => Classification + case "regression" => Regression + case _ => throw new IllegalArgumentException(s"Did not recognize Algo name: $name") + } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala index 1b505fd76eb75..cfc8192a85abd 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala @@ -17,34 +17,105 @@ package org.apache.spark.mllib.tree.configuration +import scala.collection.JavaConverters._ + import org.apache.spark.annotation.Experimental -import org.apache.spark.mllib.tree.impurity.Impurity +import org.apache.spark.mllib.tree.impurity.{Variance, Entropy, Gini, Impurity} import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ /** * :: Experimental :: * Stores all the configuration options for tree construction - * @param algo classification or regression - * @param impurity criterion used for information gain calculation - * @param maxDepth maximum depth of the tree - * @param maxBins maximum number of bins used for splitting features - * @param quantileCalculationStrategy algorithm for calculating quantiles + * @param algo Learning goal. Supported: + * [[org.apache.spark.mllib.tree.configuration.Algo.Classification]], + * [[org.apache.spark.mllib.tree.configuration.Algo.Regression]] + * @param impurity Criterion used for information gain calculation. + * Supported for Classification: [[org.apache.spark.mllib.tree.impurity.Gini]], + * [[org.apache.spark.mllib.tree.impurity.Entropy]]. + * Supported for Regression: [[org.apache.spark.mllib.tree.impurity.Variance]]. + * @param maxDepth Maximum depth of the tree. + * E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes. + * @param numClassesForClassification Number of classes for classification. + * (Ignored for regression.) + * Default value is 2 (binary classification). + * @param maxBins Maximum number of bins used for discretizing continuous features and + * for choosing how to split on features at each node. + * More bins give higher granularity. + * @param quantileCalculationStrategy Algorithm for calculating quantiles. Supported: + * [[org.apache.spark.mllib.tree.configuration.QuantileStrategy.Sort]] * @param categoricalFeaturesInfo A map storing information about the categorical variables and the * number of discrete values they take. For example, an entry (n -> * k) implies the feature n is categorical with k categories 0, * 1, 2, ... , k-1. It's important to note that features are * zero-indexed. - * @param maxMemoryInMB maximum memory in MB allocated to histogram aggregation. Default value is + * @param maxMemoryInMB Maximum memory in MB allocated to histogram aggregation. Default value is * 128 MB. - * */ @Experimental class Strategy ( val algo: Algo, val impurity: Impurity, val maxDepth: Int, + val numClassesForClassification: Int = 2, val maxBins: Int = 100, val quantileCalculationStrategy: QuantileStrategy = Sort, val categoricalFeaturesInfo: Map[Int, Int] = Map[Int, Int](), - val maxMemoryInMB: Int = 128) extends Serializable + val maxMemoryInMB: Int = 128) extends Serializable { + + if (algo == Classification) { + require(numClassesForClassification >= 2) + } + val isMulticlassClassification = + algo == Classification && numClassesForClassification > 2 + val isMulticlassWithCategoricalFeatures + = isMulticlassClassification && (categoricalFeaturesInfo.size > 0) + + /** + * Java-friendly constructor for [[org.apache.spark.mllib.tree.configuration.Strategy]] + */ + def this( + algo: Algo, + impurity: Impurity, + maxDepth: Int, + numClassesForClassification: Int, + maxBins: Int, + categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer]) { + this(algo, impurity, maxDepth, numClassesForClassification, maxBins, Sort, + categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap) + } + + /** + * Check validity of parameters. + * Throws exception if invalid. + */ + private[tree] def assertValid(): Unit = { + algo match { + case Classification => + require(numClassesForClassification >= 2, + s"DecisionTree Strategy for Classification must have numClassesForClassification >= 2," + + s" but numClassesForClassification = $numClassesForClassification.") + require(Set(Gini, Entropy).contains(impurity), + s"DecisionTree Strategy given invalid impurity for Classification: $impurity." + + s" Valid settings: Gini, Entropy") + case Regression => + require(impurity == Variance, + s"DecisionTree Strategy given invalid impurity for Regression: $impurity." + + s" Valid settings: Variance") + case _ => + throw new IllegalArgumentException( + s"DecisionTree Strategy given invalid algo parameter: $algo." + + s" Valid settings are: Classification, Regression.") + } + require(maxDepth >= 0, s"DecisionTree Strategy given invalid maxDepth parameter: $maxDepth." + + s" Valid values are integers >= 0.") + require(maxBins >= 2, s"DecisionTree Strategy given invalid maxBins parameter: $maxBins." + + s" Valid values are integers >= 2.") + categoricalFeaturesInfo.foreach { case (feature, arity) => + require(arity >= 2, + s"DecisionTree Strategy given invalid categoricalFeaturesInfo setting:" + + s" feature $feature has $arity categories. The number of categories should be >= 2.") + } + } + +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala new file mode 100644 index 0000000000000..d9eda354dc986 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.tree.impl + +import scala.collection.mutable + +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.tree.configuration.Algo._ +import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ +import org.apache.spark.mllib.tree.configuration.Strategy +import org.apache.spark.mllib.tree.impurity.Impurity +import org.apache.spark.rdd.RDD + + +/** + * Learning and dataset metadata for DecisionTree. + * + * @param numClasses For classification: labels can take values {0, ..., numClasses - 1}. + * For regression: fixed at 0 (no meaning). + * @param featureArity Map: categorical feature index --> arity. + * I.e., the feature takes values in {0, ..., arity - 1}. + */ +private[tree] class DecisionTreeMetadata( + val numFeatures: Int, + val numExamples: Long, + val numClasses: Int, + val maxBins: Int, + val featureArity: Map[Int, Int], + val unorderedFeatures: Set[Int], + val impurity: Impurity, + val quantileStrategy: QuantileStrategy) extends Serializable { + + def isUnordered(featureIndex: Int): Boolean = unorderedFeatures.contains(featureIndex) + + def isClassification: Boolean = numClasses >= 2 + + def isMulticlass: Boolean = numClasses > 2 + + def isMulticlassWithCategoricalFeatures: Boolean = isMulticlass && (featureArity.size > 0) + + def isCategorical(featureIndex: Int): Boolean = featureArity.contains(featureIndex) + + def isContinuous(featureIndex: Int): Boolean = !featureArity.contains(featureIndex) + +} + +private[tree] object DecisionTreeMetadata { + + def buildMetadata(input: RDD[LabeledPoint], strategy: Strategy): DecisionTreeMetadata = { + + val numFeatures = input.take(1)(0).features.size + val numExamples = input.count() + val numClasses = strategy.algo match { + case Classification => strategy.numClassesForClassification + case Regression => 0 + } + + val maxBins = math.min(strategy.maxBins, numExamples).toInt + val log2MaxBinsp1 = math.log(maxBins + 1) / math.log(2.0) + + val unorderedFeatures = new mutable.HashSet[Int]() + if (numClasses > 2) { + strategy.categoricalFeaturesInfo.foreach { case (f, k) => + if (k - 1 < log2MaxBinsp1) { + // Note: The above check is equivalent to checking: + // numUnorderedBins = (1 << k - 1) - 1 < maxBins + unorderedFeatures.add(f) + } else { + // TODO: Allow this case, where we simply will know nothing about some categories? + require(k < maxBins, s"maxBins (= $maxBins) should be greater than max categories " + + s"in categorical features (>= $k)") + } + } + } else { + strategy.categoricalFeaturesInfo.foreach { case (f, k) => + require(k < maxBins, s"maxBins (= $maxBins) should be greater than max categories " + + s"in categorical features (>= $k)") + } + } + + new DecisionTreeMetadata(numFeatures, numExamples, numClasses, maxBins, + strategy.categoricalFeaturesInfo, unorderedFeatures.toSet, + strategy.impurity, strategy.quantileCalculationStrategy) + } + +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/TimeTracker.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/TimeTracker.scala new file mode 100644 index 0000000000000..d215d68c4279e --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/TimeTracker.scala @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.tree.impl + +import scala.collection.mutable.{HashMap => MutableHashMap} + +import org.apache.spark.annotation.Experimental + +/** + * Time tracker implementation which holds labeled timers. + */ +@Experimental +private[tree] class TimeTracker extends Serializable { + + private val starts: MutableHashMap[String, Long] = new MutableHashMap[String, Long]() + + private val totals: MutableHashMap[String, Long] = new MutableHashMap[String, Long]() + + /** + * Starts a new timer, or re-starts a stopped timer. + */ + def start(timerLabel: String): Unit = { + val currentTime = System.nanoTime() + if (starts.contains(timerLabel)) { + throw new RuntimeException(s"TimeTracker.start(timerLabel) called again on" + + s" timerLabel = $timerLabel before that timer was stopped.") + } + starts(timerLabel) = currentTime + } + + /** + * Stops a timer and returns the elapsed time in seconds. + */ + def stop(timerLabel: String): Double = { + val currentTime = System.nanoTime() + if (!starts.contains(timerLabel)) { + throw new RuntimeException(s"TimeTracker.stop(timerLabel) called on" + + s" timerLabel = $timerLabel, but that timer was not started.") + } + val elapsed = currentTime - starts(timerLabel) + starts.remove(timerLabel) + if (totals.contains(timerLabel)) { + totals(timerLabel) += elapsed + } else { + totals(timerLabel) = elapsed + } + elapsed / 1e9 + } + + /** + * Print all timing results in seconds. + */ + override def toString: String = { + totals.map { case (label, elapsed) => + s" $label: ${elapsed / 1e9}" + }.mkString("\n") + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/TreePoint.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/TreePoint.scala new file mode 100644 index 0000000000000..170e43e222083 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/TreePoint.scala @@ -0,0 +1,185 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.tree.impl + +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.tree.model.Bin +import org.apache.spark.rdd.RDD + + +/** + * Internal representation of LabeledPoint for DecisionTree. + * This bins feature values based on a subsampled of data as follows: + * (a) Continuous features are binned into ranges. + * (b) Unordered categorical features are binned based on subsets of feature values. + * "Unordered categorical features" are categorical features with low arity used in + * multiclass classification. + * (c) Ordered categorical features are binned based on feature values. + * "Ordered categorical features" are categorical features with high arity, + * or any categorical feature used in regression or binary classification. + * + * @param label Label from LabeledPoint + * @param binnedFeatures Binned feature values. + * Same length as LabeledPoint.features, but values are bin indices. + */ +private[tree] class TreePoint(val label: Double, val binnedFeatures: Array[Int]) + extends Serializable { +} + +private[tree] object TreePoint { + + /** + * Convert an input dataset into its TreePoint representation, + * binning feature values in preparation for DecisionTree training. + * @param input Input dataset. + * @param bins Bins for features, of size (numFeatures, numBins). + * @param metadata Learning and dataset metadata + * @return TreePoint dataset representation + */ + def convertToTreeRDD( + input: RDD[LabeledPoint], + bins: Array[Array[Bin]], + metadata: DecisionTreeMetadata): RDD[TreePoint] = { + input.map { x => + TreePoint.labeledPointToTreePoint(x, bins, metadata) + } + } + + /** + * Convert one LabeledPoint into its TreePoint representation. + * @param bins Bins for features, of size (numFeatures, numBins). + */ + private def labeledPointToTreePoint( + labeledPoint: LabeledPoint, + bins: Array[Array[Bin]], + metadata: DecisionTreeMetadata): TreePoint = { + + val numFeatures = labeledPoint.features.size + val numBins = bins(0).size + val arr = new Array[Int](numFeatures) + var featureIndex = 0 + while (featureIndex < numFeatures) { + arr(featureIndex) = findBin(featureIndex, labeledPoint, metadata.isContinuous(featureIndex), + metadata.isUnordered(featureIndex), bins, metadata.featureArity) + featureIndex += 1 + } + + new TreePoint(labeledPoint.label, arr) + } + + /** + * Find bin for one (labeledPoint, feature). + * + * @param isUnorderedFeature (only applies if feature is categorical) + * @param bins Bins for features, of size (numFeatures, numBins). + * @param categoricalFeaturesInfo Map over categorical features: feature index --> feature arity + */ + private def findBin( + featureIndex: Int, + labeledPoint: LabeledPoint, + isFeatureContinuous: Boolean, + isUnorderedFeature: Boolean, + bins: Array[Array[Bin]], + categoricalFeaturesInfo: Map[Int, Int]): Int = { + + /** + * Binary search helper method for continuous feature. + */ + def binarySearchForBins(): Int = { + val binForFeatures = bins(featureIndex) + val feature = labeledPoint.features(featureIndex) + var left = 0 + var right = binForFeatures.length - 1 + while (left <= right) { + val mid = left + (right - left) / 2 + val bin = binForFeatures(mid) + val lowThreshold = bin.lowSplit.threshold + val highThreshold = bin.highSplit.threshold + if ((lowThreshold < feature) && (highThreshold >= feature)) { + return mid + } else if (lowThreshold >= feature) { + right = mid - 1 + } else { + left = mid + 1 + } + } + -1 + } + + /** + * Sequential search helper method to find bin for categorical feature in multiclass + * classification. The category is returned since each category can belong to multiple + * splits. The actual left/right child allocation per split is performed in the + * sequential phase of the bin aggregate operation. + */ + def sequentialBinSearchForUnorderedCategoricalFeatureInClassification(): Int = { + labeledPoint.features(featureIndex).toInt + } + + /** + * Sequential search helper method to find bin for categorical feature + * (for classification and regression). + */ + def sequentialBinSearchForOrderedCategoricalFeature(): Int = { + val featureCategories = categoricalFeaturesInfo(featureIndex) + val featureValue = labeledPoint.features(featureIndex) + var binIndex = 0 + while (binIndex < featureCategories) { + val bin = bins(featureIndex)(binIndex) + val categories = bin.highSplit.categories + if (categories.contains(featureValue)) { + return binIndex + } + binIndex += 1 + } + if (featureValue < 0 || featureValue >= featureCategories) { + throw new IllegalArgumentException( + s"DecisionTree given invalid data:" + + s" Feature $featureIndex is categorical with values in" + + s" {0,...,${featureCategories - 1}," + + s" but a data point gives it value $featureValue.\n" + + " Bad data point: " + labeledPoint.toString) + } + -1 + } + + if (isFeatureContinuous) { + // Perform binary search for finding bin for continuous features. + val binIndex = binarySearchForBins() + if (binIndex == -1) { + throw new RuntimeException("No bin was found for continuous feature." + + " This error can occur when given invalid data values (such as NaN)." + + s" Feature index: $featureIndex. Feature value: ${labeledPoint.features(featureIndex)}") + } + binIndex + } else { + // Perform sequential search to find bin for categorical features. + val binIndex = if (isUnorderedFeature) { + sequentialBinSearchForUnorderedCategoricalFeatureInClassification() + } else { + sequentialBinSearchForOrderedCategoricalFeature() + } + if (binIndex == -1) { + throw new RuntimeException("No bin was found for categorical feature." + + " This error can occur when given invalid data values (such as NaN)." + + s" Feature index: $featureIndex. Feature value: ${labeledPoint.features(featureIndex)}") + } + binIndex + } + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala index 60f43e9278d2a..96d2471e1f88c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala @@ -31,23 +31,46 @@ object Entropy extends Impurity { /** * :: DeveloperApi :: - * entropy calculation - * @param c0 count of instances with label 0 - * @param c1 count of instances with label 1 - * @return entropy value + * information calculation for multiclass classification + * @param counts Array[Double] with counts for each label + * @param totalCount sum of counts for all labels + * @return information value, or 0 if totalCount = 0 */ @DeveloperApi - override def calculate(c0: Double, c1: Double): Double = { - if (c0 == 0 || c1 == 0) { - 0 - } else { - val total = c0 + c1 - val f0 = c0 / total - val f1 = c1 / total - -(f0 * log2(f0)) - (f1 * log2(f1)) + override def calculate(counts: Array[Double], totalCount: Double): Double = { + if (totalCount == 0) { + return 0 } + val numClasses = counts.length + var impurity = 0.0 + var classIndex = 0 + while (classIndex < numClasses) { + val classCount = counts(classIndex) + if (classCount != 0) { + val freq = classCount / totalCount + impurity -= freq * log2(freq) + } + classIndex += 1 + } + impurity } + /** + * :: DeveloperApi :: + * variance calculation + * @param count number of instances + * @param sum sum of labels + * @param sumSquares summation of squares of the labels + * @return information value, or 0 if count = 0 + */ + @DeveloperApi override def calculate(count: Double, sum: Double, sumSquares: Double): Double = throw new UnsupportedOperationException("Entropy.calculate") + + /** + * Get this impurity instance. + * This is useful for passing impurity parameters to a Strategy in Java. + */ + def instance = this + } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala index c51d76d9b4c5b..d586f449048bb 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala @@ -30,23 +30,43 @@ object Gini extends Impurity { /** * :: DeveloperApi :: - * Gini coefficient calculation - * @param c0 count of instances with label 0 - * @param c1 count of instances with label 1 - * @return Gini coefficient value + * information calculation for multiclass classification + * @param counts Array[Double] with counts for each label + * @param totalCount sum of counts for all labels + * @return information value, or 0 if totalCount = 0 */ @DeveloperApi - override def calculate(c0: Double, c1: Double): Double = { - if (c0 == 0 || c1 == 0) { - 0 - } else { - val total = c0 + c1 - val f0 = c0 / total - val f1 = c1 / total - 1 - f0 * f0 - f1 * f1 + override def calculate(counts: Array[Double], totalCount: Double): Double = { + if (totalCount == 0) { + return 0 } + val numClasses = counts.length + var impurity = 1.0 + var classIndex = 0 + while (classIndex < numClasses) { + val freq = counts(classIndex) / totalCount + impurity -= freq * freq + classIndex += 1 + } + impurity } + /** + * :: DeveloperApi :: + * variance calculation + * @param count number of instances + * @param sum sum of labels + * @param sumSquares summation of squares of the labels + * @return information value, or 0 if count = 0 + */ + @DeveloperApi override def calculate(count: Double, sum: Double, sumSquares: Double): Double = throw new UnsupportedOperationException("Gini.calculate") + + /** + * Get this impurity instance. + * This is useful for passing impurity parameters to a Strategy in Java. + */ + def instance = this + } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Filter.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurities.scala similarity index 66% rename from mllib/src/main/scala/org/apache/spark/mllib/tree/model/Filter.scala rename to mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurities.scala index 2deaf4ae8dcab..9a6452aa13a61 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Filter.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurities.scala @@ -15,14 +15,18 @@ * limitations under the License. */ -package org.apache.spark.mllib.tree.model +package org.apache.spark.mllib.tree.impurity /** - * Filter specifying a split and type of comparison to be applied on features - * @param split split specifying the feature index, type and threshold - * @param comparison integer specifying <,=,> + * Factory for Impurity instances. */ -private[tree] case class Filter(split: Split, comparison: Int) { - // Comparison -1,0,1 signifies <.=,> - override def toString = " split = " + split + "comparison = " + comparison +private[mllib] object Impurities { + + def fromString(name: String): Impurity = name match { + case "gini" => Gini + case "entropy" => Entropy + case "variance" => Variance + case _ => throw new IllegalArgumentException(s"Did not recognize Impurity name: $name") + } + } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala index 8eab247cf0932..92b0c7b4a6fbc 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala @@ -28,13 +28,13 @@ trait Impurity extends Serializable { /** * :: DeveloperApi :: - * information calculation for binary classification - * @param c0 count of instances with label 0 - * @param c1 count of instances with label 1 - * @return information value + * information calculation for multiclass classification + * @param counts Array[Double] with counts for each label + * @param totalCount sum of counts for all labels + * @return information value, or 0 if totalCount = 0 */ @DeveloperApi - def calculate(c0 : Double, c1 : Double): Double + def calculate(counts: Array[Double], totalCount: Double): Double /** * :: DeveloperApi :: @@ -42,7 +42,7 @@ trait Impurity extends Serializable { * @param count number of instances * @param sum sum of labels * @param sumSquares summation of squares of the labels - * @return information value + * @return information value, or 0 if count = 0 */ @DeveloperApi def calculate(count: Double, sum: Double, sumSquares: Double): Double diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala index 47d07122af30f..f7d99a40eb380 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala @@ -25,7 +25,16 @@ import org.apache.spark.annotation.{DeveloperApi, Experimental} */ @Experimental object Variance extends Impurity { - override def calculate(c0: Double, c1: Double): Double = + + /** + * :: DeveloperApi :: + * information calculation for multiclass classification + * @param counts Array[Double] with counts for each label + * @param totalCount sum of counts for all labels + * @return information value, or 0 if totalCount = 0 + */ + @DeveloperApi + override def calculate(counts: Array[Double], totalCount: Double): Double = throw new UnsupportedOperationException("Variance.calculate") /** @@ -34,10 +43,21 @@ object Variance extends Impurity { * @param count number of instances * @param sum sum of labels * @param sumSquares summation of squares of the labels + * @return information value, or 0 if count = 0 */ @DeveloperApi override def calculate(count: Double, sum: Double, sumSquares: Double): Double = { + if (count == 0) { + return 0 + } val squaredLoss = sumSquares - (sum * sum) / count squaredLoss / count } + + /** + * Get this impurity instance. + * This is useful for passing impurity parameters to a Strategy in Java. + */ + def instance = this + } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Bin.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Bin.scala index 2d71e1e366069..af35d88f713e5 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Bin.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Bin.scala @@ -20,15 +20,25 @@ package org.apache.spark.mllib.tree.model import org.apache.spark.mllib.tree.configuration.FeatureType._ /** - * Used for "binning" the features bins for faster best split calculation. For a continuous - * feature, a bin is determined by a low and a high "split". For a categorical feature, - * the a bin is determined using a single label value (category). + * Used for "binning" the features bins for faster best split calculation. + * + * For a continuous feature, the bin is determined by a low and a high split, + * where an example with featureValue falls into the bin s.t. + * lowSplit.threshold < featureValue <= highSplit.threshold. + * + * For ordered categorical features, there is a 1-1-1 correspondence between + * bins, splits, and feature values. The bin is determined by category/feature value. + * However, the bins are not necessarily ordered by feature value; + * they are ordered using impurity. + * For unordered categorical features, there is a 1-1 correspondence between bins, splits, + * where bins and splits correspond to subsets of feature values (in highSplit.categories). + * * @param lowSplit signifying the lower threshold for the continuous feature to be * accepted in the bin * @param highSplit signifying the upper threshold for the continuous feature to be * accepted in the bin * @param featureType type of feature -- categorical or continuous - * @param category categorical label value accepted in the bin + * @param category categorical label value accepted in the bin for ordered features */ private[tree] case class Bin(lowSplit: Split, highSplit: Split, featureType: FeatureType, category: Double) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala index bf692ca8c4bd7..0594fd0749d21 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala @@ -24,7 +24,8 @@ import org.apache.spark.mllib.linalg.Vector /** * :: Experimental :: - * Model to store the decision tree parameters + * Decision tree model for classification or regression. + * This model stores the decision tree structure and parameters. * @param topNode root node * @param algo algorithm type -- classification or regression */ @@ -38,7 +39,7 @@ class DecisionTreeModel(val topNode: Node, val algo: Algo) extends Serializable * @return Double prediction from the trained model */ def predict(features: Vector): Double = { - topNode.predictIfLeaf(features) + topNode.predict(features) } /** @@ -50,4 +51,32 @@ class DecisionTreeModel(val topNode: Node, val algo: Algo) extends Serializable def predict(features: RDD[Vector]): RDD[Double] = { features.map(x => predict(x)) } + + /** + * Get number of nodes in tree, including leaf nodes. + */ + def numNodes: Int = { + 1 + topNode.numDescendants + } + + /** + * Get depth of tree. + * E.g.: Depth 0 means 1 leaf node. Depth 1 means 1 internal node and 2 leaf nodes. + */ + def depth: Int = { + topNode.subtreeDepth + } + + /** + * Print full model. + */ + override def toString: String = algo match { + case Classification => + s"DecisionTreeModel classifier\n" + topNode.subtreeToString(2) + case Regression => + s"DecisionTreeModel regressor\n" + topNode.subtreeToString(2) + case _ => throw new IllegalArgumentException( + s"DecisionTreeModel given unknown algo parameter: $algo.") + } + } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala index cc8a24cce9614..fb12298e0f5d3 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala @@ -27,6 +27,7 @@ import org.apache.spark.annotation.DeveloperApi * @param leftImpurity left node impurity * @param rightImpurity right node impurity * @param predict predicted value + * @param prob probability of the label (classification only) */ @DeveloperApi class InformationGainStats( @@ -34,10 +35,11 @@ class InformationGainStats( val impurity: Double, val leftImpurity: Double, val rightImpurity: Double, - val predict: Double) extends Serializable { + val predict: Double, + val prob: Double = 0.0) extends Serializable { override def toString = { - "gain = %f, impurity = %f, left impurity = %f, right impurity = %f, predict = %f" - .format(gain, impurity, leftImpurity, rightImpurity, predict) + "gain = %f, impurity = %f, left impurity = %f, right impurity = %f, predict = %f, prob = %f" + .format(gain, impurity, leftImpurity, rightImpurity, predict, prob) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala index 682f213f411a7..0eee6262781c1 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala @@ -69,26 +69,82 @@ class Node ( /** * predict value if node is not leaf - * @param feature feature value + * @param features feature value * @return predicted value */ - def predictIfLeaf(feature: Vector) : Double = { + def predict(features: Vector) : Double = { if (isLeaf) { predict } else{ if (split.get.featureType == Continuous) { - if (feature(split.get.feature) <= split.get.threshold) { - leftNode.get.predictIfLeaf(feature) + if (features(split.get.feature) <= split.get.threshold) { + leftNode.get.predict(features) } else { - rightNode.get.predictIfLeaf(feature) + rightNode.get.predict(features) } } else { - if (split.get.categories.contains(feature(split.get.feature))) { - leftNode.get.predictIfLeaf(feature) + if (split.get.categories.contains(features(split.get.feature))) { + leftNode.get.predict(features) } else { - rightNode.get.predictIfLeaf(feature) + rightNode.get.predict(features) } } } } + + /** + * Get the number of nodes in tree below this node, including leaf nodes. + * E.g., if this is a leaf, returns 0. If both children are leaves, returns 2. + */ + private[tree] def numDescendants: Int = { + if (isLeaf) { + 0 + } else { + 2 + leftNode.get.numDescendants + rightNode.get.numDescendants + } + } + + /** + * Get depth of tree from this node. + * E.g.: Depth 0 means this is a leaf node. + */ + private[tree] def subtreeDepth: Int = { + if (isLeaf) { + 0 + } else { + 1 + math.max(leftNode.get.subtreeDepth, rightNode.get.subtreeDepth) + } + } + + /** + * Recursive print function. + * @param indentFactor The number of spaces to add to each level of indentation. + */ + private[tree] def subtreeToString(indentFactor: Int = 0): String = { + + def splitToString(split: Split, left: Boolean): String = { + split.featureType match { + case Continuous => if (left) { + s"(feature ${split.feature} <= ${split.threshold})" + } else { + s"(feature ${split.feature} > ${split.threshold})" + } + case Categorical => if (left) { + s"(feature ${split.feature} in ${split.categories.mkString("{",",","}")})" + } else { + s"(feature ${split.feature} not in ${split.categories.mkString("{",",","}")})" + } + } + } + val prefix: String = " " * indentFactor + if (isLeaf) { + prefix + s"Predict: $predict\n" + } else { + prefix + s"If ${splitToString(split.get, left=true)}\n" + + leftNode.get.subtreeToString(indentFactor + 1) + + prefix + s"Else ${splitToString(split.get, left=false)}\n" + + rightNode.get.subtreeToString(indentFactor + 1) + } + } + } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala index d7ffd386c05ee..50fb48b40de3d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala @@ -24,9 +24,10 @@ import org.apache.spark.mllib.tree.configuration.FeatureType.FeatureType * :: DeveloperApi :: * Split applied to a feature * @param feature feature index - * @param threshold threshold for continuous feature + * @param threshold Threshold for continuous feature. + * Split left if feature <= threshold, else right. * @param featureType type of feature -- categorical or continuous - * @param categories accepted values for categorical variables + * @param categories Split left if categorical feature value is in this set, else right. */ @DeveloperApi case class Split( diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/LabelParsers.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LabelParsers.scala deleted file mode 100644 index e25bf18b780bf..0000000000000 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/LabelParsers.scala +++ /dev/null @@ -1,56 +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.mllib.util - -/** Trait for label parsers. */ -private trait LabelParser extends Serializable { - /** Parses a string label into a double label. */ - def parse(labelString: String): Double -} - -/** Factory methods for label parsers. */ -private object LabelParser { - def getInstance(multiclass: Boolean): LabelParser = { - if (multiclass) MulticlassLabelParser else BinaryLabelParser - } -} - -/** - * Label parser for binary labels, which outputs 1.0 (positive) if the value is greater than 0.5, - * or 0.0 (negative) otherwise. So it works with +1/-1 labeling and +1/0 labeling. - */ -private object BinaryLabelParser extends LabelParser { - /** Gets the default instance of BinaryLabelParser. */ - def getInstance(): LabelParser = this - - /** - * Parses the input label into positive (1.0) if the value is greater than 0.5, - * or negative (0.0) otherwise. - */ - override def parse(labelString: String): Double = if (labelString.toDouble > 0.5) 1.0 else 0.0 -} - -/** - * Label parser for multiclass labels, which converts the input label to double. - */ -private object MulticlassLabelParser extends LabelParser { - /** Gets the default instance of MulticlassLabelParser. */ - def getInstance(): LabelParser = this - - override def parse(labelString: String): Double = labelString.toDouble -} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala index aaf92a1a8869a..ca35100aa99c6 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala @@ -27,9 +27,11 @@ import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD import org.apache.spark.rdd.PartitionwiseSampledRDD import org.apache.spark.util.random.BernoulliSampler -import org.apache.spark.mllib.regression.{LabeledPointParser, LabeledPoint} +import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.StreamingContext +import org.apache.spark.streaming.dstream.DStream /** * Helper methods to load, save and pre-process data used in ML Lib. @@ -55,7 +57,6 @@ object MLUtils { * * @param sc Spark context * @param path file or directory path in any Hadoop-supported file system URI - * @param labelParser parser for labels * @param numFeatures number of features, which will be determined from the input data if a * nonpositive value is given. This is useful when the dataset is already split * into multiple files and you want to load them separately, because some @@ -64,10 +65,9 @@ object MLUtils { * @param minPartitions min number of partitions * @return labeled data stored as an RDD[LabeledPoint] */ - private def loadLibSVMFile( + def loadLibSVMFile( sc: SparkContext, path: String, - labelParser: LabelParser, numFeatures: Int, minPartitions: Int): RDD[LabeledPoint] = { val parsed = sc.textFile(path, minPartitions) @@ -75,7 +75,7 @@ object MLUtils { .filter(line => !(line.isEmpty || line.startsWith("#"))) .map { line => val items = line.split(' ') - val label = labelParser.parse(items.head) + val label = items.head.toDouble val (indices, values) = items.tail.map { item => val indexAndValue = item.split(':') val index = indexAndValue(0).toInt - 1 // Convert 1-based indices to 0-based. @@ -102,64 +102,46 @@ object MLUtils { // Convenient methods for `loadLibSVMFile`. - /** - * Loads labeled data in the LIBSVM format into an RDD[LabeledPoint]. - * The LIBSVM format is a text-based format used by LIBSVM and LIBLINEAR. - * Each line represents a labeled sparse feature vector using the following format: - * {{{label index1:value1 index2:value2 ...}}} - * where the indices are one-based and in ascending order. - * This method parses each line into a [[org.apache.spark.mllib.regression.LabeledPoint]], - * where the feature indices are converted to zero-based. - * - * @param sc Spark context - * @param path file or directory path in any Hadoop-supported file system URI - * @param multiclass whether the input labels contain more than two classes. If false, any label - * with value greater than 0.5 will be mapped to 1.0, or 0.0 otherwise. So it - * works for both +1/-1 and 1/0 cases. If true, the double value parsed directly - * from the label string will be used as the label value. - * @param numFeatures number of features, which will be determined from the input data if a - * nonpositive value is given. This is useful when the dataset is already split - * into multiple files and you want to load them separately, because some - * features may not present in certain files, which leads to inconsistent - * feature dimensions. - * @param minPartitions min number of partitions - * @return labeled data stored as an RDD[LabeledPoint] - */ - def loadLibSVMFile( + @deprecated("use method without multiclass argument, which no longer has effect", "1.1.0") + def loadLibSVMFile( sc: SparkContext, path: String, multiclass: Boolean, numFeatures: Int, minPartitions: Int): RDD[LabeledPoint] = - loadLibSVMFile(sc, path, LabelParser.getInstance(multiclass), numFeatures, minPartitions) + loadLibSVMFile(sc, path, numFeatures, minPartitions) /** * Loads labeled data in the LIBSVM format into an RDD[LabeledPoint], with the default number of * partitions. */ + def loadLibSVMFile( + sc: SparkContext, + path: String, + numFeatures: Int): RDD[LabeledPoint] = + loadLibSVMFile(sc, path, numFeatures, sc.defaultMinPartitions) + + @deprecated("use method without multiclass argument, which no longer has effect", "1.1.0") def loadLibSVMFile( sc: SparkContext, path: String, multiclass: Boolean, numFeatures: Int): RDD[LabeledPoint] = - loadLibSVMFile(sc, path, multiclass, numFeatures, sc.defaultMinPartitions) + loadLibSVMFile(sc, path, numFeatures) - /** - * Loads labeled data in the LIBSVM format into an RDD[LabeledPoint], with the number of features - * determined automatically and the default number of partitions. - */ + @deprecated("use method without multiclass argument, which no longer has effect", "1.1.0") def loadLibSVMFile( sc: SparkContext, path: String, multiclass: Boolean): RDD[LabeledPoint] = - loadLibSVMFile(sc, path, multiclass, -1, sc.defaultMinPartitions) + loadLibSVMFile(sc, path) /** * Loads binary labeled data in the LIBSVM format into an RDD[LabeledPoint], with number of * features determined automatically and the default number of partitions. */ def loadLibSVMFile(sc: SparkContext, path: String): RDD[LabeledPoint] = - loadLibSVMFile(sc, path, multiclass = false, -1, sc.defaultMinPartitions) + loadLibSVMFile(sc, path, -1) /** * Save labeled data in LIBSVM format. @@ -203,7 +185,7 @@ object MLUtils { * @return labeled points stored as an RDD[LabeledPoint] */ def loadLabeledPoints(sc: SparkContext, path: String, minPartitions: Int): RDD[LabeledPoint] = - sc.textFile(path, minPartitions).map(LabeledPointParser.parse) + sc.textFile(path, minPartitions).map(LabeledPoint.parse) /** * Loads labeled points saved using `RDD[LabeledPoint].saveAsTextFile` with the default number of @@ -264,8 +246,8 @@ object MLUtils { (1 to numFolds).map { fold => val sampler = new BernoulliSampler[T]((fold - 1) / numFoldsF, fold / numFoldsF, complement = false) - val validation = new PartitionwiseSampledRDD(rdd, sampler, seed) - val training = new PartitionwiseSampledRDD(rdd, sampler.cloneComplement(), seed) + val validation = new PartitionwiseSampledRDD(rdd, sampler, true, seed) + val training = new PartitionwiseSampledRDD(rdd, sampler.cloneComplement(), true, seed) (training, validation) }.toArray } diff --git a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaLogisticRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaLogisticRegressionSuite.java index faa675b59cd50..862221d48798a 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaLogisticRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaLogisticRegressionSuite.java @@ -92,8 +92,6 @@ public void runLRUsingStaticMethods() { testRDD.rdd(), 100, 1.0, 1.0); int numAccurate = validatePrediction(validationData, model); - System.out.println(numAccurate); Assert.assertTrue(numAccurate > nPoints * 4.0 / 5.0); } - } diff --git a/mllib/src/test/java/org/apache/spark/mllib/feature/JavaTfIdfSuite.java b/mllib/src/test/java/org/apache/spark/mllib/feature/JavaTfIdfSuite.java new file mode 100644 index 0000000000000..e8d99f4ae43ae --- /dev/null +++ b/mllib/src/test/java/org/apache/spark/mllib/feature/JavaTfIdfSuite.java @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.feature; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; + +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import com.google.common.collect.Lists; + +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.mllib.linalg.Vector; + +public class JavaTfIdfSuite implements Serializable { + private transient JavaSparkContext sc; + + @Before + public void setUp() { + sc = new JavaSparkContext("local", "JavaTfIdfSuite"); + } + + @After + public void tearDown() { + sc.stop(); + sc = null; + } + + @Test + public void tfIdf() { + // The tests are to check Java compatibility. + HashingTF tf = new HashingTF(); + JavaRDD> documents = sc.parallelize(Lists.newArrayList( + Lists.newArrayList("this is a sentence".split(" ")), + Lists.newArrayList("this is another sentence".split(" ")), + Lists.newArrayList("this is still a sentence".split(" "))), 2); + JavaRDD termFreqs = tf.transform(documents); + termFreqs.collect(); + IDF idf = new IDF(); + JavaRDD tfIdfs = idf.fit(termFreqs).transform(termFreqs); + List localTfIdfs = tfIdfs.collect(); + int indexOfThis = tf.indexOf("this"); + for (Vector v: localTfIdfs) { + Assert.assertEquals(0.0, v.apply(indexOfThis), 1e-15); + } + } +} diff --git a/mllib/src/test/java/org/apache/spark/mllib/feature/JavaWord2VecSuite.java b/mllib/src/test/java/org/apache/spark/mllib/feature/JavaWord2VecSuite.java new file mode 100644 index 0000000000000..fb7afe8c6434b --- /dev/null +++ b/mllib/src/test/java/org/apache/spark/mllib/feature/JavaWord2VecSuite.java @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.feature; + +import java.io.Serializable; +import java.util.List; + +import scala.Tuple2; + +import com.google.common.collect.Lists; +import com.google.common.base.Strings; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; + +public class JavaWord2VecSuite implements Serializable { + private transient JavaSparkContext sc; + + @Before + public void setUp() { + sc = new JavaSparkContext("local", "JavaWord2VecSuite"); + } + + @After + public void tearDown() { + sc.stop(); + sc = null; + } + + @Test + @SuppressWarnings("unchecked") + public void word2Vec() { + // The tests are to check Java compatibility. + String sentence = Strings.repeat("a b ", 100) + Strings.repeat("a c ", 10); + List words = Lists.newArrayList(sentence.split(" ")); + List> localDoc = Lists.newArrayList(words, words); + JavaRDD> doc = sc.parallelize(localDoc); + Word2Vec word2vec = new Word2Vec() + .setVectorSize(10) + .setSeed(42L); + Word2VecModel model = word2vec.fit(doc); + Tuple2[] syms = model.findSynonyms("a", 2); + Assert.assertEquals(2, syms.length); + Assert.assertEquals("b", syms[0]._1()); + Assert.assertEquals("c", syms[1]._1()); + } +} diff --git a/mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java b/mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java index bf2365f82044c..f6ca9643227f8 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java @@ -20,6 +20,11 @@ import java.io.Serializable; import java.util.List; +import scala.Tuple2; +import scala.Tuple3; + +import org.jblas.DoubleMatrix; + import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -28,8 +33,6 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; -import org.jblas.DoubleMatrix; - public class JavaALSSuite implements Serializable { private transient JavaSparkContext sc; @@ -44,21 +47,28 @@ public void tearDown() { sc = null; } - static void validatePrediction(MatrixFactorizationModel model, int users, int products, int features, - DoubleMatrix trueRatings, double matchThreshold, boolean implicitPrefs, DoubleMatrix truePrefs) { + static void validatePrediction( + MatrixFactorizationModel model, + int users, + int products, + int features, + DoubleMatrix trueRatings, + double matchThreshold, + boolean implicitPrefs, + DoubleMatrix truePrefs) { DoubleMatrix predictedU = new DoubleMatrix(users, features); - List> userFeatures = model.userFeatures().toJavaRDD().collect(); + List> userFeatures = model.userFeatures().toJavaRDD().collect(); for (int i = 0; i < features; ++i) { - for (scala.Tuple2 userFeature : userFeatures) { + for (Tuple2 userFeature : userFeatures) { predictedU.put((Integer)userFeature._1(), i, userFeature._2()[i]); } } DoubleMatrix predictedP = new DoubleMatrix(products, features); - List> productFeatures = + List> productFeatures = model.productFeatures().toJavaRDD().collect(); for (int i = 0; i < features; ++i) { - for (scala.Tuple2 productFeature : productFeatures) { + for (Tuple2 productFeature : productFeatures) { predictedP.put((Integer)productFeature._1(), i, productFeature._2()[i]); } } @@ -75,7 +85,8 @@ static void validatePrediction(MatrixFactorizationModel model, int users, int pr } } } else { - // For implicit prefs we use the confidence-weighted RMSE to test (ref Mahout's implicit ALS tests) + // For implicit prefs we use the confidence-weighted RMSE to test + // (ref Mahout's implicit ALS tests) double sqErr = 0.0; double denom = 0.0; for (int u = 0; u < users; ++u) { @@ -100,7 +111,7 @@ public void runALSUsingStaticMethods() { int iterations = 15; int users = 50; int products = 100; - scala.Tuple3, DoubleMatrix, DoubleMatrix> testData = ALSSuite.generateRatingsAsJavaList( + Tuple3, DoubleMatrix, DoubleMatrix> testData = ALSSuite.generateRatingsAsJavaList( users, products, features, 0.7, false, false); JavaRDD data = sc.parallelize(testData._1()); @@ -114,14 +125,14 @@ public void runALSUsingConstructor() { int iterations = 15; int users = 100; int products = 200; - scala.Tuple3, DoubleMatrix, DoubleMatrix> testData = ALSSuite.generateRatingsAsJavaList( + Tuple3, DoubleMatrix, DoubleMatrix> testData = ALSSuite.generateRatingsAsJavaList( users, products, features, 0.7, false, false); JavaRDD data = sc.parallelize(testData._1()); MatrixFactorizationModel model = new ALS().setRank(features) - .setIterations(iterations) - .run(data.rdd()); + .setIterations(iterations) + .run(data.rdd()); validatePrediction(model, users, products, features, testData._2(), 0.3, false, testData._3()); } @@ -131,7 +142,7 @@ public void runImplicitALSUsingStaticMethods() { int iterations = 15; int users = 80; int products = 160; - scala.Tuple3, DoubleMatrix, DoubleMatrix> testData = ALSSuite.generateRatingsAsJavaList( + Tuple3, DoubleMatrix, DoubleMatrix> testData = ALSSuite.generateRatingsAsJavaList( users, products, features, 0.7, true, false); JavaRDD data = sc.parallelize(testData._1()); @@ -145,7 +156,7 @@ public void runImplicitALSUsingConstructor() { int iterations = 15; int users = 100; int products = 200; - scala.Tuple3, DoubleMatrix, DoubleMatrix> testData = ALSSuite.generateRatingsAsJavaList( + Tuple3, DoubleMatrix, DoubleMatrix> testData = ALSSuite.generateRatingsAsJavaList( users, products, features, 0.7, true, false); JavaRDD data = sc.parallelize(testData._1()); @@ -163,12 +174,42 @@ public void runImplicitALSWithNegativeWeight() { int iterations = 15; int users = 80; int products = 160; - scala.Tuple3, DoubleMatrix, DoubleMatrix> testData = ALSSuite.generateRatingsAsJavaList( + Tuple3, DoubleMatrix, DoubleMatrix> testData = ALSSuite.generateRatingsAsJavaList( users, products, features, 0.7, true, true); JavaRDD data = sc.parallelize(testData._1()); - MatrixFactorizationModel model = ALS.trainImplicit(data.rdd(), features, iterations); + MatrixFactorizationModel model = new ALS().setRank(features) + .setIterations(iterations) + .setImplicitPrefs(true) + .setSeed(8675309L) + .run(data.rdd()); validatePrediction(model, users, products, features, testData._2(), 0.4, true, testData._3()); } + @Test + public void runRecommend() { + int features = 5; + int iterations = 10; + int users = 200; + int products = 50; + Tuple3, DoubleMatrix, DoubleMatrix> testData = ALSSuite.generateRatingsAsJavaList( + users, products, features, 0.7, true, false); + JavaRDD data = sc.parallelize(testData._1()); + MatrixFactorizationModel model = new ALS().setRank(features) + .setIterations(iterations) + .setImplicitPrefs(true) + .setSeed(8675309L) + .run(data.rdd()); + validateRecommendations(model.recommendProducts(1, 10), 10); + validateRecommendations(model.recommendUsers(1, 20), 20); + } + + private static void validateRecommendations(Rating[] recommendations, int howMany) { + Assert.assertEquals(howMany, recommendations.length); + for (int i = 1; i < recommendations.length; i++) { + Assert.assertTrue(recommendations[i-1].rating() >= recommendations[i].rating()); + } + Assert.assertTrue(recommendations[0].rating() > 0.7); + } + } diff --git a/mllib/src/test/java/org/apache/spark/mllib/tree/JavaDecisionTreeSuite.java b/mllib/src/test/java/org/apache/spark/mllib/tree/JavaDecisionTreeSuite.java new file mode 100644 index 0000000000000..2c281a1ee7157 --- /dev/null +++ b/mllib/src/test/java/org/apache/spark/mllib/tree/JavaDecisionTreeSuite.java @@ -0,0 +1,102 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.tree; + +import java.io.Serializable; +import java.util.HashMap; +import java.util.List; + +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.tree.configuration.Algo; +import org.apache.spark.mllib.tree.configuration.Strategy; +import org.apache.spark.mllib.tree.impurity.Gini; +import org.apache.spark.mllib.tree.model.DecisionTreeModel; + + +public class JavaDecisionTreeSuite implements Serializable { + private transient JavaSparkContext sc; + + @Before + public void setUp() { + sc = new JavaSparkContext("local", "JavaDecisionTreeSuite"); + } + + @After + public void tearDown() { + sc.stop(); + sc = null; + } + + int validatePrediction(List validationData, DecisionTreeModel model) { + int numCorrect = 0; + for (LabeledPoint point: validationData) { + Double prediction = model.predict(point.features()); + if (prediction == point.label()) { + numCorrect++; + } + } + return numCorrect; + } + + @Test + public void runDTUsingConstructor() { + List arr = DecisionTreeSuite.generateCategoricalDataPointsAsJavaList(); + JavaRDD rdd = sc.parallelize(arr); + HashMap categoricalFeaturesInfo = new HashMap(); + categoricalFeaturesInfo.put(1, 2); // feature 1 has 2 categories + + int maxDepth = 4; + int numClasses = 2; + int maxBins = 100; + Strategy strategy = new Strategy(Algo.Classification(), Gini.instance(), maxDepth, numClasses, + maxBins, categoricalFeaturesInfo); + + DecisionTree learner = new DecisionTree(strategy); + DecisionTreeModel model = learner.train(rdd.rdd()); + + int numCorrect = validatePrediction(arr, model); + Assert.assertTrue(numCorrect == rdd.count()); + } + + @Test + public void runDTUsingStaticMethods() { + List arr = DecisionTreeSuite.generateCategoricalDataPointsAsJavaList(); + JavaRDD rdd = sc.parallelize(arr); + HashMap categoricalFeaturesInfo = new HashMap(); + categoricalFeaturesInfo.put(1, 2); // feature 1 has 2 categories + + int maxDepth = 4; + int numClasses = 2; + int maxBins = 100; + Strategy strategy = new Strategy(Algo.Classification(), Gini.instance(), maxDepth, numClasses, + maxBins, categoricalFeaturesInfo); + + DecisionTreeModel model = DecisionTree$.MODULE$.train(rdd.rdd(), strategy); + + int numCorrect = validatePrediction(arr, model); + Assert.assertTrue(numCorrect == rdd.count()); + } + +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/api/python/PythonMLLibAPISuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/api/python/PythonMLLibAPISuite.scala index 642843f90204c..092d67bbc5238 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/api/python/PythonMLLibAPISuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/api/python/PythonMLLibAPISuite.scala @@ -19,11 +19,10 @@ package org.apache.spark.mllib.api.python import org.scalatest.FunSuite -import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.linalg.{Matrices, Vectors} import org.apache.spark.mllib.regression.LabeledPoint class PythonMLLibAPISuite extends FunSuite { - val py = new PythonMLLibAPI test("vector serialization") { val vectors = Seq( @@ -34,8 +33,8 @@ class PythonMLLibAPISuite extends FunSuite { Vectors.sparse(1, Array.empty[Int], Array.empty[Double]), Vectors.sparse(2, Array(1), Array(-2.0))) vectors.foreach { v => - val bytes = py.serializeDoubleVector(v) - val u = py.deserializeDoubleVector(bytes) + val bytes = SerDe.serializeDoubleVector(v) + val u = SerDe.deserializeDoubleVector(bytes) assert(u.getClass === v.getClass) assert(u === v) } @@ -50,11 +49,34 @@ class PythonMLLibAPISuite extends FunSuite { LabeledPoint(1.0, Vectors.sparse(1, Array.empty[Int], Array.empty[Double])), LabeledPoint(-0.5, Vectors.sparse(2, Array(1), Array(-2.0)))) points.foreach { p => - val bytes = py.serializeLabeledPoint(p) - val q = py.deserializeLabeledPoint(bytes) + val bytes = SerDe.serializeLabeledPoint(p) + val q = SerDe.deserializeLabeledPoint(bytes) assert(q.label === p.label) assert(q.features.getClass === p.features.getClass) assert(q.features === p.features) } } + + test("double serialization") { + for (x <- List(123.0, -10.0, 0.0, Double.MaxValue, Double.MinValue, Double.NaN)) { + val bytes = SerDe.serializeDouble(x) + val deser = SerDe.deserializeDouble(bytes) + // We use `equals` here for comparison because we cannot use `==` for NaN + assert(x.equals(deser)) + } + } + + test("matrix to 2D array") { + val values = Array[Double](0, 1.2, 3, 4.56, 7, 8) + val matrix = Matrices.dense(2, 3, values) + val arr = SerDe.to2dArray(matrix) + val expected = Array(Array[Double](0, 3, 7), Array[Double](1.2, 4.56, 8)) + assert(arr === expected) + + // Test conversion for empty matrix + val empty = Array[Double]() + val emptyMatrix = Matrices.dense(0, 0, empty) + val empty2D = SerDe.to2dArray(emptyMatrix) + assert(empty2D === Array[Array[Double]]()) + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala index 44b757b6a1fb7..862178694a50e 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala @@ -25,7 +25,8 @@ import org.scalatest.Matchers import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.regression._ -import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.mllib.util.{LocalClusterSparkContext, LocalSparkContext} +import org.apache.spark.mllib.util.TestingUtils._ object LogisticRegressionSuite { @@ -66,7 +67,7 @@ class LogisticRegressionSuite extends FunSuite with LocalSparkContext with Match } // Test if we can correctly learn A, B where Y = logistic(A + B*X) - test("logistic regression") { + test("logistic regression with SGD") { val nPoints = 10000 val A = 2.0 val B = -1.5 @@ -81,9 +82,8 @@ class LogisticRegressionSuite extends FunSuite with LocalSparkContext with Match val model = lr.run(testRDD) // Test the weights - val weight0 = model.weights(0) - assert(weight0 >= -1.60 && weight0 <= -1.40, weight0 + " not in [-1.6, -1.4]") - assert(model.intercept >= 1.9 && model.intercept <= 2.1, model.intercept + " not in [1.9, 2.1]") + assert(model.weights(0) ~== -1.52 relTol 0.01) + assert(model.intercept ~== 2.00 relTol 0.01) val validationData = LogisticRegressionSuite.generateLogisticInput(A, B, nPoints, 17) val validationRDD = sc.parallelize(validationData, 2) @@ -94,7 +94,36 @@ class LogisticRegressionSuite extends FunSuite with LocalSparkContext with Match validatePrediction(validationData.map(row => model.predict(row.features)), validationData) } - test("logistic regression with initial weights") { + // Test if we can correctly learn A, B where Y = logistic(A + B*X) + test("logistic regression with LBFGS") { + val nPoints = 10000 + val A = 2.0 + val B = -1.5 + + val testData = LogisticRegressionSuite.generateLogisticInput(A, B, nPoints, 42) + + val testRDD = sc.parallelize(testData, 2) + testRDD.cache() + val lr = new LogisticRegressionWithLBFGS().setIntercept(true) + + val model = lr.run(testRDD) + + // Test the weights + assert(model.weights(0) ~== -1.52 relTol 0.01) + assert(model.intercept ~== 2.00 relTol 0.01) + assert(model.weights(0) ~== model.weights(0) relTol 0.01) + assert(model.intercept ~== model.intercept relTol 0.01) + + val validationData = LogisticRegressionSuite.generateLogisticInput(A, B, nPoints, 17) + val validationRDD = sc.parallelize(validationData, 2) + // Test prediction on RDD. + validatePrediction(model.predict(validationRDD.map(_.features)).collect(), validationData) + + // Test prediction on Array. + validatePrediction(validationData.map(row => model.predict(row.features)), validationData) + } + + test("logistic regression with initial weights with SGD") { val nPoints = 10000 val A = 2.0 val B = -1.5 @@ -113,9 +142,9 @@ class LogisticRegressionSuite extends FunSuite with LocalSparkContext with Match val model = lr.run(testRDD, initialWeights) - val weight0 = model.weights(0) - assert(weight0 >= -1.60 && weight0 <= -1.40, weight0 + " not in [-1.6, -1.4]") - assert(model.intercept >= 1.9 && model.intercept <= 2.1, model.intercept + " not in [1.9, 2.1]") + // Test the weights + assert(model.weights(0) ~== -1.50 relTol 0.01) + assert(model.intercept ~== 1.97 relTol 0.01) val validationData = LogisticRegressionSuite.generateLogisticInput(A, B, nPoints, 17) val validationRDD = sc.parallelize(validationData, 2) @@ -125,4 +154,132 @@ class LogisticRegressionSuite extends FunSuite with LocalSparkContext with Match // Test prediction on Array. validatePrediction(validationData.map(row => model.predict(row.features)), validationData) } + + test("logistic regression with initial weights with LBFGS") { + val nPoints = 10000 + val A = 2.0 + val B = -1.5 + + val testData = LogisticRegressionSuite.generateLogisticInput(A, B, nPoints, 42) + + val initialB = -1.0 + val initialWeights = Vectors.dense(initialB) + + val testRDD = sc.parallelize(testData, 2) + testRDD.cache() + + // Use half as many iterations as the previous test. + val lr = new LogisticRegressionWithLBFGS().setIntercept(true) + + val model = lr.run(testRDD, initialWeights) + + // Test the weights + assert(model.weights(0) ~== -1.50 relTol 0.02) + assert(model.intercept ~== 1.97 relTol 0.02) + + val validationData = LogisticRegressionSuite.generateLogisticInput(A, B, nPoints, 17) + val validationRDD = sc.parallelize(validationData, 2) + // Test prediction on RDD. + validatePrediction(model.predict(validationRDD.map(_.features)).collect(), validationData) + + // Test prediction on Array. + validatePrediction(validationData.map(row => model.predict(row.features)), validationData) + } + + test("numerical stability of scaling features using logistic regression with LBFGS") { + /** + * If we rescale the features, the condition number will be changed so the convergence rate + * and the solution will not equal to the original solution multiple by the scaling factor + * which it should be. + * + * However, since in the LogisticRegressionWithLBFGS, we standardize the training dataset first, + * no matter how we multiple a scaling factor into the dataset, the convergence rate should be + * the same, and the solution should equal to the original solution multiple by the scaling + * factor. + */ + + val nPoints = 10000 + val A = 2.0 + val B = -1.5 + + val testData = LogisticRegressionSuite.generateLogisticInput(A, B, nPoints, 42) + + val initialWeights = Vectors.dense(0.0) + + val testRDD1 = sc.parallelize(testData, 2) + + val testRDD2 = sc.parallelize( + testData.map(x => LabeledPoint(x.label, Vectors.fromBreeze(x.features.toBreeze * 1.0E3))), 2) + + val testRDD3 = sc.parallelize( + testData.map(x => LabeledPoint(x.label, Vectors.fromBreeze(x.features.toBreeze * 1.0E6))), 2) + + testRDD1.cache() + testRDD2.cache() + testRDD3.cache() + + val lrA = new LogisticRegressionWithLBFGS().setIntercept(true) + val lrB = new LogisticRegressionWithLBFGS().setIntercept(true).setFeatureScaling(false) + + val modelA1 = lrA.run(testRDD1, initialWeights) + val modelA2 = lrA.run(testRDD2, initialWeights) + val modelA3 = lrA.run(testRDD3, initialWeights) + + val modelB1 = lrB.run(testRDD1, initialWeights) + val modelB2 = lrB.run(testRDD2, initialWeights) + val modelB3 = lrB.run(testRDD3, initialWeights) + + // For model trained with feature standardization, the weights should + // be the same in the scaled space. Note that the weights here are already + // in the original space, we transform back to scaled space to compare. + assert(modelA1.weights(0) ~== modelA2.weights(0) * 1.0E3 absTol 0.01) + assert(modelA1.weights(0) ~== modelA3.weights(0) * 1.0E6 absTol 0.01) + + // Training data with different scales without feature standardization + // will not yield the same result in the scaled space due to poor + // convergence rate. + assert(modelB1.weights(0) !~== modelB2.weights(0) * 1.0E3 absTol 0.1) + assert(modelB1.weights(0) !~== modelB3.weights(0) * 1.0E6 absTol 0.1) + } + +} + +class LogisticRegressionClusterSuite extends FunSuite with LocalClusterSparkContext { + + test("task size should be small in both training and prediction using SGD optimizer") { + val m = 4 + val n = 200000 + val points = sc.parallelize(0 until m, 2).mapPartitionsWithIndex { (idx, iter) => + val random = new Random(idx) + iter.map(i => LabeledPoint(1.0, Vectors.dense(Array.fill(n)(random.nextDouble())))) + }.cache() + // If we serialize data directly in the task closure, the size of the serialized task would be + // greater than 1MB and hence Spark would throw an error. + val model = LogisticRegressionWithSGD.train(points, 2) + + val predictions = model.predict(points.map(_.features)) + + // Materialize the RDDs + predictions.count() + } + + test("task size should be small in both training and prediction using LBFGS optimizer") { + val m = 4 + val n = 200000 + val points = sc.parallelize(0 until m, 2).mapPartitionsWithIndex { (idx, iter) => + val random = new Random(idx) + iter.map(i => LabeledPoint(1.0, Vectors.dense(Array.fill(n)(random.nextDouble())))) + }.cache() + // If we serialize data directly in the task closure, the size of the serialized task would be + // greater than 1MB and hence Spark would throw an error. + val lr = new LogisticRegressionWithLBFGS().setIntercept(true) + lr.optimizer.setNumIterations(2) + val model = lr.run(points) + + val predictions = model.predict(points.map(_.features)) + + // Materialize the RDDs + predictions.count() + } + } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala index 516895d04222d..06cdd04f5fdae 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala @@ -23,7 +23,7 @@ import org.scalatest.FunSuite import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.mllib.util.{LocalClusterSparkContext, LocalSparkContext} object NaiveBayesSuite { @@ -96,3 +96,21 @@ class NaiveBayesSuite extends FunSuite with LocalSparkContext { validatePrediction(validationData.map(row => model.predict(row.features)), validationData) } } + +class NaiveBayesClusterSuite extends FunSuite with LocalClusterSparkContext { + + test("task size should be small in both training and prediction") { + val m = 10 + val n = 200000 + val examples = sc.parallelize(0 until m, 2).mapPartitionsWithIndex { (idx, iter) => + val random = new Random(idx) + iter.map { i => + LabeledPoint(random.nextInt(2), Vectors.dense(Array.fill(n)(random.nextDouble()))) + } + } + // If we serialize data directly in the task closure, the size of the serialized task would be + // greater than 1MB and hence Spark would throw an error. + val model = NaiveBayes.train(examples) + val predictions = model.predict(examples.map(_.features)) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala index 886c71dde3af7..65e5df58db4c7 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala @@ -17,17 +17,16 @@ package org.apache.spark.mllib.classification -import scala.util.Random import scala.collection.JavaConversions._ - -import org.scalatest.FunSuite +import scala.util.Random import org.jblas.DoubleMatrix +import org.scalatest.FunSuite import org.apache.spark.SparkException -import org.apache.spark.mllib.regression._ -import org.apache.spark.mllib.util.LocalSparkContext import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.regression._ +import org.apache.spark.mllib.util.{LocalClusterSparkContext, LocalSparkContext} object SVMSuite { @@ -193,3 +192,19 @@ class SVMSuite extends FunSuite with LocalSparkContext { new SVMWithSGD().setValidateData(false).run(testRDDInvalid) } } + +class SVMClusterSuite extends FunSuite with LocalClusterSparkContext { + + test("task size should be small in both training and prediction") { + val m = 4 + val n = 200000 + val points = sc.parallelize(0 until m, 2).mapPartitionsWithIndex { (idx, iter) => + val random = new Random(idx) + iter.map(i => LabeledPoint(1.0, Vectors.dense(Array.fill(n)(random.nextDouble())))) + }.cache() + // If we serialize data directly in the task closure, the size of the serialized task would be + // greater than 1MB and hence Spark would throw an error. + val model = SVMWithSGD.train(points, 2) + val predictions = model.predict(points.map(_.features)) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala index 560a4ad71a4de..afa1f79b95a12 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala @@ -17,14 +17,17 @@ package org.apache.spark.mllib.clustering +import scala.util.Random + import org.scalatest.FunSuite -import org.apache.spark.mllib.util.LocalSparkContext -import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.mllib.util.{LocalClusterSparkContext, LocalSparkContext} +import org.apache.spark.mllib.util.TestingUtils._ class KMeansSuite extends FunSuite with LocalSparkContext { - import KMeans.{RANDOM, K_MEANS_PARALLEL} + import org.apache.spark.mllib.clustering.KMeans.{K_MEANS_PARALLEL, RANDOM} test("single cluster") { val data = sc.parallelize(Array( @@ -38,27 +41,53 @@ class KMeansSuite extends FunSuite with LocalSparkContext { // No matter how many runs or iterations we use, we should get one cluster, // centered at the mean of the points - var model = KMeans.train(data, k=1, maxIterations=1) - assert(model.clusterCenters.head === center) + var model = KMeans.train(data, k = 1, maxIterations = 1) + assert(model.clusterCenters.head ~== center absTol 1E-5) - model = KMeans.train(data, k=1, maxIterations=2) - assert(model.clusterCenters.head === center) + model = KMeans.train(data, k = 1, maxIterations = 2) + assert(model.clusterCenters.head ~== center absTol 1E-5) - model = KMeans.train(data, k=1, maxIterations=5) - assert(model.clusterCenters.head === center) + model = KMeans.train(data, k = 1, maxIterations = 5) + assert(model.clusterCenters.head ~== center absTol 1E-5) - model = KMeans.train(data, k=1, maxIterations=1, runs=5) - assert(model.clusterCenters.head === center) + model = KMeans.train(data, k = 1, maxIterations = 1, runs = 5) + assert(model.clusterCenters.head ~== center absTol 1E-5) - model = KMeans.train(data, k=1, maxIterations=1, runs=5) - assert(model.clusterCenters.head === center) + model = KMeans.train(data, k = 1, maxIterations = 1, runs = 5) + assert(model.clusterCenters.head ~== center absTol 1E-5) - model = KMeans.train(data, k=1, maxIterations=1, runs=1, initializationMode=RANDOM) - assert(model.clusterCenters.head === center) + model = KMeans.train(data, k = 1, maxIterations = 1, runs = 1, initializationMode = RANDOM) + assert(model.clusterCenters.head ~== center absTol 1E-5) model = KMeans.train( - data, k=1, maxIterations=1, runs=1, initializationMode=K_MEANS_PARALLEL) - assert(model.clusterCenters.head === center) + data, k = 1, maxIterations = 1, runs = 1, initializationMode = K_MEANS_PARALLEL) + assert(model.clusterCenters.head ~== center absTol 1E-5) + } + + test("no distinct points") { + val data = sc.parallelize( + Array( + Vectors.dense(1.0, 2.0, 3.0), + Vectors.dense(1.0, 2.0, 3.0), + Vectors.dense(1.0, 2.0, 3.0)), + 2) + val center = Vectors.dense(1.0, 2.0, 3.0) + + // Make sure code runs. + var model = KMeans.train(data, k=2, maxIterations=1) + assert(model.clusterCenters.size === 2) + } + + test("more clusters than points") { + val data = sc.parallelize( + Array( + Vectors.dense(1.0, 2.0, 3.0), + Vectors.dense(1.0, 3.0, 4.0)), + 2) + + // Make sure code runs. + var model = KMeans.train(data, k=3, maxIterations=1) + assert(model.clusterCenters.size === 3) } test("single cluster with big dataset") { @@ -74,27 +103,28 @@ class KMeansSuite extends FunSuite with LocalSparkContext { val center = Vectors.dense(1.0, 3.0, 4.0) - var model = KMeans.train(data, k=1, maxIterations=1) + var model = KMeans.train(data, k = 1, maxIterations = 1) assert(model.clusterCenters.size === 1) - assert(model.clusterCenters.head === center) + assert(model.clusterCenters.head ~== center absTol 1E-5) - model = KMeans.train(data, k=1, maxIterations=2) - assert(model.clusterCenters.head === center) + model = KMeans.train(data, k = 1, maxIterations = 2) + assert(model.clusterCenters.head ~== center absTol 1E-5) - model = KMeans.train(data, k=1, maxIterations=5) - assert(model.clusterCenters.head === center) + model = KMeans.train(data, k = 1, maxIterations = 5) + assert(model.clusterCenters.head ~== center absTol 1E-5) - model = KMeans.train(data, k=1, maxIterations=1, runs=5) - assert(model.clusterCenters.head === center) + model = KMeans.train(data, k = 1, maxIterations = 1, runs = 5) + assert(model.clusterCenters.head ~== center absTol 1E-5) - model = KMeans.train(data, k=1, maxIterations=1, runs=5) - assert(model.clusterCenters.head === center) + model = KMeans.train(data, k = 1, maxIterations = 1, runs = 5) + assert(model.clusterCenters.head ~== center absTol 1E-5) - model = KMeans.train(data, k=1, maxIterations=1, runs=1, initializationMode=RANDOM) - assert(model.clusterCenters.head === center) + model = KMeans.train(data, k = 1, maxIterations = 1, runs = 1, initializationMode = RANDOM) + assert(model.clusterCenters.head ~== center absTol 1E-5) - model = KMeans.train(data, k=1, maxIterations=1, runs=1, initializationMode=K_MEANS_PARALLEL) - assert(model.clusterCenters.head === center) + model = KMeans.train(data, k = 1, maxIterations = 1, runs = 1, + initializationMode = K_MEANS_PARALLEL) + assert(model.clusterCenters.head ~== center absTol 1E-5) } test("single cluster with sparse data") { @@ -119,31 +149,40 @@ class KMeansSuite extends FunSuite with LocalSparkContext { val center = Vectors.sparse(n, Seq((0, 1.0), (1, 3.0), (2, 4.0))) - var model = KMeans.train(data, k=1, maxIterations=1) - assert(model.clusterCenters.head === center) + var model = KMeans.train(data, k = 1, maxIterations = 1) + assert(model.clusterCenters.head ~== center absTol 1E-5) - model = KMeans.train(data, k=1, maxIterations=2) - assert(model.clusterCenters.head === center) + model = KMeans.train(data, k = 1, maxIterations = 2) + assert(model.clusterCenters.head ~== center absTol 1E-5) - model = KMeans.train(data, k=1, maxIterations=5) - assert(model.clusterCenters.head === center) + model = KMeans.train(data, k = 1, maxIterations = 5) + assert(model.clusterCenters.head ~== center absTol 1E-5) - model = KMeans.train(data, k=1, maxIterations=1, runs=5) - assert(model.clusterCenters.head === center) + model = KMeans.train(data, k = 1, maxIterations = 1, runs = 5) + assert(model.clusterCenters.head ~== center absTol 1E-5) - model = KMeans.train(data, k=1, maxIterations=1, runs=5) - assert(model.clusterCenters.head === center) + model = KMeans.train(data, k = 1, maxIterations = 1, runs = 5) + assert(model.clusterCenters.head ~== center absTol 1E-5) - model = KMeans.train(data, k=1, maxIterations=1, runs=1, initializationMode=RANDOM) - assert(model.clusterCenters.head === center) + model = KMeans.train(data, k = 1, maxIterations = 1, runs = 1, initializationMode = RANDOM) + assert(model.clusterCenters.head ~== center absTol 1E-5) - model = KMeans.train(data, k=1, maxIterations=1, runs=1, initializationMode=K_MEANS_PARALLEL) - assert(model.clusterCenters.head === center) + model = KMeans.train(data, k = 1, maxIterations = 1, runs = 1, + initializationMode = K_MEANS_PARALLEL) + assert(model.clusterCenters.head ~== center absTol 1E-5) data.unpersist() } test("k-means|| initialization") { + + case class VectorWithCompare(x: Vector) extends Ordered[VectorWithCompare] { + @Override def compare(that: VectorWithCompare): Int = { + if(this.x.toArray.foldLeft[Double](0.0)((acc, x) => acc + x * x) > + that.x.toArray.foldLeft[Double](0.0)((acc, x) => acc + x * x)) -1 else 1 + } + } + val points = Seq( Vectors.dense(1.0, 2.0, 6.0), Vectors.dense(1.0, 3.0, 0.0), @@ -157,16 +196,20 @@ class KMeansSuite extends FunSuite with LocalSparkContext { // it will make at least five passes, and it will give non-zero probability to each // unselected point as long as it hasn't yet selected all of them - var model = KMeans.train(rdd, k=5, maxIterations=1) - assert(Set(model.clusterCenters: _*) === Set(points: _*)) + var model = KMeans.train(rdd, k = 5, maxIterations = 1) + + assert(model.clusterCenters.sortBy(VectorWithCompare(_)) + .zip(points.sortBy(VectorWithCompare(_))).forall(x => x._1 ~== (x._2) absTol 1E-5)) // Iterations of Lloyd's should not change the answer either - model = KMeans.train(rdd, k=5, maxIterations=10) - assert(Set(model.clusterCenters: _*) === Set(points: _*)) + model = KMeans.train(rdd, k = 5, maxIterations = 10) + assert(model.clusterCenters.sortBy(VectorWithCompare(_)) + .zip(points.sortBy(VectorWithCompare(_))).forall(x => x._1 ~== (x._2) absTol 1E-5)) // Neither should more runs - model = KMeans.train(rdd, k=5, maxIterations=10, runs=5) - assert(Set(model.clusterCenters: _*) === Set(points: _*)) + model = KMeans.train(rdd, k = 5, maxIterations = 10, runs = 5) + assert(model.clusterCenters.sortBy(VectorWithCompare(_)) + .zip(points.sortBy(VectorWithCompare(_))).forall(x => x._1 ~== (x._2) absTol 1E-5)) } test("two clusters") { @@ -194,3 +237,22 @@ class KMeansSuite extends FunSuite with LocalSparkContext { } } } + +class KMeansClusterSuite extends FunSuite with LocalClusterSparkContext { + + test("task size should be small in both training and prediction") { + val m = 4 + val n = 200000 + val points = sc.parallelize(0 until m, 2).mapPartitionsWithIndex { (idx, iter) => + val random = new Random(idx) + iter.map(i => Vectors.dense(Array.fill(n)(random.nextDouble))) + }.cache() + for (initMode <- Seq(KMeans.RANDOM, KMeans.K_MEANS_PARALLEL)) { + // If we serialize data directly in the task closure, the size of the serialized task would be + // greater than 1MB and hence Spark would throw an error. + val model = KMeans.train(points, 2, 2, 1, initMode) + val predictions = model.predict(points).collect() + val cost = model.computeCost(points) + } + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/AreaUnderCurveSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/AreaUnderCurveSuite.scala index 1c9844f289fe0..994e0feb8629e 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/AreaUnderCurveSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/AreaUnderCurveSuite.scala @@ -20,27 +20,28 @@ package org.apache.spark.mllib.evaluation import org.scalatest.FunSuite import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.mllib.util.TestingUtils._ class AreaUnderCurveSuite extends FunSuite with LocalSparkContext { test("auc computation") { val curve = Seq((0.0, 0.0), (1.0, 1.0), (2.0, 3.0), (3.0, 0.0)) val auc = 4.0 - assert(AreaUnderCurve.of(curve) === auc) + assert(AreaUnderCurve.of(curve) ~== auc absTol 1E-5) val rddCurve = sc.parallelize(curve, 2) - assert(AreaUnderCurve.of(rddCurve) == auc) + assert(AreaUnderCurve.of(rddCurve) ~== auc absTol 1E-5) } test("auc of an empty curve") { val curve = Seq.empty[(Double, Double)] - assert(AreaUnderCurve.of(curve) === 0.0) + assert(AreaUnderCurve.of(curve) ~== 0.0 absTol 1E-5) val rddCurve = sc.parallelize(curve, 2) - assert(AreaUnderCurve.of(rddCurve) === 0.0) + assert(AreaUnderCurve.of(rddCurve) ~== 0.0 absTol 1E-5) } test("auc of a curve with a single point") { val curve = Seq((1.0, 1.0)) - assert(AreaUnderCurve.of(curve) === 0.0) + assert(AreaUnderCurve.of(curve) ~== 0.0 absTol 1E-5) val rddCurve = sc.parallelize(curve, 2) - assert(AreaUnderCurve.of(rddCurve) === 0.0) + assert(AreaUnderCurve.of(rddCurve) ~== 0.0 absTol 1E-5) } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetricsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetricsSuite.scala index 9d16182f9d8c4..a733f88b60b80 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetricsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetricsSuite.scala @@ -20,8 +20,15 @@ package org.apache.spark.mllib.evaluation import org.scalatest.FunSuite import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.mllib.util.TestingUtils._ class BinaryClassificationMetricsSuite extends FunSuite with LocalSparkContext { + + def cond1(x: (Double, Double)): Boolean = x._1 ~= (x._2) absTol 1E-5 + + def cond2(x: ((Double, Double), (Double, Double))): Boolean = + (x._1._1 ~= x._2._1 absTol 1E-5) && (x._1._2 ~= x._2._2 absTol 1E-5) + test("binary evaluation metrics") { val scoreAndLabels = sc.parallelize( Seq((0.1, 0.0), (0.1, 1.0), (0.4, 0.0), (0.6, 0.0), (0.6, 1.0), (0.6, 1.0), (0.8, 1.0)), 2) @@ -39,16 +46,17 @@ class BinaryClassificationMetricsSuite extends FunSuite with LocalSparkContext { val rocCurve = Seq((0.0, 0.0)) ++ fpr.zip(recall) ++ Seq((1.0, 1.0)) val pr = recall.zip(precision) val prCurve = Seq((0.0, 1.0)) ++ pr - val f1 = pr.map { case (r, p) => 2.0 * (p * r) / (p + r) } + val f1 = pr.map { case (r, p) => 2.0 * (p * r) / (p + r)} val f2 = pr.map { case (r, p) => 5.0 * (p * r) / (4.0 * p + r)} - assert(metrics.thresholds().collect().toSeq === threshold) - assert(metrics.roc().collect().toSeq === rocCurve) - assert(metrics.areaUnderROC() === AreaUnderCurve.of(rocCurve)) - assert(metrics.pr().collect().toSeq === prCurve) - assert(metrics.areaUnderPR() === AreaUnderCurve.of(prCurve)) - assert(metrics.fMeasureByThreshold().collect().toSeq === threshold.zip(f1)) - assert(metrics.fMeasureByThreshold(2.0).collect().toSeq === threshold.zip(f2)) - assert(metrics.precisionByThreshold().collect().toSeq === threshold.zip(precision)) - assert(metrics.recallByThreshold().collect().toSeq === threshold.zip(recall)) + + assert(metrics.thresholds().collect().zip(threshold).forall(cond1)) + assert(metrics.roc().collect().zip(rocCurve).forall(cond2)) + assert(metrics.areaUnderROC() ~== AreaUnderCurve.of(rocCurve) absTol 1E-5) + assert(metrics.pr().collect().zip(prCurve).forall(cond2)) + assert(metrics.areaUnderPR() ~== AreaUnderCurve.of(prCurve) absTol 1E-5) + assert(metrics.fMeasureByThreshold().collect().zip(threshold.zip(f1)).forall(cond2)) + assert(metrics.fMeasureByThreshold(2.0).collect().zip(threshold.zip(f2)).forall(cond2)) + assert(metrics.precisionByThreshold().collect().zip(threshold.zip(precision)).forall(cond2)) + assert(metrics.recallByThreshold().collect().zip(threshold.zip(recall)).forall(cond2)) } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MulticlassMetricsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MulticlassMetricsSuite.scala new file mode 100644 index 0000000000000..1ea503971c864 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MulticlassMetricsSuite.scala @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.evaluation + +import org.scalatest.FunSuite + +import org.apache.spark.mllib.linalg.Matrices +import org.apache.spark.mllib.util.LocalSparkContext + +class MulticlassMetricsSuite extends FunSuite with LocalSparkContext { + test("Multiclass evaluation metrics") { + /* + * Confusion matrix for 3-class classification with total 9 instances: + * |2|1|1| true class0 (4 instances) + * |1|3|0| true class1 (4 instances) + * |0|0|1| true class2 (1 instance) + */ + val confusionMatrix = Matrices.dense(3, 3, Array(2, 1, 0, 1, 3, 0, 1, 0, 1)) + val labels = Array(0.0, 1.0, 2.0) + val predictionAndLabels = sc.parallelize( + Seq((0.0, 0.0), (0.0, 1.0), (0.0, 0.0), (1.0, 0.0), (1.0, 1.0), + (1.0, 1.0), (1.0, 1.0), (2.0, 2.0), (2.0, 0.0)), 2) + val metrics = new MulticlassMetrics(predictionAndLabels) + val delta = 0.0000001 + val fpRate0 = 1.0 / (9 - 4) + val fpRate1 = 1.0 / (9 - 4) + val fpRate2 = 1.0 / (9 - 1) + val precision0 = 2.0 / (2 + 1) + val precision1 = 3.0 / (3 + 1) + val precision2 = 1.0 / (1 + 1) + val recall0 = 2.0 / (2 + 2) + val recall1 = 3.0 / (3 + 1) + val recall2 = 1.0 / (1 + 0) + val f1measure0 = 2 * precision0 * recall0 / (precision0 + recall0) + val f1measure1 = 2 * precision1 * recall1 / (precision1 + recall1) + val f1measure2 = 2 * precision2 * recall2 / (precision2 + recall2) + val f2measure0 = (1 + 2 * 2) * precision0 * recall0 / (2 * 2 * precision0 + recall0) + val f2measure1 = (1 + 2 * 2) * precision1 * recall1 / (2 * 2 * precision1 + recall1) + val f2measure2 = (1 + 2 * 2) * precision2 * recall2 / (2 * 2 * precision2 + recall2) + + assert(metrics.confusionMatrix.toArray.sameElements(confusionMatrix.toArray)) + assert(math.abs(metrics.falsePositiveRate(0.0) - fpRate0) < delta) + assert(math.abs(metrics.falsePositiveRate(1.0) - fpRate1) < delta) + assert(math.abs(metrics.falsePositiveRate(2.0) - fpRate2) < delta) + assert(math.abs(metrics.precision(0.0) - precision0) < delta) + assert(math.abs(metrics.precision(1.0) - precision1) < delta) + assert(math.abs(metrics.precision(2.0) - precision2) < delta) + assert(math.abs(metrics.recall(0.0) - recall0) < delta) + assert(math.abs(metrics.recall(1.0) - recall1) < delta) + assert(math.abs(metrics.recall(2.0) - recall2) < delta) + assert(math.abs(metrics.fMeasure(0.0) - f1measure0) < delta) + assert(math.abs(metrics.fMeasure(1.0) - f1measure1) < delta) + assert(math.abs(metrics.fMeasure(2.0) - f1measure2) < delta) + assert(math.abs(metrics.fMeasure(0.0, 2.0) - f2measure0) < delta) + assert(math.abs(metrics.fMeasure(1.0, 2.0) - f2measure1) < delta) + assert(math.abs(metrics.fMeasure(2.0, 2.0) - f2measure2) < delta) + + assert(math.abs(metrics.recall - + (2.0 + 3.0 + 1.0) / ((2 + 3 + 1) + (1 + 1 + 1))) < delta) + assert(math.abs(metrics.recall - metrics.precision) < delta) + assert(math.abs(metrics.recall - metrics.fMeasure) < delta) + assert(math.abs(metrics.recall - metrics.weightedRecall) < delta) + assert(math.abs(metrics.weightedFalsePositiveRate - + ((4.0 / 9) * fpRate0 + (4.0 / 9) * fpRate1 + (1.0 / 9) * fpRate2)) < delta) + assert(math.abs(metrics.weightedPrecision - + ((4.0 / 9) * precision0 + (4.0 / 9) * precision1 + (1.0 / 9) * precision2)) < delta) + assert(math.abs(metrics.weightedRecall - + ((4.0 / 9) * recall0 + (4.0 / 9) * recall1 + (1.0 / 9) * recall2)) < delta) + assert(math.abs(metrics.weightedFMeasure - + ((4.0 / 9) * f1measure0 + (4.0 / 9) * f1measure1 + (1.0 / 9) * f1measure2)) < delta) + assert(math.abs(metrics.weightedFMeasure(2.0) - + ((4.0 / 9) * f2measure0 + (4.0 / 9) * f2measure1 + (1.0 / 9) * f2measure2)) < delta) + assert(metrics.labels.sameElements(labels)) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/feature/HashingTFSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/feature/HashingTFSuite.scala new file mode 100644 index 0000000000000..a599e0d938569 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/feature/HashingTFSuite.scala @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.feature + +import org.scalatest.FunSuite + +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.util.LocalSparkContext + +class HashingTFSuite extends FunSuite with LocalSparkContext { + + test("hashing tf on a single doc") { + val hashingTF = new HashingTF(1000) + val doc = "a a b b c d".split(" ") + val n = hashingTF.numFeatures + val termFreqs = Seq( + (hashingTF.indexOf("a"), 2.0), + (hashingTF.indexOf("b"), 2.0), + (hashingTF.indexOf("c"), 1.0), + (hashingTF.indexOf("d"), 1.0)) + assert(termFreqs.map(_._1).forall(i => i >= 0 && i < n), + "index must be in range [0, #features)") + assert(termFreqs.map(_._1).toSet.size === 4, "expecting perfect hashing") + val expected = Vectors.sparse(n, termFreqs) + assert(hashingTF.transform(doc) === expected) + } + + test("hashing tf on an RDD") { + val hashingTF = new HashingTF + val localDocs: Seq[Seq[String]] = Seq( + "a a b b b c d".split(" "), + "a b c d a b c".split(" "), + "c b a c b a a".split(" ")) + val docs = sc.parallelize(localDocs, 2) + assert(hashingTF.transform(docs).collect().toSet === localDocs.map(hashingTF.transform).toSet) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/feature/IDFSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/feature/IDFSuite.scala new file mode 100644 index 0000000000000..53d9c0c640b98 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/feature/IDFSuite.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.mllib.feature + +import org.scalatest.FunSuite + +import org.apache.spark.SparkContext._ +import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vectors} +import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.mllib.util.TestingUtils._ + +class IDFSuite extends FunSuite with LocalSparkContext { + + test("idf") { + val n = 4 + val localTermFrequencies = Seq( + Vectors.sparse(n, Array(1, 3), Array(1.0, 2.0)), + Vectors.dense(0.0, 1.0, 2.0, 3.0), + Vectors.sparse(n, Array(1), Array(1.0)) + ) + val m = localTermFrequencies.size + val termFrequencies = sc.parallelize(localTermFrequencies, 2) + val idf = new IDF + val model = idf.fit(termFrequencies) + val expected = Vectors.dense(Array(0, 3, 1, 2).map { x => + math.log((m.toDouble + 1.0) / (x + 1.0)) + }) + assert(model.idf ~== expected absTol 1e-12) + val tfidf = model.transform(termFrequencies).cache().zipWithIndex().map(_.swap).collectAsMap() + assert(tfidf.size === 3) + val tfidf0 = tfidf(0L).asInstanceOf[SparseVector] + assert(tfidf0.indices === Array(1, 3)) + assert(Vectors.dense(tfidf0.values) ~== + Vectors.dense(1.0 * expected(1), 2.0 * expected(3)) absTol 1e-12) + val tfidf1 = tfidf(1L).asInstanceOf[DenseVector] + assert(Vectors.dense(tfidf1.values) ~== + Vectors.dense(0.0, 1.0 * expected(1), 2.0 * expected(2), 3.0 * expected(3)) absTol 1e-12) + val tfidf2 = tfidf(2L).asInstanceOf[SparseVector] + assert(tfidf2.indices === Array(1)) + assert(tfidf2.values(0) ~== (1.0 * expected(1)) absTol 1e-12) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/feature/NormalizerSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/feature/NormalizerSuite.scala new file mode 100644 index 0000000000000..fb76dccfdf79e --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/feature/NormalizerSuite.scala @@ -0,0 +1,120 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.feature + +import org.scalatest.FunSuite + +import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vectors} +import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.mllib.util.TestingUtils._ + +class NormalizerSuite extends FunSuite with LocalSparkContext { + + val data = Array( + Vectors.sparse(3, Seq((0, -2.0), (1, 2.3))), + Vectors.dense(0.0, 0.0, 0.0), + Vectors.dense(0.6, -1.1, -3.0), + Vectors.sparse(3, Seq((1, 0.91), (2, 3.2))), + Vectors.sparse(3, Seq((0, 5.7), (1, 0.72), (2, 2.7))), + Vectors.sparse(3, Seq()) + ) + + lazy val dataRDD = sc.parallelize(data, 3) + + test("Normalization using L1 distance") { + val l1Normalizer = new Normalizer(1) + + val data1 = data.map(l1Normalizer.transform) + val data1RDD = l1Normalizer.transform(dataRDD) + + assert((data, data1, data1RDD.collect()).zipped.forall { + case (v1: DenseVector, v2: DenseVector, v3: DenseVector) => true + case (v1: SparseVector, v2: SparseVector, v3: SparseVector) => true + case _ => false + }, "The vector type should be preserved after normalization.") + + assert((data1, data1RDD.collect()).zipped.forall((v1, v2) => v1 ~== v2 absTol 1E-5)) + + assert(data1(0).toBreeze.norm(1) ~== 1.0 absTol 1E-5) + assert(data1(2).toBreeze.norm(1) ~== 1.0 absTol 1E-5) + assert(data1(3).toBreeze.norm(1) ~== 1.0 absTol 1E-5) + assert(data1(4).toBreeze.norm(1) ~== 1.0 absTol 1E-5) + + assert(data1(0) ~== Vectors.sparse(3, Seq((0, -0.465116279), (1, 0.53488372))) absTol 1E-5) + assert(data1(1) ~== Vectors.dense(0.0, 0.0, 0.0) absTol 1E-5) + assert(data1(2) ~== Vectors.dense(0.12765957, -0.23404255, -0.63829787) absTol 1E-5) + assert(data1(3) ~== Vectors.sparse(3, Seq((1, 0.22141119), (2, 0.7785888))) absTol 1E-5) + assert(data1(4) ~== Vectors.dense(0.625, 0.07894737, 0.29605263) absTol 1E-5) + assert(data1(5) ~== Vectors.sparse(3, Seq()) absTol 1E-5) + } + + test("Normalization using L2 distance") { + val l2Normalizer = new Normalizer() + + val data2 = data.map(l2Normalizer.transform) + val data2RDD = l2Normalizer.transform(dataRDD) + + assert((data, data2, data2RDD.collect()).zipped.forall { + case (v1: DenseVector, v2: DenseVector, v3: DenseVector) => true + case (v1: SparseVector, v2: SparseVector, v3: SparseVector) => true + case _ => false + }, "The vector type should be preserved after normalization.") + + assert((data2, data2RDD.collect()).zipped.forall((v1, v2) => v1 ~== v2 absTol 1E-5)) + + assert(data2(0).toBreeze.norm(2) ~== 1.0 absTol 1E-5) + assert(data2(2).toBreeze.norm(2) ~== 1.0 absTol 1E-5) + assert(data2(3).toBreeze.norm(2) ~== 1.0 absTol 1E-5) + assert(data2(4).toBreeze.norm(2) ~== 1.0 absTol 1E-5) + + assert(data2(0) ~== Vectors.sparse(3, Seq((0, -0.65617871), (1, 0.75460552))) absTol 1E-5) + assert(data2(1) ~== Vectors.dense(0.0, 0.0, 0.0) absTol 1E-5) + assert(data2(2) ~== Vectors.dense(0.184549876, -0.3383414, -0.922749378) absTol 1E-5) + assert(data2(3) ~== Vectors.sparse(3, Seq((1, 0.27352993), (2, 0.96186349))) absTol 1E-5) + assert(data2(4) ~== Vectors.dense(0.897906166, 0.113419726, 0.42532397) absTol 1E-5) + assert(data2(5) ~== Vectors.sparse(3, Seq()) absTol 1E-5) + } + + test("Normalization using L^Inf distance.") { + val lInfNormalizer = new Normalizer(Double.PositiveInfinity) + + val dataInf = data.map(lInfNormalizer.transform) + val dataInfRDD = lInfNormalizer.transform(dataRDD) + + assert((data, dataInf, dataInfRDD.collect()).zipped.forall { + case (v1: DenseVector, v2: DenseVector, v3: DenseVector) => true + case (v1: SparseVector, v2: SparseVector, v3: SparseVector) => true + case _ => false + }, "The vector type should be preserved after normalization.") + + assert((dataInf, dataInfRDD.collect()).zipped.forall((v1, v2) => v1 ~== v2 absTol 1E-5)) + + assert(dataInf(0).toArray.map(Math.abs).max ~== 1.0 absTol 1E-5) + assert(dataInf(2).toArray.map(Math.abs).max ~== 1.0 absTol 1E-5) + assert(dataInf(3).toArray.map(Math.abs).max ~== 1.0 absTol 1E-5) + assert(dataInf(4).toArray.map(Math.abs).max ~== 1.0 absTol 1E-5) + + assert(dataInf(0) ~== Vectors.sparse(3, Seq((0, -0.86956522), (1, 1.0))) absTol 1E-5) + assert(dataInf(1) ~== Vectors.dense(0.0, 0.0, 0.0) absTol 1E-5) + assert(dataInf(2) ~== Vectors.dense(0.2, -0.36666667, -1.0) absTol 1E-5) + assert(dataInf(3) ~== Vectors.sparse(3, Seq((1, 0.284375), (2, 1.0))) absTol 1E-5) + assert(dataInf(4) ~== Vectors.dense(1.0, 0.12631579, 0.473684211) absTol 1E-5) + assert(dataInf(5) ~== Vectors.sparse(3, Seq()) absTol 1E-5) + } + +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/feature/StandardScalerSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/feature/StandardScalerSuite.scala new file mode 100644 index 0000000000000..e217b93cebbdb --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/feature/StandardScalerSuite.scala @@ -0,0 +1,194 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.feature + +import org.scalatest.FunSuite + +import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors} +import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.mllib.util.TestingUtils._ +import org.apache.spark.mllib.rdd.RDDFunctions._ +import org.apache.spark.mllib.stat.{MultivariateStatisticalSummary, MultivariateOnlineSummarizer} +import org.apache.spark.rdd.RDD + +class StandardScalerSuite extends FunSuite with LocalSparkContext { + + private def computeSummary(data: RDD[Vector]): MultivariateStatisticalSummary = { + data.treeAggregate(new MultivariateOnlineSummarizer)( + (aggregator, data) => aggregator.add(data), + (aggregator1, aggregator2) => aggregator1.merge(aggregator2)) + } + + test("Standardization with dense input") { + val data = Array( + Vectors.dense(-2.0, 2.3, 0), + Vectors.dense(0.0, -1.0, -3.0), + Vectors.dense(0.0, -5.1, 0.0), + Vectors.dense(3.8, 0.0, 1.9), + Vectors.dense(1.7, -0.6, 0.0), + Vectors.dense(0.0, 1.9, 0.0) + ) + + val dataRDD = sc.parallelize(data, 3) + + val standardizer1 = new StandardScaler(withMean = true, withStd = true) + val standardizer2 = new StandardScaler() + val standardizer3 = new StandardScaler(withMean = true, withStd = false) + + val model1 = standardizer1.fit(dataRDD) + val model2 = standardizer2.fit(dataRDD) + val model3 = standardizer3.fit(dataRDD) + + val data1 = data.map(model1.transform) + val data2 = data.map(model2.transform) + val data3 = data.map(model3.transform) + + val data1RDD = model1.transform(dataRDD) + val data2RDD = model2.transform(dataRDD) + val data3RDD = model3.transform(dataRDD) + + val summary = computeSummary(dataRDD) + val summary1 = computeSummary(data1RDD) + val summary2 = computeSummary(data2RDD) + val summary3 = computeSummary(data3RDD) + + assert((data, data1, data1RDD.collect()).zipped.forall { + case (v1: DenseVector, v2: DenseVector, v3: DenseVector) => true + case (v1: SparseVector, v2: SparseVector, v3: SparseVector) => true + case _ => false + }, "The vector type should be preserved after standardization.") + + assert((data, data2, data2RDD.collect()).zipped.forall { + case (v1: DenseVector, v2: DenseVector, v3: DenseVector) => true + case (v1: SparseVector, v2: SparseVector, v3: SparseVector) => true + case _ => false + }, "The vector type should be preserved after standardization.") + + assert((data, data3, data3RDD.collect()).zipped.forall { + case (v1: DenseVector, v2: DenseVector, v3: DenseVector) => true + case (v1: SparseVector, v2: SparseVector, v3: SparseVector) => true + case _ => false + }, "The vector type should be preserved after standardization.") + + assert((data1, data1RDD.collect()).zipped.forall((v1, v2) => v1 ~== v2 absTol 1E-5)) + assert((data2, data2RDD.collect()).zipped.forall((v1, v2) => v1 ~== v2 absTol 1E-5)) + assert((data3, data3RDD.collect()).zipped.forall((v1, v2) => v1 ~== v2 absTol 1E-5)) + + assert(summary1.mean ~== Vectors.dense(0.0, 0.0, 0.0) absTol 1E-5) + assert(summary1.variance ~== Vectors.dense(1.0, 1.0, 1.0) absTol 1E-5) + + assert(summary2.mean !~== Vectors.dense(0.0, 0.0, 0.0) absTol 1E-5) + assert(summary2.variance ~== Vectors.dense(1.0, 1.0, 1.0) absTol 1E-5) + + assert(summary3.mean ~== Vectors.dense(0.0, 0.0, 0.0) absTol 1E-5) + assert(summary3.variance ~== summary.variance absTol 1E-5) + + assert(data1(0) ~== Vectors.dense(-1.31527964, 1.023470449, 0.11637768424) absTol 1E-5) + assert(data1(3) ~== Vectors.dense(1.637735298, 0.156973995, 1.32247368462) absTol 1E-5) + assert(data2(4) ~== Vectors.dense(0.865538862, -0.22604255, 0.0) absTol 1E-5) + assert(data2(5) ~== Vectors.dense(0.0, 0.71580142, 0.0) absTol 1E-5) + assert(data3(1) ~== Vectors.dense(-0.58333333, -0.58333333, -2.8166666666) absTol 1E-5) + assert(data3(5) ~== Vectors.dense(-0.58333333, 2.316666666, 0.18333333333) absTol 1E-5) + } + + + test("Standardization with sparse input") { + val data = Array( + Vectors.sparse(3, Seq((0, -2.0), (1, 2.3))), + Vectors.sparse(3, Seq((1, -1.0), (2, -3.0))), + Vectors.sparse(3, Seq((1, -5.1))), + Vectors.sparse(3, Seq((0, 3.8), (2, 1.9))), + Vectors.sparse(3, Seq((0, 1.7), (1, -0.6))), + Vectors.sparse(3, Seq((1, 1.9))) + ) + + val dataRDD = sc.parallelize(data, 3) + + val standardizer1 = new StandardScaler(withMean = true, withStd = true) + val standardizer2 = new StandardScaler() + val standardizer3 = new StandardScaler(withMean = true, withStd = false) + + val model1 = standardizer1.fit(dataRDD) + val model2 = standardizer2.fit(dataRDD) + val model3 = standardizer3.fit(dataRDD) + + val data2 = data.map(model2.transform) + + withClue("Standardization with mean can not be applied on sparse input.") { + intercept[IllegalArgumentException] { + data.map(model1.transform) + } + } + + withClue("Standardization with mean can not be applied on sparse input.") { + intercept[IllegalArgumentException] { + data.map(model3.transform) + } + } + + val data2RDD = model2.transform(dataRDD) + + val summary2 = computeSummary(data2RDD) + + assert((data, data2, data2RDD.collect()).zipped.forall { + case (v1: DenseVector, v2: DenseVector, v3: DenseVector) => true + case (v1: SparseVector, v2: SparseVector, v3: SparseVector) => true + case _ => false + }, "The vector type should be preserved after standardization.") + + assert((data2, data2RDD.collect()).zipped.forall((v1, v2) => v1 ~== v2 absTol 1E-5)) + + assert(summary2.mean !~== Vectors.dense(0.0, 0.0, 0.0) absTol 1E-5) + assert(summary2.variance ~== Vectors.dense(1.0, 1.0, 1.0) absTol 1E-5) + + assert(data2(4) ~== Vectors.sparse(3, Seq((0, 0.865538862), (1, -0.22604255))) absTol 1E-5) + assert(data2(5) ~== Vectors.sparse(3, Seq((1, 0.71580142))) absTol 1E-5) + } + + test("Standardization with constant input") { + // When the input data is all constant, the variance is zero. The standardization against + // zero variance is not well-defined, but we decide to just set it into zero here. + val data = Array( + Vectors.dense(2.0), + Vectors.dense(2.0), + Vectors.dense(2.0) + ) + + val dataRDD = sc.parallelize(data, 2) + + val standardizer1 = new StandardScaler(withMean = true, withStd = true) + val standardizer2 = new StandardScaler(withMean = true, withStd = false) + val standardizer3 = new StandardScaler(withMean = false, withStd = true) + + val model1 = standardizer1.fit(dataRDD) + val model2 = standardizer2.fit(dataRDD) + val model3 = standardizer3.fit(dataRDD) + + val data1 = data.map(model1.transform) + val data2 = data.map(model2.transform) + val data3 = data.map(model3.transform) + + assert(data1.forall(_.toArray.forall(_ == 0.0)), + "The variance is zero, so the transformed result should be 0.0") + assert(data2.forall(_.toArray.forall(_ == 0.0)), + "The variance is zero, so the transformed result should be 0.0") + assert(data3.forall(_.toArray.forall(_ == 0.0)), + "The variance is zero, so the transformed result should be 0.0") + } + +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/feature/Word2VecSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/feature/Word2VecSuite.scala new file mode 100644 index 0000000000000..e34335d89eb75 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/feature/Word2VecSuite.scala @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.feature + +import org.scalatest.FunSuite + +import org.apache.spark.mllib.util.LocalSparkContext + +class Word2VecSuite extends FunSuite with LocalSparkContext { + + // TODO: add more tests + + test("Word2Vec") { + val sentence = "a b " * 100 + "a c " * 10 + val localDoc = Seq(sentence, sentence) + val doc = sc.parallelize(localDoc) + .map(line => line.split(" ").toSeq) + val model = new Word2Vec().setVectorSize(10).setSeed(42L).fit(doc) + val syms = model.findSynonyms("a", 2) + assert(syms.length == 2) + assert(syms(0)._1 == "b") + assert(syms(1)._1 == "c") + } + + test("Word2VecModel") { + val num = 2 + val word2VecMap = Map( + ("china", Array(0.50f, 0.50f, 0.50f, 0.50f)), + ("japan", Array(0.40f, 0.50f, 0.50f, 0.50f)), + ("taiwan", Array(0.60f, 0.50f, 0.50f, 0.50f)), + ("korea", Array(0.45f, 0.60f, 0.60f, 0.60f)) + ) + val model = new Word2VecModel(word2VecMap) + val syms = model.findSynonyms("china", num) + assert(syms.length == num) + assert(syms(0)._1 == "taiwan") + assert(syms(1)._1 == "japan") + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/BLASSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/BLASSuite.scala new file mode 100644 index 0000000000000..1952e6734ecf7 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/BLASSuite.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.mllib.linalg + +import org.scalatest.FunSuite + +import org.apache.spark.mllib.util.TestingUtils._ +import org.apache.spark.mllib.linalg.BLAS._ + +class BLASSuite extends FunSuite { + + test("copy") { + val sx = Vectors.sparse(4, Array(0, 2), Array(1.0, -2.0)) + val dx = Vectors.dense(1.0, 0.0, -2.0, 0.0) + val sy = Vectors.sparse(4, Array(0, 1, 3), Array(2.0, 1.0, 1.0)) + val dy = Array(2.0, 1.0, 0.0, 1.0) + + val dy1 = Vectors.dense(dy.clone()) + copy(sx, dy1) + assert(dy1 ~== dx absTol 1e-15) + + val dy2 = Vectors.dense(dy.clone()) + copy(dx, dy2) + assert(dy2 ~== dx absTol 1e-15) + + intercept[IllegalArgumentException] { + copy(sx, sy) + } + + intercept[IllegalArgumentException] { + copy(dx, sy) + } + + withClue("vector sizes must match") { + intercept[Exception] { + copy(sx, Vectors.dense(0.0, 1.0, 2.0)) + } + } + } + + test("scal") { + val a = 0.1 + val sx = Vectors.sparse(3, Array(0, 2), Array(1.0, -2.0)) + val dx = Vectors.dense(1.0, 0.0, -2.0) + + scal(a, sx) + assert(sx ~== Vectors.sparse(3, Array(0, 2), Array(0.1, -0.2)) absTol 1e-15) + + scal(a, dx) + assert(dx ~== Vectors.dense(0.1, 0.0, -0.2) absTol 1e-15) + } + + test("axpy") { + val alpha = 0.1 + val sx = Vectors.sparse(3, Array(0, 2), Array(1.0, -2.0)) + val dx = Vectors.dense(1.0, 0.0, -2.0) + val dy = Array(2.0, 1.0, 0.0) + val expected = Vectors.dense(2.1, 1.0, -0.2) + + val dy1 = Vectors.dense(dy.clone()) + axpy(alpha, sx, dy1) + assert(dy1 ~== expected absTol 1e-15) + + val dy2 = Vectors.dense(dy.clone()) + axpy(alpha, dx, dy2) + assert(dy2 ~== expected absTol 1e-15) + + val sy = Vectors.sparse(4, Array(0, 1), Array(2.0, 1.0)) + + intercept[IllegalArgumentException] { + axpy(alpha, sx, sy) + } + + intercept[IllegalArgumentException] { + axpy(alpha, dx, sy) + } + + withClue("vector sizes must match") { + intercept[Exception] { + axpy(alpha, sx, Vectors.dense(1.0, 2.0)) + } + } + } + + test("dot") { + val sx = Vectors.sparse(3, Array(0, 2), Array(1.0, -2.0)) + val dx = Vectors.dense(1.0, 0.0, -2.0) + val sy = Vectors.sparse(3, Array(0, 1), Array(2.0, 1.0)) + val dy = Vectors.dense(2.0, 1.0, 0.0) + + assert(dot(sx, sy) ~== 2.0 absTol 1e-15) + assert(dot(sy, sx) ~== 2.0 absTol 1e-15) + assert(dot(sx, dy) ~== 2.0 absTol 1e-15) + assert(dot(dy, sx) ~== 2.0 absTol 1e-15) + assert(dot(dx, dy) ~== 2.0 absTol 1e-15) + assert(dot(dy, dx) ~== 2.0 absTol 1e-15) + + assert(dot(sx, sx) ~== 5.0 absTol 1e-15) + assert(dot(dx, dx) ~== 5.0 absTol 1e-15) + assert(dot(sx, dx) ~== 5.0 absTol 1e-15) + assert(dot(dx, sx) ~== 5.0 absTol 1e-15) + + val sx1 = Vectors.sparse(10, Array(0, 3, 5, 7, 8), Array(1.0, 2.0, 3.0, 4.0, 5.0)) + val sx2 = Vectors.sparse(10, Array(1, 3, 6, 7, 9), Array(1.0, 2.0, 3.0, 4.0, 5.0)) + assert(dot(sx1, sx2) ~== 20.0 absTol 1e-15) + assert(dot(sx2, sx1) ~== 20.0 absTol 1e-15) + + withClue("vector sizes must match") { + intercept[Exception] { + dot(sx, Vectors.dense(2.0, 1.0)) + } + } + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala index 7972ceea1fe8a..cd651fe2d2ddf 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala @@ -125,4 +125,34 @@ class VectorsSuite extends FunSuite { } } } + + test("zeros") { + assert(Vectors.zeros(3) === Vectors.dense(0.0, 0.0, 0.0)) + } + + test("Vector.copy") { + val sv = Vectors.sparse(4, Array(0, 2), Array(1.0, 2.0)) + val svCopy = sv.copy + (sv, svCopy) match { + case (sv: SparseVector, svCopy: SparseVector) => + assert(sv.size === svCopy.size) + assert(sv.indices === svCopy.indices) + assert(sv.values === svCopy.values) + assert(!sv.indices.eq(svCopy.indices)) + assert(!sv.values.eq(svCopy.values)) + case _ => + throw new RuntimeException(s"copy returned ${svCopy.getClass} on ${sv.getClass}.") + } + + val dv = Vectors.dense(1.0, 0.0, 2.0) + val dvCopy = dv.copy + (dv, dvCopy) match { + case (dv: DenseVector, dvCopy: DenseVector) => + assert(dv.size === dvCopy.size) + assert(dv.values === dvCopy.values) + assert(!dv.values.eq(dvCopy.values)) + case _ => + throw new RuntimeException(s"copy returned ${dvCopy.getClass} on ${dv.getClass}.") + } + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala index c9f9acf4c1335..1d3a3221365cc 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala @@ -17,12 +17,13 @@ package org.apache.spark.mllib.linalg.distributed -import org.scalatest.FunSuite +import scala.util.Random import breeze.linalg.{DenseVector => BDV, DenseMatrix => BDM, norm => brzNorm, svd => brzSvd} +import org.scalatest.FunSuite -import org.apache.spark.mllib.util.LocalSparkContext import org.apache.spark.mllib.linalg.{Matrices, Vectors, Vector} +import org.apache.spark.mllib.util.{LocalClusterSparkContext, LocalSparkContext} class RowMatrixSuite extends FunSuite with LocalSparkContext { @@ -96,37 +97,44 @@ class RowMatrixSuite extends FunSuite with LocalSparkContext { test("svd of a full-rank matrix") { for (mat <- Seq(denseMat, sparseMat)) { - val localMat = mat.toBreeze() - val (localU, localSigma, localVt) = brzSvd(localMat) - val localV: BDM[Double] = localVt.t.toDenseMatrix - for (k <- 1 to n) { - val svd = mat.computeSVD(k, computeU = true) - val U = svd.U - val s = svd.s - val V = svd.V - assert(U.numRows() === m) - assert(U.numCols() === k) - assert(s.size === k) - assert(V.numRows === n) - assert(V.numCols === k) - assertColumnEqualUpToSign(U.toBreeze(), localU, k) - assertColumnEqualUpToSign(V.toBreeze.asInstanceOf[BDM[Double]], localV, k) - assert(closeToZero(s.toBreeze.asInstanceOf[BDV[Double]] - localSigma(0 until k))) + for (mode <- Seq("auto", "local-svd", "local-eigs", "dist-eigs")) { + val localMat = mat.toBreeze() + val brzSvd.SVD(localU, localSigma, localVt) = brzSvd(localMat) + val localV: BDM[Double] = localVt.t.toDenseMatrix + for (k <- 1 to n) { + val skip = (mode == "local-eigs" || mode == "dist-eigs") && k == n + if (!skip) { + val svd = mat.computeSVD(k, computeU = true, 1e-9, 300, 1e-10, mode) + val U = svd.U + val s = svd.s + val V = svd.V + assert(U.numRows() === m) + assert(U.numCols() === k) + assert(s.size === k) + assert(V.numRows === n) + assert(V.numCols === k) + assertColumnEqualUpToSign(U.toBreeze(), localU, k) + assertColumnEqualUpToSign(V.toBreeze.asInstanceOf[BDM[Double]], localV, k) + assert(closeToZero(s.toBreeze.asInstanceOf[BDV[Double]] - localSigma(0 until k))) + } + } + val svdWithoutU = mat.computeSVD(1, computeU = false, 1e-9, 300, 1e-10, mode) + assert(svdWithoutU.U === null) } - val svdWithoutU = mat.computeSVD(n) - assert(svdWithoutU.U === null) } } test("svd of a low-rank matrix") { - val rows = sc.parallelize(Array.fill(4)(Vectors.dense(1.0, 1.0)), 2) - val mat = new RowMatrix(rows, 4, 2) - val svd = mat.computeSVD(2, computeU = true) - assert(svd.s.size === 1, "should not return zero singular values") - assert(svd.U.numRows() === 4) - assert(svd.U.numCols() === 1) - assert(svd.V.numRows === 2) - assert(svd.V.numCols === 1) + val rows = sc.parallelize(Array.fill(4)(Vectors.dense(1.0, 1.0, 1.0)), 2) + val mat = new RowMatrix(rows, 4, 3) + for (mode <- Seq("auto", "local-svd", "local-eigs", "dist-eigs")) { + val svd = mat.computeSVD(2, computeU = true, 1e-6, 300, 1e-10, mode) + assert(svd.s.size === 1, s"should not return zero singular values but got ${svd.s}") + assert(svd.U.numRows() === 4) + assert(svd.U.numCols() === 1) + assert(svd.V.numRows === 3) + assert(svd.V.numCols === 1) + } } def closeToZero(G: BDM[Double]): Boolean = { @@ -186,3 +194,27 @@ class RowMatrixSuite extends FunSuite with LocalSparkContext { } } } + +class RowMatrixClusterSuite extends FunSuite with LocalClusterSparkContext { + + var mat: RowMatrix = _ + + override def beforeAll() { + super.beforeAll() + val m = 4 + val n = 200000 + val rows = sc.parallelize(0 until m, 2).mapPartitionsWithIndex { (idx, iter) => + val random = new Random(idx) + iter.map(i => Vectors.dense(Array.fill(n)(random.nextDouble()))) + } + mat = new RowMatrix(rows) + } + + test("task size should be small in svd") { + val svd = mat.computeSVD(1, computeU = true) + } + + test("task size should be small in summarize") { + val summary = mat.computeColumnSummaryStatistics() + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala index 951b4f7c6e6f4..bf040110e228b 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala @@ -17,15 +17,15 @@ package org.apache.spark.mllib.optimization -import scala.util.Random import scala.collection.JavaConversions._ +import scala.util.Random -import org.scalatest.FunSuite -import org.scalatest.Matchers +import org.scalatest.{FunSuite, Matchers} -import org.apache.spark.mllib.regression._ -import org.apache.spark.mllib.util.LocalSparkContext import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.regression._ +import org.apache.spark.mllib.util.{LocalClusterSparkContext, LocalSparkContext} +import org.apache.spark.mllib.util.TestingUtils._ object GradientDescentSuite { @@ -46,7 +46,7 @@ object GradientDescentSuite { val rnd = new Random(seed) val x1 = Array.fill[Double](nPoints)(rnd.nextGaussian()) - val unifRand = new scala.util.Random(45) + val unifRand = new Random(45) val rLogis = (0 until nPoints).map { i => val u = unifRand.nextDouble() math.log(u) - math.log(1.0-u) @@ -127,20 +127,38 @@ class GradientDescentSuite extends FunSuite with LocalSparkContext with Matchers val (newWeights1, loss1) = GradientDescent.runMiniBatchSGD( dataRDD, gradient, updater, 1, 1, regParam1, 1.0, initialWeightsWithIntercept) - def compareDouble(x: Double, y: Double, tol: Double = 1E-3): Boolean = { - math.abs(x - y) / (math.abs(y) + 1e-15) < tol - } - - assert(compareDouble( - loss1(0), - loss0(0) + (math.pow(initialWeightsWithIntercept(0), 2) + - math.pow(initialWeightsWithIntercept(1), 2)) / 2), + assert( + loss1(0) ~= (loss0(0) + (math.pow(initialWeightsWithIntercept(0), 2) + + math.pow(initialWeightsWithIntercept(1), 2)) / 2) absTol 1E-5, """For non-zero weights, the regVal should be \frac{1}{2}\sum_i w_i^2.""") assert( - compareDouble(newWeights1(0) , newWeights0(0) - initialWeightsWithIntercept(0)) && - compareDouble(newWeights1(1) , newWeights0(1) - initialWeightsWithIntercept(1)), + (newWeights1(0) ~= (newWeights0(0) - initialWeightsWithIntercept(0)) absTol 1E-5) && + (newWeights1(1) ~= (newWeights0(1) - initialWeightsWithIntercept(1)) absTol 1E-5), "The different between newWeights with/without regularization " + "should be initialWeightsWithIntercept.") } } + +class GradientDescentClusterSuite extends FunSuite with LocalClusterSparkContext { + + test("task size should be small") { + val m = 4 + val n = 200000 + val points = sc.parallelize(0 until m, 2).mapPartitionsWithIndex { (idx, iter) => + val random = new Random(idx) + iter.map(i => (1.0, Vectors.dense(Array.fill(n)(random.nextDouble())))) + }.cache() + // If we serialize data directly in the task closure, the size of the serialized task would be + // greater than 1MB and hence Spark would throw an error. + val (weights, loss) = GradientDescent.runMiniBatchSGD( + points, + new LogisticGradient, + new SquaredL2Updater, + 0.1, + 2, + 1.0, + 1.0, + Vectors.dense(new Array[Double](n))) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala index fe7a9033cd5f4..ccba004baa007 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala @@ -17,12 +17,14 @@ package org.apache.spark.mllib.optimization -import org.scalatest.FunSuite -import org.scalatest.Matchers +import scala.util.Random + +import org.scalatest.{FunSuite, Matchers} -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.util.{LocalClusterSparkContext, LocalSparkContext} +import org.apache.spark.mllib.util.TestingUtils._ class LBFGSSuite extends FunSuite with LocalSparkContext with Matchers { @@ -48,16 +50,12 @@ class LBFGSSuite extends FunSuite with LocalSparkContext with Matchers { lazy val dataRDD = sc.parallelize(data, 2).cache() - def compareDouble(x: Double, y: Double, tol: Double = 1E-3): Boolean = { - math.abs(x - y) / (math.abs(y) + 1e-15) < tol - } - test("LBFGS loss should be decreasing and match the result of Gradient Descent.") { val regParam = 0 val initialWeightsWithIntercept = Vectors.dense(1.0 +: initialWeights.toArray) val convergenceTol = 1e-12 - val maxNumIterations = 10 + val numIterations = 10 val (_, loss) = LBFGS.runLBFGS( dataRDD, @@ -65,7 +63,7 @@ class LBFGSSuite extends FunSuite with LocalSparkContext with Matchers { simpleUpdater, numCorrections, convergenceTol, - maxNumIterations, + numIterations, regParam, initialWeightsWithIntercept) @@ -101,7 +99,7 @@ class LBFGSSuite extends FunSuite with LocalSparkContext with Matchers { // Prepare another non-zero weights to compare the loss in the first iteration. val initialWeightsWithIntercept = Vectors.dense(0.3, 0.12) val convergenceTol = 1e-12 - val maxNumIterations = 10 + val numIterations = 10 val (weightLBFGS, lossLBFGS) = LBFGS.runLBFGS( dataRDD, @@ -109,7 +107,7 @@ class LBFGSSuite extends FunSuite with LocalSparkContext with Matchers { squaredL2Updater, numCorrections, convergenceTol, - maxNumIterations, + numIterations, regParam, initialWeightsWithIntercept) @@ -125,15 +123,15 @@ class LBFGSSuite extends FunSuite with LocalSparkContext with Matchers { miniBatchFrac, initialWeightsWithIntercept) - assert(compareDouble(lossGD(0), lossLBFGS(0)), + assert(lossGD(0) ~= lossLBFGS(0) absTol 1E-5, "The first losses of LBFGS and GD should be the same.") // The 2% difference here is based on observation, but is not theoretically guaranteed. - assert(compareDouble(lossGD.last, lossLBFGS.last, 0.02), + assert(lossGD.last ~= lossLBFGS.last relTol 0.02, "The last losses of LBFGS and GD should be within 2% difference.") - assert(compareDouble(weightLBFGS(0), weightGD(0), 0.02) && - compareDouble(weightLBFGS(1), weightGD(1), 0.02), + assert( + (weightLBFGS(0) ~= weightGD(0) relTol 0.02) && (weightLBFGS(1) ~= weightGD(1) relTol 0.02), "The weight differences between LBFGS and GD should be within 2%.") } @@ -142,10 +140,10 @@ class LBFGSSuite extends FunSuite with LocalSparkContext with Matchers { /** * For the first run, we set the convergenceTol to 0.0, so that the algorithm will - * run up to the maxNumIterations which is 8 here. + * run up to the numIterations which is 8 here. */ val initialWeightsWithIntercept = Vectors.dense(0.0, 0.0) - val maxNumIterations = 8 + val numIterations = 8 var convergenceTol = 0.0 val (_, lossLBFGS1) = LBFGS.runLBFGS( @@ -154,7 +152,7 @@ class LBFGSSuite extends FunSuite with LocalSparkContext with Matchers { squaredL2Updater, numCorrections, convergenceTol, - maxNumIterations, + numIterations, regParam, initialWeightsWithIntercept) @@ -169,7 +167,7 @@ class LBFGSSuite extends FunSuite with LocalSparkContext with Matchers { squaredL2Updater, numCorrections, convergenceTol, - maxNumIterations, + numIterations, regParam, initialWeightsWithIntercept) @@ -184,7 +182,7 @@ class LBFGSSuite extends FunSuite with LocalSparkContext with Matchers { squaredL2Updater, numCorrections, convergenceTol, - maxNumIterations, + numIterations, regParam, initialWeightsWithIntercept) @@ -202,12 +200,12 @@ class LBFGSSuite extends FunSuite with LocalSparkContext with Matchers { // Prepare another non-zero weights to compare the loss in the first iteration. val initialWeightsWithIntercept = Vectors.dense(0.3, 0.12) val convergenceTol = 1e-12 - val maxNumIterations = 10 + val numIterations = 10 val lbfgsOptimizer = new LBFGS(gradient, squaredL2Updater) .setNumCorrections(numCorrections) .setConvergenceTol(convergenceTol) - .setMaxNumIterations(maxNumIterations) + .setNumIterations(numIterations) .setRegParam(regParam) val weightLBFGS = lbfgsOptimizer.optimize(dataRDD, initialWeightsWithIntercept) @@ -225,8 +223,29 @@ class LBFGSSuite extends FunSuite with LocalSparkContext with Matchers { initialWeightsWithIntercept) // for class LBFGS and the optimize method, we only look at the weights - assert(compareDouble(weightLBFGS(0), weightGD(0), 0.02) && - compareDouble(weightLBFGS(1), weightGD(1), 0.02), + assert( + (weightLBFGS(0) ~= weightGD(0) relTol 0.02) && (weightLBFGS(1) ~= weightGD(1) relTol 0.02), "The weight differences between LBFGS and GD should be within 2%.") } } + +class LBFGSClusterSuite extends FunSuite with LocalClusterSparkContext { + + test("task size should be small") { + val m = 10 + val n = 200000 + val examples = sc.parallelize(0 until m, 2).mapPartitionsWithIndex { (idx, iter) => + val random = new Random(idx) + iter.map(i => (1.0, Vectors.dense(Array.fill(n)(random.nextDouble)))) + }.cache() + val lbfgs = new LBFGS(new LogisticGradient, new SquaredL2Updater) + .setNumCorrections(1) + .setConvergenceTol(1e-12) + .setNumIterations(1) + .setRegParam(1.0) + val random = new Random(0) + // If we serialize data directly in the task closure, the size of the serialized task would be + // greater than 1MB and hence Spark would throw an error. + val weights = lbfgs.optimize(examples, Vectors.dense(Array.fill(n)(random.nextDouble))) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/optimization/NNLSSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/optimization/NNLSSuite.scala index bbf385229081a..b781a6aed9a8c 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/optimization/NNLSSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/optimization/NNLSSuite.scala @@ -21,7 +21,9 @@ import scala.util.Random import org.scalatest.FunSuite -import org.jblas.{DoubleMatrix, SimpleBlas, NativeBlas} +import org.jblas.{DoubleMatrix, SimpleBlas} + +import org.apache.spark.mllib.util.TestingUtils._ class NNLSSuite extends FunSuite { /** Generate an NNLS problem whose optimal solution is the all-ones vector. */ @@ -73,7 +75,7 @@ class NNLSSuite extends FunSuite { val ws = NNLS.createWorkspace(n) val x = NNLS.solve(ata, atb, ws) for (i <- 0 until n) { - assert(Math.abs(x(i) - goodx(i)) < 1e-3) + assert(x(i) ~== goodx(i) absTol 1E-3) assert(x(i) >= 0) } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/random/RandomDataGeneratorSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/random/RandomDataGeneratorSuite.scala new file mode 100644 index 0000000000000..3df7c128af5ab --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/random/RandomDataGeneratorSuite.scala @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.random + +import org.scalatest.FunSuite + +import org.apache.spark.util.StatCounter + +// TODO update tests to use TestingUtils for floating point comparison after PR 1367 is merged +class RandomDataGeneratorSuite extends FunSuite { + + def apiChecks(gen: RandomDataGenerator[Double]) { + + // resetting seed should generate the same sequence of random numbers + gen.setSeed(42L) + val array1 = (0 until 1000).map(_ => gen.nextValue()) + gen.setSeed(42L) + val array2 = (0 until 1000).map(_ => gen.nextValue()) + assert(array1.equals(array2)) + + // newInstance should contain a difference instance of the rng + // i.e. setting difference seeds for difference instances produces different sequences of + // random numbers. + val gen2 = gen.copy() + gen.setSeed(0L) + val array3 = (0 until 1000).map(_ => gen.nextValue()) + gen2.setSeed(1L) + val array4 = (0 until 1000).map(_ => gen2.nextValue()) + // Compare arrays instead of elements since individual elements can coincide by chance but the + // sequences should differ given two different seeds. + assert(!array3.equals(array4)) + + // test that setting the same seed in the copied instance produces the same sequence of numbers + gen.setSeed(0L) + val array5 = (0 until 1000).map(_ => gen.nextValue()) + gen2.setSeed(0L) + val array6 = (0 until 1000).map(_ => gen2.nextValue()) + assert(array5.equals(array6)) + } + + def distributionChecks(gen: RandomDataGenerator[Double], + mean: Double = 0.0, + stddev: Double = 1.0, + epsilon: Double = 0.01) { + for (seed <- 0 until 5) { + gen.setSeed(seed.toLong) + val sample = (0 until 100000).map { _ => gen.nextValue()} + val stats = new StatCounter(sample) + assert(math.abs(stats.mean - mean) < epsilon) + assert(math.abs(stats.stdev - stddev) < epsilon) + } + } + + test("UniformGenerator") { + val uniform = new UniformGenerator() + apiChecks(uniform) + // Stddev of uniform distribution = (ub - lb) / math.sqrt(12) + distributionChecks(uniform, 0.5, 1 / math.sqrt(12)) + } + + test("StandardNormalGenerator") { + val normal = new StandardNormalGenerator() + apiChecks(normal) + distributionChecks(normal, 0.0, 1.0) + } + + test("PoissonGenerator") { + // mean = 0.0 will not pass the API checks since 0.0 is always deterministically produced. + for (mean <- List(1.0, 5.0, 100.0)) { + val poisson = new PoissonGenerator(mean) + apiChecks(poisson) + distributionChecks(poisson, mean, math.sqrt(mean), 0.1) + } + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/random/RandomRDDsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/random/RandomRDDsSuite.scala new file mode 100644 index 0000000000000..c50b78bcbcc61 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/random/RandomRDDsSuite.scala @@ -0,0 +1,160 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.random + +import scala.collection.mutable.ArrayBuffer + +import org.scalatest.FunSuite + +import org.apache.spark.SparkContext._ +import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.rdd.{RandomRDDPartition, RandomRDD} +import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.rdd.RDD +import org.apache.spark.util.StatCounter + +/* + * Note: avoid including APIs that do not set the seed for the RNG in unit tests + * in order to guarantee deterministic behavior. + * + * TODO update tests to use TestingUtils for floating point comparison after PR 1367 is merged + */ +class RandomRDDsSuite extends FunSuite with LocalSparkContext with Serializable { + + def testGeneratedRDD(rdd: RDD[Double], + expectedSize: Long, + expectedNumPartitions: Int, + expectedMean: Double, + expectedStddev: Double, + epsilon: Double = 0.01) { + val stats = rdd.stats() + assert(expectedSize === stats.count) + assert(expectedNumPartitions === rdd.partitions.size) + assert(math.abs(stats.mean - expectedMean) < epsilon) + assert(math.abs(stats.stdev - expectedStddev) < epsilon) + } + + // assume test RDDs are small + def testGeneratedVectorRDD(rdd: RDD[Vector], + expectedRows: Long, + expectedColumns: Int, + expectedNumPartitions: Int, + expectedMean: Double, + expectedStddev: Double, + epsilon: Double = 0.01) { + assert(expectedNumPartitions === rdd.partitions.size) + val values = new ArrayBuffer[Double]() + rdd.collect.foreach { vector => { + assert(vector.size === expectedColumns) + values ++= vector.toArray + }} + assert(expectedRows === values.size / expectedColumns) + val stats = new StatCounter(values) + assert(math.abs(stats.mean - expectedMean) < epsilon) + assert(math.abs(stats.stdev - expectedStddev) < epsilon) + } + + test("RandomRDD sizes") { + + // some cases where size % numParts != 0 to test getPartitions behaves correctly + for ((size, numPartitions) <- List((10000, 6), (12345, 1), (1000, 101))) { + val rdd = new RandomRDD(sc, size, numPartitions, new UniformGenerator, 0L) + assert(rdd.count() === size) + assert(rdd.partitions.size === numPartitions) + + // check that partition sizes are balanced + val partSizes = rdd.partitions.map(p => + p.asInstanceOf[RandomRDDPartition[Double]].size.toDouble) + + val partStats = new StatCounter(partSizes) + assert(partStats.max - partStats.min <= 1) + } + + // size > Int.MaxValue + val size = Int.MaxValue.toLong * 100L + val numPartitions = 101 + val rdd = new RandomRDD(sc, size, numPartitions, new UniformGenerator, 0L) + assert(rdd.partitions.size === numPartitions) + val count = rdd.partitions.foldLeft(0L) { (count, part) => + count + part.asInstanceOf[RandomRDDPartition[Double]].size + } + assert(count === size) + + // size needs to be positive + intercept[IllegalArgumentException] { new RandomRDD(sc, 0, 10, new UniformGenerator, 0L) } + + // numPartitions needs to be positive + intercept[IllegalArgumentException] { new RandomRDD(sc, 100, 0, new UniformGenerator, 0L) } + + // partition size needs to be <= Int.MaxValue + intercept[IllegalArgumentException] { + new RandomRDD(sc, Int.MaxValue.toLong * 100L, 99, new UniformGenerator, 0L) + } + } + + test("randomRDD for different distributions") { + val size = 100000L + val numPartitions = 10 + val poissonMean = 100.0 + + for (seed <- 0 until 5) { + val uniform = RandomRDDs.uniformRDD(sc, size, numPartitions, seed) + testGeneratedRDD(uniform, size, numPartitions, 0.5, 1 / math.sqrt(12)) + + val normal = RandomRDDs.normalRDD(sc, size, numPartitions, seed) + testGeneratedRDD(normal, size, numPartitions, 0.0, 1.0) + + val poisson = RandomRDDs.poissonRDD(sc, poissonMean, size, numPartitions, seed) + testGeneratedRDD(poisson, size, numPartitions, poissonMean, math.sqrt(poissonMean), 0.1) + } + + // mock distribution to check that partitions have unique seeds + val random = RandomRDDs.randomRDD(sc, new MockDistro(), 1000L, 1000, 0L) + assert(random.collect.size === random.collect.distinct.size) + } + + test("randomVectorRDD for different distributions") { + val rows = 1000L + val cols = 100 + val parts = 10 + val poissonMean = 100.0 + + for (seed <- 0 until 5) { + val uniform = RandomRDDs.uniformVectorRDD(sc, rows, cols, parts, seed) + testGeneratedVectorRDD(uniform, rows, cols, parts, 0.5, 1 / math.sqrt(12)) + + val normal = RandomRDDs.normalVectorRDD(sc, rows, cols, parts, seed) + testGeneratedVectorRDD(normal, rows, cols, parts, 0.0, 1.0) + + val poisson = RandomRDDs.poissonVectorRDD(sc, poissonMean, rows, cols, parts, seed) + testGeneratedVectorRDD(poisson, rows, cols, parts, poissonMean, math.sqrt(poissonMean), 0.1) + } + } +} + +private[random] class MockDistro extends RandomDataGenerator[Double] { + + var seed = 0L + + // This allows us to check that each partition has a different seed + override def nextValue(): Double = seed.toDouble + + override def setSeed(seed: Long) = this.seed = seed + + override def copy(): MockDistro = new MockDistro +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/rdd/RDDFunctionsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/rdd/RDDFunctionsSuite.scala index 3f3b10dfff35e..27a19f793242b 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/rdd/RDDFunctionsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/rdd/RDDFunctionsSuite.scala @@ -46,4 +46,22 @@ class RDDFunctionsSuite extends FunSuite with LocalSparkContext { val expected = data.flatMap(x => x).sliding(3).toList assert(sliding.collect().toList === expected) } + + test("treeAggregate") { + val rdd = sc.makeRDD(-1000 until 1000, 10) + def seqOp = (c: Long, x: Int) => c + x + def combOp = (c1: Long, c2: Long) => c1 + c2 + for (depth <- 1 until 10) { + val sum = rdd.treeAggregate(0L)(seqOp, combOp, depth) + assert(sum === -1000L) + } + } + + test("treeReduce") { + val rdd = sc.makeRDD(-1000 until 1000, 10) + for (depth <- 1 until 10) { + val sum = rdd.treeReduce(_ + _, depth) + assert(sum === -1000) + } + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala index 81bebec8c7a39..017c39edb185f 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala @@ -22,11 +22,11 @@ import scala.math.abs import scala.util.Random import org.scalatest.FunSuite - import org.jblas.DoubleMatrix -import org.apache.spark.mllib.util.LocalSparkContext import org.apache.spark.SparkContext._ +import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.mllib.recommendation.ALS.BlockStats object ALSSuite { @@ -67,8 +67,10 @@ object ALSSuite { case true => // Generate raw values from [0,9], or if negativeWeights, from [-2,7] val raw = new DoubleMatrix(users, products, - Array.fill(users * products)((if (negativeWeights) -2 else 0) + rand.nextInt(10).toDouble): _*) - val prefs = new DoubleMatrix(users, products, raw.data.map(v => if (v > 0) 1.0 else 0.0): _*) + Array.fill(users * products)( + (if (negativeWeights) -2 else 0) + rand.nextInt(10).toDouble): _*) + val prefs = + new DoubleMatrix(users, products, raw.data.map(v => if (v > 0) 1.0 else 0.0): _*) (raw, prefs) case false => (userMatrix.mmul(productMatrix), null) } @@ -160,6 +162,22 @@ class ALSSuite extends FunSuite with LocalSparkContext { testALS(100, 200, 2, 15, 0.7, 0.4, false, false, false, -1, -1, false) } + test("analyze one user block and one product block") { + val localRatings = Seq( + Rating(0, 100, 1.0), + Rating(0, 101, 2.0), + Rating(0, 102, 3.0), + Rating(1, 102, 4.0), + Rating(2, 103, 5.0)) + val ratings = sc.makeRDD(localRatings, 2) + val stats = ALS.analyzeBlocks(ratings, 1, 1) + assert(stats.size === 2) + assert(stats(0) === BlockStats("user", 0, 3, 5, 4, 3)) + assert(stats(1) === BlockStats("product", 0, 4, 5, 3, 4)) + } + + // TODO: add tests for analyzing multiple user/product blocks + /** * Test if we can correctly factorize R = U * P where U and P are of known rank. * diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/LabeledPointSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/LabeledPointSuite.scala index d9308aaba6ee1..110c44a7193fd 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/LabeledPointSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/LabeledPointSuite.scala @@ -28,12 +28,12 @@ class LabeledPointSuite extends FunSuite { LabeledPoint(1.0, Vectors.dense(1.0, 0.0)), LabeledPoint(0.0, Vectors.sparse(2, Array(1), Array(-1.0)))) points.foreach { p => - assert(p === LabeledPointParser.parse(p.toString)) + assert(p === LabeledPoint.parse(p.toString)) } } test("parse labeled points with v0.9 format") { - val point = LabeledPointParser.parse("1.0,1.0 0.0 -2.0") + val point = LabeledPoint.parse("1.0,1.0 0.0 -2.0") assert(point === LabeledPoint(1.0, Vectors.dense(1.0, 0.0, -2.0))) } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala index bfa42959c8ead..7aa96421aed87 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala @@ -17,10 +17,13 @@ package org.apache.spark.mllib.regression +import scala.util.Random + import org.scalatest.FunSuite import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.util.{LinearDataGenerator, LocalSparkContext} +import org.apache.spark.mllib.util.{LocalClusterSparkContext, LinearDataGenerator, + LocalSparkContext} class LassoSuite extends FunSuite with LocalSparkContext { @@ -113,3 +116,19 @@ class LassoSuite extends FunSuite with LocalSparkContext { validatePrediction(validationData.map(row => model.predict(row.features)), validationData) } } + +class LassoClusterSuite extends FunSuite with LocalClusterSparkContext { + + test("task size should be small in both training and prediction") { + val m = 4 + val n = 200000 + val points = sc.parallelize(0 until m, 2).mapPartitionsWithIndex { (idx, iter) => + val random = new Random(idx) + iter.map(i => LabeledPoint(1.0, Vectors.dense(Array.fill(n)(random.nextDouble())))) + }.cache() + // If we serialize data directly in the task closure, the size of the serialized task would be + // greater than 1MB and hence Spark would throw an error. + val model = LassoWithSGD.train(points, 2) + val predictions = model.predict(points.map(_.features)) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala index 7aaad7d7a3e39..4f89112b650c5 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala @@ -17,10 +17,13 @@ package org.apache.spark.mllib.regression +import scala.util.Random + import org.scalatest.FunSuite import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.util.{LinearDataGenerator, LocalSparkContext} +import org.apache.spark.mllib.util.{LocalClusterSparkContext, LinearDataGenerator, + LocalSparkContext} class LinearRegressionSuite extends FunSuite with LocalSparkContext { @@ -122,3 +125,19 @@ class LinearRegressionSuite extends FunSuite with LocalSparkContext { sparseValidationData.map(row => model.predict(row.features)), sparseValidationData) } } + +class LinearRegressionClusterSuite extends FunSuite with LocalClusterSparkContext { + + test("task size should be small in both training and prediction") { + val m = 4 + val n = 200000 + val points = sc.parallelize(0 until m, 2).mapPartitionsWithIndex { (idx, iter) => + val random = new Random(idx) + iter.map(i => LabeledPoint(1.0, Vectors.dense(Array.fill(n)(random.nextDouble())))) + }.cache() + // If we serialize data directly in the task closure, the size of the serialized task would be + // greater than 1MB and hence Spark would throw an error. + val model = LinearRegressionWithSGD.train(points, 2) + val predictions = model.predict(points.map(_.features)) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala index 67768e17fbe6d..727bbd051ff15 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala @@ -17,11 +17,14 @@ package org.apache.spark.mllib.regression -import org.scalatest.FunSuite +import scala.util.Random import org.jblas.DoubleMatrix +import org.scalatest.FunSuite -import org.apache.spark.mllib.util.{LinearDataGenerator, LocalSparkContext} +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.util.{LocalClusterSparkContext, LinearDataGenerator, + LocalSparkContext} class RidgeRegressionSuite extends FunSuite with LocalSparkContext { @@ -73,3 +76,19 @@ class RidgeRegressionSuite extends FunSuite with LocalSparkContext { "ridgeError (" + ridgeErr + ") was not less than linearError(" + linearErr + ")") } } + +class RidgeRegressionClusterSuite extends FunSuite with LocalClusterSparkContext { + + test("task size should be small in both training and prediction") { + val m = 4 + val n = 200000 + val points = sc.parallelize(0 until m, 2).mapPartitionsWithIndex { (idx, iter) => + val random = new Random(idx) + iter.map(i => LabeledPoint(1.0, Vectors.dense(Array.fill(n)(random.nextDouble())))) + }.cache() + // If we serialize data directly in the task closure, the size of the serialized task would be + // greater than 1MB and hence Spark would throw an error. + val model = RidgeRegressionWithSGD.train(points, 2) + val predictions = model.predict(points.map(_.features)) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala new file mode 100644 index 0000000000000..45e25eecf508e --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala @@ -0,0 +1,135 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.regression + +import java.io.File +import java.nio.charset.Charset + +import scala.collection.mutable.ArrayBuffer + +import com.google.common.io.Files +import org.scalatest.FunSuite + +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.util.{LinearDataGenerator, LocalSparkContext} +import org.apache.spark.streaming.{Milliseconds, StreamingContext} +import org.apache.spark.util.Utils + +class StreamingLinearRegressionSuite extends FunSuite with LocalSparkContext { + + // Assert that two values are equal within tolerance epsilon + def assertEqual(v1: Double, v2: Double, epsilon: Double) { + def errorMessage = v1.toString + " did not equal " + v2.toString + assert(math.abs(v1-v2) <= epsilon, errorMessage) + } + + // Assert that model predictions are correct + def validatePrediction(predictions: Seq[Double], input: Seq[LabeledPoint]) { + val numOffPredictions = predictions.zip(input).count { case (prediction, expected) => + // A prediction is off if the prediction is more than 0.5 away from expected value. + math.abs(prediction - expected.label) > 0.5 + } + // At least 80% of the predictions should be on. + assert(numOffPredictions < input.length / 5) + } + + // Test if we can accurately learn Y = 10*X1 + 10*X2 on streaming data + test("streaming linear regression parameter accuracy") { + + val testDir = Files.createTempDir() + val numBatches = 10 + val batchDuration = Milliseconds(1000) + val ssc = new StreamingContext(sc, batchDuration) + val data = ssc.textFileStream(testDir.toString).map(LabeledPoint.parse) + val model = new StreamingLinearRegressionWithSGD() + .setInitialWeights(Vectors.dense(0.0, 0.0)) + .setStepSize(0.1) + .setNumIterations(50) + + model.trainOn(data) + + ssc.start() + + // write data to a file stream + for (i <- 0 until numBatches) { + val samples = LinearDataGenerator.generateLinearInput( + 0.0, Array(10.0, 10.0), 100, 42 * (i + 1)) + val file = new File(testDir, i.toString) + Files.write(samples.map(x => x.toString).mkString("\n"), file, Charset.forName("UTF-8")) + Thread.sleep(batchDuration.milliseconds) + } + + ssc.stop(stopSparkContext=false) + + System.clearProperty("spark.driver.port") + Utils.deleteRecursively(testDir) + + // check accuracy of final parameter estimates + assertEqual(model.latestModel().intercept, 0.0, 0.1) + assertEqual(model.latestModel().weights(0), 10.0, 0.1) + assertEqual(model.latestModel().weights(1), 10.0, 0.1) + + // check accuracy of predictions + val validationData = LinearDataGenerator.generateLinearInput(0.0, Array(10.0, 10.0), 100, 17) + validatePrediction(validationData.map(row => model.latestModel().predict(row.features)), + validationData) + } + + // Test that parameter estimates improve when learning Y = 10*X1 on streaming data + test("streaming linear regression parameter convergence") { + + val testDir = Files.createTempDir() + val batchDuration = Milliseconds(2000) + val ssc = new StreamingContext(sc, batchDuration) + val numBatches = 5 + val data = ssc.textFileStream(testDir.toString()).map(LabeledPoint.parse) + val model = new StreamingLinearRegressionWithSGD() + .setInitialWeights(Vectors.dense(0.0)) + .setStepSize(0.1) + .setNumIterations(50) + + model.trainOn(data) + + ssc.start() + + // write data to a file stream + val history = new ArrayBuffer[Double](numBatches) + for (i <- 0 until numBatches) { + val samples = LinearDataGenerator.generateLinearInput(0.0, Array(10.0), 100, 42 * (i + 1)) + val file = new File(testDir, i.toString) + Files.write(samples.map(x => x.toString).mkString("\n"), file, Charset.forName("UTF-8")) + Thread.sleep(batchDuration.milliseconds) + // wait an extra few seconds to make sure the update finishes before new data arrive + Thread.sleep(4000) + history.append(math.abs(model.latestModel().weights(0) - 10.0)) + } + + ssc.stop(stopSparkContext=false) + + System.clearProperty("spark.driver.port") + Utils.deleteRecursively(testDir) + + val deltas = history.drop(1).zip(history.dropRight(1)) + // check error stability (it always either shrinks, or increases with small tol) + assert(deltas.forall(x => (x._1 - x._2) <= 0.1)) + // check that error shrunk on at least 2 batches + assert(deltas.map(x => if ((x._1 - x._2) < 0) 1 else 0).sum > 1) + + } + +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/stat/CorrelationSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/stat/CorrelationSuite.scala new file mode 100644 index 0000000000000..a3f76f77a5dcc --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/stat/CorrelationSuite.scala @@ -0,0 +1,141 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.stat + +import org.scalatest.FunSuite + +import breeze.linalg.{DenseMatrix => BDM, Matrix => BM} + +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.stat.correlation.{Correlations, PearsonCorrelation, + SpearmanCorrelation} +import org.apache.spark.mllib.util.LocalSparkContext + +class CorrelationSuite extends FunSuite with LocalSparkContext { + + // test input data + val xData = Array(1.0, 0.0, -2.0) + val yData = Array(4.0, 5.0, 3.0) + val zeros = new Array[Double](3) + val data = Seq( + Vectors.dense(1.0, 0.0, 0.0, -2.0), + Vectors.dense(4.0, 5.0, 0.0, 3.0), + Vectors.dense(6.0, 7.0, 0.0, 8.0), + Vectors.dense(9.0, 0.0, 0.0, 1.0) + ) + + test("corr(x, y) default, pearson") { + val x = sc.parallelize(xData) + val y = sc.parallelize(yData) + val expected = 0.6546537 + val default = Statistics.corr(x, y) + val p1 = Statistics.corr(x, y, "pearson") + assert(approxEqual(expected, default)) + assert(approxEqual(expected, p1)) + + // numPartitions >= size for input RDDs + for (numParts <- List(xData.size, xData.size * 2)) { + val x1 = sc.parallelize(xData, numParts) + val y1 = sc.parallelize(yData, numParts) + val p2 = Statistics.corr(x1, y1) + assert(approxEqual(expected, p2)) + } + + // RDD of zero variance + val z = sc.parallelize(zeros) + assert(Statistics.corr(x, z).isNaN()) + } + + test("corr(x, y) spearman") { + val x = sc.parallelize(xData) + val y = sc.parallelize(yData) + val expected = 0.5 + val s1 = Statistics.corr(x, y, "spearman") + assert(approxEqual(expected, s1)) + + // numPartitions >= size for input RDDs + for (numParts <- List(xData.size, xData.size * 2)) { + val x1 = sc.parallelize(xData, numParts) + val y1 = sc.parallelize(yData, numParts) + val s2 = Statistics.corr(x1, y1, "spearman") + assert(approxEqual(expected, s2)) + } + + // RDD of zero variance => zero variance in ranks + val z = sc.parallelize(zeros) + assert(Statistics.corr(x, z, "spearman").isNaN()) + } + + test("corr(X) default, pearson") { + val X = sc.parallelize(data) + val defaultMat = Statistics.corr(X) + val pearsonMat = Statistics.corr(X, "pearson") + val expected = BDM( + (1.00000000, 0.05564149, Double.NaN, 0.4004714), + (0.05564149, 1.00000000, Double.NaN, 0.9135959), + (Double.NaN, Double.NaN, 1.00000000, Double.NaN), + (0.40047142, 0.91359586, Double.NaN,1.0000000)) + assert(matrixApproxEqual(defaultMat.toBreeze, expected)) + assert(matrixApproxEqual(pearsonMat.toBreeze, expected)) + } + + test("corr(X) spearman") { + val X = sc.parallelize(data) + val spearmanMat = Statistics.corr(X, "spearman") + val expected = BDM( + (1.0000000, 0.1054093, Double.NaN, 0.4000000), + (0.1054093, 1.0000000, Double.NaN, 0.9486833), + (Double.NaN, Double.NaN, 1.00000000, Double.NaN), + (0.4000000, 0.9486833, Double.NaN, 1.0000000)) + assert(matrixApproxEqual(spearmanMat.toBreeze, expected)) + } + + test("method identification") { + val pearson = PearsonCorrelation + val spearman = SpearmanCorrelation + + assert(Correlations.getCorrelationFromName("pearson") === pearson) + assert(Correlations.getCorrelationFromName("spearman") === spearman) + + // Should throw IllegalArgumentException + try { + Correlations.getCorrelationFromName("kendall") + assert(false) + } catch { + case ie: IllegalArgumentException => + } + } + + def approxEqual(v1: Double, v2: Double, threshold: Double = 1e-6): Boolean = { + if (v1.isNaN) { + v2.isNaN + } else { + math.abs(v1 - v2) <= threshold + } + } + + def matrixApproxEqual(A: BM[Double], B: BM[Double], threshold: Double = 1e-6): Boolean = { + for (i <- 0 until A.rows; j <- 0 until A.cols) { + if (!approxEqual(A(i, j), B(i, j), threshold)) { + println("i, j = " + i + ", " + j + " actual: " + A(i, j) + " expected:" + B(i, j)) + return false + } + } + true + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/stat/HypothesisTestSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/stat/HypothesisTestSuite.scala new file mode 100644 index 0000000000000..6de3840b3f198 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/stat/HypothesisTestSuite.scala @@ -0,0 +1,158 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.stat + +import java.util.Random + +import org.scalatest.FunSuite + +import org.apache.spark.SparkException +import org.apache.spark.mllib.linalg.{DenseVector, Matrices, Vectors} +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.stat.test.ChiSqTest +import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.mllib.util.TestingUtils._ + +class HypothesisTestSuite extends FunSuite with LocalSparkContext { + + test("chi squared pearson goodness of fit") { + + val observed = new DenseVector(Array[Double](4, 6, 5)) + val pearson = Statistics.chiSqTest(observed) + + // Results validated against the R command `chisq.test(c(4, 6, 5), p=c(1/3, 1/3, 1/3))` + assert(pearson.statistic === 0.4) + assert(pearson.degreesOfFreedom === 2) + assert(pearson.pValue ~== 0.8187 relTol 1e-4) + assert(pearson.method === ChiSqTest.PEARSON.name) + assert(pearson.nullHypothesis === ChiSqTest.NullHypothesis.goodnessOfFit.toString) + + // different expected and observed sum + val observed1 = new DenseVector(Array[Double](21, 38, 43, 80)) + val expected1 = new DenseVector(Array[Double](3, 5, 7, 20)) + val pearson1 = Statistics.chiSqTest(observed1, expected1) + + // Results validated against the R command + // `chisq.test(c(21, 38, 43, 80), p=c(3/35, 1/7, 1/5, 4/7))` + assert(pearson1.statistic ~== 14.1429 relTol 1e-4) + assert(pearson1.degreesOfFreedom === 3) + assert(pearson1.pValue ~== 0.002717 relTol 1e-4) + assert(pearson1.method === ChiSqTest.PEARSON.name) + assert(pearson1.nullHypothesis === ChiSqTest.NullHypothesis.goodnessOfFit.toString) + + // Vectors with different sizes + val observed3 = new DenseVector(Array(1.0, 2.0, 3.0)) + val expected3 = new DenseVector(Array(1.0, 2.0, 3.0, 4.0)) + intercept[IllegalArgumentException](Statistics.chiSqTest(observed3, expected3)) + + // negative counts in observed + val negObs = new DenseVector(Array(1.0, 2.0, 3.0, -4.0)) + intercept[IllegalArgumentException](Statistics.chiSqTest(negObs, expected1)) + + // count = 0.0 in expected but not observed + val zeroExpected = new DenseVector(Array(1.0, 0.0, 3.0)) + val inf = Statistics.chiSqTest(observed, zeroExpected) + assert(inf.statistic === Double.PositiveInfinity) + assert(inf.degreesOfFreedom === 2) + assert(inf.pValue === 0.0) + assert(inf.method === ChiSqTest.PEARSON.name) + assert(inf.nullHypothesis === ChiSqTest.NullHypothesis.goodnessOfFit.toString) + + // 0.0 in expected and observed simultaneously + val zeroObserved = new DenseVector(Array(2.0, 0.0, 1.0)) + intercept[IllegalArgumentException](Statistics.chiSqTest(zeroObserved, zeroExpected)) + } + + test("chi squared pearson matrix independence") { + val data = Array(40.0, 24.0, 29.0, 56.0, 32.0, 42.0, 31.0, 10.0, 0.0, 30.0, 15.0, 12.0) + // [[40.0, 56.0, 31.0, 30.0], + // [24.0, 32.0, 10.0, 15.0], + // [29.0, 42.0, 0.0, 12.0]] + val chi = Statistics.chiSqTest(Matrices.dense(3, 4, data)) + // Results validated against R command + // `chisq.test(rbind(c(40, 56, 31, 30),c(24, 32, 10, 15), c(29, 42, 0, 12)))` + assert(chi.statistic ~== 21.9958 relTol 1e-4) + assert(chi.degreesOfFreedom === 6) + assert(chi.pValue ~== 0.001213 relTol 1e-4) + assert(chi.method === ChiSqTest.PEARSON.name) + assert(chi.nullHypothesis === ChiSqTest.NullHypothesis.independence.toString) + + // Negative counts + val negCounts = Array(4.0, 5.0, 3.0, -3.0) + intercept[IllegalArgumentException](Statistics.chiSqTest(Matrices.dense(2, 2, negCounts))) + + // Row sum = 0.0 + val rowZero = Array(0.0, 1.0, 0.0, 2.0) + intercept[IllegalArgumentException](Statistics.chiSqTest(Matrices.dense(2, 2, rowZero))) + + // Column sum = 0.0 + val colZero = Array(0.0, 0.0, 2.0, 2.0) + // IllegalArgumentException thrown here since it's thrown on driver, not inside a task + intercept[IllegalArgumentException](Statistics.chiSqTest(Matrices.dense(2, 2, colZero))) + } + + test("chi squared pearson RDD[LabeledPoint]") { + // labels: 1.0 (2 / 6), 0.0 (4 / 6) + // feature1: 0.5 (1 / 6), 1.5 (2 / 6), 3.5 (3 / 6) + // feature2: 10.0 (1 / 6), 20.0 (1 / 6), 30.0 (2 / 6), 40.0 (2 / 6) + val data = Seq( + LabeledPoint(0.0, Vectors.dense(0.5, 10.0)), + LabeledPoint(0.0, Vectors.dense(1.5, 20.0)), + LabeledPoint(1.0, Vectors.dense(1.5, 30.0)), + LabeledPoint(0.0, Vectors.dense(3.5, 30.0)), + LabeledPoint(0.0, Vectors.dense(3.5, 40.0)), + LabeledPoint(1.0, Vectors.dense(3.5, 40.0))) + for (numParts <- List(2, 4, 6, 8)) { + val chi = Statistics.chiSqTest(sc.parallelize(data, numParts)) + val feature1 = chi(0) + assert(feature1.statistic === 0.75) + assert(feature1.degreesOfFreedom === 2) + assert(feature1.pValue ~== 0.6873 relTol 1e-4) + assert(feature1.method === ChiSqTest.PEARSON.name) + assert(feature1.nullHypothesis === ChiSqTest.NullHypothesis.independence.toString) + val feature2 = chi(1) + assert(feature2.statistic === 1.5) + assert(feature2.degreesOfFreedom === 3) + assert(feature2.pValue ~== 0.6823 relTol 1e-4) + assert(feature2.method === ChiSqTest.PEARSON.name) + assert(feature2.nullHypothesis === ChiSqTest.NullHypothesis.independence.toString) + } + + // Test that the right number of results is returned + val numCols = 1001 + val sparseData = Array( + new LabeledPoint(0.0, Vectors.sparse(numCols, Seq((100, 2.0)))), + new LabeledPoint(0.1, Vectors.sparse(numCols, Seq((200, 1.0))))) + val chi = Statistics.chiSqTest(sc.parallelize(sparseData)) + assert(chi.size === numCols) + assert(chi(1000) != null) // SPARK-3087 + + // Detect continous features or labels + val random = new Random(11L) + val continuousLabel = + Seq.fill(100000)(LabeledPoint(random.nextDouble(), Vectors.dense(random.nextInt(2)))) + intercept[SparkException] { + Statistics.chiSqTest(sc.parallelize(continuousLabel, 2)) + } + val continuousFeature = + Seq.fill(100000)(LabeledPoint(random.nextInt(2), Vectors.dense(random.nextDouble()))) + intercept[SparkException] { + Statistics.chiSqTest(sc.parallelize(continuousFeature, 2)) + } + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizerSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizerSuite.scala new file mode 100644 index 0000000000000..db13f142df517 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizerSuite.scala @@ -0,0 +1,209 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.stat + +import org.scalatest.FunSuite + +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.util.TestingUtils._ + +class MultivariateOnlineSummarizerSuite extends FunSuite { + + test("basic error handing") { + val summarizer = new MultivariateOnlineSummarizer + + assert(summarizer.count === 0, "should be zero since nothing is added.") + + withClue("Getting numNonzeros from empty summarizer should throw exception.") { + intercept[IllegalArgumentException] { + summarizer.numNonzeros + } + } + + withClue("Getting variance from empty summarizer should throw exception.") { + intercept[IllegalArgumentException] { + summarizer.variance + } + } + + withClue("Getting mean from empty summarizer should throw exception.") { + intercept[IllegalArgumentException] { + summarizer.mean + } + } + + withClue("Getting max from empty summarizer should throw exception.") { + intercept[IllegalArgumentException] { + summarizer.max + } + } + + withClue("Getting min from empty summarizer should throw exception.") { + intercept[IllegalArgumentException] { + summarizer.min + } + } + + summarizer.add(Vectors.dense(-1.0, 2.0, 6.0)).add(Vectors.sparse(3, Seq((0, -2.0), (1, 6.0)))) + + withClue("Adding a new dense sample with different array size should throw exception.") { + intercept[IllegalArgumentException] { + summarizer.add(Vectors.dense(3.0, 1.0)) + } + } + + withClue("Adding a new sparse sample with different array size should throw exception.") { + intercept[IllegalArgumentException] { + summarizer.add(Vectors.sparse(5, Seq((0, -2.0), (1, 6.0)))) + } + } + + val summarizer2 = (new MultivariateOnlineSummarizer).add(Vectors.dense(1.0, -2.0, 0.0, 4.0)) + withClue("Merging a new summarizer with different dimensions should throw exception.") { + intercept[IllegalArgumentException] { + summarizer.merge(summarizer2) + } + } + } + + test("dense vector input") { + // For column 2, the maximum will be 0.0, and it's not explicitly added since we ignore all + // the zeros; it's a case we need to test. For column 3, the minimum will be 0.0 which we + // need to test as well. + val summarizer = (new MultivariateOnlineSummarizer) + .add(Vectors.dense(-1.0, 0.0, 6.0)) + .add(Vectors.dense(3.0, -3.0, 0.0)) + + assert(summarizer.mean ~== Vectors.dense(1.0, -1.5, 3.0) absTol 1E-5, "mean mismatch") + + assert(summarizer.min ~== Vectors.dense(-1.0, -3, 0.0) absTol 1E-5, "min mismatch") + + assert(summarizer.max ~== Vectors.dense(3.0, 0.0, 6.0) absTol 1E-5, "max mismatch") + + assert(summarizer.numNonzeros ~== Vectors.dense(2, 1, 1) absTol 1E-5, "numNonzeros mismatch") + + assert(summarizer.variance ~== Vectors.dense(8.0, 4.5, 18.0) absTol 1E-5, "variance mismatch") + + assert(summarizer.count === 2) + } + + test("sparse vector input") { + val summarizer = (new MultivariateOnlineSummarizer) + .add(Vectors.sparse(3, Seq((0, -1.0), (2, 6.0)))) + .add(Vectors.sparse(3, Seq((0, 3.0), (1, -3.0)))) + + assert(summarizer.mean ~== Vectors.dense(1.0, -1.5, 3.0) absTol 1E-5, "mean mismatch") + + assert(summarizer.min ~== Vectors.dense(-1.0, -3, 0.0) absTol 1E-5, "min mismatch") + + assert(summarizer.max ~== Vectors.dense(3.0, 0.0, 6.0) absTol 1E-5, "max mismatch") + + assert(summarizer.numNonzeros ~== Vectors.dense(2, 1, 1) absTol 1E-5, "numNonzeros mismatch") + + assert(summarizer.variance ~== Vectors.dense(8.0, 4.5, 18.0) absTol 1E-5, "variance mismatch") + + assert(summarizer.count === 2) + } + + test("mixing dense and sparse vector input") { + val summarizer = (new MultivariateOnlineSummarizer) + .add(Vectors.sparse(3, Seq((0, -2.0), (1, 2.3)))) + .add(Vectors.dense(0.0, -1.0, -3.0)) + .add(Vectors.sparse(3, Seq((1, -5.1)))) + .add(Vectors.dense(3.8, 0.0, 1.9)) + .add(Vectors.dense(1.7, -0.6, 0.0)) + .add(Vectors.sparse(3, Seq((1, 1.9), (2, 0.0)))) + + assert(summarizer.mean ~== + Vectors.dense(0.583333333333, -0.416666666666, -0.183333333333) absTol 1E-5, "mean mismatch") + + assert(summarizer.min ~== Vectors.dense(-2.0, -5.1, -3) absTol 1E-5, "min mismatch") + + assert(summarizer.max ~== Vectors.dense(3.8, 2.3, 1.9) absTol 1E-5, "max mismatch") + + assert(summarizer.numNonzeros ~== Vectors.dense(3, 5, 2) absTol 1E-5, "numNonzeros mismatch") + + assert(summarizer.variance ~== + Vectors.dense(3.857666666666, 7.0456666666666, 2.48166666666666) absTol 1E-5, "variance mismatch") + + assert(summarizer.count === 6) + } + + test("merging two summarizers") { + val summarizer1 = (new MultivariateOnlineSummarizer) + .add(Vectors.sparse(3, Seq((0, -2.0), (1, 2.3)))) + .add(Vectors.dense(0.0, -1.0, -3.0)) + + val summarizer2 = (new MultivariateOnlineSummarizer) + .add(Vectors.sparse(3, Seq((1, -5.1)))) + .add(Vectors.dense(3.8, 0.0, 1.9)) + .add(Vectors.dense(1.7, -0.6, 0.0)) + .add(Vectors.sparse(3, Seq((1, 1.9), (2, 0.0)))) + + val summarizer = summarizer1.merge(summarizer2) + + assert(summarizer.mean ~== + Vectors.dense(0.583333333333, -0.416666666666, -0.183333333333) absTol 1E-5, "mean mismatch") + + assert(summarizer.min ~== Vectors.dense(-2.0, -5.1, -3) absTol 1E-5, "min mismatch") + + assert(summarizer.max ~== Vectors.dense(3.8, 2.3, 1.9) absTol 1E-5, "max mismatch") + + assert(summarizer.numNonzeros ~== Vectors.dense(3, 5, 2) absTol 1E-5, "numNonzeros mismatch") + + assert(summarizer.variance ~== + Vectors.dense(3.857666666666, 7.0456666666666, 2.48166666666666) absTol 1E-5, "variance mismatch") + + assert(summarizer.count === 6) + } + + test("merging summarizer with empty summarizer") { + // If one of two is non-empty, this should return the non-empty summarizer. + // If both of them are empty, then just return the empty summarizer. + val summarizer1 = (new MultivariateOnlineSummarizer) + .add(Vectors.dense(0.0, -1.0, -3.0)).merge(new MultivariateOnlineSummarizer) + assert(summarizer1.count === 1) + + val summarizer2 = (new MultivariateOnlineSummarizer) + .merge((new MultivariateOnlineSummarizer).add(Vectors.dense(0.0, -1.0, -3.0))) + assert(summarizer2.count === 1) + + val summarizer3 = (new MultivariateOnlineSummarizer).merge(new MultivariateOnlineSummarizer) + assert(summarizer3.count === 0) + + assert(summarizer1.mean ~== Vectors.dense(0.0, -1.0, -3.0) absTol 1E-5, "mean mismatch") + + assert(summarizer2.mean ~== Vectors.dense(0.0, -1.0, -3.0) absTol 1E-5, "mean mismatch") + + assert(summarizer1.min ~== Vectors.dense(0.0, -1.0, -3.0) absTol 1E-5, "min mismatch") + + assert(summarizer2.min ~== Vectors.dense(0.0, -1.0, -3.0) absTol 1E-5, "min mismatch") + + assert(summarizer1.max ~== Vectors.dense(0.0, -1.0, -3.0) absTol 1E-5, "max mismatch") + + assert(summarizer2.max ~== Vectors.dense(0.0, -1.0, -3.0) absTol 1E-5, "max mismatch") + + assert(summarizer1.numNonzeros ~== Vectors.dense(0, 1, 1) absTol 1E-5, "numNonzeros mismatch") + + assert(summarizer2.numNonzeros ~== Vectors.dense(0, 1, 1) absTol 1E-5, "numNonzeros mismatch") + + assert(summarizer1.variance ~== Vectors.dense(0, 0, 0) absTol 1E-5, "variance mismatch") + + assert(summarizer2.variance ~== Vectors.dense(0, 0, 0) absTol 1E-5, "variance mismatch") + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala index bcb11876b8f4f..2f36fd907772c 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala @@ -17,26 +17,55 @@ package org.apache.spark.mllib.tree +import scala.collection.JavaConverters._ + import org.scalatest.FunSuite -import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.tree.impurity.{Entropy, Gini, Variance} -import org.apache.spark.mllib.tree.model.Filter -import org.apache.spark.mllib.tree.model.Split -import org.apache.spark.mllib.tree.configuration.{FeatureType, Strategy} import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.configuration.FeatureType._ +import org.apache.spark.mllib.tree.configuration.Strategy +import org.apache.spark.mllib.tree.impl.{DecisionTreeMetadata, TreePoint} +import org.apache.spark.mllib.tree.impurity.{Entropy, Gini, Variance} +import org.apache.spark.mllib.tree.model.{DecisionTreeModel, Node} import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.mllib.regression.LabeledPoint class DecisionTreeSuite extends FunSuite with LocalSparkContext { + def validateClassifier( + model: DecisionTreeModel, + input: Seq[LabeledPoint], + requiredAccuracy: Double) { + val predictions = input.map(x => model.predict(x.features)) + val numOffPredictions = predictions.zip(input).count { case (prediction, expected) => + prediction != expected.label + } + val accuracy = (input.length - numOffPredictions).toDouble / input.length + assert(accuracy >= requiredAccuracy, + s"validateClassifier calculated accuracy $accuracy but required $requiredAccuracy.") + } + + def validateRegressor( + model: DecisionTreeModel, + input: Seq[LabeledPoint], + requiredMSE: Double) { + val predictions = input.map(x => model.predict(x.features)) + val squaredError = predictions.zip(input).map { case (prediction, expected) => + val err = prediction - expected.label + err * err + }.sum + val mse = squaredError / input.length + assert(mse <= requiredMSE, s"validateRegressor calculated MSE $mse but required $requiredMSE.") + } + test("split and bin calculation") { val arr = DecisionTreeSuite.generateOrderedLabeledPointsWithLabel1() assert(arr.length === 1000) val rdd = sc.parallelize(arr) - val strategy = new Strategy(Classification, Gini, 3, 100) - val (splits, bins) = DecisionTree.findSplitsBins(rdd, strategy) + val strategy = new Strategy(Classification, Gini, 3, 2, 100) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) assert(splits.length === 2) assert(bins.length === 2) assert(splits(0).length === 99) @@ -50,10 +79,12 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val strategy = new Strategy( Classification, Gini, - maxDepth = 3, + maxDepth = 2, + numClassesForClassification = 2, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 2, 1-> 2)) - val (splits, bins) = DecisionTree.findSplitsBins(rdd, strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) assert(splits.length === 2) assert(bins.length === 2) assert(splits(0).length === 99) @@ -129,10 +160,12 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val strategy = new Strategy( Classification, Gini, - maxDepth = 3, + maxDepth = 2, + numClassesForClassification = 2, maxBins = 100, - categoricalFeaturesInfo = Map(0 -> 3, 1-> 3)) - val (splits, bins) = DecisionTree.findSplitsBins(rdd, strategy) + categoricalFeaturesInfo = Map(0 -> 3, 1 -> 3)) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) // Check splits. @@ -231,6 +264,164 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bins(1)(3) === null) } + test("extract categories from a number for multiclass classification") { + val l = DecisionTree.extractMultiClassCategories(13, 10) + assert(l.length === 3) + assert(List(3.0, 2.0, 0.0).toSeq === l.toSeq) + } + + test("split and bin calculations for unordered categorical variables with multiclass " + + "classification") { + val arr = DecisionTreeSuite.generateCategoricalDataPoints() + assert(arr.length === 1000) + val rdd = sc.parallelize(arr) + val strategy = new Strategy( + Classification, + Gini, + maxDepth = 2, + numClassesForClassification = 100, + maxBins = 100, + categoricalFeaturesInfo = Map(0 -> 3, 1-> 3)) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) + + // Expecting 2^2 - 1 = 3 bins/splits + assert(splits(0)(0).feature === 0) + assert(splits(0)(0).threshold === Double.MinValue) + assert(splits(0)(0).featureType === Categorical) + assert(splits(0)(0).categories.length === 1) + assert(splits(0)(0).categories.contains(0.0)) + assert(splits(1)(0).feature === 1) + assert(splits(1)(0).threshold === Double.MinValue) + assert(splits(1)(0).featureType === Categorical) + assert(splits(1)(0).categories.length === 1) + assert(splits(1)(0).categories.contains(0.0)) + + assert(splits(0)(1).feature === 0) + assert(splits(0)(1).threshold === Double.MinValue) + assert(splits(0)(1).featureType === Categorical) + assert(splits(0)(1).categories.length === 1) + assert(splits(0)(1).categories.contains(1.0)) + assert(splits(1)(1).feature === 1) + assert(splits(1)(1).threshold === Double.MinValue) + assert(splits(1)(1).featureType === Categorical) + assert(splits(1)(1).categories.length === 1) + assert(splits(1)(1).categories.contains(1.0)) + + assert(splits(0)(2).feature === 0) + assert(splits(0)(2).threshold === Double.MinValue) + assert(splits(0)(2).featureType === Categorical) + assert(splits(0)(2).categories.length === 2) + assert(splits(0)(2).categories.contains(0.0)) + assert(splits(0)(2).categories.contains(1.0)) + assert(splits(1)(2).feature === 1) + assert(splits(1)(2).threshold === Double.MinValue) + assert(splits(1)(2).featureType === Categorical) + assert(splits(1)(2).categories.length === 2) + assert(splits(1)(2).categories.contains(0.0)) + assert(splits(1)(2).categories.contains(1.0)) + + assert(splits(0)(3) === null) + assert(splits(1)(3) === null) + + + // Check bins. + + assert(bins(0)(0).category === Double.MinValue) + assert(bins(0)(0).lowSplit.categories.length === 0) + assert(bins(0)(0).highSplit.categories.length === 1) + assert(bins(0)(0).highSplit.categories.contains(0.0)) + assert(bins(1)(0).category === Double.MinValue) + assert(bins(1)(0).lowSplit.categories.length === 0) + assert(bins(1)(0).highSplit.categories.length === 1) + assert(bins(1)(0).highSplit.categories.contains(0.0)) + + assert(bins(0)(1).category === Double.MinValue) + assert(bins(0)(1).lowSplit.categories.length === 1) + assert(bins(0)(1).lowSplit.categories.contains(0.0)) + assert(bins(0)(1).highSplit.categories.length === 1) + assert(bins(0)(1).highSplit.categories.contains(1.0)) + assert(bins(1)(1).category === Double.MinValue) + assert(bins(1)(1).lowSplit.categories.length === 1) + assert(bins(1)(1).lowSplit.categories.contains(0.0)) + assert(bins(1)(1).highSplit.categories.length === 1) + assert(bins(1)(1).highSplit.categories.contains(1.0)) + + assert(bins(0)(2).category === Double.MinValue) + assert(bins(0)(2).lowSplit.categories.length === 1) + assert(bins(0)(2).lowSplit.categories.contains(1.0)) + assert(bins(0)(2).highSplit.categories.length === 2) + assert(bins(0)(2).highSplit.categories.contains(1.0)) + assert(bins(0)(2).highSplit.categories.contains(0.0)) + assert(bins(1)(2).category === Double.MinValue) + assert(bins(1)(2).lowSplit.categories.length === 1) + assert(bins(1)(2).lowSplit.categories.contains(1.0)) + assert(bins(1)(2).highSplit.categories.length === 2) + assert(bins(1)(2).highSplit.categories.contains(1.0)) + assert(bins(1)(2).highSplit.categories.contains(0.0)) + + assert(bins(0)(3) === null) + assert(bins(1)(3) === null) + + } + + test("split and bin calculations for ordered categorical variables with multiclass " + + "classification") { + val arr = DecisionTreeSuite.generateCategoricalDataPointsForMulticlassForOrderedFeatures() + assert(arr.length === 3000) + val rdd = sc.parallelize(arr) + val strategy = new Strategy( + Classification, + Gini, + maxDepth = 2, + numClassesForClassification = 100, + maxBins = 100, + categoricalFeaturesInfo = Map(0 -> 10, 1-> 10)) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) + + // 2^10 - 1 > 100, so categorical variables will be ordered + + assert(splits(0)(0).feature === 0) + assert(splits(0)(0).threshold === Double.MinValue) + assert(splits(0)(0).featureType === Categorical) + assert(splits(0)(0).categories.length === 1) + assert(splits(0)(0).categories.contains(1.0)) + + assert(splits(0)(1).feature === 0) + assert(splits(0)(1).threshold === Double.MinValue) + assert(splits(0)(1).featureType === Categorical) + assert(splits(0)(1).categories.length === 2) + assert(splits(0)(1).categories.contains(2.0)) + + assert(splits(0)(2).feature === 0) + assert(splits(0)(2).threshold === Double.MinValue) + assert(splits(0)(2).featureType === Categorical) + assert(splits(0)(2).categories.length === 3) + assert(splits(0)(2).categories.contains(2.0)) + assert(splits(0)(2).categories.contains(1.0)) + + assert(splits(0)(10) === null) + assert(splits(1)(10) === null) + + + // Check bins. + + assert(bins(0)(0).category === 1.0) + assert(bins(0)(0).lowSplit.categories.length === 0) + assert(bins(0)(0).highSplit.categories.length === 1) + assert(bins(0)(0).highSplit.categories.contains(1.0)) + assert(bins(0)(1).category === 2.0) + assert(bins(0)(1).lowSplit.categories.length === 1) + assert(bins(0)(1).highSplit.categories.length === 2) + assert(bins(0)(1).highSplit.categories.contains(1.0)) + assert(bins(0)(1).highSplit.categories.contains(2.0)) + + assert(bins(0)(10) === null) + + } + + test("classification stump with all categorical variables") { val arr = DecisionTreeSuite.generateCategoricalDataPoints() assert(arr.length === 1000) @@ -238,12 +429,15 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val strategy = new Strategy( Classification, Gini, - maxDepth = 3, + numClassesForClassification = 2, + maxDepth = 2, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 3, 1-> 3)) - val (splits, bins) = DecisionTree.findSplitsBins(rdd, strategy) - val bestSplits = DecisionTree.findBestSplits(rdd, new Array(7), strategy, 0, - Array[List[Filter]](), splits, bins, 10) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) + val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(7), metadata, 0, + new Array[Node](0), splits, bins, 10) val split = bestSplits(0)._1 assert(split.categories.length === 1) @@ -253,8 +447,8 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val stats = bestSplits(0)._2 assert(stats.gain > 0) - assert(stats.predict > 0.5) - assert(stats.predict < 0.7) + assert(stats.predict === 1) + assert(stats.prob === 0.6) assert(stats.impurity > 0.2) } @@ -265,12 +459,14 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val strategy = new Strategy( Regression, Variance, - maxDepth = 3, + maxDepth = 2, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 3, 1-> 3)) - val (splits, bins) = DecisionTree.findSplitsBins(rdd,strategy) - val bestSplits = DecisionTree.findBestSplits(rdd, new Array(7), strategy, 0, - Array[List[Filter]](), splits, bins, 10) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) + val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(7), metadata, 0, + new Array[Node](0), splits, bins, 10) val split = bestSplits(0)._1 assert(split.categories.length === 1) @@ -280,17 +476,34 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val stats = bestSplits(0)._2 assert(stats.gain > 0) - assert(stats.predict > 0.5) - assert(stats.predict < 0.7) + assert(stats.predict === 0.6) assert(stats.impurity > 0.2) } + test("regression stump with categorical variables of arity 2") { + val arr = DecisionTreeSuite.generateCategoricalDataPoints() + assert(arr.length === 1000) + val rdd = sc.parallelize(arr) + val strategy = new Strategy( + Regression, + Variance, + maxDepth = 2, + maxBins = 100, + categoricalFeaturesInfo = Map(0 -> 2, 1-> 2)) + + val model = DecisionTree.train(rdd, strategy) + validateRegressor(model, arr, 0.0) + assert(model.numNodes === 3) + assert(model.depth === 1) + } + test("stump with fixed label 0 for Gini") { val arr = DecisionTreeSuite.generateOrderedLabeledPointsWithLabel0() assert(arr.length === 1000) val rdd = sc.parallelize(arr) - val strategy = new Strategy(Classification, Gini, 3, 100) - val (splits, bins) = DecisionTree.findSplitsBins(rdd, strategy) + val strategy = new Strategy(Classification, Gini, 3, 2, 100) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) assert(splits.length === 2) assert(splits(0).length === 99) assert(bins.length === 2) @@ -298,11 +511,11 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(splits(0).length === 99) assert(bins(0).length === 100) - val bestSplits = DecisionTree.findBestSplits(rdd, new Array(7), strategy, 0, - Array[List[Filter]](), splits, bins, 10) + val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(7), metadata, 0, + new Array[Node](0), splits, bins, 10) assert(bestSplits.length === 1) assert(bestSplits(0)._1.feature === 0) - assert(bestSplits(0)._1.threshold === 10) assert(bestSplits(0)._2.gain === 0) assert(bestSplits(0)._2.leftImpurity === 0) assert(bestSplits(0)._2.rightImpurity === 0) @@ -312,8 +525,9 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val arr = DecisionTreeSuite.generateOrderedLabeledPointsWithLabel1() assert(arr.length === 1000) val rdd = sc.parallelize(arr) - val strategy = new Strategy(Classification, Gini, 3, 100) - val (splits, bins) = DecisionTree.findSplitsBins(rdd, strategy) + val strategy = new Strategy(Classification, Gini, 3, 2, 100) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) assert(splits.length === 2) assert(splits(0).length === 99) assert(bins.length === 2) @@ -321,11 +535,11 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(splits(0).length === 99) assert(bins(0).length === 100) - val bestSplits = DecisionTree.findBestSplits(rdd, Array(0.0), strategy, 0, - Array[List[Filter]](), splits, bins, 10) + val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) + val bestSplits = DecisionTree.findBestSplits(treeInput, Array(0.0), metadata, 0, + new Array[Node](0), splits, bins, 10) assert(bestSplits.length === 1) assert(bestSplits(0)._1.feature === 0) - assert(bestSplits(0)._1.threshold === 10) assert(bestSplits(0)._2.gain === 0) assert(bestSplits(0)._2.leftImpurity === 0) assert(bestSplits(0)._2.rightImpurity === 0) @@ -336,8 +550,9 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val arr = DecisionTreeSuite.generateOrderedLabeledPointsWithLabel0() assert(arr.length === 1000) val rdd = sc.parallelize(arr) - val strategy = new Strategy(Classification, Entropy, 3, 100) - val (splits, bins) = DecisionTree.findSplitsBins(rdd, strategy) + val strategy = new Strategy(Classification, Entropy, 3, 2, 100) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) assert(splits.length === 2) assert(splits(0).length === 99) assert(bins.length === 2) @@ -345,11 +560,11 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(splits(0).length === 99) assert(bins(0).length === 100) - val bestSplits = DecisionTree.findBestSplits(rdd, Array(0.0), strategy, 0, - Array[List[Filter]](), splits, bins, 10) + val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) + val bestSplits = DecisionTree.findBestSplits(treeInput, Array(0.0), metadata, 0, + new Array[Node](0), splits, bins, 10) assert(bestSplits.length === 1) assert(bestSplits(0)._1.feature === 0) - assert(bestSplits(0)._1.threshold === 10) assert(bestSplits(0)._2.gain === 0) assert(bestSplits(0)._2.leftImpurity === 0) assert(bestSplits(0)._2.rightImpurity === 0) @@ -360,8 +575,9 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val arr = DecisionTreeSuite.generateOrderedLabeledPointsWithLabel1() assert(arr.length === 1000) val rdd = sc.parallelize(arr) - val strategy = new Strategy(Classification, Entropy, 3, 100) - val (splits, bins) = DecisionTree.findSplitsBins(rdd, strategy) + val strategy = new Strategy(Classification, Entropy, 3, 2, 100) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) assert(splits.length === 2) assert(splits(0).length === 99) assert(bins.length === 2) @@ -369,23 +585,24 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(splits(0).length === 99) assert(bins(0).length === 100) - val bestSplits = DecisionTree.findBestSplits(rdd, Array(0.0), strategy, 0, - Array[List[Filter]](), splits, bins, 10) + val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) + val bestSplits = DecisionTree.findBestSplits(treeInput, Array(0.0), metadata, 0, + new Array[Node](0), splits, bins, 10) assert(bestSplits.length === 1) assert(bestSplits(0)._1.feature === 0) - assert(bestSplits(0)._1.threshold === 10) assert(bestSplits(0)._2.gain === 0) assert(bestSplits(0)._2.leftImpurity === 0) assert(bestSplits(0)._2.rightImpurity === 0) assert(bestSplits(0)._2.predict === 1) } - test("test second level node building with/without groups") { + test("second level node building with/without groups") { val arr = DecisionTreeSuite.generateOrderedLabeledPoints() assert(arr.length === 1000) val rdd = sc.parallelize(arr) - val strategy = new Strategy(Classification, Entropy, 3, 100) - val (splits, bins) = DecisionTree.findSplitsBins(rdd, strategy) + val strategy = new Strategy(Classification, Entropy, 3, 2, 100) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) assert(splits.length === 2) assert(splits(0).length === 99) assert(bins.length === 2) @@ -393,13 +610,19 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(splits(0).length === 99) assert(bins(0).length === 100) - val leftFilter = Filter(new Split(0, 400, FeatureType.Continuous, List()), -1) - val rightFilter = Filter(new Split(0, 400, FeatureType.Continuous, List()) ,1) - val filters = Array[List[Filter]](List(), List(leftFilter), List(rightFilter)) + // Train a 1-node model + val strategyOneNode = new Strategy(Classification, Entropy, 1, 2, 100) + val modelOneNode = DecisionTree.train(rdd, strategyOneNode) + val nodes: Array[Node] = new Array[Node](7) + nodes(0) = modelOneNode.topNode + nodes(0).leftNode = None + nodes(0).rightNode = None + val parentImpurities = Array(0.5, 0.5, 0.5) // Single group second level tree construction. - val bestSplits = DecisionTree.findBestSplits(rdd, parentImpurities, strategy, 1, filters, + val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) + val bestSplits = DecisionTree.findBestSplits(treeInput, parentImpurities, metadata, 1, nodes, splits, bins, 10) assert(bestSplits.length === 2) assert(bestSplits(0)._2.gain > 0) @@ -407,8 +630,8 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { // maxLevelForSingleGroup parameter is set to 0 to force splitting into groups for second // level tree construction. - val bestSplitsWithGroups = DecisionTree.findBestSplits(rdd, parentImpurities, strategy, 1, - filters, splits, bins, 0) + val bestSplitsWithGroups = DecisionTree.findBestSplits(treeInput, parentImpurities, metadata, 1, + nodes, splits, bins, 0) assert(bestSplitsWithGroups.length === 2) assert(bestSplitsWithGroups(0)._2.gain > 0) assert(bestSplitsWithGroups(1)._2.gain > 0) @@ -423,9 +646,166 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bestSplits(i)._2.rightImpurity === bestSplitsWithGroups(i)._2.rightImpurity) assert(bestSplits(i)._2.predict === bestSplitsWithGroups(i)._2.predict) } + } + + test("stump with categorical variables for multiclass classification") { + val arr = DecisionTreeSuite.generateCategoricalDataPointsForMulticlass() + val rdd = sc.parallelize(arr) + val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, + numClassesForClassification = 3, categoricalFeaturesInfo = Map(0 -> 3, 1 -> 3)) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + assert(strategy.isMulticlassClassification) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) + val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(31), metadata, 0, + new Array[Node](0), splits, bins, 10) + + assert(bestSplits.length === 1) + val bestSplit = bestSplits(0)._1 + assert(bestSplit.feature === 0) + assert(bestSplit.categories.length === 1) + assert(bestSplit.categories.contains(1)) + assert(bestSplit.featureType === Categorical) + } + + test("stump with 1 continuous variable for binary classification, to check off-by-1 error") { + val arr = new Array[LabeledPoint](4) + arr(0) = new LabeledPoint(0.0, Vectors.dense(0.0)) + arr(1) = new LabeledPoint(1.0, Vectors.dense(1.0)) + arr(2) = new LabeledPoint(1.0, Vectors.dense(2.0)) + arr(3) = new LabeledPoint(1.0, Vectors.dense(3.0)) + val rdd = sc.parallelize(arr) + val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, + numClassesForClassification = 2) + + val model = DecisionTree.train(rdd, strategy) + validateClassifier(model, arr, 1.0) + assert(model.numNodes === 3) + assert(model.depth === 1) + } + + test("stump with 2 continuous variables for binary classification") { + val arr = new Array[LabeledPoint](4) + arr(0) = new LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 0.0)))) + arr(1) = new LabeledPoint(1.0, Vectors.sparse(2, Seq((1, 1.0)))) + arr(2) = new LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 0.0)))) + arr(3) = new LabeledPoint(1.0, Vectors.sparse(2, Seq((1, 2.0)))) + + val input = sc.parallelize(arr) + val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, + numClassesForClassification = 2) + + val model = DecisionTree.train(input, strategy) + validateClassifier(model, arr, 1.0) + assert(model.numNodes === 3) + assert(model.depth === 1) + assert(model.topNode.split.get.feature === 1) + } + test("stump with categorical variables for multiclass classification, with just enough bins") { + val maxBins = math.pow(2, 3 - 1).toInt // just enough bins to allow unordered features + val arr = DecisionTreeSuite.generateCategoricalDataPointsForMulticlass() + val rdd = sc.parallelize(arr) + val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, + numClassesForClassification = 3, maxBins = maxBins, + categoricalFeaturesInfo = Map(0 -> 3, 1 -> 3)) + assert(strategy.isMulticlassClassification) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + + val model = DecisionTree.train(rdd, strategy) + validateClassifier(model, arr, 1.0) + assert(model.numNodes === 3) + assert(model.depth === 1) + + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) + val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(31), metadata, 0, + new Array[Node](0), splits, bins, 10) + + assert(bestSplits.length === 1) + val bestSplit = bestSplits(0)._1 + assert(bestSplit.feature === 0) + assert(bestSplit.categories.length === 1) + assert(bestSplit.categories.contains(1)) + assert(bestSplit.featureType === Categorical) + val gain = bestSplits(0)._2 + assert(gain.leftImpurity === 0) + assert(gain.rightImpurity === 0) } + test("stump with continuous variables for multiclass classification") { + val arr = DecisionTreeSuite.generateContinuousDataPointsForMulticlass() + val rdd = sc.parallelize(arr) + val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, + numClassesForClassification = 3) + assert(strategy.isMulticlassClassification) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + + val model = DecisionTree.train(rdd, strategy) + validateClassifier(model, arr, 0.9) + + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) + val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(31), metadata, 0, + new Array[Node](0), splits, bins, 10) + + assert(bestSplits.length === 1) + val bestSplit = bestSplits(0)._1 + + assert(bestSplit.feature === 1) + assert(bestSplit.featureType === Continuous) + assert(bestSplit.threshold > 1980) + assert(bestSplit.threshold < 2020) + + } + + test("stump with continuous + categorical variables for multiclass classification") { + val arr = DecisionTreeSuite.generateContinuousDataPointsForMulticlass() + val rdd = sc.parallelize(arr) + val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, + numClassesForClassification = 3, categoricalFeaturesInfo = Map(0 -> 3)) + assert(strategy.isMulticlassClassification) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + + val model = DecisionTree.train(rdd, strategy) + validateClassifier(model, arr, 0.9) + + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) + val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(31), metadata, 0, + new Array[Node](0), splits, bins, 10) + + assert(bestSplits.length === 1) + val bestSplit = bestSplits(0)._1 + + assert(bestSplit.feature === 1) + assert(bestSplit.featureType === Continuous) + assert(bestSplit.threshold > 1980) + assert(bestSplit.threshold < 2020) + } + + test("stump with categorical variables for ordered multiclass classification") { + val arr = DecisionTreeSuite.generateCategoricalDataPointsForMulticlassForOrderedFeatures() + val rdd = sc.parallelize(arr) + val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, + numClassesForClassification = 3, categoricalFeaturesInfo = Map(0 -> 10, 1 -> 10)) + assert(strategy.isMulticlassClassification) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) + val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(31), metadata, 0, + new Array[Node](0), splits, bins, 10) + + assert(bestSplits.length === 1) + val bestSplit = bestSplits(0)._1 + assert(bestSplit.feature === 0) + assert(bestSplit.categories.length === 1) + assert(bestSplit.categories.contains(1.0)) + assert(bestSplit.featureType === Categorical) + } + + } object DecisionTreeSuite { @@ -473,4 +853,51 @@ object DecisionTreeSuite { } arr } + + def generateCategoricalDataPointsAsJavaList(): java.util.List[LabeledPoint] = { + generateCategoricalDataPoints().toList.asJava + } + + def generateCategoricalDataPointsForMulticlass(): Array[LabeledPoint] = { + val arr = new Array[LabeledPoint](3000) + for (i <- 0 until 3000) { + if (i < 1000) { + arr(i) = new LabeledPoint(2.0, Vectors.dense(2.0, 2.0)) + } else if (i < 2000) { + arr(i) = new LabeledPoint(1.0, Vectors.dense(1.0, 2.0)) + } else { + arr(i) = new LabeledPoint(2.0, Vectors.dense(2.0, 2.0)) + } + } + arr + } + + def generateContinuousDataPointsForMulticlass(): Array[LabeledPoint] = { + val arr = new Array[LabeledPoint](3000) + for (i <- 0 until 3000) { + if (i < 2000) { + arr(i) = new LabeledPoint(2.0, Vectors.dense(2.0, i)) + } else { + arr(i) = new LabeledPoint(1.0, Vectors.dense(2.0, i)) + } + } + arr + } + + def generateCategoricalDataPointsForMulticlassForOrderedFeatures(): + Array[LabeledPoint] = { + val arr = new Array[LabeledPoint](3000) + for (i <- 0 until 3000) { + if (i < 1000) { + arr(i) = new LabeledPoint(2.0, Vectors.dense(2.0, 2.0)) + } else if (i < 2000) { + arr(i) = new LabeledPoint(1.0, Vectors.dense(1.0, 2.0)) + } else { + arr(i) = new LabeledPoint(1.0, Vectors.dense(2.0, 2.0)) + } + } + arr + } + + } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/LabelParsersSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/LocalClusterSparkContext.scala similarity index 56% rename from mllib/src/test/scala/org/apache/spark/mllib/util/LabelParsersSuite.scala rename to mllib/src/test/scala/org/apache/spark/mllib/util/LocalClusterSparkContext.scala index ac85677f2f014..5e9101cdd3804 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/util/LabelParsersSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/LocalClusterSparkContext.scala @@ -17,25 +17,26 @@ package org.apache.spark.mllib.util -import org.scalatest.FunSuite +import org.scalatest.{Suite, BeforeAndAfterAll} -class LabelParsersSuite extends FunSuite { - test("binary label parser") { - for (parser <- Seq(BinaryLabelParser, BinaryLabelParser.getInstance())) { - assert(parser.parse("+1") === 1.0) - assert(parser.parse("1") === 1.0) - assert(parser.parse("0") === 0.0) - assert(parser.parse("-1") === 0.0) - } +import org.apache.spark.{SparkConf, SparkContext} + +trait LocalClusterSparkContext extends BeforeAndAfterAll { self: Suite => + @transient var sc: SparkContext = _ + + override def beforeAll() { + val conf = new SparkConf() + .setMaster("local-cluster[2, 1, 512]") + .setAppName("test-cluster") + .set("spark.akka.frameSize", "1") // set to 1MB to detect direct serialization of data + sc = new SparkContext(conf) + super.beforeAll() } - test("multiclass label parser") { - for (parser <- Seq(MulticlassLabelParser, MulticlassLabelParser.getInstance())) { - assert(parser.parse("0") == 0.0) - assert(parser.parse("+1") === 1.0) - assert(parser.parse("1") === 1.0) - assert(parser.parse("2") === 2.0) - assert(parser.parse("3") === 3.0) + override def afterAll() { + if (sc != null) { + sc.stop() } + super.afterAll() } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/LocalSparkContext.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/LocalSparkContext.scala index 0d4868f3d9e42..7857d9e5ee5c4 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/util/LocalSparkContext.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/LocalSparkContext.scala @@ -20,13 +20,16 @@ package org.apache.spark.mllib.util import org.scalatest.Suite import org.scalatest.BeforeAndAfterAll -import org.apache.spark.SparkContext +import org.apache.spark.{SparkConf, SparkContext} trait LocalSparkContext extends BeforeAndAfterAll { self: Suite => @transient var sc: SparkContext = _ override def beforeAll() { - sc = new SparkContext("local", "test") + val conf = new SparkConf() + .setMaster("local") + .setAppName("test") + sc = new SparkContext(conf) super.beforeAll() } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala index c14870fb969a8..8ef2bb1bf6a78 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala @@ -63,9 +63,9 @@ class MLUtilsSuite extends FunSuite with LocalSparkContext { test("loadLibSVMFile") { val lines = """ - |+1 1:1.0 3:2.0 5:3.0 - |-1 - |-1 2:4.0 4:5.0 6:6.0 + |1 1:1.0 3:2.0 5:3.0 + |0 + |0 2:4.0 4:5.0 6:6.0 """.stripMargin val tempDir = Files.createTempDir() tempDir.deleteOnExit() @@ -73,7 +73,7 @@ class MLUtilsSuite extends FunSuite with LocalSparkContext { Files.write(lines, file, Charsets.US_ASCII) val path = tempDir.toURI.toString - val pointsWithNumFeatures = loadLibSVMFile(sc, path, multiclass = false, 6).collect() + val pointsWithNumFeatures = loadLibSVMFile(sc, path, 6).collect() val pointsWithoutNumFeatures = loadLibSVMFile(sc, path).collect() for (points <- Seq(pointsWithNumFeatures, pointsWithoutNumFeatures)) { @@ -86,11 +86,11 @@ class MLUtilsSuite extends FunSuite with LocalSparkContext { assert(points(2).features === Vectors.sparse(6, Seq((1, 4.0), (3, 5.0), (5, 6.0)))) } - val multiclassPoints = loadLibSVMFile(sc, path, multiclass = true).collect() + val multiclassPoints = loadLibSVMFile(sc, path).collect() assert(multiclassPoints.length === 3) assert(multiclassPoints(0).label === 1.0) - assert(multiclassPoints(1).label === -1.0) - assert(multiclassPoints(2).label === -1.0) + assert(multiclassPoints(1).label === 0.0) + assert(multiclassPoints(2).label === 0.0) Utils.deleteRecursively(tempDir) } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtils.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtils.scala new file mode 100644 index 0000000000000..29cc42d8cbea7 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtils.scala @@ -0,0 +1,172 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.util + +import org.apache.spark.mllib.linalg.Vector +import org.scalatest.exceptions.TestFailedException + +object TestingUtils { + + val ABS_TOL_MSG = " using absolute tolerance" + val REL_TOL_MSG = " using relative tolerance" + + /** + * Private helper function for comparing two values using relative tolerance. + * Note that if x or y is extremely close to zero, i.e., smaller than Double.MinPositiveValue, + * the relative tolerance is meaningless, so the exception will be raised to warn users. + */ + private def RelativeErrorComparison(x: Double, y: Double, eps: Double): Boolean = { + val absX = math.abs(x) + val absY = math.abs(y) + val diff = math.abs(x - y) + if (x == y) { + true + } else if (absX < Double.MinPositiveValue || absY < Double.MinPositiveValue) { + throw new TestFailedException( + s"$x or $y is extremely close to zero, so the relative tolerance is meaningless.", 0) + } else { + diff < eps * math.min(absX, absY) + } + } + + /** + * Private helper function for comparing two values using absolute tolerance. + */ + private def AbsoluteErrorComparison(x: Double, y: Double, eps: Double): Boolean = { + math.abs(x - y) < eps + } + + case class CompareDoubleRightSide( + fun: (Double, Double, Double) => Boolean, y: Double, eps: Double, method: String) + + /** + * Implicit class for comparing two double values using relative tolerance or absolute tolerance. + */ + implicit class DoubleWithAlmostEquals(val x: Double) { + + /** + * When the difference of two values are within eps, returns true; otherwise, returns false. + */ + def ~=(r: CompareDoubleRightSide): Boolean = r.fun(x, r.y, r.eps) + + /** + * When the difference of two values are within eps, returns false; otherwise, returns true. + */ + def !~=(r: CompareDoubleRightSide): Boolean = !r.fun(x, r.y, r.eps) + + /** + * Throws exception when the difference of two values are NOT within eps; + * otherwise, returns true. + */ + def ~==(r: CompareDoubleRightSide): Boolean = { + if (!r.fun(x, r.y, r.eps)) { + throw new TestFailedException( + s"Expected $x and ${r.y} to be within ${r.eps}${r.method}.", 0) + } + true + } + + /** + * Throws exception when the difference of two values are within eps; otherwise, returns true. + */ + def !~==(r: CompareDoubleRightSide): Boolean = { + if (r.fun(x, r.y, r.eps)) { + throw new TestFailedException( + s"Did not expect $x and ${r.y} to be within ${r.eps}${r.method}.", 0) + } + true + } + + /** + * Comparison using absolute tolerance. + */ + def absTol(eps: Double): CompareDoubleRightSide = CompareDoubleRightSide(AbsoluteErrorComparison, + x, eps, ABS_TOL_MSG) + + /** + * Comparison using relative tolerance. + */ + def relTol(eps: Double): CompareDoubleRightSide = CompareDoubleRightSide(RelativeErrorComparison, + x, eps, REL_TOL_MSG) + + override def toString = x.toString + } + + case class CompareVectorRightSide( + fun: (Vector, Vector, Double) => Boolean, y: Vector, eps: Double, method: String) + + /** + * Implicit class for comparing two vectors using relative tolerance or absolute tolerance. + */ + implicit class VectorWithAlmostEquals(val x: Vector) { + + /** + * When the difference of two vectors are within eps, returns true; otherwise, returns false. + */ + def ~=(r: CompareVectorRightSide): Boolean = r.fun(x, r.y, r.eps) + + /** + * When the difference of two vectors are within eps, returns false; otherwise, returns true. + */ + def !~=(r: CompareVectorRightSide): Boolean = !r.fun(x, r.y, r.eps) + + /** + * Throws exception when the difference of two vectors are NOT within eps; + * otherwise, returns true. + */ + def ~==(r: CompareVectorRightSide): Boolean = { + if (!r.fun(x, r.y, r.eps)) { + throw new TestFailedException( + s"Expected $x and ${r.y} to be within ${r.eps}${r.method} for all elements.", 0) + } + true + } + + /** + * Throws exception when the difference of two vectors are within eps; otherwise, returns true. + */ + def !~==(r: CompareVectorRightSide): Boolean = { + if (r.fun(x, r.y, r.eps)) { + throw new TestFailedException( + s"Did not expect $x and ${r.y} to be within ${r.eps}${r.method} for all elements.", 0) + } + true + } + + /** + * Comparison using absolute tolerance. + */ + def absTol(eps: Double): CompareVectorRightSide = CompareVectorRightSide( + (x: Vector, y: Vector, eps: Double) => { + x.toArray.zip(y.toArray).forall(x => x._1 ~= x._2 absTol eps) + }, x, eps, ABS_TOL_MSG) + + /** + * Comparison using relative tolerance. Note that comparing against sparse vector + * with elements having value of zero will raise exception because it involves with + * comparing against zero. + */ + def relTol(eps: Double): CompareVectorRightSide = CompareVectorRightSide( + (x: Vector, y: Vector, eps: Double) => { + x.toArray.zip(y.toArray).forall(x => x._1 ~= x._2 relTol eps) + }, x, eps, REL_TOL_MSG) + + override def toString = x.toString + } + +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtilsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtilsSuite.scala new file mode 100644 index 0000000000000..b0ecb33c28483 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtilsSuite.scala @@ -0,0 +1,182 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.util + +import org.apache.spark.mllib.linalg.Vectors +import org.scalatest.FunSuite +import org.apache.spark.mllib.util.TestingUtils._ +import org.scalatest.exceptions.TestFailedException + +class TestingUtilsSuite extends FunSuite { + + test("Comparing doubles using relative error.") { + + assert(23.1 ~== 23.52 relTol 0.02) + assert(23.1 ~== 22.74 relTol 0.02) + assert(23.1 ~= 23.52 relTol 0.02) + assert(23.1 ~= 22.74 relTol 0.02) + assert(!(23.1 !~= 23.52 relTol 0.02)) + assert(!(23.1 !~= 22.74 relTol 0.02)) + + // Should throw exception with message when test fails. + intercept[TestFailedException](23.1 !~== 23.52 relTol 0.02) + intercept[TestFailedException](23.1 !~== 22.74 relTol 0.02) + intercept[TestFailedException](23.1 ~== 23.63 relTol 0.02) + intercept[TestFailedException](23.1 ~== 22.34 relTol 0.02) + + assert(23.1 !~== 23.63 relTol 0.02) + assert(23.1 !~== 22.34 relTol 0.02) + assert(23.1 !~= 23.63 relTol 0.02) + assert(23.1 !~= 22.34 relTol 0.02) + assert(!(23.1 ~= 23.63 relTol 0.02)) + assert(!(23.1 ~= 22.34 relTol 0.02)) + + // Comparing against zero should fail the test and throw exception with message + // saying that the relative error is meaningless in this situation. + intercept[TestFailedException](0.1 ~== 0.0 relTol 0.032) + intercept[TestFailedException](0.1 ~= 0.0 relTol 0.032) + intercept[TestFailedException](0.1 !~== 0.0 relTol 0.032) + intercept[TestFailedException](0.1 !~= 0.0 relTol 0.032) + intercept[TestFailedException](0.0 ~== 0.1 relTol 0.032) + intercept[TestFailedException](0.0 ~= 0.1 relTol 0.032) + intercept[TestFailedException](0.0 !~== 0.1 relTol 0.032) + intercept[TestFailedException](0.0 !~= 0.1 relTol 0.032) + + // Comparisons of numbers very close to zero. + assert(10 * Double.MinPositiveValue ~== 9.5 * Double.MinPositiveValue relTol 0.01) + assert(10 * Double.MinPositiveValue !~== 11 * Double.MinPositiveValue relTol 0.01) + + assert(-Double.MinPositiveValue ~== 1.18 * -Double.MinPositiveValue relTol 0.012) + assert(-Double.MinPositiveValue ~== 1.38 * -Double.MinPositiveValue relTol 0.012) + } + + test("Comparing doubles using absolute error.") { + + assert(17.8 ~== 17.99 absTol 0.2) + assert(17.8 ~== 17.61 absTol 0.2) + assert(17.8 ~= 17.99 absTol 0.2) + assert(17.8 ~= 17.61 absTol 0.2) + assert(!(17.8 !~= 17.99 absTol 0.2)) + assert(!(17.8 !~= 17.61 absTol 0.2)) + + // Should throw exception with message when test fails. + intercept[TestFailedException](17.8 !~== 17.99 absTol 0.2) + intercept[TestFailedException](17.8 !~== 17.61 absTol 0.2) + intercept[TestFailedException](17.8 ~== 18.01 absTol 0.2) + intercept[TestFailedException](17.8 ~== 17.59 absTol 0.2) + + assert(17.8 !~== 18.01 absTol 0.2) + assert(17.8 !~== 17.59 absTol 0.2) + assert(17.8 !~= 18.01 absTol 0.2) + assert(17.8 !~= 17.59 absTol 0.2) + assert(!(17.8 ~= 18.01 absTol 0.2)) + assert(!(17.8 ~= 17.59 absTol 0.2)) + + // Comparisons of numbers very close to zero, and both side of zeros + assert(Double.MinPositiveValue ~== 4 * Double.MinPositiveValue absTol 5 * Double.MinPositiveValue) + assert(Double.MinPositiveValue !~== 6 * Double.MinPositiveValue absTol 5 * Double.MinPositiveValue) + + assert(-Double.MinPositiveValue ~== 3 * Double.MinPositiveValue absTol 5 * Double.MinPositiveValue) + assert(Double.MinPositiveValue !~== -4 * Double.MinPositiveValue absTol 5 * Double.MinPositiveValue) + } + + test("Comparing vectors using relative error.") { + + //Comparisons of two dense vectors + assert(Vectors.dense(Array(3.1, 3.5)) ~== Vectors.dense(Array(3.130, 3.534)) relTol 0.01) + assert(Vectors.dense(Array(3.1, 3.5)) !~== Vectors.dense(Array(3.135, 3.534)) relTol 0.01) + assert(Vectors.dense(Array(3.1, 3.5)) ~= Vectors.dense(Array(3.130, 3.534)) relTol 0.01) + assert(Vectors.dense(Array(3.1, 3.5)) !~= Vectors.dense(Array(3.135, 3.534)) relTol 0.01) + assert(!(Vectors.dense(Array(3.1, 3.5)) !~= Vectors.dense(Array(3.130, 3.534)) relTol 0.01)) + assert(!(Vectors.dense(Array(3.1, 3.5)) ~= Vectors.dense(Array(3.135, 3.534)) relTol 0.01)) + + // Should throw exception with message when test fails. + intercept[TestFailedException]( + Vectors.dense(Array(3.1, 3.5)) !~== Vectors.dense(Array(3.130, 3.534)) relTol 0.01) + + intercept[TestFailedException]( + Vectors.dense(Array(3.1, 3.5)) ~== Vectors.dense(Array(3.135, 3.534)) relTol 0.01) + + // Comparing against zero should fail the test and throw exception with message + // saying that the relative error is meaningless in this situation. + intercept[TestFailedException]( + Vectors.dense(Array(3.1, 0.01)) ~== Vectors.dense(Array(3.13, 0.0)) relTol 0.01) + + intercept[TestFailedException]( + Vectors.dense(Array(3.1, 0.01)) ~== Vectors.sparse(2, Array(0), Array(3.13)) relTol 0.01) + + // Comparisons of two sparse vectors + assert(Vectors.dense(Array(3.1, 3.5)) ~== + Vectors.sparse(2, Array(0, 1), Array(3.130, 3.534)) relTol 0.01) + + assert(Vectors.dense(Array(3.1, 3.5)) !~== + Vectors.sparse(2, Array(0, 1), Array(3.135, 3.534)) relTol 0.01) + } + + test("Comparing vectors using absolute error.") { + + //Comparisons of two dense vectors + assert(Vectors.dense(Array(3.1, 3.5, 0.0)) ~== + Vectors.dense(Array(3.1 + 1E-8, 3.5 + 2E-7, 1E-8)) absTol 1E-6) + + assert(Vectors.dense(Array(3.1, 3.5, 0.0)) !~== + Vectors.dense(Array(3.1 + 1E-5, 3.5 + 2E-7, 1 + 1E-3)) absTol 1E-6) + + assert(Vectors.dense(Array(3.1, 3.5, 0.0)) ~= + Vectors.dense(Array(3.1 + 1E-8, 3.5 + 2E-7, 1E-8)) absTol 1E-6) + + assert(Vectors.dense(Array(3.1, 3.5, 0.0)) !~= + Vectors.dense(Array(3.1 + 1E-5, 3.5 + 2E-7, 1 + 1E-3)) absTol 1E-6) + + assert(!(Vectors.dense(Array(3.1, 3.5, 0.0)) !~= + Vectors.dense(Array(3.1 + 1E-8, 3.5 + 2E-7, 1E-8)) absTol 1E-6)) + + assert(!(Vectors.dense(Array(3.1, 3.5, 0.0)) ~= + Vectors.dense(Array(3.1 + 1E-5, 3.5 + 2E-7, 1 + 1E-3)) absTol 1E-6)) + + // Should throw exception with message when test fails. + intercept[TestFailedException](Vectors.dense(Array(3.1, 3.5, 0.0)) !~== + Vectors.dense(Array(3.1 + 1E-8, 3.5 + 2E-7, 1E-8)) absTol 1E-6) + + intercept[TestFailedException](Vectors.dense(Array(3.1, 3.5, 0.0)) ~== + Vectors.dense(Array(3.1 + 1E-5, 3.5 + 2E-7, 1 + 1E-3)) absTol 1E-6) + + // Comparisons of two sparse vectors + assert(Vectors.sparse(3, Array(0, 2), Array(3.1, 2.4)) ~== + Vectors.sparse(3, Array(0, 2), Array(3.1 + 1E-8, 2.4 + 1E-7)) absTol 1E-6) + + assert(Vectors.sparse(3, Array(0, 2), Array(3.1 + 1E-8, 2.4 + 1E-7)) ~== + Vectors.sparse(3, Array(0, 2), Array(3.1, 2.4)) absTol 1E-6) + + assert(Vectors.sparse(3, Array(0, 2), Array(3.1, 2.4)) !~== + Vectors.sparse(3, Array(0, 2), Array(3.1 + 1E-3, 2.4)) absTol 1E-6) + + assert(Vectors.sparse(3, Array(0, 2), Array(3.1 + 1E-3, 2.4)) !~== + Vectors.sparse(3, Array(0, 2), Array(3.1, 2.4)) absTol 1E-6) + + // Comparisons of a dense vector and a sparse vector + assert(Vectors.sparse(3, Array(0, 2), Array(3.1, 2.4)) ~== + Vectors.dense(Array(3.1 + 1E-8, 0, 2.4 + 1E-7)) absTol 1E-6) + + assert(Vectors.dense(Array(3.1 + 1E-8, 0, 2.4 + 1E-7)) ~== + Vectors.sparse(3, Array(0, 2), Array(3.1, 2.4)) absTol 1E-6) + + assert(Vectors.sparse(3, Array(0, 2), Array(3.1, 2.4)) !~== + Vectors.dense(Array(3.1, 1E-3, 2.4)) absTol 1E-6) + } +} diff --git a/pom.xml b/pom.xml index 05f76d566e9d1..ef12c8f1a5c49 100644 --- a/pom.xml +++ b/pom.xml @@ -100,6 +100,7 @@ external/twitter external/kafka external/flume + external/flume-sink external/zeromq external/mqtt examples @@ -110,9 +111,10 @@ UTF-8 1.6 - + spark 2.10.4 2.10 + 2.0.1 0.18.1 shaded-protobuf org.spark-project.akka @@ -132,6 +134,8 @@ 3.0.0 1.7.6 0.7.1 + 1.8.3 + 1.1.0 64m 512m @@ -139,11 +143,10 @@ - maven-repo + central Maven Repository - - http://repo.maven.apache.org/maven2 + https://repo1.maven.org/maven2 true @@ -209,7 +212,7 @@ spring-releases Spring Release Repository - http://repo.spring.io/libs-release + https://repo.spring.io/libs-release true @@ -218,6 +221,18 @@ + + + central + https://repo1.maven.org/maven2 + + true + + + false + + + @@ -252,9 +267,15 @@ 3.3.2 - commons-codec - commons-codec - 1.5 + commons-codec + commons-codec + 1.5 + + + org.apache.commons + commons-math3 + 3.3 + test com.google.code.findbugs @@ -295,7 +316,12 @@ org.xerial.snappy snappy-java - 1.0.5 + 1.1.1.3 + + + net.jpountz.lz4 + lz4 + 1.2.0 com.clearspring.analytics @@ -535,6 +561,10 @@ org.mortbay.jetty servlet-api-2.5 + + javax.servlet + servlet-api + junit junit @@ -605,7 +635,6 @@ net.java.dev.jets3t jets3t ${jets3t.version} - runtime commons-logging @@ -618,6 +647,10 @@ hadoop-yarn-api ${yarn.version} + + javax.servlet + servlet-api + asm asm @@ -760,7 +793,7 @@ net.alchim31.maven scala-maven-plugin - 3.1.6 + 3.2.0 scala-compile-first @@ -806,6 +839,15 @@ -target ${java.version} + + + + org.scalamacros + paradise_${scala.version} + ${scala.macros.version} + + @@ -839,10 +881,11 @@ ${project.build.directory}/SparkTestSuite.txt -Xmx3g -XX:MaxPermSize=${MaxPermGen} -XX:ReservedCodeCacheSize=512m - - ${session.executionRootDirectory} - 1 - + + true + ${session.executionRootDirectory} + 1 + @@ -945,6 +988,30 @@ org.apache.maven.plugins maven-source-plugin + + org.scalastyle + scalastyle-maven-plugin + 0.4.0 + + false + true + false + false + ${basedir}/src/main/scala + ${basedir}/src/test/scala + scalastyle-config.xml + scalastyle-output.xml + UTF-8 + + + + package + + check + + + + @@ -958,6 +1025,14 @@ + + + kinesis-asl + + extras/kinesis-asl + + + java8-tests @@ -1103,5 +1178,15 @@ + + hive-thriftserver + + false + + + sql/hive-thriftserver + + + diff --git a/project/MimaBuild.scala b/project/MimaBuild.scala index bb2d73741c3bf..034ba6a7bf50f 100644 --- a/project/MimaBuild.scala +++ b/project/MimaBuild.scala @@ -15,13 +15,16 @@ * limitations under the License. */ +import sbt._ +import sbt.Keys.version + import com.typesafe.tools.mima.core._ import com.typesafe.tools.mima.core.MissingClassProblem import com.typesafe.tools.mima.core.MissingTypesProblem import com.typesafe.tools.mima.core.ProblemFilters._ import com.typesafe.tools.mima.plugin.MimaKeys.{binaryIssueFilters, previousArtifact} import com.typesafe.tools.mima.plugin.MimaPlugin.mimaDefaultSettings -import sbt._ + object MimaBuild { @@ -53,7 +56,7 @@ object MimaBuild { excludePackage("org.apache.spark." + packageName) } - def ignoredABIProblems(base: File) = { + def ignoredABIProblems(base: File, currentSparkVersion: String) = { // Excludes placed here will be used for all Spark versions val defaultExcludes = Seq() @@ -77,11 +80,16 @@ object MimaBuild { } defaultExcludes ++ ignoredClasses.flatMap(excludeClass) ++ - ignoredMembers.flatMap(excludeMember) ++ MimaExcludes.excludes + ignoredMembers.flatMap(excludeMember) ++ MimaExcludes.excludes(currentSparkVersion) + } + + def mimaSettings(sparkHome: File, projectRef: ProjectRef) = { + val organization = "org.apache.spark" + val previousSparkVersion = "1.0.0" + val fullId = "spark-" + projectRef.project + "_2.10" + mimaDefaultSettings ++ + Seq(previousArtifact := Some(organization % fullId % previousSparkVersion), + binaryIssueFilters ++= ignoredABIProblems(sparkHome, version.value)) } - def mimaSettings(sparkHome: File) = mimaDefaultSettings ++ Seq( - previousArtifact := None, - binaryIssueFilters ++= ignoredABIProblems(sparkHome) - ) } diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 1621833e124f5..300589394b96f 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -31,8 +31,8 @@ import com.typesafe.tools.mima.core._ * MimaBuild.excludeSparkClass("graphx.util.collection.GraphXPrimitiveKeyOpenHashMap") */ object MimaExcludes { - val excludes = - SparkBuild.SPARK_VERSION match { + def excludes(version: String) = + version match { case v if v.startsWith("1.1") => Seq( MimaBuild.excludeSparkPackage("deploy"), @@ -59,10 +59,35 @@ object MimaExcludes { ProblemFilters.exclude[MissingMethodProblem]( "org.apache.spark.api.java.JavaDoubleRDD.countApproxDistinct$default$1"), ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.storage.MemoryStore.Entry"), + "org.apache.spark.storage.MemoryStore.Entry") + ) ++ + Seq( + // Serializer interface change. See SPARK-3045. + ProblemFilters.exclude[IncompatibleTemplateDefProblem]( + "org.apache.spark.serializer.DeserializationStream"), + ProblemFilters.exclude[IncompatibleTemplateDefProblem]( + "org.apache.spark.serializer.Serializer"), + ProblemFilters.exclude[IncompatibleTemplateDefProblem]( + "org.apache.spark.serializer.SerializationStream"), + ProblemFilters.exclude[IncompatibleTemplateDefProblem]( + "org.apache.spark.serializer.SerializerInstance") + )++ + Seq( + // Renamed putValues -> putArray + putIterator + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.storage.MemoryStore.putValues"), + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.storage.DiskStore.putValues"), + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.storage.TachyonStore.putValues") + ) ++ + Seq( ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.rdd.PairRDDFunctions.org$apache$spark$rdd$PairRDDFunctions$$" - + "createZero$1") + "org.apache.spark.streaming.flume.FlumeReceiver.this"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]( + "org.apache.spark.streaming.kafka.KafkaUtils.createStream"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]( + "org.apache.spark.streaming.kafka.KafkaReceiver.this") ) ++ Seq( // Ignore some private methods in ALS. ProblemFilters.exclude[MissingMethodProblem]( @@ -70,14 +95,48 @@ object MimaExcludes { ProblemFilters.exclude[MissingMethodProblem]( // The only public constructor is the one without arguments. "org.apache.spark.mllib.recommendation.ALS.this"), ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.mllib.recommendation.ALS.org$apache$spark$mllib$recommendation$ALS$$$default$7") + "org.apache.spark.mllib.recommendation.ALS.org$apache$spark$mllib$recommendation$ALS$$$default$7"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]( + "org.apache.spark.mllib.recommendation.ALS.org$apache$spark$mllib$recommendation$ALS$^dateFeatures") ) ++ + MimaBuild.excludeSparkClass("mllib.linalg.distributed.ColumnStatisticsAggregator") ++ MimaBuild.excludeSparkClass("rdd.ZippedRDD") ++ MimaBuild.excludeSparkClass("rdd.ZippedPartition") ++ MimaBuild.excludeSparkClass("util.SerializableHyperLogLog") ++ MimaBuild.excludeSparkClass("storage.Values") ++ MimaBuild.excludeSparkClass("storage.Entry") ++ - MimaBuild.excludeSparkClass("storage.MemoryStore$Entry") + MimaBuild.excludeSparkClass("storage.MemoryStore$Entry") ++ + Seq( + ProblemFilters.exclude[IncompatibleMethTypeProblem]( + "org.apache.spark.mllib.tree.impurity.Gini.calculate"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]( + "org.apache.spark.mllib.tree.impurity.Entropy.calculate"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]( + "org.apache.spark.mllib.tree.impurity.Variance.calculate") + ) ++ + Seq ( // Package-private classes removed in SPARK-2341 + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.util.BinaryLabelParser"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.util.BinaryLabelParser$"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.util.LabelParser"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.util.LabelParser$"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.util.MulticlassLabelParser"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.util.MulticlassLabelParser$") + ) ++ + Seq( // package-private classes removed in MLlib + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.mllib.regression.GeneralizedLinearAlgorithm.org$apache$spark$mllib$regression$GeneralizedLinearAlgorithm$$prependOne") + ) ++ + Seq( // new Vector methods in MLlib (binary compatible assuming users do not implement Vector) + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.mllib.linalg.Vector.copy") + ) ++ + Seq( // synthetic methods generated in LabeledPoint + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.mllib.regression.LabeledPoint$"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.mllib.regression.LabeledPoint.apply"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.mllib.regression.LabeledPoint.toString") + ) ++ + Seq ( // Scala 2.11 compatibility fix + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.StreamingContext.$default$2") + ) case v if v.startsWith("1.0") => Seq( MimaBuild.excludeSparkPackage("api.java"), diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 599714233c18f..63a285b81a60c 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -15,498 +15,200 @@ * limitations under the License. */ +import scala.util.Properties +import scala.collection.JavaConversions._ + import sbt._ import sbt.Classpaths.publishTask import sbt.Keys._ -import sbtassembly.Plugin._ -import AssemblyKeys._ -import scala.util.Properties +import sbtunidoc.Plugin.genjavadocSettings import org.scalastyle.sbt.ScalastylePlugin.{Settings => ScalaStyleSettings} -import com.typesafe.tools.mima.plugin.MimaKeys.previousArtifact -import sbtunidoc.Plugin._ -import UnidocKeys._ - -import scala.collection.JavaConversions._ - -// For Sonatype publishing -// import com.jsuereth.pgp.sbtplugin.PgpKeys._ - -object SparkBuild extends Build { - val SPARK_VERSION = "1.1.0-SNAPSHOT" - val SPARK_VERSION_SHORT = SPARK_VERSION.replaceAll("-SNAPSHOT", "") - - // Hadoop version to build against. For example, "1.0.4" for Apache releases, or - // "2.0.0-mr1-cdh4.2.0" for Cloudera Hadoop. Note that these variables can be set - // through the environment variables SPARK_HADOOP_VERSION and SPARK_YARN. - val DEFAULT_HADOOP_VERSION = "1.0.4" - - // Whether the Hadoop version to build against is 2.2.x, or a variant of it. This can be set - // through the SPARK_IS_NEW_HADOOP environment variable. - val DEFAULT_IS_NEW_HADOOP = false - - val DEFAULT_YARN = false - - val DEFAULT_HIVE = false - - // HBase version; set as appropriate. - val HBASE_VERSION = "0.94.6" - - // Target JVM version - val SCALAC_JVM_VERSION = "jvm-1.6" - val JAVAC_JVM_VERSION = "1.6" - - lazy val root = Project("root", file("."), settings = rootSettings) aggregate(allProjects: _*) +import com.typesafe.sbt.pom.{PomBuild, SbtPomKeys} +import net.virtualvoid.sbt.graph.Plugin.graphSettings - lazy val core = Project("core", file("core"), settings = coreSettings) +object BuildCommons { - /** Following project only exists to pull previous artifacts of Spark for generating - Mima ignores. For more information see: SPARK 2071 */ - lazy val oldDeps = Project("oldDeps", file("dev"), settings = oldDepsSettings) + private val buildLocation = file(".").getAbsoluteFile.getParentFile - def replDependencies = Seq[ProjectReference](core, graphx, bagel, mllib, sql) ++ maybeHiveRef + val allProjects@Seq(bagel, catalyst, core, graphx, hive, hiveThriftServer, mllib, repl, + sql, streaming, streamingFlumeSink, streamingFlume, streamingKafka, streamingMqtt, + streamingTwitter, streamingZeromq) = + Seq("bagel", "catalyst", "core", "graphx", "hive", "hive-thriftserver", "mllib", "repl", + "sql", "streaming", "streaming-flume-sink", "streaming-flume", "streaming-kafka", + "streaming-mqtt", "streaming-twitter", "streaming-zeromq").map(ProjectRef(buildLocation, _)) - lazy val repl = Project("repl", file("repl"), settings = replSettings) - .dependsOn(replDependencies.map(a => a: sbt.ClasspathDep[sbt.ProjectReference]): _*) + val optionallyEnabledProjects@Seq(yarn, yarnStable, yarnAlpha, java8Tests, sparkGangliaLgpl, sparkKinesisAsl) = + Seq("yarn", "yarn-stable", "yarn-alpha", "java8-tests", "ganglia-lgpl", "kinesis-asl") + .map(ProjectRef(buildLocation, _)) - lazy val tools = Project("tools", file("tools"), settings = toolsSettings) dependsOn(core) dependsOn(streaming) + val assemblyProjects@Seq(assembly, examples) = Seq("assembly", "examples") + .map(ProjectRef(buildLocation, _)) - lazy val bagel = Project("bagel", file("bagel"), settings = bagelSettings) dependsOn(core) - - lazy val graphx = Project("graphx", file("graphx"), settings = graphxSettings) dependsOn(core) - - lazy val catalyst = Project("catalyst", file("sql/catalyst"), settings = catalystSettings) dependsOn(core) - - lazy val sql = Project("sql", file("sql/core"), settings = sqlCoreSettings) dependsOn(core) dependsOn(catalyst % "compile->compile;test->test") - - lazy val hive = Project("hive", file("sql/hive"), settings = hiveSettings) dependsOn(sql) - - lazy val maybeHive: Seq[ClasspathDependency] = if (isHiveEnabled) Seq(hive) else Seq() - lazy val maybeHiveRef: Seq[ProjectReference] = if (isHiveEnabled) Seq(hive) else Seq() - - lazy val streaming = Project("streaming", file("streaming"), settings = streamingSettings) dependsOn(core) + val tools = ProjectRef(buildLocation, "tools") + // Root project. + val spark = ProjectRef(buildLocation, "spark") + val sparkHome = buildLocation +} - lazy val mllib = Project("mllib", file("mllib"), settings = mllibSettings) dependsOn(core) +object SparkBuild extends PomBuild { - lazy val assemblyProj = Project("assembly", file("assembly"), settings = assemblyProjSettings) - .dependsOn(core, graphx, bagel, mllib, streaming, repl, sql) dependsOn(maybeYarn: _*) dependsOn(maybeHive: _*) dependsOn(maybeGanglia: _*) + import BuildCommons._ + import scala.collection.mutable.Map - lazy val assembleDepsTask = TaskKey[Unit]("assemble-deps") - lazy val assembleDeps = assembleDepsTask := { - println() - println("**** NOTE ****") - println("'sbt/sbt assemble-deps' is no longer supported.") - println("Instead create a normal assembly and:") - println(" export SPARK_PREPEND_CLASSES=1 (toggle on)") - println(" unset SPARK_PREPEND_CLASSES (toggle off)") - println() - } + val projectsMap: Map[String, Seq[Setting[_]]] = Map.empty - // A configuration to set an alternative publishLocalConfiguration - lazy val MavenCompile = config("m2r") extend(Compile) - lazy val publishLocalBoth = TaskKey[Unit]("publish-local", "publish local for m2 and ivy") - val sparkHome = System.getProperty("user.dir") - - // Allows build configuration to be set through environment variables - lazy val hadoopVersion = Properties.envOrElse("SPARK_HADOOP_VERSION", DEFAULT_HADOOP_VERSION) - lazy val isNewHadoop = Properties.envOrNone("SPARK_IS_NEW_HADOOP") match { - case None => { - val isNewHadoopVersion = "^2\\.[2-9]+".r.findFirstIn(hadoopVersion).isDefined - (isNewHadoopVersion|| DEFAULT_IS_NEW_HADOOP) + // Provides compatibility for older versions of the Spark build + def backwardCompatibility = { + import scala.collection.mutable + var isAlphaYarn = false + var profiles: mutable.Seq[String] = mutable.Seq.empty + if (Properties.envOrNone("SPARK_GANGLIA_LGPL").isDefined) { + println("NOTE: SPARK_GANGLIA_LGPL is deprecated, please use -Pspark-ganglia-lgpl flag.") + profiles ++= Seq("spark-ganglia-lgpl") + } + if (Properties.envOrNone("SPARK_HIVE").isDefined) { + println("NOTE: SPARK_HIVE is deprecated, please use -Phive flag.") + profiles ++= Seq("hive") } - case Some(v) => v.toBoolean + Properties.envOrNone("SPARK_HADOOP_VERSION") match { + case Some(v) => + if (v.matches("0.23.*")) isAlphaYarn = true + println("NOTE: SPARK_HADOOP_VERSION is deprecated, please use -Dhadoop.version=" + v) + System.setProperty("hadoop.version", v) + case None => + } + if (Properties.envOrNone("SPARK_YARN").isDefined) { + if(isAlphaYarn) { + println("NOTE: SPARK_YARN is deprecated, please use -Pyarn-alpha flag.") + profiles ++= Seq("yarn-alpha") + } + else { + println("NOTE: SPARK_YARN is deprecated, please use -Pyarn flag.") + profiles ++= Seq("yarn") + } + } + profiles } - lazy val isYarnEnabled = Properties.envOrNone("SPARK_YARN") match { - case None => DEFAULT_YARN - case Some(v) => v.toBoolean + override val profiles = Properties.envOrNone("SBT_MAVEN_PROFILES") match { + case None => backwardCompatibility + case Some(v) => + if (backwardCompatibility.nonEmpty) + println("Note: We ignore environment variables, when use of profile is detected in " + + "conjunction with environment variable.") + v.split("(\\s+|,)").filterNot(_.isEmpty).map(_.trim.replaceAll("-P", "")).toSeq } - lazy val hadoopClient = if (hadoopVersion.startsWith("0.20.") || hadoopVersion == "1.0.0") "hadoop-core" else "hadoop-client" - val maybeAvro = if (hadoopVersion.startsWith("0.23.")) Seq("org.apache.avro" % "avro" % "1.7.4") else Seq() - lazy val isHiveEnabled = Properties.envOrNone("SPARK_HIVE") match { - case None => DEFAULT_HIVE - case Some(v) => v.toBoolean + Properties.envOrNone("SBT_MAVEN_PROPERTIES") match { + case Some(v) => + v.split("(\\s+|,)").filterNot(_.isEmpty).map(_.split("=")).foreach(x => System.setProperty(x(0), x(1))) + case _ => } - // Include Ganglia integration if the user has enabled Ganglia - // This is isolated from the normal build due to LGPL-licensed code in the library - lazy val isGangliaEnabled = Properties.envOrNone("SPARK_GANGLIA_LGPL").isDefined - lazy val gangliaProj = Project("spark-ganglia-lgpl", file("extras/spark-ganglia-lgpl"), settings = gangliaSettings).dependsOn(core) - val maybeGanglia: Seq[ClasspathDependency] = if (isGangliaEnabled) Seq(gangliaProj) else Seq() - val maybeGangliaRef: Seq[ProjectReference] = if (isGangliaEnabled) Seq(gangliaProj) else Seq() - - // Include the Java 8 project if the JVM version is 8+ - lazy val javaVersion = System.getProperty("java.specification.version") - lazy val isJava8Enabled = javaVersion.toDouble >= "1.8".toDouble - val maybeJava8Tests = if (isJava8Enabled) Seq[ProjectReference](java8Tests) else Seq[ProjectReference]() - lazy val java8Tests = Project("java8-tests", file("extras/java8-tests"), settings = java8TestsSettings). - dependsOn(core) dependsOn(streaming % "compile->compile;test->test") + override val userPropertiesMap = System.getProperties.toMap - // Include the YARN project if the user has enabled YARN - lazy val yarnAlpha = Project("yarn-alpha", file("yarn/alpha"), settings = yarnAlphaSettings) dependsOn(core) - lazy val yarn = Project("yarn", file("yarn/stable"), settings = yarnSettings) dependsOn(core) - - lazy val maybeYarn: Seq[ClasspathDependency] = if (isYarnEnabled) Seq(if (isNewHadoop) yarn else yarnAlpha) else Seq() - lazy val maybeYarnRef: Seq[ProjectReference] = if (isYarnEnabled) Seq(if (isNewHadoop) yarn else yarnAlpha) else Seq() - - lazy val externalTwitter = Project("external-twitter", file("external/twitter"), settings = twitterSettings) - .dependsOn(streaming % "compile->compile;test->test") - - lazy val externalKafka = Project("external-kafka", file("external/kafka"), settings = kafkaSettings) - .dependsOn(streaming % "compile->compile;test->test") - - lazy val externalFlume = Project("external-flume", file("external/flume"), settings = flumeSettings) - .dependsOn(streaming % "compile->compile;test->test") - - lazy val externalZeromq = Project("external-zeromq", file("external/zeromq"), settings = zeromqSettings) - .dependsOn(streaming % "compile->compile;test->test") - - lazy val externalMqtt = Project("external-mqtt", file("external/mqtt"), settings = mqttSettings) - .dependsOn(streaming % "compile->compile;test->test") - - lazy val allExternal = Seq[ClasspathDependency](externalTwitter, externalKafka, externalFlume, externalZeromq, externalMqtt) - lazy val allExternalRefs = Seq[ProjectReference](externalTwitter, externalKafka, externalFlume, externalZeromq, externalMqtt) - - lazy val examples = Project("examples", file("examples"), settings = examplesSettings) - .dependsOn(core, mllib, graphx, bagel, streaming, hive) dependsOn(allExternal: _*) - - // Everything except assembly, hive, tools, java8Tests and examples belong to packageProjects - lazy val packageProjects = Seq[ProjectReference](core, repl, bagel, streaming, mllib, graphx, catalyst, sql) ++ maybeYarnRef ++ maybeHiveRef ++ maybeGangliaRef - - lazy val allProjects = packageProjects ++ allExternalRefs ++ - Seq[ProjectReference](examples, tools, assemblyProj) ++ maybeJava8Tests + lazy val MavenCompile = config("m2r") extend(Compile) + lazy val publishLocalBoth = TaskKey[Unit]("publish-local", "publish local for m2 and ivy") - def sharedSettings = Defaults.defaultSettings ++ MimaBuild.mimaSettings(file(sparkHome)) ++ Seq( - organization := "org.apache.spark", - version := SPARK_VERSION, - scalaVersion := "2.10.4", - scalacOptions := Seq("-Xmax-classfile-name", "120", "-unchecked", "-deprecation", "-feature", - "-target:" + SCALAC_JVM_VERSION), - javacOptions := Seq("-target", JAVAC_JVM_VERSION, "-source", JAVAC_JVM_VERSION), - unmanagedJars in Compile <<= baseDirectory map { base => (base / "lib" ** "*.jar").classpath }, + lazy val sharedSettings = graphSettings ++ ScalaStyleSettings ++ genjavadocSettings ++ Seq ( + javaHome := Properties.envOrNone("JAVA_HOME").map(file), + incOptions := incOptions.value.withNameHashing(true), retrieveManaged := true, - javaHome := Properties.envOrNone("JAVA_HOME").map(file), - // This is to add convenience of enabling sbt -Dsbt.offline=true for making the build offline. - offline := "true".equalsIgnoreCase(sys.props("sbt.offline")), retrievePattern := "[type]s/[artifact](-[revision])(-[classifier]).[ext]", - transitiveClassifiers in Scope.GlobalScope := Seq("sources"), - testListeners <<= target.map(t => Seq(new eu.henkelmann.sbt.JUnitXmlTestsListener(t.getAbsolutePath))), - incOptions := incOptions.value.withNameHashing(true), - // Fork new JVMs for tests and set Java options for those - fork := true, - javaOptions in Test += "-Dspark.home=" + sparkHome, - javaOptions in Test += "-Dspark.testing=1", - javaOptions in Test += "-Dsun.io.serialization.extendedDebugInfo=true", - javaOptions in Test ++= System.getProperties.filter(_._1 startsWith "spark").map { case (k,v) => s"-D$k=$v" }.toSeq, - javaOptions in Test ++= "-Xmx3g -XX:PermSize=128M -XX:MaxNewSize=256m -XX:MaxPermSize=1g".split(" ").toSeq, - javaOptions += "-Xmx3g", - // Show full stack trace and duration in test cases. - testOptions in Test += Tests.Argument("-oDF"), - // Remove certain packages from Scaladoc - scalacOptions in (Compile, doc) := Seq( - "-groups", - "-skip-packages", Seq( - "akka", - "org.apache.spark.api.python", - "org.apache.spark.network", - "org.apache.spark.deploy", - "org.apache.spark.util.collection" - ).mkString(":"), - "-doc-title", "Spark " + SPARK_VERSION_SHORT + " ScalaDoc" - ), - - // Only allow one test at a time, even across projects, since they run in the same JVM - concurrentRestrictions in Global += Tags.limit(Tags.Test, 1), - - resolvers ++= Seq( - // HTTPS is unavailable for Maven Central - "Maven Repository" at "http://repo.maven.apache.org/maven2", - "Apache Repository" at "https://repository.apache.org/content/repositories/releases", - "JBoss Repository" at "https://repository.jboss.org/nexus/content/repositories/releases/", - "MQTT Repository" at "https://repo.eclipse.org/content/repositories/paho-releases/", - "Cloudera Repository" at "http://repository.cloudera.com/artifactory/cloudera-repos/", - "Pivotal Repository" at "http://repo.spring.io/libs-release/", - // For Sonatype publishing - // "sonatype-snapshots" at "https://oss.sonatype.org/content/repositories/snapshots", - // "sonatype-staging" at "https://oss.sonatype.org/service/local/staging/deploy/maven2/", - // also check the local Maven repository ~/.m2 - Resolver.mavenLocal - ), - publishMavenStyle := true, - - // useGpg in Global := true, - - pomExtra := ( - - org.apache - apache - 14 - - http://spark.apache.org/ - - - Apache 2.0 License - http://www.apache.org/licenses/LICENSE-2.0.html - repo - - - - scm:git:git@github.com:apache/spark.git - scm:git:git@github.com:apache/spark.git - - - - matei - Matei Zaharia - matei.zaharia@gmail.com - http://www.cs.berkeley.edu/~matei - Apache Software Foundation - http://spark.apache.org - - - - JIRA - https://issues.apache.org/jira/browse/SPARK - - ), - - /* - publishTo <<= version { (v: String) => - val nexus = "https://oss.sonatype.org/" - if (v.trim.endsWith("SNAPSHOT")) - Some("sonatype-snapshots" at nexus + "content/repositories/snapshots") - else - Some("sonatype-staging" at nexus + "service/local/staging/deploy/maven2") - }, - - */ - - libraryDependencies ++= Seq( - "io.netty" % "netty-all" % "4.0.17.Final", - "org.eclipse.jetty" % "jetty-server" % jettyVersion, - "org.eclipse.jetty" % "jetty-util" % jettyVersion, - "org.eclipse.jetty" % "jetty-plus" % jettyVersion, - "org.eclipse.jetty" % "jetty-security" % jettyVersion, - "org.scalatest" %% "scalatest" % "2.1.5" % "test", - "org.scalacheck" %% "scalacheck" % "1.11.3" % "test", - "com.novocode" % "junit-interface" % "0.10" % "test", - "org.easymock" % "easymockclassextension" % "3.1" % "test", - "org.mockito" % "mockito-all" % "1.9.0" % "test", - "junit" % "junit" % "4.10" % "test", - // Needed by cglib which is needed by easymock. - "asm" % "asm" % "3.3.1" % "test" - ), - - testOptions += Tests.Argument(TestFrameworks.JUnit, "-v", "-a"), - parallelExecution := true, - /* Workaround for issue #206 (fixed after SBT 0.11.0) */ - watchTransitiveSources <<= Defaults.inDependencies[Task[Seq[File]]](watchSources.task, - const(std.TaskExtra.constant(Nil)), aggregate = true, includeRoot = true) apply { _.join.map(_.flatten) }, - - otherResolvers := Seq(Resolver.file("dotM2", file(Path.userHome + "/.m2/repository"))), + + resolvers += Resolver.mavenLocal, + otherResolvers <<= SbtPomKeys.mvnLocalRepository(dotM2 => Seq(Resolver.file("dotM2", dotM2))), publishLocalConfiguration in MavenCompile <<= (packagedArtifacts, deliverLocal, ivyLoggingLevel) map { (arts, _, level) => new PublishConfiguration(None, "dotM2", arts, Seq(), level) }, publishMavenStyle in MavenCompile := true, publishLocal in MavenCompile <<= publishTask(publishLocalConfiguration in MavenCompile, deliverLocal), publishLocalBoth <<= Seq(publishLocal in MavenCompile, publishLocal).dependOn - ) ++ net.virtualvoid.sbt.graph.Plugin.graphSettings ++ ScalaStyleSettings ++ genjavadocSettings - - val akkaVersion = "2.2.3-shaded-protobuf" - val chillVersion = "0.3.6" - val codahaleMetricsVersion = "3.0.0" - val jblasVersion = "1.2.3" - val jets3tVersion = if ("^2\\.[3-9]+".r.findFirstIn(hadoopVersion).isDefined) "0.9.0" else "0.7.1" - val jettyVersion = "8.1.14.v20131031" - val hiveVersion = "0.12.0" - val parquetVersion = "1.4.3" - val slf4jVersion = "1.7.5" - - val excludeJBossNetty = ExclusionRule(organization = "org.jboss.netty") - val excludeIONetty = ExclusionRule(organization = "io.netty") - val excludeEclipseJetty = ExclusionRule(organization = "org.eclipse.jetty") - val excludeAsm = ExclusionRule(organization = "org.ow2.asm") - val excludeOldAsm = ExclusionRule(organization = "asm") - val excludeCommonsLogging = ExclusionRule(organization = "commons-logging") - val excludeSLF4J = ExclusionRule(organization = "org.slf4j") - val excludeScalap = ExclusionRule(organization = "org.scala-lang", artifact = "scalap") - val excludeHadoop = ExclusionRule(organization = "org.apache.hadoop") - val excludeCurator = ExclusionRule(organization = "org.apache.curator") - val excludePowermock = ExclusionRule(organization = "org.powermock") - val excludeFastutil = ExclusionRule(organization = "it.unimi.dsi") - val excludeJruby = ExclusionRule(organization = "org.jruby") - val excludeThrift = ExclusionRule(organization = "org.apache.thrift") - val excludeServletApi = ExclusionRule(organization = "javax.servlet", artifact = "servlet-api") - val excludeJUnit = ExclusionRule(organization = "junit") - - def sparkPreviousArtifact(id: String, organization: String = "org.apache.spark", - version: String = "1.0.0", crossVersion: String = "2.10"): Option[sbt.ModuleID] = { - val fullId = if (crossVersion.isEmpty) id else id + "_" + crossVersion - Some(organization % fullId % version) // the artifact to compare binary compatibility with - } - - def coreSettings = sharedSettings ++ Seq( - name := "spark-core", - libraryDependencies ++= Seq( - "com.google.guava" % "guava" % "14.0.1", - "org.apache.commons" % "commons-lang3" % "3.3.2", - "org.apache.commons" % "commons-math3" % "3.3" % "test", - "com.google.code.findbugs" % "jsr305" % "1.3.9", - "log4j" % "log4j" % "1.2.17", - "org.slf4j" % "slf4j-api" % slf4jVersion, - "org.slf4j" % "slf4j-log4j12" % slf4jVersion, - "org.slf4j" % "jul-to-slf4j" % slf4jVersion, - "org.slf4j" % "jcl-over-slf4j" % slf4jVersion, - "commons-daemon" % "commons-daemon" % "1.0.10", // workaround for bug HADOOP-9407 - "com.ning" % "compress-lzf" % "1.0.0", - "org.xerial.snappy" % "snappy-java" % "1.0.5", - "org.spark-project.akka" %% "akka-remote" % akkaVersion, - "org.spark-project.akka" %% "akka-slf4j" % akkaVersion, - "org.spark-project.akka" %% "akka-testkit" % akkaVersion % "test", - "org.json4s" %% "json4s-jackson" % "3.2.6" excludeAll(excludeScalap), - "colt" % "colt" % "1.2.0", - "org.apache.mesos" % "mesos" % "0.18.1" classifier("shaded-protobuf") exclude("com.google.protobuf", "protobuf-java"), - "commons-net" % "commons-net" % "2.2", - "net.java.dev.jets3t" % "jets3t" % jets3tVersion excludeAll(excludeCommonsLogging), - "commons-codec" % "commons-codec" % "1.5", // Prevent jets3t from including the older version of commons-codec - "org.apache.derby" % "derby" % "10.4.2.0" % "test", - "org.apache.hadoop" % hadoopClient % hadoopVersion excludeAll(excludeJBossNetty, excludeAsm, excludeCommonsLogging, excludeSLF4J, excludeOldAsm, excludeServletApi), - "org.apache.curator" % "curator-recipes" % "2.4.0" excludeAll(excludeJBossNetty), - "com.codahale.metrics" % "metrics-core" % codahaleMetricsVersion, - "com.codahale.metrics" % "metrics-jvm" % codahaleMetricsVersion, - "com.codahale.metrics" % "metrics-json" % codahaleMetricsVersion, - "com.codahale.metrics" % "metrics-graphite" % codahaleMetricsVersion, - "com.twitter" %% "chill" % chillVersion excludeAll(excludeAsm), - "com.twitter" % "chill-java" % chillVersion excludeAll(excludeAsm), - "org.tachyonproject" % "tachyon" % "0.4.1-thrift" excludeAll(excludeHadoop, excludeCurator, excludeEclipseJetty, excludePowermock), - "com.clearspring.analytics" % "stream" % "2.7.0" excludeAll(excludeFastutil), // Only HyperLogLogPlus is used, which does not depend on fastutil. - "org.spark-project" % "pyrolite" % "2.0.1", - "net.sf.py4j" % "py4j" % "0.8.1" - ), - libraryDependencies ++= maybeAvro, - assembleDeps, - previousArtifact := sparkPreviousArtifact("spark-core") ) - // Create a colon-separate package list adding "org.apache.spark" in front of all of them, - // for easier specification of JavaDoc package groups - def packageList(names: String*): String = { - names.map(s => "org.apache.spark." + s).mkString(":") + def enable(settings: Seq[Setting[_]])(projectRef: ProjectRef) = { + val existingSettings = projectsMap.getOrElse(projectRef.project, Seq[Setting[_]]()) + projectsMap += (projectRef.project -> (existingSettings ++ settings)) } - def rootSettings = sharedSettings ++ scalaJavaUnidocSettings ++ Seq( - publish := {}, + // Note ordering of these settings matter. + /* Enable shared settings on all projects */ + (allProjects ++ optionallyEnabledProjects ++ assemblyProjects).foreach(enable(sharedSettings)) - unidocProjectFilter in (ScalaUnidoc, unidoc) := - inAnyProject -- inProjects(repl, examples, tools, catalyst, yarn, yarnAlpha), - unidocProjectFilter in (JavaUnidoc, unidoc) := - inAnyProject -- inProjects(repl, examples, bagel, graphx, catalyst, tools, yarn, yarnAlpha), + /* Enable tests settings for all projects except examples, assembly and tools */ + (allProjects ++ optionallyEnabledProjects).foreach(enable(TestSettings.settings)) - // Skip class names containing $ and some internal packages in Javadocs - unidocAllSources in (JavaUnidoc, unidoc) := { - (unidocAllSources in (JavaUnidoc, unidoc)).value - .map(_.filterNot(_.getName.contains("$"))) - .map(_.filterNot(_.getCanonicalPath.contains("akka"))) - .map(_.filterNot(_.getCanonicalPath.contains("deploy"))) - .map(_.filterNot(_.getCanonicalPath.contains("network"))) - .map(_.filterNot(_.getCanonicalPath.contains("executor"))) - .map(_.filterNot(_.getCanonicalPath.contains("python"))) - .map(_.filterNot(_.getCanonicalPath.contains("collection"))) - }, + // TODO: Add Sql to mima checks + allProjects.filterNot(x => Seq(spark, sql, hive, hiveThriftServer, catalyst, repl, + streamingFlumeSink).contains(x)).foreach(x => enable(MimaBuild.mimaSettings(sparkHome, x))(x)) - // Javadoc options: create a window title, and group key packages on index page - javacOptions in doc := Seq( - "-windowtitle", "Spark " + SPARK_VERSION_SHORT + " JavaDoc", - "-public", - "-group", "Core Java API", packageList("api.java", "api.java.function"), - "-group", "Spark Streaming", packageList( - "streaming.api.java", "streaming.flume", "streaming.kafka", - "streaming.mqtt", "streaming.twitter", "streaming.zeromq" - ), - "-group", "MLlib", packageList( - "mllib.classification", "mllib.clustering", "mllib.evaluation.binary", "mllib.linalg", - "mllib.linalg.distributed", "mllib.optimization", "mllib.rdd", "mllib.recommendation", - "mllib.regression", "mllib.stat", "mllib.tree", "mllib.tree.configuration", - "mllib.tree.impurity", "mllib.tree.model", "mllib.util" - ), - "-group", "Spark SQL", packageList("sql.api.java", "sql.hive.api.java"), - "-noqualifier", "java.lang" - ) - ) + /* Enable Assembly for all assembly projects */ + assemblyProjects.foreach(enable(Assembly.settings)) - def replSettings = sharedSettings ++ Seq( - name := "spark-repl", - libraryDependencies <+= scalaVersion(v => "org.scala-lang" % "scala-compiler" % v), - libraryDependencies <+= scalaVersion(v => "org.scala-lang" % "jline" % v), - libraryDependencies <+= scalaVersion(v => "org.scala-lang" % "scala-reflect" % v) - ) + /* Enable unidoc only for the root spark project */ + enable(Unidoc.settings)(spark) - def examplesSettings = sharedSettings ++ Seq( - name := "spark-examples", - jarName in assembly <<= version map { - v => "spark-examples-" + v + "-hadoop" + hadoopVersion + ".jar" }, - libraryDependencies ++= Seq( - "com.twitter" %% "algebird-core" % "0.1.11", - "org.apache.hbase" % "hbase" % HBASE_VERSION excludeAll(excludeIONetty, excludeJBossNetty, excludeAsm, excludeOldAsm, excludeCommonsLogging, excludeJruby), - "org.apache.cassandra" % "cassandra-all" % "1.2.6" - exclude("com.google.guava", "guava") - exclude("com.googlecode.concurrentlinkedhashmap", "concurrentlinkedhashmap-lru") - exclude("com.ning","compress-lzf") - exclude("io.netty", "netty") - exclude("jline","jline") - exclude("org.apache.cassandra.deps", "avro") - excludeAll(excludeSLF4J, excludeIONetty), - "com.github.scopt" %% "scopt" % "3.2.0" - ) - ) ++ assemblySettings ++ extraAssemblySettings - - def toolsSettings = sharedSettings ++ Seq( - name := "spark-tools", - libraryDependencies <+= scalaVersion(v => "org.scala-lang" % "scala-compiler" % v), - libraryDependencies <+= scalaVersion(v => "org.scala-lang" % "scala-reflect" % v ) - ) ++ assemblySettings ++ extraAssemblySettings - - def graphxSettings = sharedSettings ++ Seq( - name := "spark-graphx", - previousArtifact := sparkPreviousArtifact("spark-graphx"), - libraryDependencies ++= Seq( - "org.jblas" % "jblas" % jblasVersion - ) - ) + /* Catalyst macro settings */ + enable(Catalyst.settings)(catalyst) - def bagelSettings = sharedSettings ++ Seq( - name := "spark-bagel", - previousArtifact := sparkPreviousArtifact("spark-bagel") - ) + /* Spark SQL Core console settings */ + enable(SQL.settings)(sql) - def mllibSettings = sharedSettings ++ Seq( - name := "spark-mllib", - previousArtifact := sparkPreviousArtifact("spark-mllib"), - libraryDependencies ++= Seq( - "org.jblas" % "jblas" % jblasVersion, - "org.scalanlp" %% "breeze" % "0.7" excludeAll(excludeJUnit) - ) - ) + /* Hive console settings */ + enable(Hive.settings)(hive) - def catalystSettings = sharedSettings ++ Seq( - name := "catalyst", - // The mechanics of rewriting expression ids to compare trees in some test cases makes - // assumptions about the the expression ids being contiguous. Running tests in parallel breaks - // this non-deterministically. TODO: FIX THIS. - parallelExecution in Test := false, - libraryDependencies ++= Seq( - "com.typesafe" %% "scalalogging-slf4j" % "1.0.1" - ) + enable(Flume.settings)(streamingFlumeSink) + + // TODO: move this to its upstream project. + override def projectDefinitions(baseDirectory: File): Seq[Project] = { + super.projectDefinitions(baseDirectory).map { x => + if (projectsMap.exists(_._1 == x.id)) x.settings(projectsMap(x.id): _*) + else x.settings(Seq[Setting[_]](): _*) + } ++ Seq[Project](OldDeps.project) + } + +} + +object Flume { + lazy val settings = sbtavro.SbtAvro.avroSettings +} + +/** + * Following project only exists to pull previous artifacts of Spark for generating + * Mima ignores. For more information see: SPARK 2071 + */ +object OldDeps { + + lazy val project = Project("oldDeps", file("dev"), settings = oldDepsSettings) + + def versionArtifact(id: String): Option[sbt.ModuleID] = { + val fullId = id + "_2.10" + Some("org.apache.spark" % fullId % "1.0.0") + } + + def oldDepsSettings() = Defaults.defaultSettings ++ Seq( + name := "old-deps", + scalaVersion := "2.10.4", + retrieveManaged := true, + retrievePattern := "[type]s/[artifact](-[revision])(-[classifier]).[ext]", + libraryDependencies := Seq("spark-streaming-mqtt", "spark-streaming-zeromq", + "spark-streaming-flume", "spark-streaming-kafka", "spark-streaming-twitter", + "spark-streaming", "spark-mllib", "spark-bagel", "spark-graphx", + "spark-core").map(versionArtifact(_).get intransitive()) ) +} - def sqlCoreSettings = sharedSettings ++ Seq( - name := "spark-sql", - libraryDependencies ++= Seq( - "com.twitter" % "parquet-column" % parquetVersion, - "com.twitter" % "parquet-hadoop" % parquetVersion, - "com.fasterxml.jackson.core" % "jackson-databind" % "2.3.0" // json4s-jackson 3.2.6 requires jackson-databind 2.3.0. - ), +object Catalyst { + lazy val settings = Seq( + addCompilerPlugin("org.scalamacros" % "paradise" % "2.0.1" cross CrossVersion.full), + // Quasiquotes break compiling scala doc... + // TODO: Investigate fixing this. + sources in (Compile, doc) ~= (_ filter (_.getName contains "codegen"))) +} + +object SQL { + lazy val settings = Seq( initialCommands in console := """ |import org.apache.spark.sql.catalyst.analysis._ @@ -521,18 +223,14 @@ object SparkBuild extends Build { |import org.apache.spark.sql.test.TestSQLContext._ |import org.apache.spark.sql.parquet.ParquetTestData""".stripMargin ) +} + +object Hive { + + lazy val settings = Seq( - // Since we don't include hive in the main assembly this project also acts as an alternative - // assembly jar. - def hiveSettings = sharedSettings ++ Seq( - name := "spark-hive", javaOptions += "-XX:MaxPermSize=1g", - libraryDependencies ++= Seq( - "org.spark-project.hive" % "hive-metastore" % hiveVersion, - "org.spark-project.hive" % "hive-exec" % hiveVersion excludeAll(excludeCommonsLogging), - "org.spark-project.hive" % "hive-serde" % hiveVersion - ), - // Multiple queries rely on the TestHive singleton. See comments there for more details. + // Multiple queries rely on the TestHive singleton. See comments there for more details. parallelExecution in Test := false, // Supporting all SerDes requires us to depend on deprecated APIs, so we turn off the warnings // only for this subproject. @@ -555,67 +253,16 @@ object SparkBuild extends Build { |import org.apache.spark.sql.parquet.ParquetTestData""".stripMargin ) - def streamingSettings = sharedSettings ++ Seq( - name := "spark-streaming", - previousArtifact := sparkPreviousArtifact("spark-streaming") - ) - - def yarnCommonSettings = sharedSettings ++ Seq( - unmanagedSourceDirectories in Compile <++= baseDirectory { base => - Seq( - base / "../common/src/main/scala" - ) - }, - - unmanagedSourceDirectories in Test <++= baseDirectory { base => - Seq( - base / "../common/src/test/scala" - ) - } - - ) ++ extraYarnSettings - - def yarnAlphaSettings = yarnCommonSettings ++ Seq( - name := "spark-yarn-alpha" - ) - - def yarnSettings = yarnCommonSettings ++ Seq( - name := "spark-yarn" - ) - - def gangliaSettings = sharedSettings ++ Seq( - name := "spark-ganglia-lgpl", - libraryDependencies += "com.codahale.metrics" % "metrics-ganglia" % "3.0.0" - ) - - def java8TestsSettings = sharedSettings ++ Seq( - name := "java8-tests", - javacOptions := Seq("-target", "1.8", "-source", "1.8"), - testOptions += Tests.Argument(TestFrameworks.JUnit, "-v", "-a") - ) - - // Conditionally include the YARN dependencies because some tools look at all sub-projects and will complain - // if we refer to nonexistent dependencies (e.g. hadoop-yarn-api from a Hadoop version without YARN). - def extraYarnSettings = if(isYarnEnabled) yarnEnabledSettings else Seq() - - def yarnEnabledSettings = Seq( - libraryDependencies ++= Seq( - // Exclude rule required for all ? - "org.apache.hadoop" % hadoopClient % hadoopVersion excludeAll(excludeJBossNetty, excludeAsm, excludeOldAsm), - "org.apache.hadoop" % "hadoop-yarn-api" % hadoopVersion excludeAll(excludeJBossNetty, excludeAsm, excludeOldAsm, excludeCommonsLogging), - "org.apache.hadoop" % "hadoop-yarn-common" % hadoopVersion excludeAll(excludeJBossNetty, excludeAsm, excludeOldAsm, excludeCommonsLogging), - "org.apache.hadoop" % "hadoop-yarn-client" % hadoopVersion excludeAll(excludeJBossNetty, excludeAsm, excludeOldAsm, excludeCommonsLogging), - "org.apache.hadoop" % "hadoop-yarn-server-web-proxy" % hadoopVersion excludeAll(excludeJBossNetty, excludeAsm, excludeOldAsm, excludeCommonsLogging, excludeServletApi) - ) - ) +} - def assemblyProjSettings = sharedSettings ++ Seq( - name := "spark-assembly", - jarName in assembly <<= version map { v => "spark-assembly-" + v + "-hadoop" + hadoopVersion + ".jar" } - ) ++ assemblySettings ++ extraAssemblySettings +object Assembly { + import sbtassembly.Plugin._ + import AssemblyKeys._ - def extraAssemblySettings() = Seq( + lazy val settings = assemblySettings ++ Seq( test in assembly := {}, + jarName in assembly <<= (version, moduleName) map { (v, mName) => mName + "-"+v + "-hadoop" + + Option(System.getProperty("hadoop.version")).getOrElse("1.0.4") + ".jar" }, mergeStrategy in assembly := { case PathList("org", "datanucleus", xs @ _*) => MergeStrategy.discard case m if m.toLowerCase.endsWith("manifest.mf") => MergeStrategy.discard @@ -627,57 +274,98 @@ object SparkBuild extends Build { } ) - def oldDepsSettings() = Defaults.defaultSettings ++ Seq( - name := "old-deps", - scalaVersion := "2.10.4", - retrieveManaged := true, - retrievePattern := "[type]s/[artifact](-[revision])(-[classifier]).[ext]", - libraryDependencies := Seq("spark-streaming-mqtt", "spark-streaming-zeromq", - "spark-streaming-flume", "spark-streaming-kafka", "spark-streaming-twitter", - "spark-streaming", "spark-mllib", "spark-bagel", "spark-graphx", - "spark-core").map(sparkPreviousArtifact(_).get intransitive()) - ) +} - def twitterSettings() = sharedSettings ++ Seq( - name := "spark-streaming-twitter", - previousArtifact := sparkPreviousArtifact("spark-streaming-twitter"), - libraryDependencies ++= Seq( - "org.twitter4j" % "twitter4j-stream" % "3.0.3" - ) - ) +object Unidoc { - def kafkaSettings() = sharedSettings ++ Seq( - name := "spark-streaming-kafka", - previousArtifact := sparkPreviousArtifact("spark-streaming-kafka"), - libraryDependencies ++= Seq( - "com.github.sgroschupf" % "zkclient" % "0.1", - "org.apache.kafka" %% "kafka" % "0.8.0" - exclude("com.sun.jdmk", "jmxtools") - exclude("com.sun.jmx", "jmxri") - exclude("net.sf.jopt-simple", "jopt-simple") - excludeAll(excludeSLF4J) - ) - ) + import BuildCommons._ + import sbtunidoc.Plugin._ + import UnidocKeys._ - def flumeSettings() = sharedSettings ++ Seq( - name := "spark-streaming-flume", - previousArtifact := sparkPreviousArtifact("spark-streaming-flume"), - libraryDependencies ++= Seq( - "org.apache.flume" % "flume-ng-sdk" % "1.4.0" % "compile" excludeAll(excludeIONetty, excludeThrift) + // for easier specification of JavaDoc package groups + private def packageList(names: String*): String = { + names.map(s => "org.apache.spark." + s).mkString(":") + } + + lazy val settings = scalaJavaUnidocSettings ++ Seq ( + publish := {}, + + unidocProjectFilter in(ScalaUnidoc, unidoc) := + inAnyProject -- inProjects(OldDeps.project, repl, examples, tools, catalyst, yarn, yarnAlpha), + unidocProjectFilter in(JavaUnidoc, unidoc) := + inAnyProject -- inProjects(OldDeps.project, repl, bagel, graphx, examples, tools, catalyst, yarn, yarnAlpha), + + // Skip class names containing $ and some internal packages in Javadocs + unidocAllSources in (JavaUnidoc, unidoc) := { + (unidocAllSources in (JavaUnidoc, unidoc)).value + .map(_.filterNot(_.getName.contains("$"))) + .map(_.filterNot(_.getCanonicalPath.contains("akka"))) + .map(_.filterNot(_.getCanonicalPath.contains("deploy"))) + .map(_.filterNot(_.getCanonicalPath.contains("network"))) + .map(_.filterNot(_.getCanonicalPath.contains("shuffle"))) + .map(_.filterNot(_.getCanonicalPath.contains("executor"))) + .map(_.filterNot(_.getCanonicalPath.contains("python"))) + .map(_.filterNot(_.getCanonicalPath.contains("collection"))) + }, + + // Javadoc options: create a window title, and group key packages on index page + javacOptions in doc := Seq( + "-windowtitle", "Spark " + version.value.replaceAll("-SNAPSHOT", "") + " JavaDoc", + "-public", + "-group", "Core Java API", packageList("api.java", "api.java.function"), + "-group", "Spark Streaming", packageList( + "streaming.api.java", "streaming.flume", "streaming.kafka", + "streaming.mqtt", "streaming.twitter", "streaming.zeromq" + ), + "-group", "MLlib", packageList( + "mllib.classification", "mllib.clustering", "mllib.evaluation.binary", "mllib.linalg", + "mllib.linalg.distributed", "mllib.optimization", "mllib.rdd", "mllib.recommendation", + "mllib.regression", "mllib.stat", "mllib.tree", "mllib.tree.configuration", + "mllib.tree.impurity", "mllib.tree.model", "mllib.util" + ), + "-group", "Spark SQL", packageList("sql.api.java", "sql.api.java.types", "sql.hive.api.java"), + "-noqualifier", "java.lang" ) ) +} + +object TestSettings { + import BuildCommons._ + + lazy val settings = Seq ( + // Fork new JVMs for tests and set Java options for those + fork := true, + javaOptions in Test += "-Dspark.test.home=" + sparkHome, + javaOptions in Test += "-Dspark.testing=1", + javaOptions in Test += "-Dspark.ports.maxRetries=100", + javaOptions in Test += "-Dspark.ui.port=0", + javaOptions in Test += "-Dsun.io.serialization.extendedDebugInfo=true", + javaOptions in Test ++= System.getProperties.filter(_._1 startsWith "spark") + .map { case (k,v) => s"-D$k=$v" }.toSeq, + javaOptions in Test ++= "-Xmx3g -XX:PermSize=128M -XX:MaxNewSize=256m -XX:MaxPermSize=1g" + .split(" ").toSeq, + javaOptions += "-Xmx3g", - def zeromqSettings() = sharedSettings ++ Seq( - name := "spark-streaming-zeromq", - previousArtifact := sparkPreviousArtifact("spark-streaming-zeromq"), - libraryDependencies ++= Seq( - "org.spark-project.akka" %% "akka-zeromq" % akkaVersion + // Show full stack trace and duration in test cases. + testOptions in Test += Tests.Argument("-oDF"), + testOptions += Tests.Argument(TestFrameworks.JUnit, "-v", "-a"), + // Enable Junit testing. + libraryDependencies += "com.novocode" % "junit-interface" % "0.9" % "test", + // Only allow one test at a time, even across projects, since they run in the same JVM + parallelExecution in Test := false, + concurrentRestrictions in Global += Tags.limit(Tags.Test, 1), + // Remove certain packages from Scaladoc + scalacOptions in (Compile, doc) := Seq( + "-groups", + "-skip-packages", Seq( + "akka", + "org.apache.spark.api.python", + "org.apache.spark.network", + "org.apache.spark.deploy", + "org.apache.spark.util.collection" + ).mkString(":"), + "-doc-title", "Spark " + version.value.replaceAll("-SNAPSHOT", "") + " ScalaDoc" ) ) - def mqttSettings() = streamingSettings ++ Seq( - name := "spark-streaming-mqtt", - previousArtifact := sparkPreviousArtifact("spark-streaming-mqtt"), - libraryDependencies ++= Seq("org.eclipse.paho" % "mqtt-client" % "0.4.0") - ) } diff --git a/project/build.properties b/project/build.properties index bcde13f4362a7..c12ef652adfcb 100644 --- a/project/build.properties +++ b/project/build.properties @@ -14,4 +14,4 @@ # See the License for the specific language governing permissions and # limitations under the License. # -sbt.version=0.13.2 +sbt.version=0.13.5 diff --git a/project/plugins.sbt b/project/plugins.sbt index d3ac4bf335e87..2a61f56c2ea60 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -23,4 +23,6 @@ addSbtPlugin("com.typesafe" % "sbt-mima-plugin" % "0.1.6") addSbtPlugin("com.alpinenow" % "junit_xml_listener" % "0.5.1") -addSbtPlugin("com.eed3si9n" % "sbt-unidoc" % "0.3.0") +addSbtPlugin("com.eed3si9n" % "sbt-unidoc" % "0.3.1") + +addSbtPlugin("com.cavorite" % "sbt-avro" % "0.3.2") diff --git a/project/project/SparkPluginBuild.scala b/project/project/SparkPluginBuild.scala index e9fba641eb8a1..3ef2d5451da0d 100644 --- a/project/project/SparkPluginBuild.scala +++ b/project/project/SparkPluginBuild.scala @@ -24,8 +24,10 @@ import sbt.Keys._ * becomes available for scalastyle sbt plugin. */ object SparkPluginDef extends Build { - lazy val root = Project("plugins", file(".")) dependsOn(sparkStyle) + lazy val root = Project("plugins", file(".")) dependsOn(sparkStyle, sbtPomReader) lazy val sparkStyle = Project("spark-style", file("spark-style"), settings = styleSettings) + lazy val sbtPomReader = uri("https://github.com/ScrapCodes/sbt-pom-reader.git") + // There is actually no need to publish this artifact. def styleSettings = Defaults.defaultSettings ++ Seq ( name := "spark-style", diff --git a/python/epydoc.conf b/python/epydoc.conf index b73860bad8263..51c0faf359939 100644 --- a/python/epydoc.conf +++ b/python/epydoc.conf @@ -35,4 +35,4 @@ private: no exclude: pyspark.cloudpickle pyspark.worker pyspark.join pyspark.java_gateway pyspark.examples pyspark.shell pyspark.tests pyspark.rddsampler pyspark.daemon pyspark.mllib._common - pyspark.mllib.tests + pyspark.mllib.tests pyspark.shuffle diff --git a/python/lib/py4j-0.8.1-src.zip b/python/lib/py4j-0.8.1-src.zip deleted file mode 100644 index 2069a328d1f2e..0000000000000 Binary files a/python/lib/py4j-0.8.1-src.zip and /dev/null differ diff --git a/python/lib/py4j-0.8.2.1-src.zip b/python/lib/py4j-0.8.2.1-src.zip new file mode 100644 index 0000000000000..5203b84d9119e Binary files /dev/null and b/python/lib/py4j-0.8.2.1-src.zip differ diff --git a/python/pyspark/__init__.py b/python/pyspark/__init__.py index 07df8697bd1a8..c58555fc9d2c5 100644 --- a/python/pyspark/__init__.py +++ b/python/pyspark/__init__.py @@ -49,6 +49,16 @@ Main entry point for accessing data stored in Apache Hive.. """ +# The following block allows us to import python's random instead of mllib.random for scripts in +# mllib that depend on top level pyspark packages, which transitively depend on python's random. +# Since Python's import logic looks for modules in the current package first, we eliminate +# mllib.random as a candidate for C{import random} by removing the first search path, the script's +# location, in order to force the loader to look in Python's top-level modules for C{random}. +import sys +s = sys.path.pop(0) +import random +sys.path.insert(0, s) + from pyspark.conf import SparkConf from pyspark.context import SparkContext from pyspark.sql import SQLContext @@ -59,4 +69,5 @@ from pyspark.storagelevel import StorageLevel -__all__ = ["SparkConf", "SparkContext", "SQLContext", "RDD", "SchemaRDD", "SparkFiles", "StorageLevel", "Row"] +__all__ = ["SparkConf", "SparkContext", "SQLContext", "RDD", "SchemaRDD", + "SparkFiles", "StorageLevel", "Row"] diff --git a/python/pyspark/accumulators.py b/python/pyspark/accumulators.py index 2204e9c9ca701..f133cf6f7befc 100644 --- a/python/pyspark/accumulators.py +++ b/python/pyspark/accumulators.py @@ -86,6 +86,7 @@ Exception:... """ +import select import struct import SocketServer import threading @@ -109,6 +110,7 @@ def _deserialize_accumulator(aid, zero_value, accum_param): class Accumulator(object): + """ A shared variable that can be accumulated, i.e., has a commutative and associative "add" operation. Worker tasks on a Spark cluster can add values to an Accumulator with the C{+=} @@ -165,6 +167,7 @@ def __repr__(self): class AccumulatorParam(object): + """ Helper object that defines how to accumulate values of a given type. """ @@ -185,6 +188,7 @@ def addInPlace(self, value1, value2): class AddingAccumulatorParam(AccumulatorParam): + """ An AccumulatorParam that uses the + operators to add values. Designed for simple types such as integers, floats, and lists. Requires the zero value for the underlying type @@ -209,19 +213,42 @@ def addInPlace(self, value1, value2): class _UpdateRequestHandler(SocketServer.StreamRequestHandler): + + """ + This handler will keep polling updates from the same socket until the + server is shutdown. + """ + def handle(self): from pyspark.accumulators import _accumulatorRegistry - num_updates = read_int(self.rfile) - for _ in range(num_updates): - (aid, update) = pickleSer._read_with_length(self.rfile) - _accumulatorRegistry[aid] += update - # Write a byte in acknowledgement - self.wfile.write(struct.pack("!b", 1)) + while not self.server.server_shutdown: + # Poll every 1 second for new data -- don't block in case of shutdown. + r, _, _ = select.select([self.rfile], [], [], 1) + if self.rfile in r: + num_updates = read_int(self.rfile) + for _ in range(num_updates): + (aid, update) = pickleSer._read_with_length(self.rfile) + _accumulatorRegistry[aid] += update + # Write a byte in acknowledgement + self.wfile.write(struct.pack("!b", 1)) + + +class AccumulatorServer(SocketServer.TCPServer): + + """ + A simple TCP server that intercepts shutdown() in order to interrupt + our continuous polling on the handler. + """ + server_shutdown = False + + def shutdown(self): + self.server_shutdown = True + SocketServer.TCPServer.shutdown(self) def _start_update_server(): """Start a TCP server to receive accumulator updates in a daemon thread, and returns it""" - server = SocketServer.TCPServer(("localhost", 0), _UpdateRequestHandler) + server = AccumulatorServer(("localhost", 0), _UpdateRequestHandler) thread = threading.Thread(target=server.serve_forever) thread.daemon = True thread.start() diff --git a/python/pyspark/broadcast.py b/python/pyspark/broadcast.py index 43f40f8783bfd..675a2fcd2ff4e 100644 --- a/python/pyspark/broadcast.py +++ b/python/pyspark/broadcast.py @@ -21,18 +21,16 @@ >>> b = sc.broadcast([1, 2, 3, 4, 5]) >>> b.value [1, 2, 3, 4, 5] - ->>> from pyspark.broadcast import _broadcastRegistry ->>> _broadcastRegistry[b.bid] = b ->>> from cPickle import dumps, loads ->>> loads(dumps(b)).value -[1, 2, 3, 4, 5] - >>> sc.parallelize([0, 0]).flatMap(lambda x: b.value).collect() [1, 2, 3, 4, 5, 1, 2, 3, 4, 5] +>>> b.unpersist() >>> large_broadcast = sc.broadcast(list(range(10000))) """ +import os + +from pyspark.serializers import CompressedSerializer, PickleSerializer + # Holds broadcasted data received from Java, keyed by its id. _broadcastRegistry = {} @@ -45,23 +43,45 @@ def _from_id(bid): class Broadcast(object): + """ A broadcast variable created with L{SparkContext.broadcast()}. Access its value through C{.value}. """ - def __init__(self, bid, value, java_broadcast=None, pickle_registry=None): + def __init__(self, bid, value, java_broadcast=None, + pickle_registry=None, path=None): """ Should not be called directly by users -- use L{SparkContext.broadcast()} instead. """ - self.value = value self.bid = bid + if path is None: + self.value = value self._jbroadcast = java_broadcast self._pickle_registry = pickle_registry + self.path = path + + def unpersist(self, blocking=False): + self._jbroadcast.unpersist(blocking) + os.unlink(self.path) def __reduce__(self): self._pickle_registry.add(self) return (_from_id, (self.bid, )) + + def __getattr__(self, item): + if item == 'value' and self.path is not None: + ser = CompressedSerializer(PickleSerializer()) + value = ser.load_stream(open(self.path)).next() + self.value = value + return value + + raise AttributeError(item) + + +if __name__ == "__main__": + import doctest + doctest.testmod() diff --git a/python/pyspark/cloudpickle.py b/python/pyspark/cloudpickle.py index eb5dbb8de2b39..68062483dedaa 100644 --- a/python/pyspark/cloudpickle.py +++ b/python/pyspark/cloudpickle.py @@ -243,10 +243,10 @@ def save_function(self, obj, name=None, pack=struct.pack): # if func is lambda, def'ed at prompt, is in main, or is nested, then # we'll pickle the actual function object rather than simply saving a # reference (as is done in default pickler), via save_function_tuple. - if islambda(obj) or obj.func_code.co_filename == '' or themodule == None: + if islambda(obj) or obj.func_code.co_filename == '' or themodule is None: #Force server to import modules that have been imported in main modList = None - if themodule == None and not self.savedForceImports: + if themodule is None and not self.savedForceImports: mainmod = sys.modules['__main__'] if useForcedImports and hasattr(mainmod,'___pyc_forcedImports__'): modList = list(mainmod.___pyc_forcedImports__) @@ -560,8 +560,9 @@ class ItemGetterType(ctypes.Structure): ] - itemgetter_obj = ctypes.cast(ctypes.c_void_p(id(obj)), ctypes.POINTER(ItemGetterType)).contents - return self.save_reduce(operator.itemgetter, (itemgetter_obj.item,)) + obj = ctypes.cast(ctypes.c_void_p(id(obj)), ctypes.POINTER(ItemGetterType)).contents + return self.save_reduce(operator.itemgetter, + obj.item if obj.nitems > 1 else (obj.item,)) if PyObject_HEAD: dispatch[operator.itemgetter] = save_itemgetter diff --git a/python/pyspark/conf.py b/python/pyspark/conf.py index 8eff4a242a529..fb716f6753a45 100644 --- a/python/pyspark/conf.py +++ b/python/pyspark/conf.py @@ -30,7 +30,7 @@ u'local' >>> sc.appName u'My app' ->>> sc.sparkHome == None +>>> sc.sparkHome is None True >>> conf = SparkConf(loadDefaults=False) @@ -50,11 +50,13 @@ spark.executorEnv.VAR4=value4 spark.home=/path >>> sorted(conf.getAll(), key=lambda p: p[0]) -[(u'spark.executorEnv.VAR1', u'value1'), (u'spark.executorEnv.VAR3', u'value3'), (u'spark.executorEnv.VAR4', u'value4'), (u'spark.home', u'/path')] +[(u'spark.executorEnv.VAR1', u'value1'), (u'spark.executorEnv.VAR3', u'value3'), \ +(u'spark.executorEnv.VAR4', u'value4'), (u'spark.home', u'/path')] """ class SparkConf(object): + """ Configuration for a Spark application. Used to set various Spark parameters as key-value pairs. @@ -99,6 +101,12 @@ def set(self, key, value): self._jconf.set(key, unicode(value)) return self + def setIfMissing(self, key, value): + """Set a configuration property, if not already set.""" + if self.get(key) is None: + self.set(key, value) + return self + def setMaster(self, value): """Set master URL to connect to.""" self._jconf.setMaster(value) @@ -116,11 +124,11 @@ def setSparkHome(self, value): def setExecutorEnv(self, key=None, value=None, pairs=None): """Set an environment variable to be passed to executors.""" - if (key != None and pairs != None) or (key == None and pairs == None): + if (key is not None and pairs is not None) or (key is None and pairs is None): raise Exception("Either pass one key-value pair or a list of pairs") - elif key != None: + elif key is not None: self._jconf.setExecutorEnv(key, value) - elif pairs != None: + elif pairs is not None: for (k, v) in pairs: self._jconf.setExecutorEnv(k, v) return self @@ -137,7 +145,7 @@ def setAll(self, pairs): def get(self, key, defaultValue=None): """Get the configured value for some key, or return a default otherwise.""" - if defaultValue == None: # Py4J doesn't call the right get() if we pass None + if defaultValue is None: # Py4J doesn't call the right get() if we pass None if not self._jconf.contains(key): return None return self._jconf.get(key) diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 95c54e7a5ad63..a90870ed3a353 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -29,7 +29,7 @@ from pyspark.files import SparkFiles from pyspark.java_gateway import launch_gateway from pyspark.serializers import PickleSerializer, BatchedSerializer, UTF8Deserializer, \ - PairDeserializer + PairDeserializer, CompressedSerializer from pyspark.storagelevel import StorageLevel from pyspark import rdd from pyspark.rdd import RDD @@ -37,7 +37,17 @@ from py4j.java_collections import ListConverter +# These are special default configs for PySpark, they will overwrite +# the default ones for Spark if they are not configured by user. +DEFAULT_CONFIGS = { + "spark.serializer": "org.apache.spark.serializer.KryoSerializer", + "spark.serializer.objectStreamReset": 100, + "spark.rdd.compress": True, +} + + class SparkContext(object): + """ Main entry point for Spark functionality. A SparkContext represents the connection to a Spark cluster, and can be used to create L{RDD}s and @@ -50,12 +60,12 @@ class SparkContext(object): _next_accum_id = 0 _active_spark_context = None _lock = Lock() - _python_includes = None # zip and egg files that need to be added to PYTHONPATH - + _python_includes = None # zip and egg files that need to be added to PYTHONPATH + _default_batch_size_for_serialized_input = 10 def __init__(self, master=None, appName=None, sparkHome=None, pyFiles=None, - environment=None, batchSize=1024, serializer=PickleSerializer(), conf=None, - gateway=None): + environment=None, batchSize=1024, serializer=PickleSerializer(), conf=None, + gateway=None): """ Create a new SparkContext. At least the master and app name should be set, either through the named parameters here or through C{conf}. @@ -75,7 +85,7 @@ def __init__(self, master=None, appName=None, sparkHome=None, pyFiles=None, @param serializer: The serializer for RDDs. @param conf: A L{SparkConf} object setting Spark properties. @param gateway: Use an existing gateway and JVM, otherwise a new JVM - will be instatiated. + will be instantiated. >>> from pyspark.context import SparkContext @@ -92,7 +102,16 @@ def __init__(self, master=None, appName=None, sparkHome=None, pyFiles=None, tempNamedTuple = namedtuple("Callsite", "function file linenum") self._callsite = tempNamedTuple(function=None, file=None, linenum=None) SparkContext._ensure_initialized(self, gateway=gateway) - + try: + self._do_init(master, appName, sparkHome, pyFiles, environment, batchSize, serializer, + conf) + except: + # If an error occurs, clean up in order to allow future SparkContext creation: + self.stop() + raise + + def _do_init(self, master, appName, sparkHome, pyFiles, environment, batchSize, serializer, + conf): self.environment = environment or {} self._conf = conf or SparkConf(_jvm=self._jvm) self._batchSize = batchSize # -1 represents an unlimited batch size @@ -113,6 +132,8 @@ def __init__(self, master=None, appName=None, sparkHome=None, pyFiles=None, if environment: for key, value in environment.iteritems(): self._conf.setExecutorEnv(key, value) + for key, value in DEFAULT_CONFIGS.items(): + self._conf.setIfMissing(key, value) # Check that we have at least the required parameters if not self._conf.contains("spark.master"): @@ -138,8 +159,8 @@ def __init__(self, master=None, appName=None, sparkHome=None, pyFiles=None, self._accumulatorServer = accumulators._start_update_server() (host, port) = self._accumulatorServer.server_address self._javaAccumulator = self._jsc.accumulator( - self._jvm.java.util.ArrayList(), - self._jvm.PythonAccumulatorParam(host, port)) + self._jvm.java.util.ArrayList(), + self._jvm.PythonAccumulatorParam(host, port)) self.pythonExec = os.environ.get("PYSPARK_PYTHON", 'python') @@ -165,7 +186,7 @@ def __init__(self, master=None, appName=None, sparkHome=None, pyFiles=None, (dirname, filename) = os.path.split(path) self._python_includes.append(filename) sys.path.append(path) - if not dirname in sys.path: + if dirname not in sys.path: sys.path.append(dirname) # Create a temporary directory inside spark.local.dir: @@ -192,15 +213,19 @@ def _ensure_initialized(cls, instance=None, gateway=None): SparkContext._writeToFile = SparkContext._jvm.PythonRDD.writeToFile if instance: - if SparkContext._active_spark_context and SparkContext._active_spark_context != instance: + if (SparkContext._active_spark_context and + SparkContext._active_spark_context != instance): currentMaster = SparkContext._active_spark_context.master currentAppName = SparkContext._active_spark_context.appName callsite = SparkContext._active_spark_context._callsite # Raise error if there is already a running Spark context - raise ValueError("Cannot run multiple SparkContexts at once; existing SparkContext(app=%s, master=%s)" \ - " created by %s at %s:%s " \ - % (currentAppName, currentMaster, callsite.function, callsite.file, callsite.linenum)) + raise ValueError( + "Cannot run multiple SparkContexts at once; " + "existing SparkContext(app=%s, master=%s)" + " created by %s at %s:%s " + % (currentAppName, currentMaster, + callsite.function, callsite.file, callsite.linenum)) else: SparkContext._active_spark_context = instance @@ -213,6 +238,13 @@ def setSystemProperty(cls, key, value): SparkContext._ensure_initialized() SparkContext._jvm.java.lang.System.setProperty(key, value) + @property + def version(self): + """ + The version of Spark on which this application is running. + """ + return self._jsc.version() + @property def defaultParallelism(self): """ @@ -228,17 +260,14 @@ def defaultMinPartitions(self): """ return self._jsc.sc().defaultMinPartitions() - def __del__(self): - self.stop() - def stop(self): """ Shut down the SparkContext. """ - if self._jsc: + if getattr(self, "_jsc", None): self._jsc.stop() self._jsc = None - if self._accumulatorServer: + if getattr(self, "_accumulatorServer", None): self._accumulatorServer.shutdown() self._accumulatorServer = None with SparkContext._lock: @@ -290,7 +319,7 @@ def textFile(self, name, minPartitions=None): Read a text file from HDFS, a local file system (available on all nodes), or any Hadoop-supported file system URI, and return it as an RDD of Strings. - + >>> path = os.path.join(tempdir, "sample-text.txt") >>> with open(path, "w") as testFile: ... testFile.write("Hello world!") @@ -351,7 +380,7 @@ def _dictToJavaMap(self, d): return jm def sequenceFile(self, path, keyClass=None, valueClass=None, keyConverter=None, - valueConverter=None, minSplits=None): + valueConverter=None, minSplits=None, batchSize=None): """ Read a Hadoop SequenceFile with arbitrary key and value Writable class from HDFS, a local file system (available on all nodes), or any Hadoop-supported file system URI. @@ -371,14 +400,18 @@ def sequenceFile(self, path, keyClass=None, valueClass=None, keyConverter=None, @param valueConverter: @param minSplits: minimum splits in dataset (default min(2, sc.defaultParallelism)) + @param batchSize: The number of Python objects represented as a single + Java object. (default sc._default_batch_size_for_serialized_input) """ minSplits = minSplits or min(self.defaultParallelism, 2) + batchSize = max(1, batchSize or self._default_batch_size_for_serialized_input) + ser = BatchedSerializer(PickleSerializer()) if (batchSize > 1) else PickleSerializer() jrdd = self._jvm.PythonRDD.sequenceFile(self._jsc, path, keyClass, valueClass, - keyConverter, valueConverter, minSplits) - return RDD(jrdd, self, PickleSerializer()) + keyConverter, valueConverter, minSplits, batchSize) + return RDD(jrdd, self, ser) def newAPIHadoopFile(self, path, inputFormatClass, keyClass, valueClass, keyConverter=None, - valueConverter=None, conf=None): + valueConverter=None, conf=None, batchSize=None): """ Read a 'new API' Hadoop InputFormat with arbitrary key and value class from HDFS, a local file system (available on all nodes), or any Hadoop-supported file system URI. @@ -398,14 +431,19 @@ def newAPIHadoopFile(self, path, inputFormatClass, keyClass, valueClass, keyConv @param valueConverter: (None by default) @param conf: Hadoop configuration, passed in as a dict (None by default) + @param batchSize: The number of Python objects represented as a single + Java object. (default sc._default_batch_size_for_serialized_input) """ jconf = self._dictToJavaMap(conf) + batchSize = max(1, batchSize or self._default_batch_size_for_serialized_input) + ser = BatchedSerializer(PickleSerializer()) if (batchSize > 1) else PickleSerializer() jrdd = self._jvm.PythonRDD.newAPIHadoopFile(self._jsc, path, inputFormatClass, keyClass, - valueClass, keyConverter, valueConverter, jconf) - return RDD(jrdd, self, PickleSerializer()) + valueClass, keyConverter, valueConverter, + jconf, batchSize) + return RDD(jrdd, self, ser) def newAPIHadoopRDD(self, inputFormatClass, keyClass, valueClass, keyConverter=None, - valueConverter=None, conf=None): + valueConverter=None, conf=None, batchSize=None): """ Read a 'new API' Hadoop InputFormat with arbitrary key and value class, from an arbitrary Hadoop configuration, which is passed in as a Python dict. @@ -422,14 +460,19 @@ def newAPIHadoopRDD(self, inputFormatClass, keyClass, valueClass, keyConverter=N @param valueConverter: (None by default) @param conf: Hadoop configuration, passed in as a dict (None by default) + @param batchSize: The number of Python objects represented as a single + Java object. (default sc._default_batch_size_for_serialized_input) """ jconf = self._dictToJavaMap(conf) + batchSize = max(1, batchSize or self._default_batch_size_for_serialized_input) + ser = BatchedSerializer(PickleSerializer()) if (batchSize > 1) else PickleSerializer() jrdd = self._jvm.PythonRDD.newAPIHadoopRDD(self._jsc, inputFormatClass, keyClass, - valueClass, keyConverter, valueConverter, jconf) - return RDD(jrdd, self, PickleSerializer()) + valueClass, keyConverter, valueConverter, + jconf, batchSize) + return RDD(jrdd, self, ser) def hadoopFile(self, path, inputFormatClass, keyClass, valueClass, keyConverter=None, - valueConverter=None, conf=None): + valueConverter=None, conf=None, batchSize=None): """ Read an 'old' Hadoop InputFormat with arbitrary key and value class from HDFS, a local file system (available on all nodes), or any Hadoop-supported file system URI. @@ -449,14 +492,19 @@ def hadoopFile(self, path, inputFormatClass, keyClass, valueClass, keyConverter= @param valueConverter: (None by default) @param conf: Hadoop configuration, passed in as a dict (None by default) + @param batchSize: The number of Python objects represented as a single + Java object. (default sc._default_batch_size_for_serialized_input) """ jconf = self._dictToJavaMap(conf) + batchSize = max(1, batchSize or self._default_batch_size_for_serialized_input) + ser = BatchedSerializer(PickleSerializer()) if (batchSize > 1) else PickleSerializer() jrdd = self._jvm.PythonRDD.hadoopFile(self._jsc, path, inputFormatClass, keyClass, - valueClass, keyConverter, valueConverter, jconf) - return RDD(jrdd, self, PickleSerializer()) + valueClass, keyConverter, valueConverter, + jconf, batchSize) + return RDD(jrdd, self, ser) def hadoopRDD(self, inputFormatClass, keyClass, valueClass, keyConverter=None, - valueConverter=None, conf=None): + valueConverter=None, conf=None, batchSize=None): """ Read an 'old' Hadoop InputFormat with arbitrary key and value class, from an arbitrary Hadoop configuration, which is passed in as a Python dict. @@ -473,11 +521,16 @@ def hadoopRDD(self, inputFormatClass, keyClass, valueClass, keyConverter=None, @param valueConverter: (None by default) @param conf: Hadoop configuration, passed in as a dict (None by default) + @param batchSize: The number of Python objects represented as a single + Java object. (default sc._default_batch_size_for_serialized_input) """ jconf = self._dictToJavaMap(conf) - jrdd = self._jvm.PythonRDD.hadoopRDD(self._jsc, inputFormatClass, keyClass, valueClass, - keyConverter, valueConverter, jconf) - return RDD(jrdd, self, PickleSerializer()) + batchSize = max(1, batchSize or self._default_batch_size_for_serialized_input) + ser = BatchedSerializer(PickleSerializer()) if (batchSize > 1) else PickleSerializer() + jrdd = self._jvm.PythonRDD.hadoopRDD(self._jsc, inputFormatClass, keyClass, + valueClass, keyConverter, valueConverter, + jconf, batchSize) + return RDD(jrdd, self, ser) def _checkpointFile(self, name, input_deserializer): jrdd = self._jsc.checkpointFile(name) @@ -507,21 +560,25 @@ def union(self, rdds): first = rdds[0]._jrdd rest = [x._jrdd for x in rdds[1:]] rest = ListConverter().convert(rest, self._gateway._gateway_client) - return RDD(self._jsc.union(first, rest), self, - rdds[0]._jrdd_deserializer) + return RDD(self._jsc.union(first, rest), self, rdds[0]._jrdd_deserializer) def broadcast(self, value): """ Broadcast a read-only variable to the cluster, returning a L{Broadcast} - object for reading it in distributed functions. The variable will be - sent to each cluster only once. + object for reading it in distributed functions. The variable will + be sent to each cluster only once. + + :keep: Keep the `value` in driver or not. """ - pickleSer = PickleSerializer() - pickled = pickleSer.dumps(value) - jbroadcast = self._jsc.broadcast(bytearray(pickled)) - return Broadcast(jbroadcast.id(), value, jbroadcast, - self._pickled_broadcast_vars) + ser = CompressedSerializer(PickleSerializer()) + # pass large object by py4j is very slow and need much memory + tempFile = NamedTemporaryFile(delete=False, dir=self._temp_dir) + ser.dump_stream([value], tempFile) + tempFile.close() + jbroadcast = self._jvm.PythonRDD.readBroadcastFromFile(self._jsc, tempFile.name) + return Broadcast(jbroadcast.id(), None, jbroadcast, + self._pickled_broadcast_vars, tempFile.name) def accumulator(self, value, accum_param=None): """ @@ -562,7 +619,7 @@ def addFile(self, path): >>> def func(iterator): ... with open(SparkFiles.get("test.txt")) as testFile: ... fileVal = int(testFile.readline()) - ... return [x * 100 for x in iterator] + ... return [x * fileVal for x in iterator] >>> sc.parallelize([1, 2, 3, 4]).mapPartitions(func).collect() [100, 200, 300, 400] """ @@ -584,11 +641,12 @@ def addPyFile(self, path): HTTP, HTTPS or FTP URI. """ self.addFile(path) - (dirname, filename) = os.path.split(path) # dirname may be directory or HDFS/S3 prefix + (dirname, filename) = os.path.split(path) # dirname may be directory or HDFS/S3 prefix if filename.endswith('.zip') or filename.endswith('.ZIP') or filename.endswith('.egg'): self._python_includes.append(filename) - sys.path.append(os.path.join(SparkFiles.getRootDirectory(), filename)) # for tests in local mode + # for tests in local mode + sys.path.append(os.path.join(SparkFiles.getRootDirectory(), filename)) def setCheckpointDir(self, dirName): """ @@ -649,9 +707,9 @@ def setJobGroup(self, groupId, description, interruptOnCancel=False): Cancelled If interruptOnCancel is set to true for the job group, then job cancellation will result - in Thread.interrupt() being called on the job's executor threads. This is useful to help ensure - that the tasks are actually stopped in a timely manner, but is off by default due to HDFS-1208, - where HDFS may respond to Thread.interrupt() by marking nodes as dead. + in Thread.interrupt() being called on the job's executor threads. This is useful to help + ensure that the tasks are actually stopped in a timely manner, but is off by default due + to HDFS-1208, where HDFS may respond to Thread.interrupt() by marking nodes as dead. """ self._jsc.setJobGroup(groupId, description, interruptOnCancel) @@ -688,7 +746,7 @@ def cancelAllJobs(self): """ self._jsc.sc().cancelAllJobs() - def runJob(self, rdd, partitionFunc, partitions = None, allowLocal = False): + def runJob(self, rdd, partitionFunc, partitions=None, allowLocal=False): """ Executes the given partitionFunc on the specified set of partitions, returning the result as an array of elements. @@ -703,7 +761,7 @@ def runJob(self, rdd, partitionFunc, partitions = None, allowLocal = False): >>> sc.runJob(myRDD, lambda part: [x * x for x in part], [0, 2], True) [0, 1, 16, 25] """ - if partitions == None: + if partitions is None: partitions = range(rdd._jrdd.partitions().size()) javaPartitions = ListConverter().convert(partitions, self._gateway._gateway_client) @@ -714,6 +772,7 @@ def runJob(self, rdd, partitionFunc, partitions = None, allowLocal = False): it = self._jvm.PythonRDD.runJob(self._jsc.sc(), mappedRDD._jrdd, javaPartitions, allowLocal) return list(mappedRDD._collect_iterator_through_file(it)) + def _test(): import atexit import doctest diff --git a/python/pyspark/daemon.py b/python/pyspark/daemon.py index 5eb1c63bf206b..22ab8d30c0ae3 100644 --- a/python/pyspark/daemon.py +++ b/python/pyspark/daemon.py @@ -15,64 +15,40 @@ # limitations under the License. # +import numbers import os import signal +import select import socket import sys import traceback -import multiprocessing -from ctypes import c_bool -from errno import EINTR, ECHILD +import time +from errno import EINTR, ECHILD, EAGAIN from socket import AF_INET, SOCK_STREAM, SOMAXCONN from signal import SIGHUP, SIGTERM, SIGCHLD, SIG_DFL, SIG_IGN from pyspark.worker import main as worker_main -from pyspark.serializers import write_int - -try: - POOLSIZE = multiprocessing.cpu_count() -except NotImplementedError: - POOLSIZE = 4 - -exit_flag = multiprocessing.Value(c_bool, False) - - -def should_exit(): - global exit_flag - return exit_flag.value +from pyspark.serializers import read_int, write_int def compute_real_exit_code(exit_code): - # SystemExit's code can be integer or string, but os._exit only accepts integers - import numbers - if isinstance(exit_code, numbers.Integral): - return exit_code - else: - return 1 + # SystemExit's code can be integer or string, but os._exit only accepts integers + if isinstance(exit_code, numbers.Integral): + return exit_code + else: + return 1 -def worker(listen_sock): +def worker(sock): + """ + Called by a worker process after the fork(). + """ # Redirect stdout to stderr os.dup2(2, 1) - sys.stdout = sys.stderr # The sys.stdout object is different from file descriptor 1 + sys.stdout = sys.stderr # The sys.stdout object is different from file descriptor 1 - # Manager sends SIGHUP to request termination of workers in the pool - def handle_sighup(*args): - assert should_exit() - signal.signal(SIGHUP, handle_sighup) - - # Cleanup zombie children - def handle_sigchld(*args): - pid = status = None - try: - while (pid, status) != (0, 0): - pid, status = os.waitpid(0, os.WNOHANG) - except EnvironmentError as err: - if err.errno == EINTR: - # retry - handle_sigchld() - elif err.errno != ECHILD: - raise - signal.signal(SIGCHLD, handle_sigchld) + signal.signal(SIGHUP, SIG_DFL) + signal.signal(SIGCHLD, SIG_DFL) + signal.signal(SIGTERM, SIG_DFL) # Blocks until the socket is closed by draining the input stream # until it raises an exception or returns EOF. @@ -85,55 +61,35 @@ def waitSocketClose(sock): except: pass - # Handle clients - while not should_exit(): - # Wait until a client arrives or we have to exit - sock = None - while not should_exit() and sock is None: - try: - sock, addr = listen_sock.accept() - except EnvironmentError as err: - if err.errno != EINTR: - raise + # Read the socket using fdopen instead of socket.makefile() because the latter + # seems to be very slow; note that we need to dup() the file descriptor because + # otherwise writes also cause a seek that makes us miss data on the read side. + infile = os.fdopen(os.dup(sock.fileno()), "a+", 65536) + outfile = os.fdopen(os.dup(sock.fileno()), "a+", 65536) + exit_code = 0 + try: + # Acknowledge that the fork was successful + write_int(os.getpid(), outfile) + outfile.flush() + worker_main(infile, outfile) + except SystemExit as exc: + exit_code = exc.code + finally: + outfile.flush() + # The Scala side will close the socket upon task completion. + waitSocketClose(sock) + os._exit(compute_real_exit_code(exit_code)) - if sock is not None: - # Fork a child to handle the client. - # The client is handled in the child so that the manager - # never receives SIGCHLD unless a worker crashes. - if os.fork() == 0: - # Leave the worker pool - signal.signal(SIGHUP, SIG_DFL) - signal.signal(SIGCHLD, SIG_DFL) - listen_sock.close() - # Read the socket using fdopen instead of socket.makefile() because the latter - # seems to be very slow; note that we need to dup() the file descriptor because - # otherwise writes also cause a seek that makes us miss data on the read side. - infile = os.fdopen(os.dup(sock.fileno()), "a+", 65536) - outfile = os.fdopen(os.dup(sock.fileno()), "a+", 65536) - exit_code = 0 - try: - worker_main(infile, outfile) - except SystemExit as exc: - exit_code = exc.code - finally: - outfile.flush() - # The Scala side will close the socket upon task completion. - waitSocketClose(sock) - os._exit(compute_real_exit_code(exit_code)) - else: - sock.close() - - -def launch_worker(listen_sock): - if os.fork() == 0: - try: - worker(listen_sock) - except Exception as err: - traceback.print_exc() - os._exit(1) - else: - assert should_exit() - os._exit(0) + +# Cleanup zombie children +def cleanup_dead_children(): + try: + while True: + pid, _ = os.waitpid(0, os.WNOHANG) + if not pid: + break + except: + pass def manager(): @@ -143,51 +99,85 @@ def manager(): # Create a listening socket on the AF_INET loopback interface listen_sock = socket.socket(AF_INET, SOCK_STREAM) listen_sock.bind(('127.0.0.1', 0)) - listen_sock.listen(max(1024, 2 * POOLSIZE, SOMAXCONN)) + listen_sock.listen(max(1024, SOMAXCONN)) listen_host, listen_port = listen_sock.getsockname() write_int(listen_port, sys.stdout) - # Launch initial worker pool - for idx in range(POOLSIZE): - launch_worker(listen_sock) - listen_sock.close() - - def shutdown(): - global exit_flag - exit_flag.value = True - - # Gracefully exit on SIGTERM, don't die on SIGHUP - signal.signal(SIGTERM, lambda signum, frame: shutdown()) - signal.signal(SIGHUP, SIG_IGN) + def shutdown(code): + signal.signal(SIGTERM, SIG_DFL) + # Send SIGHUP to notify workers of shutdown + os.kill(0, SIGHUP) + exit(code) - # Cleanup zombie children - def handle_sigchld(*args): - try: - pid, status = os.waitpid(0, os.WNOHANG) - if status != 0 and not should_exit(): - raise RuntimeError("worker crashed: %s, %s" % (pid, status)) - except EnvironmentError as err: - if err.errno not in (ECHILD, EINTR): - raise - signal.signal(SIGCHLD, handle_sigchld) + def handle_sigterm(*args): + shutdown(1) + signal.signal(SIGTERM, handle_sigterm) # Gracefully exit on SIGTERM + signal.signal(SIGHUP, SIG_IGN) # Don't die on SIGHUP # Initialization complete sys.stdout.close() try: - while not should_exit(): + while True: try: - # Spark tells us to exit by closing stdin - if os.read(0, 512) == '': - shutdown() - except EnvironmentError as err: - if err.errno != EINTR: - shutdown() + ready_fds = select.select([0, listen_sock], [], [], 1)[0] + except select.error as ex: + if ex[0] == EINTR: + continue + else: + raise + + # cleanup in signal handler will cause deadlock + cleanup_dead_children() + + if 0 in ready_fds: + try: + worker_pid = read_int(sys.stdin) + except EOFError: + # Spark told us to exit by closing stdin + shutdown(0) + try: + os.kill(worker_pid, signal.SIGKILL) + except OSError: + pass # process already died + + if listen_sock in ready_fds: + try: + sock, _ = listen_sock.accept() + except OSError as e: + if e.errno == EINTR: + continue raise + + # Launch a worker process + try: + pid = os.fork() + except OSError as e: + if e.errno in (EAGAIN, EINTR): + time.sleep(1) + pid = os.fork() # error here will shutdown daemon + else: + outfile = sock.makefile('w') + write_int(e.errno, outfile) # Signal that the fork failed + outfile.flush() + outfile.close() + sock.close() + continue + + if pid == 0: + # in child process + listen_sock.close() + try: + worker(sock) + except: + traceback.print_exc() + os._exit(1) + else: + os._exit(0) + else: + sock.close() + finally: - signal.signal(SIGTERM, SIG_DFL) - exit_flag.value = True - # Send SIGHUP to notify workers of shutdown - os.kill(0, SIGHUP) + shutdown(1) if __name__ == '__main__': diff --git a/python/pyspark/files.py b/python/pyspark/files.py index 57ee14eeb7776..331de9a9b2212 100644 --- a/python/pyspark/files.py +++ b/python/pyspark/files.py @@ -19,6 +19,7 @@ class SparkFiles(object): + """ Resolves paths to files added through L{SparkContext.addFile()}. diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py index 0dbead4415b02..099cc1db0ba52 100644 --- a/python/pyspark/java_gateway.py +++ b/python/pyspark/java_gateway.py @@ -24,6 +24,7 @@ from threading import Thread from py4j.java_gateway import java_import, JavaGateway, GatewayClient + def launch_gateway(): SPARK_HOME = os.environ["SPARK_HOME"] @@ -38,7 +39,7 @@ def launch_gateway(): submit_args = os.environ.get("PYSPARK_SUBMIT_ARGS") submit_args = submit_args if submit_args is not None else "" submit_args = shlex.split(submit_args) - command = [os.path.join(SPARK_HOME, script), "pyspark-shell"] + submit_args + command = [os.path.join(SPARK_HOME, script)] + submit_args + ["pyspark-shell"] if not on_windows: # Don't send ctrl-c / SIGINT to the Java gateway: def preexec_func(): @@ -56,7 +57,7 @@ def preexec_func(): (stdout, _) = proc.communicate() exit_code = proc.poll() error_msg = "Launching GatewayServer failed" - error_msg += " with exit code %d!" % exit_code if exit_code else "! " + error_msg += " with exit code %d! " % exit_code if exit_code else "! " error_msg += "(Warning: unexpected output detected.)\n\n" error_msg += gateway_port + stdout raise Exception(error_msg) @@ -64,6 +65,7 @@ def preexec_func(): # Create a thread to echo output from the GatewayServer, which is required # for Java log output to show up: class EchoOutputThread(Thread): + def __init__(self, stream): Thread.__init__(self) self.daemon = True @@ -82,6 +84,9 @@ def run(self): java_import(gateway.jvm, "org.apache.spark.SparkConf") java_import(gateway.jvm, "org.apache.spark.api.java.*") java_import(gateway.jvm, "org.apache.spark.api.python.*") + java_import(gateway.jvm, "org.apache.spark.streaming.api.java.*") + java_import(gateway.jvm, "org.apache.spark.streaming.api.python.*") + java_import(gateway.jvm, "org.apache.spark.streaming.*") java_import(gateway.jvm, "org.apache.spark.mllib.api.python.*") java_import(gateway.jvm, "org.apache.spark.sql.SQLContext") java_import(gateway.jvm, "org.apache.spark.sql.hive.HiveContext") diff --git a/python/pyspark/join.py b/python/pyspark/join.py index 5f3a7e71f7866..b0f1cc1927066 100644 --- a/python/pyspark/join.py +++ b/python/pyspark/join.py @@ -33,10 +33,11 @@ from pyspark.resultiterable import ResultIterable + def _do_python_join(rdd, other, numPartitions, dispatch): vs = rdd.map(lambda (k, v): (k, (1, v))) ws = other.map(lambda (k, v): (k, (2, v))) - return vs.union(ws).groupByKey(numPartitions).flatMapValues(lambda x : dispatch(x.__iter__())) + return vs.union(ws).groupByKey(numPartitions).flatMapValues(lambda x: dispatch(x.__iter__())) def python_join(rdd, other, numPartitions): @@ -85,6 +86,7 @@ def make_mapper(i): vrdds = [rdd.map(make_mapper(i)) for i, rdd in enumerate(rdds)] union_vrdds = reduce(lambda acc, other: acc.union(other), vrdds) rdd_len = len(vrdds) + def dispatch(seq): bufs = [[] for i in range(rdd_len)] for (n, v) in seq: diff --git a/python/pyspark/mllib/_common.py b/python/pyspark/mllib/_common.py index e609b60a0f968..bb60d3d0c8463 100644 --- a/python/pyspark/mllib/_common.py +++ b/python/pyspark/mllib/_common.py @@ -16,6 +16,7 @@ # import struct +import sys import numpy from numpy import ndarray, float64, int64, int32, array_equal, array from pyspark import SparkContext, RDD @@ -78,6 +79,14 @@ LABELED_POINT_MAGIC = 4 +# Workaround for SPARK-2954: before Python 2.7, struct.unpack couldn't unpack bytearray()s. +if sys.version_info[:2] <= (2, 6): + def _unpack(fmt, string): + return struct.unpack(fmt, buffer(string)) +else: + _unpack = struct.unpack + + def _deserialize_numpy_array(shape, ba, offset, dtype=float64): """ Deserialize a numpy array of the given type from an offset in @@ -97,8 +106,28 @@ def _deserialize_numpy_array(shape, ba, offset, dtype=float64): return ar.copy() +def _serialize_double(d): + """ + Serialize a double (float or numpy.float64) into a mutually understood format. + """ + if type(d) == float or type(d) == float64 or type(d) == int or type(d) == long: + d = float64(d) + ba = bytearray(8) + _copyto(d, buffer=ba, offset=0, shape=[1], dtype=float64) + return ba + else: + raise TypeError("_serialize_double called on non-float input") + + def _serialize_double_vector(v): - """Serialize a double vector into a mutually understood format. + """ + Serialize a double vector into a mutually understood format. + + Note: we currently do not use a magic byte for double for storage + efficiency. This should be reconsidered when we add Ser/De for other + 8-byte types (e.g. Long), for safety. The corresponding deserializer, + _deserialize_double, needs to be modified as well if the serialization + scheme changes. >>> x = array([1,2,3]) >>> y = _deserialize_double_vector(_serialize_double_vector(x)) @@ -148,6 +177,32 @@ def _serialize_sparse_vector(v): return ba +def _deserialize_double(ba, offset=0): + """Deserialize a double from a mutually understood format. + + >>> import sys + >>> _deserialize_double(_serialize_double(123.0)) == 123.0 + True + >>> _deserialize_double(_serialize_double(float64(0.0))) == 0.0 + True + >>> _deserialize_double(_serialize_double(1)) == 1.0 + True + >>> _deserialize_double(_serialize_double(1L)) == 1.0 + True + >>> x = sys.float_info.max + >>> _deserialize_double(_serialize_double(sys.float_info.max)) == x + True + >>> y = float64(sys.float_info.max) + >>> _deserialize_double(_serialize_double(sys.float_info.max)) == y + True + """ + if type(ba) != bytearray: + raise TypeError("_deserialize_double called on a %s; wanted bytearray" % type(ba)) + if len(ba) - offset != 8: + raise TypeError("_deserialize_double called on a %d-byte array; wanted 8 bytes." % nb) + return _unpack("d", ba[offset:])[0] + + def _deserialize_double_vector(ba, offset=0): """Deserialize a double vector from a mutually understood format. @@ -164,7 +219,7 @@ def _deserialize_double_vector(ba, offset=0): nb = len(ba) - offset if nb < 5: raise TypeError("_deserialize_double_vector called on a %d-byte array, " - "which is too short" % nb) + "which is too short" % nb) if ba[offset] == DENSE_VECTOR_MAGIC: return _deserialize_dense_vector(ba, offset) elif ba[offset] == SPARSE_VECTOR_MAGIC: @@ -272,6 +327,7 @@ def _serialize_labeled_point(p): header_float[0] = p.label return header + serialized_features + def _deserialize_labeled_point(ba, offset=0): """Deserialize a LabeledPoint from a mutually understood format.""" from pyspark.mllib.regression import LabeledPoint @@ -283,6 +339,7 @@ def _deserialize_labeled_point(ba, offset=0): features = _deserialize_double_vector(ba, offset + 9) return LabeledPoint(label, features) + def _copyto(array, buffer, offset, shape, dtype): """ Copy the contents of a vector to a destination bytearray at the @@ -295,22 +352,35 @@ def _copyto(array, buffer, offset, shape, dtype): temp_array[...] = array -def _get_unmangled_rdd(data, serializer): +def _get_unmangled_rdd(data, serializer, cache=True): + """ + :param cache: If True, the serialized RDD is cached. (default = True) + WARNING: Users should unpersist() this later! + """ dataBytes = data.map(serializer) dataBytes._bypass_serializer = True - dataBytes.cache() # TODO: users should unpersist() this later! + if cache: + dataBytes.cache() return dataBytes -# Map a pickled Python RDD of Python dense or sparse vectors to a Java RDD of -# _serialized_double_vectors -def _get_unmangled_double_vector_rdd(data): - return _get_unmangled_rdd(data, _serialize_double_vector) +def _get_unmangled_double_vector_rdd(data, cache=True): + """ + Map a pickled Python RDD of Python dense or sparse vectors to a Java RDD of + _serialized_double_vectors. + :param cache: If True, the serialized RDD is cached. (default = True) + WARNING: Users should unpersist() this later! + """ + return _get_unmangled_rdd(data, _serialize_double_vector, cache) -# Map a pickled Python RDD of LabeledPoint to a Java RDD of _serialized_labeled_points -def _get_unmangled_labeled_point_rdd(data): - return _get_unmangled_rdd(data, _serialize_labeled_point) +def _get_unmangled_labeled_point_rdd(data, cache=True): + """ + Map a pickled Python RDD of LabeledPoint to a Java RDD of _serialized_labeled_points. + :param cache: If True, the serialized RDD is cached. (default = True) + WARNING: Users should unpersist() this later! + """ + return _get_unmangled_rdd(data, _serialize_labeled_point, cache) # Common functions for dealing with and training linear models @@ -332,7 +402,7 @@ def _linear_predictor_typecheck(x, coeffs): if x.size != coeffs.shape[0]: raise RuntimeError("Got sparse vector of size %d; wanted %d" % ( x.size, coeffs.shape[0])) - elif (type(x) == RDD): + elif isinstance(x, RDD): raise RuntimeError("Bulk predict not yet supported.") else: raise TypeError("Argument of type " + type(x).__name__ + " unsupported") @@ -382,6 +452,7 @@ def _serialize_rating(r): class RatingDeserializer(Serializer): + def loads(self, stream): length = struct.unpack("!i", stream.read(4))[0] ba = stream.read(length) diff --git a/python/pyspark/mllib/classification.py b/python/pyspark/mllib/classification.py index 1c0c536c4fb3d..ffdda7ee19302 100644 --- a/python/pyspark/mllib/classification.py +++ b/python/pyspark/mllib/classification.py @@ -31,6 +31,7 @@ class LogisticRegressionModel(LinearModel): + """A linear binary classification model derived from logistic regression. >>> data = [ @@ -60,25 +61,57 @@ class LogisticRegressionModel(LinearModel): >>> lrm.predict(SparseVector(2, {1: 0.0})) <= 0 True """ + def predict(self, x): _linear_predictor_typecheck(x, self._coeff) margin = _dot(x, self._coeff) + self._intercept - prob = 1/(1 + exp(-margin)) + if margin > 0: + prob = 1 / (1 + exp(-margin)) + else: + exp_margin = exp(margin) + prob = exp_margin / (1 + exp_margin) return 1 if prob > 0.5 else 0 class LogisticRegressionWithSGD(object): + @classmethod - def train(cls, data, iterations=100, step=1.0, miniBatchFraction=1.0, initialWeights=None): - """Train a logistic regression model on the given data.""" + def train(cls, data, iterations=100, step=1.0, miniBatchFraction=1.0, + initialWeights=None, regParam=1.0, regType=None, intercept=False): + """ + Train a logistic regression model on the given data. + + @param data: The training data. + @param iterations: The number of iterations (default: 100). + @param step: The step parameter used in SGD + (default: 1.0). + @param miniBatchFraction: Fraction of data to be used for each SGD + iteration. + @param initialWeights: The initial weights (default: None). + @param regParam: The regularizer parameter (default: 1.0). + @param regType: The type of regularizer used for training + our model. + Allowed values: "l1" for using L1Updater, + "l2" for using + SquaredL2Updater, + "none" for no regularizer. + (default: "none") + @param intercept: Boolean parameter which indicates the use + or not of the augmented representation for + training data (i.e. whether bias features + are activated or not). + """ sc = data.context + if regType is None: + regType = "none" train_func = lambda d, i: sc._jvm.PythonMLLibAPI().trainLogisticRegressionModelWithSGD( - d._jrdd, iterations, step, miniBatchFraction, i) + d._jrdd, iterations, step, miniBatchFraction, i, regParam, regType, intercept) return _regression_train_wrapper(sc, train_func, LogisticRegressionModel, data, initialWeights) class SVMModel(LinearModel): + """A support vector machine. >>> data = [ @@ -102,6 +135,7 @@ class SVMModel(LinearModel): >>> svm.predict(SparseVector(2, {0: -1.0})) <= 0 True """ + def predict(self, x): _linear_predictor_typecheck(x, self._coeff) margin = _dot(x, self._coeff) + self._intercept @@ -109,17 +143,43 @@ def predict(self, x): class SVMWithSGD(object): + @classmethod def train(cls, data, iterations=100, step=1.0, regParam=1.0, - miniBatchFraction=1.0, initialWeights=None): - """Train a support vector machine on the given data.""" + miniBatchFraction=1.0, initialWeights=None, regType=None, intercept=False): + """ + Train a support vector machine on the given data. + + @param data: The training data. + @param iterations: The number of iterations (default: 100). + @param step: The step parameter used in SGD + (default: 1.0). + @param regParam: The regularizer parameter (default: 1.0). + @param miniBatchFraction: Fraction of data to be used for each SGD + iteration. + @param initialWeights: The initial weights (default: None). + @param regType: The type of regularizer used for training + our model. + Allowed values: "l1" for using L1Updater, + "l2" for using + SquaredL2Updater, + "none" for no regularizer. + (default: "none") + @param intercept: Boolean parameter which indicates the use + or not of the augmented representation for + training data (i.e. whether bias features + are activated or not). + """ sc = data.context + if regType is None: + regType = "none" train_func = lambda d, i: sc._jvm.PythonMLLibAPI().trainSVMModelWithSGD( - d._jrdd, iterations, step, regParam, miniBatchFraction, i) + d._jrdd, iterations, step, regParam, miniBatchFraction, i, regType, intercept) return _regression_train_wrapper(sc, train_func, SVMModel, data, initialWeights) class NaiveBayesModel(object): + """ Model for Naive Bayes classifiers. @@ -160,6 +220,7 @@ def predict(self, x): class NaiveBayes(object): + @classmethod def train(cls, data, lambda_=1.0): """ diff --git a/python/pyspark/mllib/clustering.py b/python/pyspark/mllib/clustering.py index b380e8f6c8725..a0630d1d5c58b 100644 --- a/python/pyspark/mllib/clustering.py +++ b/python/pyspark/mllib/clustering.py @@ -27,6 +27,7 @@ class KMeansModel(object): + """A clustering model derived from the k-means method. >>> data = array([0.0,0.0, 1.0,1.0, 9.0,8.0, 8.0,9.0]).reshape(4,2) @@ -55,6 +56,7 @@ class KMeansModel(object): >>> type(model.clusterCenters) """ + def __init__(self, centers): self.centers = centers @@ -76,6 +78,7 @@ def predict(self, x): class KMeans(object): + @classmethod def train(cls, data, k, maxIterations=100, runs=1, initializationMode="k-means||"): """Train a k-means clustering model.""" diff --git a/python/pyspark/mllib/linalg.py b/python/pyspark/mllib/linalg.py index db39ed0acdb66..9a239abfbbeb1 100644 --- a/python/pyspark/mllib/linalg.py +++ b/python/pyspark/mllib/linalg.py @@ -27,6 +27,7 @@ class SparseVector(object): + """ A simple sparse vector class for passing data to MLlib. Users may alternatively pass SciPy's {scipy.sparse} data types. @@ -192,6 +193,7 @@ def __ne__(self, other): class Vectors(object): + """ Factory methods for working with vectors. Note that dense vectors are simply represented as NumPy array objects, so there is no need @@ -247,6 +249,7 @@ def stringify(vector): else: return "[" + ",".join([str(v) for v in vector]) + "]" + def _test(): import doctest (failure_count, test_count) = doctest.testmod(optionflags=doctest.ELLIPSIS) @@ -254,4 +257,8 @@ def _test(): exit(-1) if __name__ == "__main__": + # remove current path from list of search paths to avoid importing mllib.random + # for C{import random}, which is done in an external dependency of pyspark during doctests. + import sys + sys.path.pop(0) _test() diff --git a/python/pyspark/mllib/random.py b/python/pyspark/mllib/random.py new file mode 100644 index 0000000000000..3f3b19053d32e --- /dev/null +++ b/python/pyspark/mllib/random.py @@ -0,0 +1,183 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +""" +Python package for random data generation. +""" + + +from pyspark.rdd import RDD +from pyspark.mllib._common import _deserialize_double, _deserialize_double_vector +from pyspark.serializers import NoOpSerializer + + +class RandomRDDs: + """ + Generator methods for creating RDDs comprised of i.i.d samples from + some distribution. + """ + + @staticmethod + def uniformRDD(sc, size, numPartitions=None, seed=None): + """ + Generates an RDD comprised of i.i.d. samples from the + uniform distribution on [0.0, 1.0]. + + To transform the distribution in the generated RDD from U[0.0, 1.0] + to U[a, b], use + C{RandomRDDs.uniformRDD(sc, n, p, seed)\ + .map(lambda v: a + (b - a) * v)} + + >>> x = RandomRDDs.uniformRDD(sc, 100).collect() + >>> len(x) + 100 + >>> max(x) <= 1.0 and min(x) >= 0.0 + True + >>> RandomRDDs.uniformRDD(sc, 100, 4).getNumPartitions() + 4 + >>> parts = RandomRDDs.uniformRDD(sc, 100, seed=4).getNumPartitions() + >>> parts == sc.defaultParallelism + True + """ + jrdd = sc._jvm.PythonMLLibAPI().uniformRDD(sc._jsc, size, numPartitions, seed) + uniform = RDD(jrdd, sc, NoOpSerializer()) + return uniform.map(lambda bytes: _deserialize_double(bytearray(bytes))) + + @staticmethod + def normalRDD(sc, size, numPartitions=None, seed=None): + """ + Generates an RDD comprised of i.i.d samples from the standard normal + distribution. + + To transform the distribution in the generated RDD from standard normal + to some other normal N(mean, sigma), use + C{RandomRDDs.normal(sc, n, p, seed)\ + .map(lambda v: mean + sigma * v)} + + >>> x = RandomRDDs.normalRDD(sc, 1000, seed=1L) + >>> stats = x.stats() + >>> stats.count() + 1000L + >>> abs(stats.mean() - 0.0) < 0.1 + True + >>> abs(stats.stdev() - 1.0) < 0.1 + True + """ + jrdd = sc._jvm.PythonMLLibAPI().normalRDD(sc._jsc, size, numPartitions, seed) + normal = RDD(jrdd, sc, NoOpSerializer()) + return normal.map(lambda bytes: _deserialize_double(bytearray(bytes))) + + @staticmethod + def poissonRDD(sc, mean, size, numPartitions=None, seed=None): + """ + Generates an RDD comprised of i.i.d samples from the Poisson + distribution with the input mean. + + >>> mean = 100.0 + >>> x = RandomRDDs.poissonRDD(sc, mean, 1000, seed=1L) + >>> stats = x.stats() + >>> stats.count() + 1000L + >>> abs(stats.mean() - mean) < 0.5 + True + >>> from math import sqrt + >>> abs(stats.stdev() - sqrt(mean)) < 0.5 + True + """ + jrdd = sc._jvm.PythonMLLibAPI().poissonRDD(sc._jsc, mean, size, numPartitions, seed) + poisson = RDD(jrdd, sc, NoOpSerializer()) + return poisson.map(lambda bytes: _deserialize_double(bytearray(bytes))) + + @staticmethod + def uniformVectorRDD(sc, numRows, numCols, numPartitions=None, seed=None): + """ + Generates an RDD comprised of vectors containing i.i.d samples drawn + from the uniform distribution on [0.0 1.0]. + + >>> import numpy as np + >>> mat = np.matrix(RandomRDDs.uniformVectorRDD(sc, 10, 10).collect()) + >>> mat.shape + (10, 10) + >>> mat.max() <= 1.0 and mat.min() >= 0.0 + True + >>> RandomRDDs.uniformVectorRDD(sc, 10, 10, 4).getNumPartitions() + 4 + """ + jrdd = sc._jvm.PythonMLLibAPI() \ + .uniformVectorRDD(sc._jsc, numRows, numCols, numPartitions, seed) + uniform = RDD(jrdd, sc, NoOpSerializer()) + return uniform.map(lambda bytes: _deserialize_double_vector(bytearray(bytes))) + + @staticmethod + def normalVectorRDD(sc, numRows, numCols, numPartitions=None, seed=None): + """ + Generates an RDD comprised of vectors containing i.i.d samples drawn + from the standard normal distribution. + + >>> import numpy as np + >>> mat = np.matrix(RandomRDDs.normalVectorRDD(sc, 100, 100, seed=1L).collect()) + >>> mat.shape + (100, 100) + >>> abs(mat.mean() - 0.0) < 0.1 + True + >>> abs(mat.std() - 1.0) < 0.1 + True + """ + jrdd = sc._jvm.PythonMLLibAPI() \ + .normalVectorRDD(sc._jsc, numRows, numCols, numPartitions, seed) + normal = RDD(jrdd, sc, NoOpSerializer()) + return normal.map(lambda bytes: _deserialize_double_vector(bytearray(bytes))) + + @staticmethod + def poissonVectorRDD(sc, mean, numRows, numCols, numPartitions=None, seed=None): + """ + Generates an RDD comprised of vectors containing i.i.d samples drawn + from the Poisson distribution with the input mean. + + >>> import numpy as np + >>> mean = 100.0 + >>> rdd = RandomRDDs.poissonVectorRDD(sc, mean, 100, 100, seed=1L) + >>> mat = np.mat(rdd.collect()) + >>> mat.shape + (100, 100) + >>> abs(mat.mean() - mean) < 0.5 + True + >>> from math import sqrt + >>> abs(mat.std() - sqrt(mean)) < 0.5 + True + """ + jrdd = sc._jvm.PythonMLLibAPI() \ + .poissonVectorRDD(sc._jsc, mean, numRows, numCols, numPartitions, seed) + poisson = RDD(jrdd, sc, NoOpSerializer()) + return poisson.map(lambda bytes: _deserialize_double_vector(bytearray(bytes))) + + +def _test(): + import doctest + from pyspark.context import SparkContext + globs = globals().copy() + # The small batch size here ensures that we see multiple batches, + # even in these small test examples: + globs['sc'] = SparkContext('local[2]', 'PythonTest', batchSize=2) + (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) + globs['sc'].stop() + if failure_count: + exit(-1) + + +if __name__ == "__main__": + _test() diff --git a/python/pyspark/mllib/recommendation.py b/python/pyspark/mllib/recommendation.py index 6c385042ffa5f..e863fc249ec36 100644 --- a/python/pyspark/mllib/recommendation.py +++ b/python/pyspark/mllib/recommendation.py @@ -26,6 +26,7 @@ class MatrixFactorizationModel(object): + """A matrix factorisation model trained by regularized alternating least-squares. @@ -58,6 +59,7 @@ def predictAll(self, usersProducts): class ALS(object): + @classmethod def train(cls, ratings, rank, iterations=5, lambda_=0.01, blocks=-1): sc = ratings.context diff --git a/python/pyspark/mllib/regression.py b/python/pyspark/mllib/regression.py index b84bc531dec8c..d8792cf44872f 100644 --- a/python/pyspark/mllib/regression.py +++ b/python/pyspark/mllib/regression.py @@ -27,6 +27,7 @@ class LabeledPoint(object): + """ The features and labels of a data point. @@ -34,6 +35,7 @@ class LabeledPoint(object): @param features: Vector of features for this point (NumPy array, list, pyspark.mllib.linalg.SparseVector, or scipy.sparse column matrix) """ + def __init__(self, label, features): self.label = label if (type(features) == ndarray or type(features) == SparseVector @@ -49,7 +51,9 @@ def __str__(self): class LinearModel(object): + """A linear model that has a vector of coefficients and an intercept.""" + def __init__(self, weights, intercept): self._coeff = weights self._intercept = intercept @@ -64,6 +68,7 @@ def intercept(self): class LinearRegressionModelBase(LinearModel): + """A linear regression model. >>> lrmb = LinearRegressionModelBase(array([1.0, 2.0]), 0.1) @@ -72,6 +77,7 @@ class LinearRegressionModelBase(LinearModel): >>> abs(lrmb.predict(SparseVector(2, {0: -1.03, 1: 7.777})) - 14.624) < 1e-6 True """ + def predict(self, x): """Predict the value of the dependent variable given a vector x""" """containing values for the independent variables.""" @@ -80,6 +86,7 @@ def predict(self, x): class LinearRegressionModel(LinearRegressionModelBase): + """A linear regression model derived from a least-squares fit. >>> from pyspark.mllib.regression import LabeledPoint @@ -111,17 +118,43 @@ class LinearRegressionModel(LinearRegressionModelBase): class LinearRegressionWithSGD(object): + @classmethod - def train(cls, data, iterations=100, step=1.0, - miniBatchFraction=1.0, initialWeights=None): - """Train a linear regression model on the given data.""" + def train(cls, data, iterations=100, step=1.0, miniBatchFraction=1.0, + initialWeights=None, regParam=1.0, regType=None, intercept=False): + """ + Train a linear regression model on the given data. + + @param data: The training data. + @param iterations: The number of iterations (default: 100). + @param step: The step parameter used in SGD + (default: 1.0). + @param miniBatchFraction: Fraction of data to be used for each SGD + iteration. + @param initialWeights: The initial weights (default: None). + @param regParam: The regularizer parameter (default: 1.0). + @param regType: The type of regularizer used for training + our model. + Allowed values: "l1" for using L1Updater, + "l2" for using + SquaredL2Updater, + "none" for no regularizer. + (default: "none") + @param intercept: Boolean parameter which indicates the use + or not of the augmented representation for + training data (i.e. whether bias features + are activated or not). + """ sc = data.context + if regType is None: + regType = "none" train_f = lambda d, i: sc._jvm.PythonMLLibAPI().trainLinearRegressionModelWithSGD( - d._jrdd, iterations, step, miniBatchFraction, i) + d._jrdd, iterations, step, miniBatchFraction, i, regParam, regType, intercept) return _regression_train_wrapper(sc, train_f, LinearRegressionModel, data, initialWeights) class LassoModel(LinearRegressionModelBase): + """A linear regression model derived from a least-squares fit with an l_1 penalty term. @@ -154,6 +187,7 @@ class LassoModel(LinearRegressionModelBase): class LassoWithSGD(object): + @classmethod def train(cls, data, iterations=100, step=1.0, regParam=1.0, miniBatchFraction=1.0, initialWeights=None): @@ -165,6 +199,7 @@ def train(cls, data, iterations=100, step=1.0, regParam=1.0, class RidgeRegressionModel(LinearRegressionModelBase): + """A linear regression model derived from a least-squares fit with an l_2 penalty term. @@ -197,6 +232,7 @@ class RidgeRegressionModel(LinearRegressionModelBase): class RidgeRegressionWithSGD(object): + @classmethod def train(cls, data, iterations=100, step=1.0, regParam=1.0, miniBatchFraction=1.0, initialWeights=None): diff --git a/python/pyspark/mllib/stat.py b/python/pyspark/mllib/stat.py new file mode 100644 index 0000000000000..a73abc5ff90df --- /dev/null +++ b/python/pyspark/mllib/stat.py @@ -0,0 +1,169 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +""" +Python package for statistical functions in MLlib. +""" + +from pyspark.mllib._common import \ + _get_unmangled_double_vector_rdd, _get_unmangled_rdd, \ + _serialize_double, _serialize_double_vector, \ + _deserialize_double, _deserialize_double_matrix, _deserialize_double_vector + + +class MultivariateStatisticalSummary(object): + + """ + Trait for multivariate statistical summary of a data matrix. + """ + + def __init__(self, sc, java_summary): + """ + :param sc: Spark context + :param java_summary: Handle to Java summary object + """ + self._sc = sc + self._java_summary = java_summary + + def __del__(self): + self._sc._gateway.detach(self._java_summary) + + def mean(self): + return _deserialize_double_vector(self._java_summary.mean()) + + def variance(self): + return _deserialize_double_vector(self._java_summary.variance()) + + def count(self): + return self._java_summary.count() + + def numNonzeros(self): + return _deserialize_double_vector(self._java_summary.numNonzeros()) + + def max(self): + return _deserialize_double_vector(self._java_summary.max()) + + def min(self): + return _deserialize_double_vector(self._java_summary.min()) + + +class Statistics(object): + + @staticmethod + def colStats(X): + """ + Computes column-wise summary statistics for the input RDD[Vector]. + + >>> from linalg import Vectors + >>> rdd = sc.parallelize([Vectors.dense([2, 0, 0, -2]), + ... Vectors.dense([4, 5, 0, 3]), + ... Vectors.dense([6, 7, 0, 8])]) + >>> cStats = Statistics.colStats(rdd) + >>> cStats.mean() + array([ 4., 4., 0., 3.]) + >>> cStats.variance() + array([ 4., 13., 0., 25.]) + >>> cStats.count() + 3L + >>> cStats.numNonzeros() + array([ 3., 2., 0., 3.]) + >>> cStats.max() + array([ 6., 7., 0., 8.]) + >>> cStats.min() + array([ 2., 0., 0., -2.]) + """ + sc = X.ctx + Xser = _get_unmangled_double_vector_rdd(X) + cStats = sc._jvm.PythonMLLibAPI().colStats(Xser._jrdd) + return MultivariateStatisticalSummary(sc, cStats) + + @staticmethod + def corr(x, y=None, method=None): + """ + Compute the correlation (matrix) for the input RDD(s) using the + specified method. + Methods currently supported: I{pearson (default), spearman}. + + If a single RDD of Vectors is passed in, a correlation matrix + comparing the columns in the input RDD is returned. Use C{method=} + to specify the method to be used for single RDD inout. + If two RDDs of floats are passed in, a single float is returned. + + >>> x = sc.parallelize([1.0, 0.0, -2.0], 2) + >>> y = sc.parallelize([4.0, 5.0, 3.0], 2) + >>> zeros = sc.parallelize([0.0, 0.0, 0.0], 2) + >>> abs(Statistics.corr(x, y) - 0.6546537) < 1e-7 + True + >>> Statistics.corr(x, y) == Statistics.corr(x, y, "pearson") + True + >>> Statistics.corr(x, y, "spearman") + 0.5 + >>> from math import isnan + >>> isnan(Statistics.corr(x, zeros)) + True + >>> from linalg import Vectors + >>> rdd = sc.parallelize([Vectors.dense([1, 0, 0, -2]), Vectors.dense([4, 5, 0, 3]), + ... Vectors.dense([6, 7, 0, 8]), Vectors.dense([9, 0, 0, 1])]) + >>> Statistics.corr(rdd) + array([[ 1. , 0.05564149, nan, 0.40047142], + [ 0.05564149, 1. , nan, 0.91359586], + [ nan, nan, 1. , nan], + [ 0.40047142, 0.91359586, nan, 1. ]]) + >>> Statistics.corr(rdd, method="spearman") + array([[ 1. , 0.10540926, nan, 0.4 ], + [ 0.10540926, 1. , nan, 0.9486833 ], + [ nan, nan, 1. , nan], + [ 0.4 , 0.9486833 , nan, 1. ]]) + >>> try: + ... Statistics.corr(rdd, "spearman") + ... print "Method name as second argument without 'method=' shouldn't be allowed." + ... except TypeError: + ... pass + """ + sc = x.ctx + # Check inputs to determine whether a single value or a matrix is needed for output. + # Since it's legal for users to use the method name as the second argument, we need to + # check if y is used to specify the method name instead. + if type(y) == str: + raise TypeError("Use 'method=' to specify method name.") + if not y: + try: + Xser = _get_unmangled_double_vector_rdd(x) + except TypeError: + raise TypeError("corr called on a single RDD not consisted of Vectors.") + resultMat = sc._jvm.PythonMLLibAPI().corr(Xser._jrdd, method) + return _deserialize_double_matrix(resultMat) + else: + xSer = _get_unmangled_rdd(x, _serialize_double) + ySer = _get_unmangled_rdd(y, _serialize_double) + result = sc._jvm.PythonMLLibAPI().corr(xSer._jrdd, ySer._jrdd, method) + return result + + +def _test(): + import doctest + from pyspark import SparkContext + globs = globals().copy() + globs['sc'] = SparkContext('local[4]', 'PythonTest', batchSize=2) + (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) + globs['sc'].stop() + if failure_count: + exit(-1) + + +if __name__ == "__main__": + _test() diff --git a/python/pyspark/mllib/tests.py b/python/pyspark/mllib/tests.py index 37ccf1d590743..8a851bd35c0e8 100644 --- a/python/pyspark/mllib/tests.py +++ b/python/pyspark/mllib/tests.py @@ -19,8 +19,13 @@ Fuller unit tests for Python MLlib. """ +import sys from numpy import array, array_equal -import unittest + +if sys.version_info[:2] <= (2, 6): + import unittest2 as unittest +else: + import unittest from pyspark.mllib._common import _convert_vector, _serialize_double_vector, \ _deserialize_double_vector, _dot, _squared_distance @@ -39,6 +44,7 @@ class VectorTests(unittest.TestCase): + def test_serialize(self): sv = SparseVector(4, {1: 1, 3: 2}) dv = array([1., 2., 3., 4.]) @@ -81,6 +87,7 @@ def test_squared_distance(self): class ListTests(PySparkTestCase): + """ Test MLlib algorithms on plain lists, to make sure they're passed through as NumPy arrays. @@ -100,6 +107,7 @@ def test_clustering(self): def test_classification(self): from pyspark.mllib.classification import LogisticRegressionWithSGD, SVMWithSGD, NaiveBayes + from pyspark.mllib.tree import DecisionTree data = [ LabeledPoint(0.0, [1, 0, 0]), LabeledPoint(1.0, [0, 1, 1]), @@ -127,9 +135,19 @@ def test_classification(self): self.assertTrue(nb_model.predict(features[2]) <= 0) self.assertTrue(nb_model.predict(features[3]) > 0) + categoricalFeaturesInfo = {0: 3} # feature 0 has 3 categories + dt_model = \ + DecisionTree.trainClassifier(rdd, numClasses=2, + categoricalFeaturesInfo=categoricalFeaturesInfo) + self.assertTrue(dt_model.predict(features[0]) <= 0) + self.assertTrue(dt_model.predict(features[1]) > 0) + self.assertTrue(dt_model.predict(features[2]) <= 0) + self.assertTrue(dt_model.predict(features[3]) > 0) + def test_regression(self): from pyspark.mllib.regression import LinearRegressionWithSGD, LassoWithSGD, \ RidgeRegressionWithSGD + from pyspark.mllib.tree import DecisionTree data = [ LabeledPoint(-1.0, [0, -1]), LabeledPoint(1.0, [0, 1]), @@ -157,9 +175,18 @@ def test_regression(self): self.assertTrue(rr_model.predict(features[2]) <= 0) self.assertTrue(rr_model.predict(features[3]) > 0) + categoricalFeaturesInfo = {0: 2} # feature 0 has 2 categories + dt_model = \ + DecisionTree.trainRegressor(rdd, categoricalFeaturesInfo=categoricalFeaturesInfo) + self.assertTrue(dt_model.predict(features[0]) <= 0) + self.assertTrue(dt_model.predict(features[1]) > 0) + self.assertTrue(dt_model.predict(features[2]) <= 0) + self.assertTrue(dt_model.predict(features[3]) > 0) + @unittest.skipIf(not _have_scipy, "SciPy not installed") class SciPyTests(PySparkTestCase): + """ Test both vector operations and MLlib algorithms with SciPy sparse matrices, if SciPy is available. @@ -229,6 +256,7 @@ def test_clustering(self): def test_classification(self): from pyspark.mllib.classification import LogisticRegressionWithSGD, SVMWithSGD, NaiveBayes + from pyspark.mllib.tree import DecisionTree data = [ LabeledPoint(0.0, self.scipy_matrix(2, {0: 1.0})), LabeledPoint(1.0, self.scipy_matrix(2, {1: 1.0})), @@ -256,9 +284,18 @@ def test_classification(self): self.assertTrue(nb_model.predict(features[2]) <= 0) self.assertTrue(nb_model.predict(features[3]) > 0) + categoricalFeaturesInfo = {0: 3} # feature 0 has 3 categories + dt_model = DecisionTree.trainClassifier(rdd, numClasses=2, + categoricalFeaturesInfo=categoricalFeaturesInfo) + self.assertTrue(dt_model.predict(features[0]) <= 0) + self.assertTrue(dt_model.predict(features[1]) > 0) + self.assertTrue(dt_model.predict(features[2]) <= 0) + self.assertTrue(dt_model.predict(features[3]) > 0) + def test_regression(self): from pyspark.mllib.regression import LinearRegressionWithSGD, LassoWithSGD, \ RidgeRegressionWithSGD + from pyspark.mllib.tree import DecisionTree data = [ LabeledPoint(-1.0, self.scipy_matrix(2, {1: -1.0})), LabeledPoint(1.0, self.scipy_matrix(2, {1: 1.0})), @@ -286,6 +323,13 @@ def test_regression(self): self.assertTrue(rr_model.predict(features[2]) <= 0) self.assertTrue(rr_model.predict(features[3]) > 0) + categoricalFeaturesInfo = {0: 2} # feature 0 has 2 categories + dt_model = DecisionTree.trainRegressor(rdd, categoricalFeaturesInfo=categoricalFeaturesInfo) + self.assertTrue(dt_model.predict(features[0]) <= 0) + self.assertTrue(dt_model.predict(features[1]) > 0) + self.assertTrue(dt_model.predict(features[2]) <= 0) + self.assertTrue(dt_model.predict(features[3]) > 0) + if __name__ == "__main__": if not _have_scipy: diff --git a/python/pyspark/mllib/tree.py b/python/pyspark/mllib/tree.py new file mode 100644 index 0000000000000..e1a4671709b7d --- /dev/null +++ b/python/pyspark/mllib/tree.py @@ -0,0 +1,207 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from py4j.java_collections import MapConverter + +from pyspark import SparkContext, RDD +from pyspark.mllib._common import \ + _get_unmangled_rdd, _get_unmangled_double_vector_rdd, _serialize_double_vector, \ + _deserialize_labeled_point, _get_unmangled_labeled_point_rdd, \ + _deserialize_double +from pyspark.mllib.regression import LabeledPoint +from pyspark.serializers import NoOpSerializer + + +class DecisionTreeModel(object): + + """ + A decision tree model for classification or regression. + + EXPERIMENTAL: This is an experimental API. + It will probably be modified for Spark v1.2. + """ + + def __init__(self, sc, java_model): + """ + :param sc: Spark context + :param java_model: Handle to Java model object + """ + self._sc = sc + self._java_model = java_model + + def __del__(self): + self._sc._gateway.detach(self._java_model) + + def predict(self, x): + """ + Predict the label of one or more examples. + :param x: Data point (feature vector), + or an RDD of data points (feature vectors). + """ + pythonAPI = self._sc._jvm.PythonMLLibAPI() + if isinstance(x, RDD): + # Bulk prediction + if x.count() == 0: + return self._sc.parallelize([]) + dataBytes = _get_unmangled_double_vector_rdd(x, cache=False) + jSerializedPreds = \ + pythonAPI.predictDecisionTreeModel(self._java_model, + dataBytes._jrdd) + serializedPreds = RDD(jSerializedPreds, self._sc, NoOpSerializer()) + return serializedPreds.map(lambda bytes: _deserialize_double(bytearray(bytes))) + else: + # Assume x is a single data point. + x_ = _serialize_double_vector(x) + return pythonAPI.predictDecisionTreeModel(self._java_model, x_) + + def numNodes(self): + return self._java_model.numNodes() + + def depth(self): + return self._java_model.depth() + + def __str__(self): + return self._java_model.toString() + + +class DecisionTree(object): + + """ + Learning algorithm for a decision tree model + for classification or regression. + + EXPERIMENTAL: This is an experimental API. + It will probably be modified for Spark v1.2. + + Example usage: + >>> from numpy import array, ndarray + >>> from pyspark.mllib.regression import LabeledPoint + >>> from pyspark.mllib.tree import DecisionTree + >>> from pyspark.mllib.linalg import SparseVector + >>> + >>> data = [ + ... LabeledPoint(0.0, [0.0]), + ... LabeledPoint(1.0, [1.0]), + ... LabeledPoint(1.0, [2.0]), + ... LabeledPoint(1.0, [3.0]) + ... ] + >>> + >>> model = DecisionTree.trainClassifier(sc.parallelize(data), numClasses=2) + >>> print(model) + DecisionTreeModel classifier + If (feature 0 <= 0.5) + Predict: 0.0 + Else (feature 0 > 0.5) + Predict: 1.0 + + >>> model.predict(array([1.0])) > 0 + True + >>> model.predict(array([0.0])) == 0 + True + >>> sparse_data = [ + ... LabeledPoint(0.0, SparseVector(2, {0: 0.0})), + ... LabeledPoint(1.0, SparseVector(2, {1: 1.0})), + ... LabeledPoint(0.0, SparseVector(2, {0: 0.0})), + ... LabeledPoint(1.0, SparseVector(2, {1: 2.0})) + ... ] + >>> + >>> model = DecisionTree.trainRegressor(sc.parallelize(sparse_data)) + >>> model.predict(array([0.0, 1.0])) == 1 + True + >>> model.predict(array([0.0, 0.0])) == 0 + True + >>> model.predict(SparseVector(2, {1: 1.0})) == 1 + True + >>> model.predict(SparseVector(2, {1: 0.0})) == 0 + True + """ + + @staticmethod + def trainClassifier(data, numClasses, categoricalFeaturesInfo, + impurity="gini", maxDepth=4, maxBins=100): + """ + Train a DecisionTreeModel for classification. + + :param data: Training data: RDD of LabeledPoint. + Labels are integers {0,1,...,numClasses}. + :param numClasses: Number of classes for classification. + :param categoricalFeaturesInfo: Map from categorical feature index + to number of categories. + Any feature not in this map + is treated as continuous. + :param impurity: Supported values: "entropy" or "gini" + :param maxDepth: Max depth of tree. + E.g., depth 0 means 1 leaf node. + Depth 1 means 1 internal node + 2 leaf nodes. + :param maxBins: Number of bins used for finding splits at each node. + :return: DecisionTreeModel + """ + sc = data.context + dataBytes = _get_unmangled_labeled_point_rdd(data) + categoricalFeaturesInfoJMap = \ + MapConverter().convert(categoricalFeaturesInfo, + sc._gateway._gateway_client) + model = sc._jvm.PythonMLLibAPI().trainDecisionTreeModel( + dataBytes._jrdd, "classification", + numClasses, categoricalFeaturesInfoJMap, + impurity, maxDepth, maxBins) + dataBytes.unpersist() + return DecisionTreeModel(sc, model) + + @staticmethod + def trainRegressor(data, categoricalFeaturesInfo, + impurity="variance", maxDepth=4, maxBins=100): + """ + Train a DecisionTreeModel for regression. + + :param data: Training data: RDD of LabeledPoint. + Labels are real numbers. + :param categoricalFeaturesInfo: Map from categorical feature index + to number of categories. + Any feature not in this map + is treated as continuous. + :param impurity: Supported values: "variance" + :param maxDepth: Max depth of tree. + E.g., depth 0 means 1 leaf node. + Depth 1 means 1 internal node + 2 leaf nodes. + :param maxBins: Number of bins used for finding splits at each node. + :return: DecisionTreeModel + """ + sc = data.context + dataBytes = _get_unmangled_labeled_point_rdd(data) + categoricalFeaturesInfoJMap = \ + MapConverter().convert(categoricalFeaturesInfo, + sc._gateway._gateway_client) + model = sc._jvm.PythonMLLibAPI().trainDecisionTreeModel( + dataBytes._jrdd, "regression", + 0, categoricalFeaturesInfoJMap, + impurity, maxDepth, maxBins) + dataBytes.unpersist() + return DecisionTreeModel(sc, model) + + +def _test(): + import doctest + globs = globals().copy() + globs['sc'] = SparkContext('local[4]', 'PythonTest', batchSize=2) + (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) + globs['sc'].stop() + if failure_count: + exit(-1) + +if __name__ == "__main__": + _test() diff --git a/python/pyspark/mllib/util.py b/python/pyspark/mllib/util.py index e24c144f458bd..4962d05491c03 100644 --- a/python/pyspark/mllib/util.py +++ b/python/pyspark/mllib/util.py @@ -16,6 +16,7 @@ # import numpy as np +import warnings from pyspark.mllib.linalg import Vectors, SparseVector from pyspark.mllib.regression import LabeledPoint @@ -24,21 +25,24 @@ from pyspark.serializers import NoOpSerializer - class MLUtils: + """ Helper methods to load, save and pre-process data used in MLlib. """ @staticmethod def _parse_libsvm_line(line, multiclass): + warnings.warn("deprecated", DeprecationWarning) + return _parse_libsvm_line(line) + + @staticmethod + def _parse_libsvm_line(line): """ Parses a line in LIBSVM format into (label, indices, values). """ items = line.split(None) label = float(items[0]) - if not multiclass: - label = 1.0 if label > 0.5 else 0.0 nnz = len(items) - 1 indices = np.zeros(nnz, dtype=np.int32) values = np.zeros(nnz) @@ -67,6 +71,11 @@ def _convert_labeled_point_to_libsvm(p): @staticmethod def loadLibSVMFile(sc, path, multiclass=False, numFeatures=-1, minPartitions=None): + warnings.warn("deprecated", DeprecationWarning) + return loadLibSVMFile(sc, path, numFeatures, minPartitions) + + @staticmethod + def loadLibSVMFile(sc, path, numFeatures=-1, minPartitions=None): """ Loads labeled data in the LIBSVM format into an RDD of LabeledPoint. The LIBSVM format is a text-based format used by @@ -82,13 +91,6 @@ def loadLibSVMFile(sc, path, multiclass=False, numFeatures=-1, minPartitions=Non @param sc: Spark context @param path: file or directory path in any Hadoop-supported file system URI - @param multiclass: whether the input labels contain more than - two classes. If false, any label with value - greater than 0.5 will be mapped to 1.0, or - 0.0 otherwise. So it works for both +1/-1 and - 1/0 cases. If true, the double value parsed - directly from the label string will be used - as the label value. @param numFeatures: number of features, which will be determined from the input data if a nonpositive value is given. This is useful when the dataset is @@ -106,7 +108,6 @@ def loadLibSVMFile(sc, path, multiclass=False, numFeatures=-1, minPartitions=Non >>> tempFile.write("+1 1:1.0 3:2.0 5:3.0\\n-1\\n-1 2:4.0 4:5.0 6:6.0") >>> tempFile.flush() >>> examples = MLUtils.loadLibSVMFile(sc, tempFile.name).collect() - >>> multiclass_examples = MLUtils.loadLibSVMFile(sc, tempFile.name, True).collect() >>> tempFile.close() >>> type(examples[0]) == LabeledPoint True @@ -115,20 +116,18 @@ def loadLibSVMFile(sc, path, multiclass=False, numFeatures=-1, minPartitions=Non >>> type(examples[1]) == LabeledPoint True >>> print examples[1] - (0.0,(6,[],[])) + (-1.0,(6,[],[])) >>> type(examples[2]) == LabeledPoint True >>> print examples[2] - (0.0,(6,[1,3,5],[4.0,5.0,6.0])) - >>> multiclass_examples[1].label - -1.0 + (-1.0,(6,[1,3,5],[4.0,5.0,6.0])) """ lines = sc.textFile(path, minPartitions) - parsed = lines.map(lambda l: MLUtils._parse_libsvm_line(l, multiclass)) + parsed = lines.map(lambda l: MLUtils._parse_libsvm_line(l)) if numFeatures <= 0: parsed.cache() - numFeatures = parsed.map(lambda x: 0 if x[1].size == 0 else x[1][-1]).reduce(max) + 1 + numFeatures = parsed.map(lambda x: -1 if x[1].size == 0 else x[1][-1]).reduce(max) + 1 return parsed.map(lambda x: LabeledPoint(x[0], Vectors.sparse(numFeatures, x[1], x[2]))) @staticmethod @@ -154,7 +153,6 @@ def saveAsLibSVMFile(data, dir): lines = data.map(lambda p: MLUtils._convert_labeled_point_to_libsvm(p)) lines.saveAsTextFile(dir) - @staticmethod def loadLabeledPoints(sc, path, minPartitions=None): """ diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index f64f48e3a4c9c..240381e5bae12 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -30,24 +30,56 @@ from threading import Thread import warnings import heapq +import bisect from random import Random from math import sqrt, log from pyspark.serializers import NoOpSerializer, CartesianDeserializer, \ BatchedSerializer, CloudPickleSerializer, PairDeserializer, \ - PickleSerializer, pack_long + PickleSerializer, pack_long, CompressedSerializer from pyspark.join import python_join, python_left_outer_join, \ python_right_outer_join, python_cogroup from pyspark.statcounter import StatCounter -from pyspark.rddsampler import RDDSampler +from pyspark.rddsampler import RDDSampler, RDDStratifiedSampler from pyspark.storagelevel import StorageLevel from pyspark.resultiterable import ResultIterable +from pyspark.shuffle import Aggregator, InMemoryMerger, ExternalMerger, \ + get_used_memory from py4j.java_collections import ListConverter, MapConverter __all__ = ["RDD"] +# TODO: for Python 3.3+, PYTHONHASHSEED should be reset to disable randomized +# hash for string +def portable_hash(x): + """ + This function returns consistant hash code for builtin types, especially + for None and tuple with None. + + The algrithm is similar to that one used by CPython 2.7 + + >>> portable_hash(None) + 0 + >>> portable_hash((None, 1)) + 219750521 + """ + if x is None: + return 0 + if isinstance(x, tuple): + h = 0x345678 + for i in x: + h ^= portable_hash(i) + h *= 1000003 + h &= 0xffffffff + h ^= len(x) + if h == -1: + h = -2 + return h + return hash(x) + + def _extract_concise_traceback(): """ This function returns the traceback info for a callsite, returns a dict @@ -69,16 +101,19 @@ def _extract_concise_traceback(): file, line, fun, what = tb[0] return callsite(function=fun, file=file, linenum=line) sfile, sline, sfun, swhat = tb[first_spark_frame] - ufile, uline, ufun, uwhat = tb[first_spark_frame-1] + ufile, uline, ufun, uwhat = tb[first_spark_frame - 1] return callsite(function=sfun, file=ufile, linenum=uline) _spark_stack_depth = 0 + class _JavaStackTrace(object): + def __init__(self, sc): tb = _extract_concise_traceback() if tb is not None: - self._traceback = "%s at %s:%s" % (tb.function, tb.file, tb.linenum) + self._traceback = "%s at %s:%s" % ( + tb.function, tb.file, tb.linenum) else: self._traceback = "Error! Could not extract traceback info" self._context = sc @@ -95,9 +130,12 @@ def __exit__(self, type, value, tb): if _spark_stack_depth == 0: self._context._jsc.setCallSite(None) + class MaxHeapQ(object): + """ An implementation of MaxHeap. + >>> import pyspark.rdd >>> heap = pyspark.rdd.MaxHeapQ(5) >>> [heap.insert(i) for i in range(10)] @@ -117,14 +155,14 @@ class MaxHeapQ(object): """ def __init__(self, maxsize): - # we start from q[1], this makes calculating children as trivial as 2 * k + # We start from q[1], so its children are always 2 * k self.q = [0] self.maxsize = maxsize def _swim(self, k): - while (k > 1) and (self.q[k/2] < self.q[k]): - self._swap(k, k/2) - k = k/2 + while (k > 1) and (self.q[k / 2] < self.q[k]): + self._swap(k, k / 2) + k = k / 2 def _swap(self, i, j): t = self.q[i] @@ -162,7 +200,25 @@ def _replaceRoot(self, value): self.q[1] = value self._sink(1) + +def _parse_memory(s): + """ + Parse a memory string in the format supported by Java (e.g. 1g, 200m) and + return the value in MB + + >>> _parse_memory("256m") + 256 + >>> _parse_memory("2g") + 2048 + """ + units = {'g': 1024, 'm': 1, 't': 1 << 20, 'k': 1.0 / 1024} + if s[-1] not in units: + raise ValueError("invalid format: " + s) + return int(float(s[:-1]) * units[s[-1].lower()]) + + class RDD(object): + """ A Resilient Distributed Dataset (RDD), the basic abstraction in Spark. Represents an immutable, partitioned collection of elements that can be @@ -177,6 +233,13 @@ def __init__(self, jrdd, ctx, jrdd_deserializer): self._jrdd_deserializer = jrdd_deserializer self._id = jrdd.id() + def _toPickleSerialization(self): + if (self._jrdd_deserializer == PickleSerializer() or + self._jrdd_deserializer == BatchedSerializer(PickleSerializer())): + return self + else: + return self._reserialize(BatchedSerializer(PickleSerializer(), 10)) + def id(self): """ A unique ID for this RDD (within its SparkContext). @@ -195,10 +258,10 @@ def context(self): def cache(self): """ - Persist this RDD with the default storage level (C{MEMORY_ONLY}). + Persist this RDD with the default storage level (C{MEMORY_ONLY_SER}). """ self.is_cached = True - self._jrdd.cache() + self.persist(StorageLevel.MEMORY_ONLY_SER) return self def persist(self, storageLevel): @@ -257,8 +320,9 @@ def map(self, f, preservesPartitioning=False): >>> sorted(rdd.map(lambda x: (x, 1)).collect()) [('a', 1), ('b', 1), ('c', 1)] """ - def func(split, iterator): return imap(f, iterator) - return PipelinedRDD(self, func, preservesPartitioning) + def func(_, iterator): + return imap(f, iterator) + return self.mapPartitionsWithIndex(func, preservesPartitioning) def flatMap(self, f, preservesPartitioning=False): """ @@ -271,7 +335,8 @@ def flatMap(self, f, preservesPartitioning=False): >>> sorted(rdd.flatMap(lambda x: [(x, x), (x, x)]).collect()) [(2, 2), (2, 2), (3, 3), (3, 3), (4, 4), (4, 4)] """ - def func(s, iterator): return chain.from_iterable(imap(f, iterator)) + def func(s, iterator): + return chain.from_iterable(imap(f, iterator)) return self.mapPartitionsWithIndex(func, preservesPartitioning) def mapPartitions(self, f, preservesPartitioning=False): @@ -283,7 +348,8 @@ def mapPartitions(self, f, preservesPartitioning=False): >>> rdd.mapPartitions(f).collect() [3, 7] """ - def func(s, iterator): return f(iterator) + def func(s, iterator): + return f(iterator) return self.mapPartitionsWithIndex(func) def mapPartitionsWithIndex(self, f, preservesPartitioning=False): @@ -311,17 +377,18 @@ def mapPartitionsWithSplit(self, f, preservesPartitioning=False): 6 """ warnings.warn("mapPartitionsWithSplit is deprecated; " - "use mapPartitionsWithIndex instead", DeprecationWarning, stacklevel=2) + "use mapPartitionsWithIndex instead", DeprecationWarning, stacklevel=2) return self.mapPartitionsWithIndex(f, preservesPartitioning) def getNumPartitions(self): - """ - Returns the number of partitions in RDD - >>> rdd = sc.parallelize([1, 2, 3, 4], 2) - >>> rdd.getNumPartitions() - 2 - """ - return self._jrdd.partitions().size() + """ + Returns the number of partitions in RDD + + >>> rdd = sc.parallelize([1, 2, 3, 4], 2) + >>> rdd.getNumPartitions() + 2 + """ + return self._jrdd.partitions().size() def filter(self, f): """ @@ -331,7 +398,8 @@ def filter(self, f): >>> rdd.filter(lambda x: x % 2 == 0).collect() [2, 4] """ - def func(iterator): return ifilter(f, iterator) + def func(iterator): + return ifilter(f, iterator) return self.mapPartitions(func) def distinct(self): @@ -353,7 +421,7 @@ def sample(self, withReplacement, fraction, seed=None): >>> sc.parallelize(range(0, 100)).sample(False, 0.1, 2).collect() #doctest: +SKIP [2, 3, 20, 21, 24, 41, 42, 66, 67, 89, 90, 98] """ - assert fraction >= 0.0, "Invalid fraction value: %s" % fraction + assert fraction >= 0.0, "Negative fraction value: %s" % fraction return self.mapPartitionsWithIndex(RDDSampler(withReplacement, fraction, seed).func, True) # this is ported from scala/spark/RDD.scala @@ -391,9 +459,11 @@ def takeSample(self, withReplacement, num, seed=None): maxSampleSize = sys.maxint - int(numStDev * sqrt(sys.maxint)) if num > maxSampleSize: - raise ValueError("Sample size cannot be greater than %d." % maxSampleSize) + raise ValueError( + "Sample size cannot be greater than %d." % maxSampleSize) - fraction = RDD._computeFractionForSampleSize(num, initialCount, withReplacement) + fraction = RDD._computeFractionForSampleSize( + num, initialCount, withReplacement) samples = self.sample(withReplacement, fraction, seed).collect() # If the first sample didn't turn out large enough, keep trying to take samples; @@ -499,55 +569,58 @@ def __add__(self, other): raise TypeError return self.union(other) - def sortByKey(self, ascending=True, numPartitions=None, keyfunc = lambda x: x): + def sortByKey(self, ascending=True, numPartitions=None, keyfunc=lambda x: x): """ Sorts this RDD, which is assumed to consist of (key, value) pairs. + # noqa >>> tmp = [('a', 1), ('b', 2), ('1', 3), ('d', 4), ('2', 5)] + >>> sc.parallelize(tmp).sortByKey(True, 1).collect() + [('1', 3), ('2', 5), ('a', 1), ('b', 2), ('d', 4)] >>> sc.parallelize(tmp).sortByKey(True, 2).collect() [('1', 3), ('2', 5), ('a', 1), ('b', 2), ('d', 4)] >>> tmp2 = [('Mary', 1), ('had', 2), ('a', 3), ('little', 4), ('lamb', 5)] >>> tmp2.extend([('whose', 6), ('fleece', 7), ('was', 8), ('white', 9)]) >>> sc.parallelize(tmp2).sortByKey(True, 3, keyfunc=lambda k: k.lower()).collect() - [('a', 3), ('fleece', 7), ('had', 2), ('lamb', 5), ('little', 4), ('Mary', 1), ('was', 8), ('white', 9), ('whose', 6)] + [('a', 3), ('fleece', 7), ('had', 2), ('lamb', 5),...('white', 9), ('whose', 6)] """ if numPartitions is None: numPartitions = self._defaultReducePartitions() - bounds = list() + if numPartitions == 1: + if self.getNumPartitions() > 1: + self = self.coalesce(1) + + def sort(iterator): + return sorted(iterator, reverse=(not ascending), key=lambda (k, v): keyfunc(k)) + + return self.mapPartitions(sort) # first compute the boundary of each part via sampling: we want to partition # the key-space into bins such that the bins have roughly the same # number of (key, value) pairs falling into them - if numPartitions > 1: - rddSize = self.count() - maxSampleSize = numPartitions * 20.0 # constant from Spark's RangePartitioner - fraction = min(maxSampleSize / max(rddSize, 1), 1.0) + rddSize = self.count() + maxSampleSize = numPartitions * 20.0 # constant from Spark's RangePartitioner + fraction = min(maxSampleSize / max(rddSize, 1), 1.0) + samples = self.sample(False, fraction, 1).map(lambda (k, v): k).collect() + samples = sorted(samples, reverse=(not ascending), key=keyfunc) - samples = self.sample(False, fraction, 1).map(lambda (k, v): k).collect() - samples = sorted(samples, reverse=(not ascending), key=keyfunc) - - # we have numPartitions many parts but one of the them has - # an implicit boundary - for i in range(0, numPartitions - 1): - index = (len(samples) - 1) * (i + 1) / numPartitions - bounds.append(samples[index]) + # we have numPartitions many parts but one of the them has + # an implicit boundary + bounds = [samples[len(samples) * (i + 1) / numPartitions] + for i in range(0, numPartitions - 1)] def rangePartitionFunc(k): - p = 0 - while p < len(bounds) and keyfunc(k) > bounds[p]: - p += 1 + p = bisect.bisect_left(bounds, keyfunc(k)) if ascending: return p else: - return numPartitions-1-p + return numPartitions - 1 - p def mapFunc(iterator): - yield sorted(iterator, reverse=(not ascending), key=lambda (k, v): keyfunc(k)) + return sorted(iterator, reverse=(not ascending), key=lambda (k, v): keyfunc(k)) - return (self.partitionBy(numPartitions, partitionFunc=rangePartitionFunc) - .mapPartitions(mapFunc,preservesPartitioning=True) - .flatMap(lambda x: x, preservesPartitioning=True)) + return self.partitionBy(numPartitions, rangePartitionFunc).mapPartitions(mapFunc, True) def sortBy(self, keyfunc, ascending=True, numPartitions=None): """ @@ -570,7 +643,8 @@ def glom(self): >>> sorted(rdd.glom().collect()) [[1, 2], [3, 4]] """ - def func(iterator): yield list(iterator) + def func(iterator): + yield list(iterator) return self.mapPartitions(func) def cartesian(self, other): @@ -607,7 +681,9 @@ def pipe(self, command, env={}): ['1', '2', '', '3'] """ def func(iterator): - pipe = Popen(shlex.split(command), env=env, stdin=PIPE, stdout=PIPE) + pipe = Popen( + shlex.split(command), env=env, stdin=PIPE, stdout=PIPE) + def pipe_objs(out): for obj in iterator: out.write(str(obj).rstrip('\n') + '\n') @@ -646,7 +722,7 @@ def collect(self): Return a list that contains all of the elements in this RDD. """ with _JavaStackTrace(self.context) as st: - bytesInJava = self._jrdd.collect().iterator() + bytesInJava = self._jrdd.collect().iterator() return list(self._collect_iterator_through_file(bytesInJava)) def _collect_iterator_through_file(self, iterator): @@ -736,7 +812,6 @@ def func(iterator): return self.mapPartitions(func).fold(zeroValue, combOp) - def max(self): """ Find the maximum item in this RDD. @@ -844,6 +919,7 @@ def countPartition(iterator): for obj in iterator: counts[obj] += 1 yield counts + def mergeMaps(m1, m2): for (k, v) in m2.iteritems(): m1[k] += v @@ -888,22 +964,22 @@ def takeOrdered(self, num, key=None): def topNKeyedElems(iterator, key_=None): q = MaxHeapQ(num) for k in iterator: - if key_ != None: + if key_ is not None: k = (key_(k), k) q.insert(k) yield q.getElements() def unKey(x, key_=None): - if key_ != None: + if key_ is not None: x = [i[1] for i in x] return x def merge(a, b): return next(topNKeyedElems(a + b)) - result = self.mapPartitions(lambda i: topNKeyedElems(i, key)).reduce(merge) + result = self.mapPartitions( + lambda i: topNKeyedElems(i, key)).reduce(merge) return sorted(unKey(result, key), key=key) - def take(self, num): """ Take the first num elements of the RDD. @@ -947,7 +1023,8 @@ def takeUpToNumLeft(iterator): yield next(iterator) taken += 1 - p = range(partsScanned, min(partsScanned + numPartsToTry, totalParts)) + p = range( + partsScanned, min(partsScanned + numPartsToTry, totalParts)) res = self.context.runJob(self, takeUpToNumLeft, p, True) items += res @@ -964,6 +1041,117 @@ def first(self): """ return self.take(1)[0] + def saveAsNewAPIHadoopDataset(self, conf, keyConverter=None, valueConverter=None): + """ + Output a Python RDD of key-value pairs (of form C{RDD[(K, V)]}) to any Hadoop file + system, using the new Hadoop OutputFormat API (mapreduce package). Keys/values are + converted for output using either user specified converters or, by default, + L{org.apache.spark.api.python.JavaToWritableConverter}. + + @param conf: Hadoop job configuration, passed in as a dict + @param keyConverter: (None by default) + @param valueConverter: (None by default) + """ + jconf = self.ctx._dictToJavaMap(conf) + pickledRDD = self._toPickleSerialization() + batched = isinstance(pickledRDD._jrdd_deserializer, BatchedSerializer) + self.ctx._jvm.PythonRDD.saveAsHadoopDataset(pickledRDD._jrdd, batched, jconf, + keyConverter, valueConverter, True) + + def saveAsNewAPIHadoopFile(self, path, outputFormatClass, keyClass=None, valueClass=None, + keyConverter=None, valueConverter=None, conf=None): + """ + Output a Python RDD of key-value pairs (of form C{RDD[(K, V)]}) to any Hadoop file + system, using the new Hadoop OutputFormat API (mapreduce package). Key and value types + will be inferred if not specified. Keys and values are converted for output using either + user specified converters or L{org.apache.spark.api.python.JavaToWritableConverter}. The + C{conf} is applied on top of the base Hadoop conf associated with the SparkContext + of this RDD to create a merged Hadoop MapReduce job configuration for saving the data. + + @param path: path to Hadoop file + @param outputFormatClass: fully qualified classname of Hadoop OutputFormat + (e.g. "org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat") + @param keyClass: fully qualified classname of key Writable class + (e.g. "org.apache.hadoop.io.IntWritable", None by default) + @param valueClass: fully qualified classname of value Writable class + (e.g. "org.apache.hadoop.io.Text", None by default) + @param keyConverter: (None by default) + @param valueConverter: (None by default) + @param conf: Hadoop job configuration, passed in as a dict (None by default) + """ + jconf = self.ctx._dictToJavaMap(conf) + pickledRDD = self._toPickleSerialization() + batched = isinstance(pickledRDD._jrdd_deserializer, BatchedSerializer) + self.ctx._jvm.PythonRDD.saveAsNewAPIHadoopFile(pickledRDD._jrdd, batched, path, + outputFormatClass, + keyClass, valueClass, + keyConverter, valueConverter, jconf) + + def saveAsHadoopDataset(self, conf, keyConverter=None, valueConverter=None): + """ + Output a Python RDD of key-value pairs (of form C{RDD[(K, V)]}) to any Hadoop file + system, using the old Hadoop OutputFormat API (mapred package). Keys/values are + converted for output using either user specified converters or, by default, + L{org.apache.spark.api.python.JavaToWritableConverter}. + + @param conf: Hadoop job configuration, passed in as a dict + @param keyConverter: (None by default) + @param valueConverter: (None by default) + """ + jconf = self.ctx._dictToJavaMap(conf) + pickledRDD = self._toPickleSerialization() + batched = isinstance(pickledRDD._jrdd_deserializer, BatchedSerializer) + self.ctx._jvm.PythonRDD.saveAsHadoopDataset(pickledRDD._jrdd, batched, jconf, + keyConverter, valueConverter, False) + + def saveAsHadoopFile(self, path, outputFormatClass, keyClass=None, valueClass=None, + keyConverter=None, valueConverter=None, conf=None, + compressionCodecClass=None): + """ + Output a Python RDD of key-value pairs (of form C{RDD[(K, V)]}) to any Hadoop file + system, using the old Hadoop OutputFormat API (mapred package). Key and value types + will be inferred if not specified. Keys and values are converted for output using either + user specified converters or L{org.apache.spark.api.python.JavaToWritableConverter}. The + C{conf} is applied on top of the base Hadoop conf associated with the SparkContext + of this RDD to create a merged Hadoop MapReduce job configuration for saving the data. + + @param path: path to Hadoop file + @param outputFormatClass: fully qualified classname of Hadoop OutputFormat + (e.g. "org.apache.hadoop.mapred.SequenceFileOutputFormat") + @param keyClass: fully qualified classname of key Writable class + (e.g. "org.apache.hadoop.io.IntWritable", None by default) + @param valueClass: fully qualified classname of value Writable class + (e.g. "org.apache.hadoop.io.Text", None by default) + @param keyConverter: (None by default) + @param valueConverter: (None by default) + @param conf: (None by default) + @param compressionCodecClass: (None by default) + """ + jconf = self.ctx._dictToJavaMap(conf) + pickledRDD = self._toPickleSerialization() + batched = isinstance(pickledRDD._jrdd_deserializer, BatchedSerializer) + self.ctx._jvm.PythonRDD.saveAsHadoopFile(pickledRDD._jrdd, batched, path, + outputFormatClass, + keyClass, valueClass, + keyConverter, valueConverter, + jconf, compressionCodecClass) + + def saveAsSequenceFile(self, path, compressionCodecClass=None): + """ + Output a Python RDD of key-value pairs (of form C{RDD[(K, V)]}) to any Hadoop file + system, using the L{org.apache.hadoop.io.Writable} types that we convert from the + RDD's key and value types. The mechanism is as follows: + 1. Pyrolite is used to convert pickled Python RDD into RDD of Java objects. + 2. Keys and values of this Java RDD are converted to Writables and written out. + + @param path: path to sequence file + @param compressionCodecClass: (None by default) + """ + pickledRDD = self._toPickleSerialization() + batched = isinstance(pickledRDD._jrdd_deserializer, BatchedSerializer) + self.ctx._jvm.PythonRDD.saveAsSequenceFile(pickledRDD._jrdd, batched, + path, compressionCodecClass) + def saveAsPickleFile(self, path, batchSize=10): """ Save this RDD as a SequenceFile of serialized objects. The serializer @@ -977,7 +1165,7 @@ def saveAsPickleFile(self, path, batchSize=10): [1, 2, 'rdd', 'spark'] """ self._reserialize(BatchedSerializer(PickleSerializer(), - batchSize))._jrdd.saveAsObjectFile(path) + batchSize))._jrdd.saveAsObjectFile(path) def saveAsTextFile(self, path): """ @@ -1004,7 +1192,7 @@ def func(split, iterator): if not isinstance(x, basestring): x = unicode(x) yield x.encode("utf-8") - keyed = PipelinedRDD(self, func) + keyed = self.mapPartitionsWithIndex(func) keyed._bypass_serializer = True keyed._jrdd.map(self.ctx._jvm.BytesToString()).saveAsTextFile(path) @@ -1025,6 +1213,7 @@ def collectAsMap(self): def keys(self): """ Return an RDD with the keys of each tuple. + >>> m = sc.parallelize([(1, 2), (3, 4)]).keys() >>> m.collect() [1, 3] @@ -1034,6 +1223,7 @@ def keys(self): def values(self): """ Return an RDD with the values of each tuple. + >>> m = sc.parallelize([(1, 2), (3, 4)]).values() >>> m.collect() [2, 4] @@ -1075,6 +1265,7 @@ def reducePartition(iterator): for (k, v) in iterator: m[k] = v if k not in m else func(m[k], v) yield m + def mergeMaps(m1, m2): for (k, v) in m2.iteritems(): m1[k] = v if k not in m1 else func(m1[k], v) @@ -1144,7 +1335,9 @@ def rightOuterJoin(self, other, numPartitions=None): return python_right_outer_join(self, other, numPartitions) # TODO: add option to control map-side combining - def partitionBy(self, numPartitions, partitionFunc=None): + # portable_hash is used as default, because builtin hash of None is different + # cross machines. + def partitionBy(self, numPartitions, partitionFunc=portable_hash): """ Return a copy of the RDD partitioned using the specified partitioner. @@ -1156,31 +1349,60 @@ def partitionBy(self, numPartitions, partitionFunc=None): if numPartitions is None: numPartitions = self._defaultReducePartitions() - if partitionFunc is None: - partitionFunc = lambda x: 0 if x is None else hash(x) - # Transferring O(n) objects to Java is too expensive. Instead, we'll - # form the hash buckets in Python, transferring O(numPartitions) objects - # to Java. Each object is a (splitNumber, [objects]) pair. + # Transferring O(n) objects to Java is too expensive. + # Instead, we'll form the hash buckets in Python, + # transferring O(numPartitions) objects to Java. + # Each object is a (splitNumber, [objects]) pair. + # In order to avoid too huge objects, the objects are + # grouped into chunks. outputSerializer = self.ctx._unbatched_serializer + + limit = (_parse_memory(self.ctx._conf.get( + "spark.python.worker.memory", "512m")) / 2) + def add_shuffle_key(split, iterator): buckets = defaultdict(list) + c, batch = 0, min(10 * numPartitions, 1000) for (k, v) in iterator: buckets[partitionFunc(k) % numPartitions].append((k, v)) + c += 1 + + # check used memory and avg size of chunk of objects + if (c % 1000 == 0 and get_used_memory() > limit + or c > batch): + n, size = len(buckets), 0 + for split in buckets.keys(): + yield pack_long(split) + d = outputSerializer.dumps(buckets[split]) + del buckets[split] + yield d + size += len(d) + + avg = (size / n) >> 20 + # let 1M < avg < 10M + if avg < 1: + batch *= 1.5 + elif avg > 10: + batch = max(batch / 1.5, 1) + c = 0 + for (split, items) in buckets.iteritems(): yield pack_long(split) yield outputSerializer.dumps(items) - keyed = PipelinedRDD(self, add_shuffle_key) + + keyed = self.mapPartitionsWithIndex(add_shuffle_key) keyed._bypass_serializer = True with _JavaStackTrace(self.context) as st: - pairRDD = self.ctx._jvm.PairwiseRDD(keyed._jrdd.rdd()).asJavaPairRDD() + pairRDD = self.ctx._jvm.PairwiseRDD( + keyed._jrdd.rdd()).asJavaPairRDD() partitioner = self.ctx._jvm.PythonPartitioner(numPartitions, id(partitionFunc)) jrdd = pairRDD.partitionBy(partitioner).values() rdd = RDD(jrdd, self.ctx, BatchedSerializer(outputSerializer)) - # This is required so that id(partitionFunc) remains unique, even if - # partitionFunc is a lambda: + # This is required so that id(partitionFunc) remains unique, + # even if partitionFunc is a lambda: rdd._partitionFunc = partitionFunc return rdd @@ -1213,40 +1435,46 @@ def combineByKey(self, createCombiner, mergeValue, mergeCombiners, """ if numPartitions is None: numPartitions = self._defaultReducePartitions() + + serializer = self.ctx.serializer + spill = (self.ctx._conf.get("spark.shuffle.spill", 'True').lower() + == 'true') + memory = _parse_memory(self.ctx._conf.get( + "spark.python.worker.memory", "512m")) + agg = Aggregator(createCombiner, mergeValue, mergeCombiners) + def combineLocally(iterator): - combiners = {} - for x in iterator: - (k, v) = x - if k not in combiners: - combiners[k] = createCombiner(v) - else: - combiners[k] = mergeValue(combiners[k], v) - return combiners.iteritems() + merger = ExternalMerger(agg, memory * 0.9, serializer) \ + if spill else InMemoryMerger(agg) + merger.mergeValues(iterator) + return merger.iteritems() + locally_combined = self.mapPartitions(combineLocally) shuffled = locally_combined.partitionBy(numPartitions) + def _mergeCombiners(iterator): - combiners = {} - for (k, v) in iterator: - if not k in combiners: - combiners[k] = v - else: - combiners[k] = mergeCombiners(combiners[k], v) - return combiners.iteritems() + merger = ExternalMerger(agg, memory, serializer) \ + if spill else InMemoryMerger(agg) + merger.mergeCombiners(iterator) + return merger.iteritems() + return shuffled.mapPartitions(_mergeCombiners) def aggregateByKey(self, zeroValue, seqFunc, combFunc, numPartitions=None): """ - 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 + 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 createZero(): - return copy.deepcopy(zeroValue) + return copy.deepcopy(zeroValue) - return self.combineByKey(lambda v: seqFunc(createZero(), v), seqFunc, combFunc, numPartitions) + return self.combineByKey( + lambda v: seqFunc(createZero(), v), seqFunc, combFunc, numPartitions) def foldByKey(self, zeroValue, func, numPartitions=None): """ @@ -1261,11 +1489,10 @@ def foldByKey(self, zeroValue, func, numPartitions=None): [('a', 2), ('b', 1)] """ def createZero(): - return copy.deepcopy(zeroValue) + return copy.deepcopy(zeroValue) return self.combineByKey(lambda v: func(createZero(), v), func, func, numPartitions) - # TODO: support variant with custom partitioner def groupByKey(self, numPartitions=None): """ @@ -1289,10 +1516,11 @@ def mergeValue(xs, x): return xs def mergeCombiners(a, b): - return a + b + a.extend(b) + return a return self.combineByKey(createCombiner, mergeValue, mergeCombiners, - numPartitions).mapValues(lambda x: ResultIterable(x)) + numPartitions).mapValues(lambda x: ResultIterable(x)) # TODO: add tests def flatMapValues(self, f): @@ -1352,6 +1580,27 @@ def cogroup(self, other, numPartitions=None): """ return python_cogroup((self, other), numPartitions) + def sampleByKey(self, withReplacement, fractions, seed=None): + """ + Return a subset of this RDD sampled by key (via stratified sampling). + Create a sample of this RDD using variable sampling rates for + different keys as specified by fractions, a key to sampling rate map. + + >>> fractions = {"a": 0.2, "b": 0.1} + >>> rdd = sc.parallelize(fractions.keys()).cartesian(sc.parallelize(range(0, 1000))) + >>> sample = dict(rdd.sampleByKey(False, fractions, 2).groupByKey().collect()) + >>> 100 < len(sample["a"]) < 300 and 50 < len(sample["b"]) < 150 + True + >>> max(sample["a"]) <= 999 and min(sample["a"]) >= 0 + True + >>> max(sample["b"]) <= 999 and min(sample["b"]) >= 0 + True + """ + for fraction in fractions.values(): + assert fraction >= 0.0, "Negative fraction value: %s" % fraction + return self.mapPartitionsWithIndex( + RDDStratifiedSampler(withReplacement, fractions, seed).func, True) + def subtractByKey(self, other, numPartitions=None): """ Return each (key, value) pair in C{self} that has no pair with matching @@ -1362,7 +1611,8 @@ def subtractByKey(self, other, numPartitions=None): >>> sorted(x.subtractByKey(y).collect()) [('b', 4), ('b', 5)] """ - filter_func = lambda (key, vals): len(vals[0]) > 0 and len(vals[1]) == 0 + def filter_func((key, vals)): + return len(vals[0]) > 0 and len(vals[1]) == 0 map_func = lambda (key, vals): [(key, val) for val in vals[0]] return self.cogroup(other, numPartitions).filter(filter_func).flatMap(map_func) @@ -1375,8 +1625,9 @@ def subtract(self, other, numPartitions=None): >>> sorted(x.subtract(y).collect()) [('a', 1), ('b', 4), ('b', 5)] """ - rdd = other.map(lambda x: (x, True)) # note: here 'True' is just a placeholder - return self.map(lambda x: (x, True)).subtractByKey(rdd).map(lambda tpl: tpl[0]) # note: here 'True' is just a placeholder + # note: here 'True' is just a placeholder + rdd = other.map(lambda x: (x, True)) + return self.map(lambda x: (x, True)).subtractByKey(rdd).map(lambda tpl: tpl[0]) def keyBy(self, f): """ @@ -1397,6 +1648,7 @@ def repartition(self, numPartitions): Internally, this uses a shuffle to redistribute data. If you are decreasing the number of partitions in this RDD, consider using `coalesce`, which can avoid performing a shuffle. + >>> rdd = sc.parallelize([1,2,3,4,5,6,7], 4) >>> sorted(rdd.glom().collect()) [[1], [2, 3], [4, 5], [6, 7]] @@ -1411,6 +1663,7 @@ def repartition(self, numPartitions): def coalesce(self, numPartitions, shuffle=False): """ Return a new RDD that is reduced into `numPartitions` partitions. + >>> sc.parallelize([1, 2, 3, 4, 5], 3).glom().collect() [[1], [2, 3], [4, 5]] >>> sc.parallelize([1, 2, 3, 4, 5], 3).coalesce(1).glom().collect() @@ -1434,7 +1687,7 @@ def zip(self, other): """ pairRDD = self._jrdd.zip(other._jrdd) deserializer = PairDeserializer(self._jrdd_deserializer, - other._jrdd_deserializer) + other._jrdd_deserializer) return RDD(pairRDD, self.ctx, deserializer) def name(self): @@ -1449,6 +1702,7 @@ def name(self): def setName(self, name): """ Assign a name to this RDD. + >>> rdd1 = sc.parallelize([1,2]) >>> rdd1.setName('RDD1') >>> rdd1.name() @@ -1503,9 +1757,12 @@ def _defaultReducePartitions(self): # keys in the pairs. This could be an expensive operation, since those # hashes aren't retained. + class PipelinedRDD(RDD): + """ Pipelined maps: + >>> rdd = sc.parallelize([1, 2, 3, 4]) >>> rdd.map(lambda x: 2 * x).cache().map(lambda x: 2 * x).collect() [4, 8, 12, 16] @@ -1519,6 +1776,7 @@ class PipelinedRDD(RDD): >>> rdd.flatMap(lambda x: [x, x]).reduce(add) 20 """ + def __init__(self, prev, func, preservesPartitioning=False): if not isinstance(prev, PipelinedRDD) or not prev._is_pipelinable(): # This transformation is the first in its stage: @@ -1528,6 +1786,7 @@ def __init__(self, prev, func, preservesPartitioning=False): self._prev_jrdd_deserializer = prev._jrdd_deserializer else: prev_func = prev.func + def pipeline_func(split, iterator): return func(split, prev_func(split, iterator)) self.func = pipeline_func @@ -1551,20 +1810,21 @@ def _jrdd(self): self._jrdd_deserializer = NoOpSerializer() command = (self.func, self._prev_jrdd_deserializer, self._jrdd_deserializer) - pickled_command = CloudPickleSerializer().dumps(command) + ser = CompressedSerializer(CloudPickleSerializer()) + pickled_command = ser.dumps(command) broadcast_vars = ListConverter().convert( [x._jbroadcast for x in self.ctx._pickled_broadcast_vars], self.ctx._gateway._gateway_client) self.ctx._pickled_broadcast_vars.clear() - class_tag = self._prev_jrdd.classTag() env = MapConverter().convert(self.ctx.environment, self.ctx._gateway._gateway_client) includes = ListConverter().convert(self.ctx._python_includes, - self.ctx._gateway._gateway_client) + self.ctx._gateway._gateway_client) python_rdd = self.ctx._jvm.PythonRDD(self._prev_jrdd.rdd(), - bytearray(pickled_command), env, includes, self.preservesPartitioning, - self.ctx.pythonExec, broadcast_vars, self.ctx._javaAccumulator, - class_tag) + bytearray(pickled_command), + env, includes, self.preservesPartitioning, + self.ctx.pythonExec, + broadcast_vars, self.ctx._javaAccumulator) self._jrdd_val = python_rdd.asJavaRDD() return self._jrdd_val @@ -1579,7 +1839,8 @@ def _test(): # The small batch size here ensures that we see multiple batches, # even in these small test examples: globs['sc'] = SparkContext('local[4]', 'PythonTest', batchSize=2) - (failure_count, test_count) = doctest.testmod(globs=globs,optionflags=doctest.ELLIPSIS) + (failure_count, test_count) = doctest.testmod( + globs=globs, optionflags=doctest.ELLIPSIS) globs['sc'].stop() if failure_count: exit(-1) diff --git a/python/pyspark/rddsampler.py b/python/pyspark/rddsampler.py index 845a267e311c5..55e247da0e4dc 100644 --- a/python/pyspark/rddsampler.py +++ b/python/pyspark/rddsampler.py @@ -18,18 +18,21 @@ import sys import random -class RDDSampler(object): - def __init__(self, withReplacement, fraction, seed=None): + +class RDDSamplerBase(object): + + def __init__(self, withReplacement, seed=None): try: import numpy self._use_numpy = True except ImportError: - print >> sys.stderr, "NumPy does not appear to be installed. Falling back to default random generator for sampling." + print >> sys.stderr, ( + "NumPy does not appear to be installed. " + "Falling back to default random generator for sampling.") self._use_numpy = False self._seed = seed if seed is not None else random.randint(0, sys.maxint) self._withReplacement = withReplacement - self._fraction = fraction self._random = None self._split = None self._rand_initialized = False @@ -61,7 +64,7 @@ def getUniformSample(self, split): def getPoissonSample(self, split, mean): if not self._rand_initialized or split != self._split: self.initRandomGenerator(split) - + if self._use_numpy: return self._random.poisson(mean) else: @@ -80,23 +83,31 @@ def getPoissonSample(self, split, mean): num_arrivals += 1 return (num_arrivals - 1) - + def shuffle(self, vals): - if self._random == None: + if self._random is None: self.initRandomGenerator(0) # this should only ever called on the master so # the split does not matter - + if self._use_numpy: self._random.shuffle(vals) else: self._random.shuffle(vals, self._random.random) + +class RDDSampler(RDDSamplerBase): + + def __init__(self, withReplacement, fraction, seed=None): + RDDSamplerBase.__init__(self, withReplacement, seed) + self._fraction = fraction + def func(self, split, iterator): - if self._withReplacement: + if self._withReplacement: for obj in iterator: - # For large datasets, the expected number of occurrences of each element in a sample with - # replacement is Poisson(frac). We use that to get a count for each element. - count = self.getPoissonSample(split, mean = self._fraction) + # For large datasets, the expected number of occurrences of each element in + # a sample with replacement is Poisson(frac). We use that to get a count for + # each element. + count = self.getPoissonSample(split, mean=self._fraction) for _ in range(0, count): yield obj else: @@ -104,6 +115,23 @@ def func(self, split, iterator): if self.getUniformSample(split) <= self._fraction: yield obj - - - + +class RDDStratifiedSampler(RDDSamplerBase): + + def __init__(self, withReplacement, fractions, seed=None): + RDDSamplerBase.__init__(self, withReplacement, seed) + self._fractions = fractions + + def func(self, split, iterator): + if self._withReplacement: + for key, val in iterator: + # For large datasets, the expected number of occurrences of each element in + # a sample with replacement is Poisson(frac). We use that to get a count for + # each element. + count = self.getPoissonSample(split, mean=self._fractions[key]) + for _ in range(0, count): + yield key, val + else: + for key, val in iterator: + if self.getUniformSample(split) <= self._fractions[key]: + yield key, val diff --git a/python/pyspark/resultiterable.py b/python/pyspark/resultiterable.py index 7f418f8d2e29a..ef04c82866e6c 100644 --- a/python/pyspark/resultiterable.py +++ b/python/pyspark/resultiterable.py @@ -19,15 +19,20 @@ import collections + class ResultIterable(collections.Iterable): + """ A special result iterable. This is used because the standard iterator can not be pickled """ + def __init__(self, data): self.data = data self.index = 0 self.maxindex = len(data) + def __iter__(self): return iter(self.data) + def __len__(self): return len(self.data) diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index b253807974a2e..74870c0edcf99 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -65,6 +65,10 @@ import marshal import struct import sys +import types +import collections +import zlib + from pyspark import cloudpickle @@ -91,7 +95,6 @@ def load_stream(self, stream): """ raise NotImplementedError - def _load_stream_without_unbatching(self, stream): return self.load_stream(stream) @@ -109,6 +112,7 @@ def __ne__(self, other): class FramedSerializer(Serializer): + """ Serializer that writes objects as a stream of (length, data) pairs, where C{length} is a 32-bit integer and data is C{length} bytes. @@ -160,6 +164,7 @@ def loads(self, obj): class BatchedSerializer(Serializer): + """ Serializes a stream of objects in batches by calling its wrapped Serializer with streams of objects. @@ -194,17 +199,18 @@ def load_stream(self, stream): return chain.from_iterable(self._load_stream_without_unbatching(stream)) def _load_stream_without_unbatching(self, stream): - return self.serializer.load_stream(stream) + return self.serializer.load_stream(stream) def __eq__(self, other): - return isinstance(other, BatchedSerializer) and \ - other.serializer == self.serializer + return (isinstance(other, BatchedSerializer) and + other.serializer == self.serializer) def __str__(self): return "BatchedSerializer<%s>" % str(self.serializer) class CartesianDeserializer(FramedSerializer): + """ Deserializes the JavaRDD cartesian() of two PythonRDDs. """ @@ -229,8 +235,8 @@ def load_stream(self, stream): yield pair def __eq__(self, other): - return isinstance(other, CartesianDeserializer) and \ - self.key_ser == other.key_ser and self.val_ser == other.val_ser + return (isinstance(other, CartesianDeserializer) and + self.key_ser == other.key_ser and self.val_ser == other.val_ser) def __str__(self): return "CartesianDeserializer<%s, %s>" % \ @@ -238,6 +244,7 @@ def __str__(self): class PairDeserializer(CartesianDeserializer): + """ Deserializes the JavaRDD zip() of two PythonRDDs. """ @@ -252,21 +259,87 @@ def load_stream(self, stream): yield pair def __eq__(self, other): - return isinstance(other, PairDeserializer) and \ - self.key_ser == other.key_ser and self.val_ser == other.val_ser + return (isinstance(other, PairDeserializer) and + self.key_ser == other.key_ser and self.val_ser == other.val_ser) def __str__(self): - return "PairDeserializer<%s, %s>" % \ - (str(self.key_ser), str(self.val_ser)) + return "PairDeserializer<%s, %s>" % (str(self.key_ser), str(self.val_ser)) class NoOpSerializer(FramedSerializer): - def loads(self, obj): return obj - def dumps(self, obj): return obj + def loads(self, obj): + return obj + + def dumps(self, obj): + return obj + + +# Hook namedtuple, make it picklable + +__cls = {} + + +def _restore(name, fields, value): + """ Restore an object of namedtuple""" + k = (name, fields) + cls = __cls.get(k) + if cls is None: + cls = collections.namedtuple(name, fields) + __cls[k] = cls + return cls(*value) + + +def _hack_namedtuple(cls): + """ Make class generated by namedtuple picklable """ + name = cls.__name__ + fields = cls._fields + + def __reduce__(self): + return (_restore, (name, fields, tuple(self))) + cls.__reduce__ = __reduce__ + return cls + + +def _hijack_namedtuple(): + """ Hack namedtuple() to make it picklable """ + # hijack only one time + if hasattr(collections.namedtuple, "__hijack"): + return + + global _old_namedtuple # or it will put in closure + + def _copy_func(f): + return types.FunctionType(f.func_code, f.func_globals, f.func_name, + f.func_defaults, f.func_closure) + + _old_namedtuple = _copy_func(collections.namedtuple) + + def namedtuple(*args, **kwargs): + cls = _old_namedtuple(*args, **kwargs) + return _hack_namedtuple(cls) + + # replace namedtuple with new one + collections.namedtuple.func_globals["_old_namedtuple"] = _old_namedtuple + collections.namedtuple.func_globals["_hack_namedtuple"] = _hack_namedtuple + collections.namedtuple.func_code = namedtuple.func_code + collections.namedtuple.__hijack = 1 + + # hack the cls already generated by namedtuple + # those created in other module can be pickled as normal, + # so only hack those in __main__ module + for n, o in sys.modules["__main__"].__dict__.iteritems(): + if (type(o) is type and o.__base__ is tuple + and hasattr(o, "_fields") + and "__reduce__" not in o.__dict__): + _hack_namedtuple(o) # hack inplace + + +_hijack_namedtuple() class PickleSerializer(FramedSerializer): + """ Serializes objects using Python's cPickle serializer: @@ -276,15 +349,20 @@ class PickleSerializer(FramedSerializer): not be as fast as more specialized serializers. """ - def dumps(self, obj): return cPickle.dumps(obj, 2) + def dumps(self, obj): + return cPickle.dumps(obj, 2) + loads = cPickle.loads + class CloudPickleSerializer(PickleSerializer): - def dumps(self, obj): return cloudpickle.dumps(obj, 2) + def dumps(self, obj): + return cloudpickle.dumps(obj, 2) class MarshalSerializer(FramedSerializer): + """ Serializes objects using Python's Marshal serializer: @@ -297,7 +375,53 @@ class MarshalSerializer(FramedSerializer): loads = marshal.loads +class AutoSerializer(FramedSerializer): + + """ + Choose marshal or cPickle as serialization protocol autumatically + """ + + def __init__(self): + FramedSerializer.__init__(self) + self._type = None + + def dumps(self, obj): + if self._type is not None: + return 'P' + cPickle.dumps(obj, -1) + try: + return 'M' + marshal.dumps(obj) + except Exception: + self._type = 'P' + return 'P' + cPickle.dumps(obj, -1) + + def loads(self, obj): + _type = obj[0] + if _type == 'M': + return marshal.loads(obj[1:]) + elif _type == 'P': + return cPickle.loads(obj[1:]) + else: + raise ValueError("invalid sevialization type: %s" % _type) + + +class CompressedSerializer(FramedSerializer): + """ + compress the serialized data + """ + + def __init__(self, serializer): + FramedSerializer.__init__(self) + self.serializer = serializer + + def dumps(self, obj): + return zlib.compress(self.serializer.dumps(obj), 1) + + def loads(self, obj): + return self.serializer.loads(zlib.decompress(obj)) + + class UTF8Deserializer(Serializer): + """ Deserializes streams written by String.getBytes. """ diff --git a/python/pyspark/shell.py b/python/pyspark/shell.py index ebd714db7a918..e1e7cd954189f 100644 --- a/python/pyspark/shell.py +++ b/python/pyspark/shell.py @@ -35,7 +35,8 @@ from pyspark.storagelevel import StorageLevel # this is the equivalent of ADD_JARS -add_files = os.environ.get("ADD_FILES").split(',') if os.environ.get("ADD_FILES") != None else None +add_files = (os.environ.get("ADD_FILES").split(',') + if os.environ.get("ADD_FILES") is not None else None) if os.environ.get("SPARK_EXECUTOR_URI"): SparkContext.setSystemProperty("spark.executor.uri", os.environ["SPARK_EXECUTOR_URI"]) @@ -55,7 +56,7 @@ platform.python_build()[1])) print("SparkContext available as sc.") -if add_files != None: +if add_files is not None: print("Adding files: [%s]" % ", ".join(add_files)) # The ./bin/pyspark script stores the old PYTHONSTARTUP value in OLD_PYTHONSTARTUP, diff --git a/python/pyspark/shuffle.py b/python/pyspark/shuffle.py new file mode 100644 index 0000000000000..2c68cd4921deb --- /dev/null +++ b/python/pyspark/shuffle.py @@ -0,0 +1,439 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import os +import sys +import platform +import shutil +import warnings +import gc + +from pyspark.serializers import BatchedSerializer, PickleSerializer + +try: + import psutil + + def get_used_memory(): + """ Return the used memory in MB """ + process = psutil.Process(os.getpid()) + if hasattr(process, "memory_info"): + info = process.memory_info() + else: + info = process.get_memory_info() + return info.rss >> 20 +except ImportError: + + def get_used_memory(): + """ Return the used memory in MB """ + if platform.system() == 'Linux': + for line in open('/proc/self/status'): + if line.startswith('VmRSS:'): + return int(line.split()[1]) >> 10 + else: + warnings.warn("Please install psutil to have better " + "support with spilling") + if platform.system() == "Darwin": + import resource + rss = resource.getrusage(resource.RUSAGE_SELF).ru_maxrss + return rss >> 20 + # TODO: support windows + return 0 + + +class Aggregator(object): + + """ + Aggregator has tree functions to merge values into combiner. + + createCombiner: (value) -> combiner + mergeValue: (combine, value) -> combiner + mergeCombiners: (combiner, combiner) -> combiner + """ + + def __init__(self, createCombiner, mergeValue, mergeCombiners): + self.createCombiner = createCombiner + self.mergeValue = mergeValue + self.mergeCombiners = mergeCombiners + + +class SimpleAggregator(Aggregator): + + """ + SimpleAggregator is useful for the cases that combiners have + same type with values + """ + + def __init__(self, combiner): + Aggregator.__init__(self, lambda x: x, combiner, combiner) + + +class Merger(object): + + """ + Merge shuffled data together by aggregator + """ + + def __init__(self, aggregator): + self.agg = aggregator + + def mergeValues(self, iterator): + """ Combine the items by creator and combiner """ + raise NotImplementedError + + def mergeCombiners(self, iterator): + """ Merge the combined items by mergeCombiner """ + raise NotImplementedError + + def iteritems(self): + """ Return the merged items ad iterator """ + raise NotImplementedError + + +class InMemoryMerger(Merger): + + """ + In memory merger based on in-memory dict. + """ + + def __init__(self, aggregator): + Merger.__init__(self, aggregator) + self.data = {} + + def mergeValues(self, iterator): + """ Combine the items by creator and combiner """ + # speed up attributes lookup + d, creator = self.data, self.agg.createCombiner + comb = self.agg.mergeValue + for k, v in iterator: + d[k] = comb(d[k], v) if k in d else creator(v) + + def mergeCombiners(self, iterator): + """ Merge the combined items by mergeCombiner """ + # speed up attributes lookup + d, comb = self.data, self.agg.mergeCombiners + for k, v in iterator: + d[k] = comb(d[k], v) if k in d else v + + def iteritems(self): + """ Return the merged items ad iterator """ + return self.data.iteritems() + + +class ExternalMerger(Merger): + + """ + External merger will dump the aggregated data into disks when + memory usage goes above the limit, then merge them together. + + This class works as follows: + + - It repeatedly combine the items and save them in one dict in + memory. + + - When the used memory goes above memory limit, it will split + the combined data into partitions by hash code, dump them + into disk, one file per partition. + + - Then it goes through the rest of the iterator, combine items + into different dict by hash. Until the used memory goes over + memory limit, it dump all the dicts into disks, one file per + dict. Repeat this again until combine all the items. + + - Before return any items, it will load each partition and + combine them seperately. Yield them before loading next + partition. + + - During loading a partition, if the memory goes over limit, + it will partition the loaded data and dump them into disks + and load them partition by partition again. + + `data` and `pdata` are used to hold the merged items in memory. + At first, all the data are merged into `data`. Once the used + memory goes over limit, the items in `data` are dumped indo + disks, `data` will be cleared, all rest of items will be merged + into `pdata` and then dumped into disks. Before returning, all + the items in `pdata` will be dumped into disks. + + Finally, if any items were spilled into disks, each partition + will be merged into `data` and be yielded, then cleared. + + >>> agg = SimpleAggregator(lambda x, y: x + y) + >>> merger = ExternalMerger(agg, 10) + >>> N = 10000 + >>> merger.mergeValues(zip(xrange(N), xrange(N)) * 10) + >>> assert merger.spills > 0 + >>> sum(v for k,v in merger.iteritems()) + 499950000 + + >>> merger = ExternalMerger(agg, 10) + >>> merger.mergeCombiners(zip(xrange(N), xrange(N)) * 10) + >>> assert merger.spills > 0 + >>> sum(v for k,v in merger.iteritems()) + 499950000 + """ + + # the max total partitions created recursively + MAX_TOTAL_PARTITIONS = 4096 + + def __init__(self, aggregator, memory_limit=512, serializer=None, + localdirs=None, scale=1, partitions=59, batch=1000): + Merger.__init__(self, aggregator) + self.memory_limit = memory_limit + # default serializer is only used for tests + self.serializer = serializer or \ + BatchedSerializer(PickleSerializer(), 1024) + self.localdirs = localdirs or self._get_dirs() + # number of partitions when spill data into disks + self.partitions = partitions + # check the memory after # of items merged + self.batch = batch + # scale is used to scale down the hash of key for recursive hash map + self.scale = scale + # unpartitioned merged data + self.data = {} + # partitioned merged data, list of dicts + self.pdata = [] + # number of chunks dumped into disks + self.spills = 0 + # randomize the hash of key, id(o) is the address of o (aligned by 8) + self._seed = id(self) + 7 + + def _get_dirs(self): + """ Get all the directories """ + path = os.environ.get("SPARK_LOCAL_DIR", "/tmp") + dirs = path.split(",") + return [os.path.join(d, "python", str(os.getpid()), str(id(self))) + for d in dirs] + + def _get_spill_dir(self, n): + """ Choose one directory for spill by number n """ + return os.path.join(self.localdirs[n % len(self.localdirs)], str(n)) + + def _next_limit(self): + """ + Return the next memory limit. If the memory is not released + after spilling, it will dump the data only when the used memory + starts to increase. + """ + return max(self.memory_limit, get_used_memory() * 1.05) + + def mergeValues(self, iterator): + """ Combine the items by creator and combiner """ + iterator = iter(iterator) + # speedup attribute lookup + creator, comb = self.agg.createCombiner, self.agg.mergeValue + d, c, batch = self.data, 0, self.batch + + for k, v in iterator: + d[k] = comb(d[k], v) if k in d else creator(v) + + c += 1 + if c % batch == 0 and get_used_memory() > self.memory_limit: + self._spill() + self._partitioned_mergeValues(iterator, self._next_limit()) + break + + def _partition(self, key): + """ Return the partition for key """ + return hash((key, self._seed)) % self.partitions + + def _partitioned_mergeValues(self, iterator, limit=0): + """ Partition the items by key, then combine them """ + # speedup attribute lookup + creator, comb = self.agg.createCombiner, self.agg.mergeValue + c, pdata, hfun, batch = 0, self.pdata, self._partition, self.batch + + for k, v in iterator: + d = pdata[hfun(k)] + d[k] = comb(d[k], v) if k in d else creator(v) + if not limit: + continue + + c += 1 + if c % batch == 0 and get_used_memory() > limit: + self._spill() + limit = self._next_limit() + + def mergeCombiners(self, iterator, check=True): + """ Merge (K,V) pair by mergeCombiner """ + iterator = iter(iterator) + # speedup attribute lookup + d, comb, batch = self.data, self.agg.mergeCombiners, self.batch + c = 0 + for k, v in iterator: + d[k] = comb(d[k], v) if k in d else v + if not check: + continue + + c += 1 + if c % batch == 0 and get_used_memory() > self.memory_limit: + self._spill() + self._partitioned_mergeCombiners(iterator, self._next_limit()) + break + + def _partitioned_mergeCombiners(self, iterator, limit=0): + """ Partition the items by key, then merge them """ + comb, pdata = self.agg.mergeCombiners, self.pdata + c, hfun = 0, self._partition + for k, v in iterator: + d = pdata[hfun(k)] + d[k] = comb(d[k], v) if k in d else v + if not limit: + continue + + c += 1 + if c % self.batch == 0 and get_used_memory() > limit: + self._spill() + limit = self._next_limit() + + def _spill(self): + """ + dump already partitioned data into disks. + + It will dump the data in batch for better performance. + """ + path = self._get_spill_dir(self.spills) + if not os.path.exists(path): + os.makedirs(path) + + if not self.pdata: + # The data has not been partitioned, it will iterator the + # dataset once, write them into different files, has no + # additional memory. It only called when the memory goes + # above limit at the first time. + + # open all the files for writing + streams = [open(os.path.join(path, str(i)), 'w') + for i in range(self.partitions)] + + for k, v in self.data.iteritems(): + h = self._partition(k) + # put one item in batch, make it compatitable with load_stream + # it will increase the memory if dump them in batch + self.serializer.dump_stream([(k, v)], streams[h]) + + for s in streams: + s.close() + + self.data.clear() + self.pdata = [{} for i in range(self.partitions)] + + else: + for i in range(self.partitions): + p = os.path.join(path, str(i)) + with open(p, "w") as f: + # dump items in batch + self.serializer.dump_stream(self.pdata[i].iteritems(), f) + self.pdata[i].clear() + + self.spills += 1 + gc.collect() # release the memory as much as possible + + def iteritems(self): + """ Return all merged items as iterator """ + if not self.pdata and not self.spills: + return self.data.iteritems() + return self._external_items() + + def _external_items(self): + """ Return all partitioned items as iterator """ + assert not self.data + if any(self.pdata): + self._spill() + hard_limit = self._next_limit() + + try: + for i in range(self.partitions): + self.data = {} + for j in range(self.spills): + path = self._get_spill_dir(j) + p = os.path.join(path, str(i)) + # do not check memory during merging + self.mergeCombiners(self.serializer.load_stream(open(p)), + False) + + # limit the total partitions + if (self.scale * self.partitions < self.MAX_TOTAL_PARTITIONS + and j < self.spills - 1 + and get_used_memory() > hard_limit): + self.data.clear() # will read from disk again + gc.collect() # release the memory as much as possible + for v in self._recursive_merged_items(i): + yield v + return + + for v in self.data.iteritems(): + yield v + self.data.clear() + gc.collect() + + # remove the merged partition + for j in range(self.spills): + path = self._get_spill_dir(j) + os.remove(os.path.join(path, str(i))) + + finally: + self._cleanup() + + def _cleanup(self): + """ Clean up all the files in disks """ + for d in self.localdirs: + shutil.rmtree(d, True) + + def _recursive_merged_items(self, start): + """ + merge the partitioned items and return the as iterator + + If one partition can not be fit in memory, then them will be + partitioned and merged recursively. + """ + # make sure all the data are dumps into disks. + assert not self.data + if any(self.pdata): + self._spill() + assert self.spills > 0 + + for i in range(start, self.partitions): + subdirs = [os.path.join(d, "parts", str(i)) + for d in self.localdirs] + m = ExternalMerger(self.agg, self.memory_limit, self.serializer, + subdirs, self.scale * self.partitions) + m.pdata = [{} for _ in range(self.partitions)] + limit = self._next_limit() + + for j in range(self.spills): + path = self._get_spill_dir(j) + p = os.path.join(path, str(i)) + m._partitioned_mergeCombiners( + self.serializer.load_stream(open(p))) + + if get_used_memory() > limit: + m._spill() + limit = self._next_limit() + + for v in m._external_items(): + yield v + + # remove the merged partition + for j in range(self.spills): + path = self._get_spill_dir(j) + os.remove(os.path.join(path, str(i))) + + +if __name__ == "__main__": + import doctest + doctest.testmod() diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index ffe177576f363..d4ca0cc8f336e 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -15,31 +15,910 @@ # limitations under the License. # + +import sys +import types +import itertools +import warnings +import decimal +import datetime +import keyword +import warnings +from array import array +from operator import itemgetter + from pyspark.rdd import RDD, PipelinedRDD -from pyspark.serializers import BatchedSerializer, PickleSerializer +from pyspark.serializers import BatchedSerializer, PickleSerializer, CloudPickleSerializer + +from itertools import chain, ifilter, imap from py4j.protocol import Py4JError +from py4j.java_collections import ListConverter, MapConverter + + +__all__ = [ + "StringType", "BinaryType", "BooleanType", "TimestampType", "DecimalType", + "DoubleType", "FloatType", "ByteType", "IntegerType", "LongType", + "ShortType", "ArrayType", "MapType", "StructField", "StructType", + "SQLContext", "HiveContext", "LocalHiveContext", "TestHiveContext", + "SchemaRDD", "Row"] + + +class DataType(object): + + """Spark SQL DataType""" + + def __repr__(self): + return self.__class__.__name__ + + def __hash__(self): + return hash(str(self)) + + def __eq__(self, other): + return (isinstance(other, self.__class__) and + self.__dict__ == other.__dict__) + + def __ne__(self, other): + return not self.__eq__(other) + + +class PrimitiveTypeSingleton(type): + + """Metaclass for PrimitiveType""" + + _instances = {} + + def __call__(cls): + if cls not in cls._instances: + cls._instances[cls] = super(PrimitiveTypeSingleton, cls).__call__() + return cls._instances[cls] + + +class PrimitiveType(DataType): + + """Spark SQL PrimitiveType""" + + __metaclass__ = PrimitiveTypeSingleton + + def __eq__(self, other): + # because they should be the same object + return self is other + + +class StringType(PrimitiveType): + + """Spark SQL StringType + + The data type representing string values. + """ + + +class BinaryType(PrimitiveType): + + """Spark SQL BinaryType + + The data type representing bytearray values. + """ + + +class BooleanType(PrimitiveType): + + """Spark SQL BooleanType + + The data type representing bool values. + """ + + +class TimestampType(PrimitiveType): + + """Spark SQL TimestampType + + The data type representing datetime.datetime values. + """ + + +class DecimalType(PrimitiveType): + + """Spark SQL DecimalType + + The data type representing decimal.Decimal values. + """ + + +class DoubleType(PrimitiveType): + + """Spark SQL DoubleType + + The data type representing float values. + """ + + +class FloatType(PrimitiveType): + + """Spark SQL FloatType + + The data type representing single precision floating-point values. + """ + + +class ByteType(PrimitiveType): + + """Spark SQL ByteType + + The data type representing int values with 1 singed byte. + """ + + +class IntegerType(PrimitiveType): + + """Spark SQL IntegerType + + The data type representing int values. + """ + + +class LongType(PrimitiveType): + + """Spark SQL LongType + + The data type representing long values. If the any value is + beyond the range of [-9223372036854775808, 9223372036854775807], + please use DecimalType. + """ + + +class ShortType(PrimitiveType): + + """Spark SQL ShortType + + The data type representing int values with 2 signed bytes. + """ + + +class ArrayType(DataType): + + """Spark SQL ArrayType + + The data type representing list values. An ArrayType object + comprises two fields, elementType (a DataType) and containsNull (a bool). + The field of elementType is used to specify the type of array elements. + The field of containsNull is used to specify if the array has None values. + + """ + + def __init__(self, elementType, containsNull=False): + """Creates an ArrayType + + :param elementType: the data type of elements. + :param containsNull: indicates whether the list contains None values. + + >>> ArrayType(StringType) == ArrayType(StringType, False) + True + >>> ArrayType(StringType, True) == ArrayType(StringType) + False + """ + self.elementType = elementType + self.containsNull = containsNull + + def __str__(self): + return "ArrayType(%s,%s)" % (self.elementType, + str(self.containsNull).lower()) + + +class MapType(DataType): + + """Spark SQL MapType + + The data type representing dict values. A MapType object comprises + three fields, keyType (a DataType), valueType (a DataType) and + valueContainsNull (a bool). + + The field of keyType is used to specify the type of keys in the map. + The field of valueType is used to specify the type of values in the map. + The field of valueContainsNull is used to specify if values of this + map has None values. + + For values of a MapType column, keys are not allowed to have None values. + + """ + + def __init__(self, keyType, valueType, valueContainsNull=True): + """Creates a MapType + :param keyType: the data type of keys. + :param valueType: the data type of values. + :param valueContainsNull: indicates whether values contains + null values. + + >>> (MapType(StringType, IntegerType) + ... == MapType(StringType, IntegerType, True)) + True + >>> (MapType(StringType, IntegerType, False) + ... == MapType(StringType, FloatType)) + False + """ + self.keyType = keyType + self.valueType = valueType + self.valueContainsNull = valueContainsNull + + def __repr__(self): + return "MapType(%s,%s,%s)" % (self.keyType, self.valueType, + str(self.valueContainsNull).lower()) + + +class StructField(DataType): + + """Spark SQL StructField + + Represents a field in a StructType. + A StructField object comprises three fields, name (a string), + dataType (a DataType) and nullable (a bool). The field of name + is the name of a StructField. The field of dataType specifies + the data type of a StructField. + + The field of nullable specifies if values of a StructField can + contain None values. + + """ + + def __init__(self, name, dataType, nullable): + """Creates a StructField + :param name: the name of this field. + :param dataType: the data type of this field. + :param nullable: indicates whether values of this field + can be null. + + >>> (StructField("f1", StringType, True) + ... == StructField("f1", StringType, True)) + True + >>> (StructField("f1", StringType, True) + ... == StructField("f2", StringType, True)) + False + """ + self.name = name + self.dataType = dataType + self.nullable = nullable + + def __repr__(self): + return "StructField(%s,%s,%s)" % (self.name, self.dataType, + str(self.nullable).lower()) + + +class StructType(DataType): + + """Spark SQL StructType + + The data type representing rows. + A StructType object comprises a list of L{StructField}s. + + """ + + def __init__(self, fields): + """Creates a StructType + + >>> struct1 = StructType([StructField("f1", StringType, True)]) + >>> struct2 = StructType([StructField("f1", StringType, True)]) + >>> struct1 == struct2 + True + >>> struct1 = StructType([StructField("f1", StringType, True)]) + >>> struct2 = StructType([StructField("f1", StringType, True), + ... [StructField("f2", IntegerType, False)]]) + >>> struct1 == struct2 + False + """ + self.fields = fields + + def __repr__(self): + return ("StructType(List(%s))" % + ",".join(str(field) for field in self.fields)) + + +def _parse_datatype_list(datatype_list_string): + """Parses a list of comma separated data types.""" + index = 0 + datatype_list = [] + start = 0 + depth = 0 + while index < len(datatype_list_string): + if depth == 0 and datatype_list_string[index] == ",": + datatype_string = datatype_list_string[start:index].strip() + datatype_list.append(_parse_datatype_string(datatype_string)) + start = index + 1 + elif datatype_list_string[index] == "(": + depth += 1 + elif datatype_list_string[index] == ")": + depth -= 1 + + index += 1 + + # Handle the last data type + datatype_string = datatype_list_string[start:index].strip() + datatype_list.append(_parse_datatype_string(datatype_string)) + return datatype_list + + +_all_primitive_types = dict((k, v) for k, v in globals().iteritems() + if type(v) is PrimitiveTypeSingleton and v.__base__ == PrimitiveType) + + +def _parse_datatype_string(datatype_string): + """Parses the given data type string. + + >>> def check_datatype(datatype): + ... scala_datatype = sqlCtx._ssql_ctx.parseDataType(str(datatype)) + ... python_datatype = _parse_datatype_string( + ... scala_datatype.toString()) + ... return datatype == python_datatype + >>> all(check_datatype(cls()) for cls in _all_primitive_types.values()) + True + >>> # Simple ArrayType. + >>> simple_arraytype = ArrayType(StringType(), True) + >>> check_datatype(simple_arraytype) + True + >>> # Simple MapType. + >>> simple_maptype = MapType(StringType(), LongType()) + >>> check_datatype(simple_maptype) + True + >>> # Simple StructType. + >>> simple_structtype = StructType([ + ... StructField("a", DecimalType(), False), + ... StructField("b", BooleanType(), True), + ... StructField("c", LongType(), True), + ... StructField("d", BinaryType(), False)]) + >>> check_datatype(simple_structtype) + True + >>> # Complex StructType. + >>> complex_structtype = StructType([ + ... StructField("simpleArray", simple_arraytype, True), + ... StructField("simpleMap", simple_maptype, True), + ... StructField("simpleStruct", simple_structtype, True), + ... StructField("boolean", BooleanType(), False)]) + >>> check_datatype(complex_structtype) + True + >>> # Complex ArrayType. + >>> complex_arraytype = ArrayType(complex_structtype, True) + >>> check_datatype(complex_arraytype) + True + >>> # Complex MapType. + >>> complex_maptype = MapType(complex_structtype, + ... complex_arraytype, False) + >>> check_datatype(complex_maptype) + True + """ + index = datatype_string.find("(") + if index == -1: + # It is a primitive type. + index = len(datatype_string) + type_or_field = datatype_string[:index] + rest_part = datatype_string[index + 1:len(datatype_string) - 1].strip() + + if type_or_field in _all_primitive_types: + return _all_primitive_types[type_or_field]() + + elif type_or_field == "ArrayType": + last_comma_index = rest_part.rfind(",") + containsNull = True + if rest_part[last_comma_index + 1:].strip().lower() == "false": + containsNull = False + elementType = _parse_datatype_string( + rest_part[:last_comma_index].strip()) + return ArrayType(elementType, containsNull) + + elif type_or_field == "MapType": + last_comma_index = rest_part.rfind(",") + valueContainsNull = True + if rest_part[last_comma_index + 1:].strip().lower() == "false": + valueContainsNull = False + keyType, valueType = _parse_datatype_list( + rest_part[:last_comma_index].strip()) + return MapType(keyType, valueType, valueContainsNull) + + elif type_or_field == "StructField": + first_comma_index = rest_part.find(",") + name = rest_part[:first_comma_index].strip() + last_comma_index = rest_part.rfind(",") + nullable = True + if rest_part[last_comma_index + 1:].strip().lower() == "false": + nullable = False + dataType = _parse_datatype_string( + rest_part[first_comma_index + 1:last_comma_index].strip()) + return StructField(name, dataType, nullable) + + elif type_or_field == "StructType": + # rest_part should be in the format like + # List(StructField(field1,IntegerType,false)). + field_list_string = rest_part[rest_part.find("(") + 1:-1] + fields = _parse_datatype_list(field_list_string) + return StructType(fields) + + +# Mapping Python types to Spark SQL DateType +_type_mappings = { + bool: BooleanType, + int: IntegerType, + long: LongType, + float: DoubleType, + str: StringType, + unicode: StringType, + decimal.Decimal: DecimalType, + datetime.datetime: TimestampType, + datetime.date: TimestampType, + datetime.time: TimestampType, +} + + +def _infer_type(obj): + """Infer the DataType from obj""" + if obj is None: + raise ValueError("Can not infer type for None") + + dataType = _type_mappings.get(type(obj)) + if dataType is not None: + return dataType() + + if isinstance(obj, dict): + if not obj: + raise ValueError("Can not infer type for empty dict") + key, value = obj.iteritems().next() + return MapType(_infer_type(key), _infer_type(value), True) + elif isinstance(obj, (list, array)): + if not obj: + raise ValueError("Can not infer type for empty list/array") + return ArrayType(_infer_type(obj[0]), True) + else: + try: + return _infer_schema(obj) + except ValueError: + raise ValueError("not supported type: %s" % type(obj)) + + +def _infer_schema(row): + """Infer the schema from dict/namedtuple/object""" + if isinstance(row, dict): + items = sorted(row.items()) + + elif isinstance(row, tuple): + if hasattr(row, "_fields"): # namedtuple + items = zip(row._fields, tuple(row)) + elif hasattr(row, "__FIELDS__"): # Row + items = zip(row.__FIELDS__, tuple(row)) + elif all(isinstance(x, tuple) and len(x) == 2 for x in row): + items = row + else: + raise ValueError("Can't infer schema from tuple") + + elif hasattr(row, "__dict__"): # object + items = sorted(row.__dict__.items()) + + else: + raise ValueError("Can not infer schema for type: %s" % type(row)) + + fields = [StructField(k, _infer_type(v), True) for k, v in items] + return StructType(fields) + + +def _create_converter(obj, dataType): + """Create an converter to drop the names of fields in obj """ + if isinstance(dataType, ArrayType): + conv = _create_converter(obj[0], dataType.elementType) + return lambda row: map(conv, row) + + elif isinstance(dataType, MapType): + value = obj.values()[0] + conv = _create_converter(value, dataType.valueType) + return lambda row: dict((k, conv(v)) for k, v in row.iteritems()) + + elif not isinstance(dataType, StructType): + return lambda x: x + + # dataType must be StructType + names = [f.name for f in dataType.fields] + + if isinstance(obj, dict): + conv = lambda o: tuple(o.get(n) for n in names) + + elif isinstance(obj, tuple): + if hasattr(obj, "_fields"): # namedtuple + conv = tuple + elif hasattr(obj, "__FIELDS__"): + conv = tuple + elif all(isinstance(x, tuple) and len(x) == 2 for x in obj): + conv = lambda o: tuple(v for k, v in o) + else: + raise ValueError("unexpected tuple") + + elif hasattr(obj, "__dict__"): # object + conv = lambda o: [o.__dict__.get(n, None) for n in names] + + if all(isinstance(f.dataType, PrimitiveType) for f in dataType.fields): + return conv + + row = conv(obj) + convs = [_create_converter(v, f.dataType) + for v, f in zip(row, dataType.fields)] + + def nested_conv(row): + return tuple(f(v) for f, v in zip(convs, conv(row))) + + return nested_conv + + +def _drop_schema(rows, schema): + """ all the names of fields, becoming tuples""" + iterator = iter(rows) + row = iterator.next() + converter = _create_converter(row, schema) + yield converter(row) + for i in iterator: + yield converter(i) + + +_BRACKETS = {'(': ')', '[': ']', '{': '}'} + + +def _split_schema_abstract(s): + """ + split the schema abstract into fields + + >>> _split_schema_abstract("a b c") + ['a', 'b', 'c'] + >>> _split_schema_abstract("a(a b)") + ['a(a b)'] + >>> _split_schema_abstract("a b[] c{a b}") + ['a', 'b[]', 'c{a b}'] + >>> _split_schema_abstract(" ") + [] + """ + + r = [] + w = '' + brackets = [] + for c in s: + if c == ' ' and not brackets: + if w: + r.append(w) + w = '' + else: + w += c + if c in _BRACKETS: + brackets.append(c) + elif c in _BRACKETS.values(): + if not brackets or c != _BRACKETS[brackets.pop()]: + raise ValueError("unexpected " + c) + + if brackets: + raise ValueError("brackets not closed: %s" % brackets) + if w: + r.append(w) + return r + -__all__ = ["SQLContext", "HiveContext", "LocalHiveContext", "TestHiveContext", "SchemaRDD", "Row"] +def _parse_field_abstract(s): + """ + Parse a field in schema abstract + + >>> _parse_field_abstract("a") + StructField(a,None,true) + >>> _parse_field_abstract("b(c d)") + StructField(b,StructType(...c,None,true),StructField(d... + >>> _parse_field_abstract("a[]") + StructField(a,ArrayType(None,true),true) + >>> _parse_field_abstract("a{[]}") + StructField(a,MapType(None,ArrayType(None,true),true),true) + """ + if set(_BRACKETS.keys()) & set(s): + idx = min((s.index(c) for c in _BRACKETS if c in s)) + name = s[:idx] + return StructField(name, _parse_schema_abstract(s[idx:]), True) + else: + return StructField(s, None, True) + + +def _parse_schema_abstract(s): + """ + parse abstract into schema + + >>> _parse_schema_abstract("a b c") + StructType...a...b...c... + >>> _parse_schema_abstract("a[b c] b{}") + StructType...a,ArrayType...b...c...b,MapType... + >>> _parse_schema_abstract("c{} d{a b}") + StructType...c,MapType...d,MapType...a...b... + >>> _parse_schema_abstract("a b(t)").fields[1] + StructField(b,StructType(List(StructField(t,None,true))),true) + """ + s = s.strip() + if not s: + return + + elif s.startswith('('): + return _parse_schema_abstract(s[1:-1]) + + elif s.startswith('['): + return ArrayType(_parse_schema_abstract(s[1:-1]), True) + + elif s.startswith('{'): + return MapType(None, _parse_schema_abstract(s[1:-1])) + + parts = _split_schema_abstract(s) + fields = [_parse_field_abstract(p) for p in parts] + return StructType(fields) + + +def _infer_schema_type(obj, dataType): + """ + Fill the dataType with types infered from obj + + >>> schema = _parse_schema_abstract("a b c") + >>> row = (1, 1.0, "str") + >>> _infer_schema_type(row, schema) + StructType...IntegerType...DoubleType...StringType... + >>> row = [[1], {"key": (1, 2.0)}] + >>> schema = _parse_schema_abstract("a[] b{c d}") + >>> _infer_schema_type(row, schema) + StructType...a,ArrayType...b,MapType(StringType,...c,IntegerType... + """ + if dataType is None: + return _infer_type(obj) + + if not obj: + raise ValueError("Can not infer type from empty value") + + if isinstance(dataType, ArrayType): + eType = _infer_schema_type(obj[0], dataType.elementType) + return ArrayType(eType, True) + + elif isinstance(dataType, MapType): + k, v = obj.iteritems().next() + return MapType(_infer_type(k), + _infer_schema_type(v, dataType.valueType)) + + elif isinstance(dataType, StructType): + fs = dataType.fields + assert len(fs) == len(obj), \ + "Obj(%s) have different length with fields(%s)" % (obj, fs) + fields = [StructField(f.name, _infer_schema_type(o, f.dataType), True) + for o, f in zip(obj, fs)] + return StructType(fields) + + else: + raise ValueError("Unexpected dataType: %s" % dataType) + + +_acceptable_types = { + BooleanType: (bool,), + ByteType: (int, long), + ShortType: (int, long), + IntegerType: (int, long), + LongType: (long,), + FloatType: (float,), + DoubleType: (float,), + DecimalType: (decimal.Decimal,), + StringType: (str, unicode), + TimestampType: (datetime.datetime,), + ArrayType: (list, tuple, array), + MapType: (dict,), + StructType: (tuple, list), +} + + +def _verify_type(obj, dataType): + """ + Verify the type of obj against dataType, raise an exception if + they do not match. + + >>> _verify_type(None, StructType([])) + >>> _verify_type("", StringType()) + >>> _verify_type(0, IntegerType()) + >>> _verify_type(range(3), ArrayType(ShortType())) + >>> _verify_type(set(), ArrayType(StringType())) # doctest: +IGNORE_EXCEPTION_DETAIL + Traceback (most recent call last): + ... + TypeError:... + >>> _verify_type({}, MapType(StringType(), IntegerType())) + >>> _verify_type((), StructType([])) + >>> _verify_type([], StructType([])) + >>> _verify_type([1], StructType([])) # doctest: +IGNORE_EXCEPTION_DETAIL + Traceback (most recent call last): + ... + ValueError:... + """ + # all objects are nullable + if obj is None: + return + + _type = type(dataType) + if _type not in _acceptable_types: + return + + if type(obj) not in _acceptable_types[_type]: + raise TypeError("%s can not accept abject in type %s" + % (dataType, type(obj))) + + if isinstance(dataType, ArrayType): + for i in obj: + _verify_type(i, dataType.elementType) + + elif isinstance(dataType, MapType): + for k, v in obj.iteritems(): + _verify_type(k, dataType.keyType) + _verify_type(v, dataType.valueType) + + elif isinstance(dataType, StructType): + if len(obj) != len(dataType.fields): + raise ValueError("Length of object (%d) does not match with" + "length of fields (%d)" % (len(obj), len(dataType.fields))) + for v, f in zip(obj, dataType.fields): + _verify_type(v, f.dataType) + + +_cached_cls = {} + + +def _restore_object(dataType, obj): + """ Restore object during unpickling. """ + # use id(dataType) as key to speed up lookup in dict + # Because of batched pickling, dataType will be the + # same object in mose cases. + k = id(dataType) + cls = _cached_cls.get(k) + if cls is None: + # use dataType as key to avoid create multiple class + cls = _cached_cls.get(dataType) + if cls is None: + cls = _create_cls(dataType) + _cached_cls[dataType] = cls + _cached_cls[k] = cls + return cls(obj) + + +def _create_object(cls, v): + """ Create an customized object with class `cls`. """ + return cls(v) if v is not None else v + + +def _create_getter(dt, i): + """ Create a getter for item `i` with schema """ + cls = _create_cls(dt) + + def getter(self): + return _create_object(cls, self[i]) + + return getter + + +def _has_struct(dt): + """Return whether `dt` is or has StructType in it""" + if isinstance(dt, StructType): + return True + elif isinstance(dt, ArrayType): + return _has_struct(dt.elementType) + elif isinstance(dt, MapType): + return _has_struct(dt.valueType) + return False + + +def _create_properties(fields): + """Create properties according to fields""" + ps = {} + for i, f in enumerate(fields): + name = f.name + if (name.startswith("__") and name.endswith("__") + or keyword.iskeyword(name)): + warnings.warn("field name %s can not be accessed in Python," + "use position to access it instead" % name) + if _has_struct(f.dataType): + # delay creating object until accessing it + getter = _create_getter(f.dataType, i) + else: + getter = itemgetter(i) + ps[name] = property(getter) + return ps + + +def _create_cls(dataType): + """ + Create an class by dataType + + The created class is similar to namedtuple, but can have nested schema. + + >>> schema = _parse_schema_abstract("a b c") + >>> row = (1, 1.0, "str") + >>> schema = _infer_schema_type(row, schema) + >>> obj = _create_cls(schema)(row) + >>> import pickle + >>> pickle.loads(pickle.dumps(obj)) + Row(a=1, b=1.0, c='str') + + >>> row = [[1], {"key": (1, 2.0)}] + >>> schema = _parse_schema_abstract("a[] b{c d}") + >>> schema = _infer_schema_type(row, schema) + >>> obj = _create_cls(schema)(row) + >>> pickle.loads(pickle.dumps(obj)) + Row(a=[1], b={'key': Row(c=1, d=2.0)}) + """ + + if isinstance(dataType, ArrayType): + cls = _create_cls(dataType.elementType) + + class List(list): + + def __getitem__(self, i): + # create object with datetype + return _create_object(cls, list.__getitem__(self, i)) + + def __repr__(self): + # call collect __repr__ for nested objects + return "[%s]" % (", ".join(repr(self[i]) + for i in range(len(self)))) + + def __reduce__(self): + return list.__reduce__(self) + + return List + + elif isinstance(dataType, MapType): + vcls = _create_cls(dataType.valueType) + + class Dict(dict): + + def __getitem__(self, k): + # create object with datetype + return _create_object(vcls, dict.__getitem__(self, k)) + + def __repr__(self): + # call collect __repr__ for nested objects + return "{%s}" % (", ".join("%r: %r" % (k, self[k]) + for k in self)) + + def __reduce__(self): + return dict.__reduce__(self) + + return Dict + + elif not isinstance(dataType, StructType): + raise Exception("unexpected data type: %s" % dataType) + + class Row(tuple): + + """ Row in SchemaRDD """ + __DATATYPE__ = dataType + __FIELDS__ = tuple(f.name for f in dataType.fields) + __slots__ = () + + # create property for fast access + locals().update(_create_properties(dataType.fields)) + + def __repr__(self): + # call collect __repr__ for nested objects + return ("Row(%s)" % ", ".join("%s=%r" % (n, getattr(self, n)) + for n in self.__FIELDS__)) + + def __reduce__(self): + return (_restore_object, (self.__DATATYPE__, tuple(self))) + + return Row class SQLContext: + """Main entry point for SparkSQL functionality. A SQLContext can be used create L{SchemaRDD}s, register L{SchemaRDD}s as tables, execute SQL over tables, cache tables, and read parquet files. """ - def __init__(self, sparkContext, sqlContext = None): + def __init__(self, sparkContext, sqlContext=None): """Create a new SQLContext. @param sparkContext: The SparkContext to wrap. + @param sqlContext: An optional JVM Scala SQLContext. If set, we do not instatiate a new + SQLContext in the JVM, instead we make all calls to this object. >>> srdd = sqlCtx.inferSchema(rdd) >>> sqlCtx.inferSchema(srdd) # doctest: +IGNORE_EXCEPTION_DETAIL Traceback (most recent call last): ... - ValueError:... + TypeError:... >>> bad_rdd = sc.parallelize([1,2,3]) >>> sqlCtx.inferSchema(bad_rdd) # doctest: +IGNORE_EXCEPTION_DETAIL @@ -47,17 +926,23 @@ def __init__(self, sparkContext, sqlContext = None): ... ValueError:... - >>> allTypes = sc.parallelize([{"int" : 1, "string" : "string", "double" : 1.0, "long": 1L, - ... "boolean" : True}]) - >>> srdd = sqlCtx.inferSchema(allTypes).map(lambda x: (x.int, x.string, x.double, x.long, - ... x.boolean)) - >>> srdd.collect()[0] - (1, u'string', 1.0, 1, True) + >>> from datetime import datetime + >>> allTypes = sc.parallelize([Row(i=1, s="string", d=1.0, l=1L, + ... b=True, list=[1, 2, 3], dict={"s": 0}, row=Row(a=1), + ... time=datetime(2014, 8, 1, 14, 1, 5))]) + >>> srdd = sqlCtx.inferSchema(allTypes) + >>> srdd.registerTempTable("allTypes") + >>> sqlCtx.sql('select i+1, d+1, not b, list[1], dict["s"], time, row.a ' + ... 'from allTypes where b and i > 0').collect() + [Row(c0=2, c1=2.0, c2=False, c3=2, c4=0...8, 1, 14, 1, 5), a=1)] + >>> srdd.map(lambda x: (x.i, x.s, x.d, x.l, x.b, x.time, + ... x.row.a, x.list)).collect() + [(1, u'string', 1.0, 1, True, ...(2014, 8, 1, 14, 1, 5), 1, [1, 2, 3])] """ self._sc = sparkContext self._jsc = self._sc._jsc self._jvm = self._sc._jvm - self._pythonToJavaMap = self._jvm.PythonRDD.pythonToJavaMap + self._pythonToJava = self._jvm.PythonRDD.pythonToJavaArray if sqlContext: self._scala_SQLContext = sqlContext @@ -73,38 +958,169 @@ def _ssql_ctx(self): self._scala_SQLContext = self._jvm.SQLContext(self._jsc.sc()) return self._scala_SQLContext + def registerFunction(self, name, f, returnType=StringType()): + """Registers a lambda function as a UDF so it can be used in SQL statements. + + In addition to a name and the function itself, the return type can be optionally specified. + When the return type is not given it default to a string and conversion will automatically + be done. For any other return type, the produced object must match the specified type. + + >>> sqlCtx.registerFunction("stringLengthString", lambda x: len(x)) + >>> sqlCtx.sql("SELECT stringLengthString('test')").collect() + [Row(c0=u'4')] + >>> sqlCtx.registerFunction("stringLengthInt", lambda x: len(x), IntegerType()) + >>> sqlCtx.sql("SELECT stringLengthInt('test')").collect() + [Row(c0=4)] + >>> sqlCtx.registerFunction("twoArgs", lambda x, y: len(x) + y, IntegerType()) + >>> sqlCtx.sql("SELECT twoArgs('test', 1)").collect() + [Row(c0=5)] + """ + func = lambda _, it: imap(lambda x: f(*x), it) + command = (func, + BatchedSerializer(PickleSerializer(), 1024), + BatchedSerializer(PickleSerializer(), 1024)) + env = MapConverter().convert(self._sc.environment, + self._sc._gateway._gateway_client) + includes = ListConverter().convert(self._sc._python_includes, + self._sc._gateway._gateway_client) + self._ssql_ctx.registerPython(name, + bytearray(CloudPickleSerializer().dumps(command)), + env, + includes, + self._sc.pythonExec, + self._sc._javaAccumulator, + str(returnType)) + def inferSchema(self, rdd): - """Infer and apply a schema to an RDD of L{dict}s. + """Infer and apply a schema to an RDD of L{Row}s. + + We peek at the first row of the RDD to determine the fields' names + and types. Nested collections are supported, which include array, + dict, list, Row, tuple, namedtuple, or object. + + All the rows in `rdd` should have the same type with the first one, + or it will cause runtime exceptions. - We peek at the first row of the RDD to determine the fields names - and types, and then use that to extract all the dictionaries. Nested - collections are supported, which include array, dict, list, set, and - tuple. + Each row could be L{pyspark.sql.Row} object or namedtuple or objects, + using dict is deprecated. + >>> rdd = sc.parallelize( + ... [Row(field1=1, field2="row1"), + ... Row(field1=2, field2="row2"), + ... Row(field1=3, field2="row3")]) >>> srdd = sqlCtx.inferSchema(rdd) - >>> srdd.collect() == [{"field1" : 1, "field2" : "row1"}, {"field1" : 2, "field2": "row2"}, - ... {"field1" : 3, "field2": "row3"}] - True + >>> srdd.collect()[0] + Row(field1=1, field2=u'row1') - >>> from array import array + >>> NestedRow = Row("f1", "f2") + >>> nestedRdd1 = sc.parallelize([ + ... NestedRow(array('i', [1, 2]), {"row1": 1.0}), + ... NestedRow(array('i', [2, 3]), {"row2": 2.0})]) >>> srdd = sqlCtx.inferSchema(nestedRdd1) - >>> srdd.collect() == [{"f1" : array('i', [1, 2]), "f2" : {"row1" : 1.0}}, - ... {"f1" : array('i', [2, 3]), "f2" : {"row2" : 2.0}}] - True + >>> srdd.collect() + [Row(f1=[1, 2], f2={u'row1': 1.0}), ..., f2={u'row2': 2.0})] + >>> nestedRdd2 = sc.parallelize([ + ... NestedRow([[1, 2], [2, 3]], [1, 2]), + ... NestedRow([[2, 3], [3, 4]], [2, 3])]) >>> srdd = sqlCtx.inferSchema(nestedRdd2) - >>> srdd.collect() == [{"f1" : [[1, 2], [2, 3]], "f2" : set([1, 2]), "f3" : (1, 2)}, - ... {"f1" : [[2, 3], [3, 4]], "f2" : set([2, 3]), "f3" : (2, 3)}] - True + >>> srdd.collect() + [Row(f1=[[1, 2], [2, 3]], f2=[1, 2]), ..., f2=[2, 3])] """ - if (rdd.__class__ is SchemaRDD): - raise ValueError("Cannot apply schema to %s" % SchemaRDD.__name__) - elif not isinstance(rdd.first(), dict): - raise ValueError("Only RDDs with dictionaries can be converted to %s: %s" % - (SchemaRDD.__name__, rdd.first())) - jrdd = self._pythonToJavaMap(rdd._jrdd) - srdd = self._ssql_ctx.inferSchema(jrdd.rdd()) + if isinstance(rdd, SchemaRDD): + raise TypeError("Cannot apply schema to SchemaRDD") + + first = rdd.first() + if not first: + raise ValueError("The first row in RDD is empty, " + "can not infer schema") + if type(first) is dict: + warnings.warn("Using RDD of dict to inferSchema is deprecated," + "please use pyspark.Row instead") + + schema = _infer_schema(first) + rdd = rdd.mapPartitions(lambda rows: _drop_schema(rows, schema)) + return self.applySchema(rdd, schema) + + def applySchema(self, rdd, schema): + """ + Applies the given schema to the given RDD of L{tuple} or L{list}s. + + These tuples or lists can contain complex nested structures like + lists, maps or nested rows. + + The schema should be a StructType. + + It is important that the schema matches the types of the objects + in each row or exceptions could be thrown at runtime. + + >>> rdd2 = sc.parallelize([(1, "row1"), (2, "row2"), (3, "row3")]) + >>> schema = StructType([StructField("field1", IntegerType(), False), + ... StructField("field2", StringType(), False)]) + >>> srdd = sqlCtx.applySchema(rdd2, schema) + >>> sqlCtx.registerRDDAsTable(srdd, "table1") + >>> srdd2 = sqlCtx.sql("SELECT * from table1") + >>> srdd2.collect() + [Row(field1=1, field2=u'row1'),..., Row(field1=3, field2=u'row3')] + + >>> from datetime import datetime + >>> rdd = sc.parallelize([(127, -128L, -32768, 32767, 2147483647L, 1.0, + ... datetime(2010, 1, 1, 1, 1, 1), + ... {"a": 1}, (2,), [1, 2, 3], None)]) + >>> schema = StructType([ + ... StructField("byte1", ByteType(), False), + ... StructField("byte2", ByteType(), False), + ... StructField("short1", ShortType(), False), + ... StructField("short2", ShortType(), False), + ... StructField("int", IntegerType(), False), + ... StructField("float", FloatType(), False), + ... StructField("time", TimestampType(), False), + ... StructField("map", + ... MapType(StringType(), IntegerType(), False), False), + ... StructField("struct", + ... StructType([StructField("b", ShortType(), False)]), False), + ... StructField("list", ArrayType(ByteType(), False), False), + ... StructField("null", DoubleType(), True)]) + >>> srdd = sqlCtx.applySchema(rdd, schema) + >>> results = srdd.map( + ... lambda x: (x.byte1, x.byte2, x.short1, x.short2, x.int, x.float, x.time, + ... x.map["a"], x.struct.b, x.list, x.null)) + >>> results.collect()[0] + (127, -128, -32768, 32767, 2147483647, 1.0, ...(2010, 1, 1, 1, 1, 1), 1, 2, [1, 2, 3], None) + + >>> srdd.registerTempTable("table2") + >>> sqlCtx.sql( + ... "SELECT byte1 - 1 AS byte1, byte2 + 1 AS byte2, " + + ... "short1 + 1 AS short1, short2 - 1 AS short2, int - 1 AS int, " + + ... "float + 1.5 as float FROM table2").collect() + [Row(byte1=126, byte2=-127, short1=-32767, short2=32766, int=2147483646, float=2.5)] + + >>> rdd = sc.parallelize([(127, -32768, 1.0, + ... datetime(2010, 1, 1, 1, 1, 1), + ... {"a": 1}, (2,), [1, 2, 3])]) + >>> abstract = "byte short float time map{} struct(b) list[]" + >>> schema = _parse_schema_abstract(abstract) + >>> typedSchema = _infer_schema_type(rdd.first(), schema) + >>> srdd = sqlCtx.applySchema(rdd, typedSchema) + >>> srdd.collect() + [Row(byte=127, short=-32768, float=1.0, time=..., list=[1, 2, 3])] + """ + + if isinstance(rdd, SchemaRDD): + raise TypeError("Cannot apply schema to SchemaRDD") + + if not isinstance(schema, StructType): + raise TypeError("schema should be StructType") + + # take the first few rows to verify schema + rows = rdd.take(10) + for row in rows: + _verify_type(row, schema) + + batched = isinstance(rdd._jrdd_deserializer, BatchedSerializer) + jrdd = self._pythonToJava(rdd._jrdd, batched) + srdd = self._ssql_ctx.applySchemaToPythonRDD(jrdd.rdd(), str(schema)) return SchemaRDD(srdd, self) def registerRDDAsTable(self, rdd, tableName): @@ -137,11 +1153,16 @@ def parquetFile(self, path): jschema_rdd = self._ssql_ctx.parquetFile(path) return SchemaRDD(jschema_rdd, self) + def jsonFile(self, path, schema=None): + """ + Loads a text file storing one JSON object per line as a + L{SchemaRDD}. + + If the schema is provided, applies the given schema to this + JSON dataset. - def jsonFile(self, path): - """Loads a text file storing one JSON object per line, - returning the result as a L{SchemaRDD}. - It goes through the entire dataset once to determine the schema. + Otherwise, it goes through the entire dataset once to determine + the schema. >>> import tempfile, shutil >>> jsonFile = tempfile.mkdtemp() @@ -150,42 +1171,113 @@ def jsonFile(self, path): >>> for json in jsonStrings: ... print>>ofn, json >>> ofn.close() - >>> srdd = sqlCtx.jsonFile(jsonFile) - >>> sqlCtx.registerRDDAsTable(srdd, "table1") + >>> srdd1 = sqlCtx.jsonFile(jsonFile) + >>> sqlCtx.registerRDDAsTable(srdd1, "table1") >>> srdd2 = sqlCtx.sql( - ... "SELECT field1 AS f1, field2 as f2, field3 as f3, field6 as f4 from table1") - >>> srdd2.collect() == [ - ... {"f1":1, "f2":"row1", "f3":{"field4":11, "field5": None}, "f4":None}, - ... {"f1":2, "f2":None, "f3":{"field4":22, "field5": [10, 11]}, "f4":[{"field7": "row2"}]}, - ... {"f1":None, "f2":"row3", "f3":{"field4":33, "field5": []}, "f4":None}] - True + ... "SELECT field1 AS f1, field2 as f2, field3 as f3, " + ... "field6 as f4 from table1") + >>> for r in srdd2.collect(): + ... print r + Row(f1=1, f2=u'row1', f3=Row(field4=11, field5=None), f4=None) + Row(f1=2, f2=None, f3=Row(field4=22,..., f4=[Row(field7=u'row2')]) + Row(f1=None, f2=u'row3', f3=Row(field4=33, field5=[]), f4=None) + >>> srdd3 = sqlCtx.jsonFile(jsonFile, srdd1.schema()) + >>> sqlCtx.registerRDDAsTable(srdd3, "table2") + >>> srdd4 = sqlCtx.sql( + ... "SELECT field1 AS f1, field2 as f2, field3 as f3, " + ... "field6 as f4 from table2") + >>> for r in srdd4.collect(): + ... print r + Row(f1=1, f2=u'row1', f3=Row(field4=11, field5=None), f4=None) + Row(f1=2, f2=None, f3=Row(field4=22,..., f4=[Row(field7=u'row2')]) + Row(f1=None, f2=u'row3', f3=Row(field4=33, field5=[]), f4=None) + >>> schema = StructType([ + ... StructField("field2", StringType(), True), + ... StructField("field3", + ... StructType([ + ... StructField("field5", + ... ArrayType(IntegerType(), False), True)]), False)]) + >>> srdd5 = sqlCtx.jsonFile(jsonFile, schema) + >>> sqlCtx.registerRDDAsTable(srdd5, "table3") + >>> srdd6 = sqlCtx.sql( + ... "SELECT field2 AS f1, field3.field5 as f2, " + ... "field3.field5[0] as f3 from table3") + >>> srdd6.collect() + [Row(f1=u'row1', f2=None, f3=None)...Row(f1=u'row3', f2=[], f3=None)] """ - jschema_rdd = self._ssql_ctx.jsonFile(path) + if schema is None: + jschema_rdd = self._ssql_ctx.jsonFile(path) + else: + scala_datatype = self._ssql_ctx.parseDataType(str(schema)) + jschema_rdd = self._ssql_ctx.jsonFile(path, scala_datatype) return SchemaRDD(jschema_rdd, self) - def jsonRDD(self, rdd): - """Loads an RDD storing one JSON object per string, returning the result as a L{SchemaRDD}. - It goes through the entire dataset once to determine the schema. + def jsonRDD(self, rdd, schema=None): + """Loads an RDD storing one JSON object per string as a L{SchemaRDD}. - >>> srdd = sqlCtx.jsonRDD(json) - >>> sqlCtx.registerRDDAsTable(srdd, "table1") + If the schema is provided, applies the given schema to this + JSON dataset. + + Otherwise, it goes through the entire dataset once to determine + the schema. + + >>> srdd1 = sqlCtx.jsonRDD(json) + >>> sqlCtx.registerRDDAsTable(srdd1, "table1") >>> srdd2 = sqlCtx.sql( - ... "SELECT field1 AS f1, field2 as f2, field3 as f3, field6 as f4 from table1") - >>> srdd2.collect() == [ - ... {"f1":1, "f2":"row1", "f3":{"field4":11, "field5": None}, "f4":None}, - ... {"f1":2, "f2":None, "f3":{"field4":22, "field5": [10, 11]}, "f4":[{"field7": "row2"}]}, - ... {"f1":None, "f2":"row3", "f3":{"field4":33, "field5": []}, "f4":None}] - True + ... "SELECT field1 AS f1, field2 as f2, field3 as f3, " + ... "field6 as f4 from table1") + >>> for r in srdd2.collect(): + ... print r + Row(f1=1, f2=u'row1', f3=Row(field4=11, field5=None), f4=None) + Row(f1=2, f2=None, f3=Row(field4=22..., f4=[Row(field7=u'row2')]) + Row(f1=None, f2=u'row3', f3=Row(field4=33, field5=[]), f4=None) + >>> srdd3 = sqlCtx.jsonRDD(json, srdd1.schema()) + >>> sqlCtx.registerRDDAsTable(srdd3, "table2") + >>> srdd4 = sqlCtx.sql( + ... "SELECT field1 AS f1, field2 as f2, field3 as f3, " + ... "field6 as f4 from table2") + >>> for r in srdd4.collect(): + ... print r + Row(f1=1, f2=u'row1', f3=Row(field4=11, field5=None), f4=None) + Row(f1=2, f2=None, f3=Row(field4=22..., f4=[Row(field7=u'row2')]) + Row(f1=None, f2=u'row3', f3=Row(field4=33, field5=[]), f4=None) + >>> schema = StructType([ + ... StructField("field2", StringType(), True), + ... StructField("field3", + ... StructType([ + ... StructField("field5", + ... ArrayType(IntegerType(), False), True)]), False)]) + >>> srdd5 = sqlCtx.jsonRDD(json, schema) + >>> sqlCtx.registerRDDAsTable(srdd5, "table3") + >>> srdd6 = sqlCtx.sql( + ... "SELECT field2 AS f1, field3.field5 as f2, " + ... "field3.field5[0] as f3 from table3") + >>> srdd6.collect() + [Row(f1=u'row1', f2=None,...Row(f1=u'row3', f2=[], f3=None)] + + >>> sqlCtx.jsonRDD(sc.parallelize(['{}', + ... '{"key0": {"key1": "value1"}}'])).collect() + [Row(key0=None), Row(key0=Row(key1=u'value1'))] + >>> sqlCtx.jsonRDD(sc.parallelize(['{"key0": null}', + ... '{"key0": {"key1": "value1"}}'])).collect() + [Row(key0=None), Row(key0=Row(key1=u'value1'))] """ - def func(split, iterator): + + def func(iterator): for x in iterator: if not isinstance(x, basestring): x = unicode(x) - yield x.encode("utf-8") - keyed = PipelinedRDD(rdd, func) + if isinstance(x, unicode): + x = x.encode("utf-8") + yield x + keyed = rdd.mapPartitions(func) keyed._bypass_serializer = True jrdd = keyed._jrdd.map(self._jvm.BytesToString()) - jschema_rdd = self._ssql_ctx.jsonRDD(jrdd.rdd()) + if schema is None: + jschema_rdd = self._ssql_ctx.jsonRDD(jrdd.rdd()) + else: + scala_datatype = self._ssql_ctx.parseDataType(str(schema)) + jschema_rdd = self._ssql_ctx.jsonRDD(jrdd.rdd(), scala_datatype) return SchemaRDD(jschema_rdd, self) def sql(self, sqlQuery): @@ -194,9 +1286,8 @@ def sql(self, sqlQuery): >>> srdd = sqlCtx.inferSchema(rdd) >>> sqlCtx.registerRDDAsTable(srdd, "table1") >>> srdd2 = sqlCtx.sql("SELECT field1 AS f1, field2 as f2 from table1") - >>> srdd2.collect() == [{"f1" : 1, "f2" : "row1"}, {"f1" : 2, "f2": "row2"}, - ... {"f1" : 3, "f2": "row3"}] - True + >>> srdd2.collect() + [Row(f1=1, f2=u'row1'), Row(f1=2, f2=u'row2'), Row(f1=3, f2=u'row3')] """ return SchemaRDD(self._ssql_ctx.sql(sqlQuery), self) @@ -221,12 +1312,25 @@ def uncacheTable(self, tableName): class HiveContext(SQLContext): + """A variant of Spark SQL that integrates with data stored in Hive. Configuration for Hive is read from hive-site.xml on the classpath. It supports running both SQL and HiveQL commands. """ + def __init__(self, sparkContext, hiveContext=None): + """Create a new HiveContext. + + @param sparkContext: The SparkContext to wrap. + @param hiveContext: An optional JVM Scala HiveContext. If set, we do not instatiate a new + HiveContext in the JVM, instead we make all calls to this object. + """ + SQLContext.__init__(self, sparkContext) + + if hiveContext: + self._scala_HiveContext = hiveContext + @property def _ssql_ctx(self): try: @@ -234,26 +1338,34 @@ def _ssql_ctx(self): self._scala_HiveContext = self._get_hive_ctx() return self._scala_HiveContext except Py4JError as e: - raise Exception("You must build Spark with Hive. Export 'SPARK_HIVE=true' and run " \ - "sbt/sbt assembly" , e) + raise Exception("You must build Spark with Hive. " + "Export 'SPARK_HIVE=true' and run " + "sbt/sbt assembly", e) def _get_hive_ctx(self): return self._jvm.HiveContext(self._jsc.sc()) def hiveql(self, hqlQuery): """ - Runs a query expressed in HiveQL, returning the result as a L{SchemaRDD}. + DEPRECATED: Use sql() """ + warnings.warn("hiveql() is deprecated as the sql function now parses using HiveQL by" + + "default. The SQL dialect for parsing can be set using 'spark.sql.dialect'", + DeprecationWarning) return SchemaRDD(self._ssql_ctx.hiveql(hqlQuery), self) def hql(self, hqlQuery): """ - Runs a query expressed in HiveQL, returning the result as a L{SchemaRDD}. + DEPRECATED: Use sql() """ + warnings.warn("hql() is deprecated as the sql function now parses using HiveQL by" + + "default. The SQL dialect for parsing can be set using 'spark.sql.dialect'", + DeprecationWarning) return self.hiveql(hqlQuery) class LocalHiveContext(HiveContext): + """Starts up an instance of hive where metadata is stored locally. An in-process metadata data is created with data stored in ./metadata. @@ -262,13 +1374,17 @@ class LocalHiveContext(HiveContext): >>> import os >>> hiveCtx = LocalHiveContext(sc) >>> try: - ... supress = hiveCtx.hql("DROP TABLE src") + ... supress = hiveCtx.sql("DROP TABLE src") ... except Exception: ... pass - >>> kv1 = os.path.join(os.environ["SPARK_HOME"], 'examples/src/main/resources/kv1.txt') - >>> supress = hiveCtx.hql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") - >>> supress = hiveCtx.hql("LOAD DATA LOCAL INPATH '%s' INTO TABLE src" % kv1) - >>> results = hiveCtx.hql("FROM src SELECT value").map(lambda r: int(r.value.split('_')[1])) + >>> kv1 = os.path.join(os.environ["SPARK_HOME"], + ... 'examples/src/main/resources/kv1.txt') + >>> supress = hiveCtx.sql( + ... "CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") + >>> supress = hiveCtx.sql("LOAD DATA LOCAL INPATH '%s' INTO TABLE src" + ... % kv1) + >>> results = hiveCtx.sql("FROM src SELECT value" + ... ).map(lambda r: int(r.value.split('_')[1])) >>> num = results.count() >>> reduce_sum = results.reduce(lambda x, y: x + y) >>> num @@ -277,6 +1393,11 @@ class LocalHiveContext(HiveContext): 130091 """ + def __init__(self, sparkContext, sqlContext=None): + HiveContext.__init__(self, sparkContext, sqlContext) + warnings.warn("LocalHiveContext is deprecated. " + "Use HiveContext instead.", DeprecationWarning) + def _get_hive_ctx(self): return self._jvm.LocalHiveContext(self._jsc.sc()) @@ -287,28 +1408,87 @@ def _get_hive_ctx(self): return self._jvm.TestHiveContext(self._jsc.sc()) -# TODO: Investigate if it is more efficient to use a namedtuple. One problem is that named tuples -# are custom classes that must be generated per Schema. -class Row(dict): - """A row in L{SchemaRDD}. +def _create_row(fields, values): + row = Row(*values) + row.__FIELDS__ = fields + return row + - An extended L{dict} that takes a L{dict} in its constructor, and - exposes those items as fields. +class Row(tuple): - >>> r = Row({"hello" : "world", "foo" : "bar"}) - >>> r.hello - 'world' - >>> r.foo - 'bar' """ + A row in L{SchemaRDD}. The fields in it can be accessed like attributes. + + Row can be used to create a row object by using named arguments, + the fields will be sorted by names. + + >>> row = Row(name="Alice", age=11) + >>> row + Row(age=11, name='Alice') + >>> row.name, row.age + ('Alice', 11) + + Row also can be used to create another Row like class, then it + could be used to create Row objects, such as - def __init__(self, d): - d.update(self.__dict__) - self.__dict__ = d - dict.__init__(self, d) + >>> Person = Row("name", "age") + >>> Person + + >>> Person("Alice", 11) + Row(name='Alice', age=11) + """ + + def __new__(self, *args, **kwargs): + if args and kwargs: + raise ValueError("Can not use both args " + "and kwargs to create Row") + if args: + # create row class or objects + return tuple.__new__(self, args) + + elif kwargs: + # create row objects + names = sorted(kwargs.keys()) + values = tuple(kwargs[n] for n in names) + row = tuple.__new__(self, values) + row.__FIELDS__ = names + return row + + else: + raise ValueError("No args or kwargs") + + # let obect acs like class + def __call__(self, *args): + """create new Row object""" + return _create_row(self, args) + + def __getattr__(self, item): + if item.startswith("__"): + raise AttributeError(item) + try: + # it will be slow when it has many fields, + # but this will not be used in normal cases + idx = self.__FIELDS__.index(item) + return self[idx] + except IndexError: + raise AttributeError(item) + + def __reduce__(self): + if hasattr(self, "__FIELDS__"): + return (_create_row, (self.__FIELDS__, tuple(self))) + else: + return tuple.__reduce__(self) + + def __repr__(self): + if hasattr(self, "__FIELDS__"): + return "Row(%s)" % ", ".join("%s=%r" % (k, v) + for k, v in zip(self.__FIELDS__, self)) + else: + return "" % ", ".join(self) class SchemaRDD(RDD): + """An RDD of L{Row} objects that has an associated schema. The underlying JVM object is a SchemaRDD, not a PythonRDD, so we can @@ -319,6 +1499,10 @@ class SchemaRDD(RDD): implementation is an RDD composed of Java objects. Instead it is converted to a PythonRDD in the JVM, on which Python operations can be done. + + This class receives raw tuples from Java but assigns a class to it in + all its data-collection methods (mapPartitionsWithIndex, collect, take, + etc) so that PySpark sees them as Row objects with named fields. """ def __init__(self, jschema_rdd, sql_ctx): @@ -329,7 +1513,8 @@ def __init__(self, jschema_rdd, sql_ctx): self.is_cached = False self.is_checkpointed = False self.ctx = self.sql_ctx._sc - self._jrdd_deserializer = self.ctx.serializer + # the _jrdd is created by javaToPython(), serialized by pickle + self._jrdd_deserializer = BatchedSerializer(PickleSerializer()) @property def _jrdd(self): @@ -339,7 +1524,7 @@ def _jrdd(self): L{pyspark.rdd.RDD} super class (map, filter, etc.). """ if not hasattr(self, '_lazy_jrdd'): - self._lazy_jrdd = self._toPython()._jrdd + self._lazy_jrdd = self._jschema_rdd.javaToPython() return self._lazy_jrdd @property @@ -363,21 +1548,25 @@ def saveAsParquetFile(self, path): """ self._jschema_rdd.saveAsParquetFile(path) - def registerAsTable(self, name): + def registerTempTable(self, name): """Registers this RDD as a temporary table using the given name. The lifetime of this temporary table is tied to the L{SQLContext} that was used to create this SchemaRDD. >>> srdd = sqlCtx.inferSchema(rdd) - >>> srdd.registerAsTable("test") + >>> srdd.registerTempTable("test") >>> srdd2 = sqlCtx.sql("select * from test") >>> sorted(srdd.collect()) == sorted(srdd2.collect()) True """ - self._jschema_rdd.registerAsTable(name) + self._jschema_rdd.registerTempTable(name) + + def registerAsTable(self, name): + warnings.warn("Use registerTempTable instead of registerAsTable.", DeprecationWarning) + self.registerTempTable(name) - def insertInto(self, tableName, overwrite = False): + def insertInto(self, tableName, overwrite=False): """Inserts the contents of this SchemaRDD into the specified table. Optionally overwriting any existing data. @@ -388,6 +1577,11 @@ def saveAsTable(self, tableName): """Creates a new table with the contents of this SchemaRDD.""" self._jschema_rdd.saveAsTable(tableName) + def schema(self): + """Returns the schema of this SchemaRDD (represented by + a L{StructType}).""" + return _parse_datatype_string(self._jschema_rdd.schema().toString()) + def schemaString(self): """Returns the output schema in the tree format.""" return self._jschema_rdd.schemaString() @@ -411,19 +1605,45 @@ def count(self): """ return self._jschema_rdd.count() - def _toPython(self): - # We have to import the Row class explicitly, so that the reference Pickler has is - # pyspark.sql.Row instead of __main__.Row - from pyspark.sql import Row - jrdd = self._jschema_rdd.javaToPython() - # TODO: This is inefficient, we should construct the Python Row object - # in Java land in the javaToPython function. May require a custom - # pickle serializer in Pyrolite - return RDD(jrdd, self._sc, BatchedSerializer( - PickleSerializer())).map(lambda d: Row(d)) - - # We override the default cache/persist/checkpoint behavior as we want to cache the underlying - # SchemaRDD object in the JVM, not the PythonRDD checkpointed by the super class + def collect(self): + """ + Return a list that contains all of the rows in this RDD. + + Each object in the list is on Row, the fields can be accessed as + attributes. + """ + rows = RDD.collect(self) + cls = _create_cls(self.schema()) + return map(cls, rows) + + # Convert each object in the RDD to a Row with the right class + # for this SchemaRDD, so that fields can be accessed as attributes. + def mapPartitionsWithIndex(self, f, preservesPartitioning=False): + """ + Return a new RDD by applying a function to each partition of this RDD, + while tracking the index of the original partition. + + >>> rdd = sc.parallelize([1, 2, 3, 4], 4) + >>> def f(splitIndex, iterator): yield splitIndex + >>> rdd.mapPartitionsWithIndex(f).sum() + 6 + """ + rdd = RDD(self._jrdd, self._sc, self._jrdd_deserializer) + + schema = self.schema() + import pickle + pickle.loads(pickle.dumps(schema)) + + def applySchema(_, it): + cls = _create_cls(schema) + return itertools.imap(cls, it) + + objrdd = rdd.mapPartitionsWithIndex(applySchema, preservesPartitioning) + return objrdd.mapPartitionsWithIndex(f, preservesPartitioning) + + # We override the default cache/persist/checkpoint behavior + # as we want to cache the underlying SchemaRDD object in the JVM, + # not the PythonRDD checkpointed by the super class def cache(self): self.is_cached = True self._jschema_rdd.cache() @@ -435,9 +1655,9 @@ def persist(self, storageLevel): self._jschema_rdd.persist(javaStorageLevel) return self - def unpersist(self): + def unpersist(self, blocking=True): self.is_cached = False - self._jschema_rdd.unpersist() + self._jschema_rdd.unpersist(blocking) return self def checkpoint(self): @@ -478,35 +1698,42 @@ def subtract(self, other, numPartitions=None): if numPartitions is None: rdd = self._jschema_rdd.subtract(other._jschema_rdd) else: - rdd = self._jschema_rdd.subtract(other._jschema_rdd, numPartitions) + rdd = self._jschema_rdd.subtract(other._jschema_rdd, + numPartitions) return SchemaRDD(rdd, self.sql_ctx) else: raise ValueError("Can only subtract another SchemaRDD") + def _test(): import doctest from array import array from pyspark.context import SparkContext - globs = globals().copy() + # let doctest run in pyspark.sql, so DataTypes can be picklable + import pyspark.sql + from pyspark.sql import Row, SQLContext + globs = pyspark.sql.__dict__.copy() # The small batch size here ensures that we see multiple batches, # even in these small test examples: sc = SparkContext('local[4]', 'PythonTest', batchSize=2) globs['sc'] = sc globs['sqlCtx'] = SQLContext(sc) - globs['rdd'] = sc.parallelize([{"field1" : 1, "field2" : "row1"}, - {"field1" : 2, "field2": "row2"}, {"field1" : 3, "field2": "row3"}]) - jsonStrings = ['{"field1": 1, "field2": "row1", "field3":{"field4":11}}', - '{"field1" : 2, "field3":{"field4":22, "field5": [10, 11]}, "field6":[{"field7": "row2"}]}', - '{"field1" : null, "field2": "row3", "field3":{"field4":33, "field5": []}}'] + globs['rdd'] = sc.parallelize( + [Row(field1=1, field2="row1"), + Row(field1=2, field2="row2"), + Row(field1=3, field2="row3")] + ) + jsonStrings = [ + '{"field1": 1, "field2": "row1", "field3":{"field4":11}}', + '{"field1" : 2, "field3":{"field4":22, "field5": [10, 11]},' + '"field6":[{"field7": "row2"}]}', + '{"field1" : null, "field2": "row3", ' + '"field3":{"field4":33, "field5": []}}' + ] globs['jsonStrings'] = jsonStrings globs['json'] = sc.parallelize(jsonStrings) - globs['nestedRdd1'] = sc.parallelize([ - {"f1" : array('i', [1, 2]), "f2" : {"row1" : 1.0}}, - {"f1" : array('i', [2, 3]), "f2" : {"row2" : 2.0}}]) - globs['nestedRdd2'] = sc.parallelize([ - {"f1" : [[1, 2], [2, 3]], "f2" : set([1, 2]), "f3" : (1, 2)}, - {"f1" : [[2, 3], [3, 4]], "f2" : set([2, 3]), "f3" : (2, 3)}]) - (failure_count, test_count) = doctest.testmod(globs=globs,optionflags=doctest.ELLIPSIS) + (failure_count, test_count) = doctest.testmod( + pyspark.sql, globs=globs, optionflags=doctest.ELLIPSIS) globs['sc'].stop() if failure_count: exit(-1) @@ -514,4 +1741,3 @@ def _test(): if __name__ == "__main__": _test() - diff --git a/python/pyspark/statcounter.py b/python/pyspark/statcounter.py index 080325061a697..1e597d64e03fe 100644 --- a/python/pyspark/statcounter.py +++ b/python/pyspark/statcounter.py @@ -20,29 +20,35 @@ import copy import math +try: + from numpy import maximum, minimum, sqrt +except ImportError: + maximum = max + minimum = min + sqrt = math.sqrt + + class StatCounter(object): - + def __init__(self, values=[]): self.n = 0L # Running count of our values self.mu = 0.0 # Running mean of our values self.m2 = 0.0 # Running variance numerator (sum of (x - mean)^2) self.maxValue = float("-inf") self.minValue = float("inf") - + for v in values: self.merge(v) - + # Add a value into this StatCounter, updating the internal statistics. def merge(self, value): delta = value - self.mu self.n += 1 self.mu += delta / self.n self.m2 += delta * (value - self.mu) - if self.maxValue < value: - self.maxValue = value - if self.minValue > value: - self.minValue = value - + self.maxValue = maximum(self.maxValue, value) + self.minValue = minimum(self.minValue, value) + return self # Merge another StatCounter into this one, adding up the internal statistics. @@ -50,7 +56,7 @@ def mergeStats(self, other): if not isinstance(other, StatCounter): raise Exception("Can only merge Statcounters!") - if other is self: # reference equality holds + if other is self: # reference equality holds self.merge(copy.deepcopy(other)) # Avoid overwriting fields in a weird order else: if self.n == 0: @@ -59,8 +65,8 @@ def mergeStats(self, other): self.n = other.n self.maxValue = other.maxValue self.minValue = other.minValue - - elif other.n != 0: + + elif other.n != 0: delta = other.mu - self.mu if other.n * 10 < self.n: self.mu = self.mu + (delta * other.n) / (self.n + other.n) @@ -68,10 +74,10 @@ def mergeStats(self, other): self.mu = other.mu - (delta * self.n) / (self.n + other.n) else: self.mu = (self.mu * self.n + other.mu * other.n) / (self.n + other.n) - - self.maxValue = max(self.maxValue, other.maxValue) - self.minValue = min(self.minValue, other.minValue) - + + self.maxValue = maximum(self.maxValue, other.maxValue) + self.minValue = minimum(self.minValue, other.minValue) + self.m2 += other.m2 + (delta * delta * self.n * other.n) / (self.n + other.n) self.n += other.n return self @@ -94,7 +100,7 @@ def min(self): def max(self): return self.maxValue - + # Return the variance of the values. def variance(self): if self.n == 0: @@ -114,15 +120,15 @@ def sampleVariance(self): # Return the standard deviation of the values. def stdev(self): - return math.sqrt(self.variance()) + return sqrt(self.variance()) # # Return the sample standard deviation of the values, which corrects for bias in estimating the # variance by dividing by N-1 instead of N. # def sampleStdev(self): - return math.sqrt(self.sampleVariance()) + return sqrt(self.sampleVariance()) def __repr__(self): - return "(count: %s, mean: %s, stdev: %s, max: %s, min: %s)" % (self.count(), self.mean(), self.stdev(), self.max(), self.min()) - + return ("(count: %s, mean: %s, stdev: %s, max: %s, min: %s)" % + (self.count(), self.mean(), self.stdev(), self.max(), self.min())) diff --git a/python/pyspark/storagelevel.py b/python/pyspark/storagelevel.py index 3a18ea54eae4c..2aa0fb9d2c1ed 100644 --- a/python/pyspark/storagelevel.py +++ b/python/pyspark/storagelevel.py @@ -17,7 +17,9 @@ __all__ = ["StorageLevel"] + class StorageLevel: + """ Flags for controlling the storage of an RDD. Each StorageLevel records whether to use memory, whether to drop the RDD to disk if it falls out of memory, whether to keep the data in memory @@ -25,7 +27,7 @@ class StorageLevel: Also contains static constants for some commonly used storage levels, such as MEMORY_ONLY. """ - def __init__(self, useDisk, useMemory, useOffHeap, deserialized, replication = 1): + def __init__(self, useDisk, useMemory, useOffHeap, deserialized, replication=1): self.useDisk = useDisk self.useMemory = useMemory self.useOffHeap = useOffHeap @@ -55,4 +57,4 @@ def __str__(self): StorageLevel.MEMORY_AND_DISK_2 = StorageLevel(True, True, False, True, 2) StorageLevel.MEMORY_AND_DISK_SER = StorageLevel(True, True, False, False) StorageLevel.MEMORY_AND_DISK_SER_2 = StorageLevel(True, True, False, False, 2) -StorageLevel.OFF_HEAP = StorageLevel(False, False, True, False, 1) \ No newline at end of file +StorageLevel.OFF_HEAP = StorageLevel(False, False, True, False, 1) diff --git a/python/pyspark/streaming/__init__.py b/python/pyspark/streaming/__init__.py new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/python/pyspark/streaming/context.py b/python/pyspark/streaming/context.py new file mode 100644 index 0000000000000..6b6cc653dfa6f --- /dev/null +++ b/python/pyspark/streaming/context.py @@ -0,0 +1,155 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import sys +from signal import signal, SIGTERM, SIGINT + +from pyspark.serializers import PickleSerializer, BatchedSerializer, UTF8Deserializer +from pyspark.context import SparkContext +from pyspark.streaming.dstream import DStream + +from py4j.java_collections import ListConverter + + +class StreamingContext(object): + """ + Main entry point for Spark Streaming functionality. A StreamingContext represents the + connection to a Spark cluster, and can be used to create L{DStream}s and + broadcast variables on that cluster. + """ + + def __init__(self, master=None, appName=None, sparkHome=None, pyFiles=None, + environment=None, batchSize=1024, serializer=PickleSerializer(), conf=None, + gateway=None, sparkContext=None, duration=None): + """ + Create a new StreamingContext. At least the master and app name and duration + should be set, either through the named parameters here or through C{conf}. + + @param master: Cluster URL to connect to + (e.g. mesos://host:port, spark://host:port, local[4]). + @param appName: A name for your job, to display on the cluster web UI. + @param sparkHome: Location where Spark is installed on cluster nodes. + @param pyFiles: Collection of .zip or .py files to send to the cluster + and add to PYTHONPATH. These can be paths on the local file + system or HDFS, HTTP, HTTPS, or FTP URLs. + @param environment: A dictionary of environment variables to set on + worker nodes. + @param batchSize: The number of Python objects represented as a single + Java object. Set 1 to disable batching or -1 to use an + unlimited batch size. + @param serializer: The serializer for RDDs. + @param conf: A L{SparkConf} object setting Spark properties. + @param gateway: Use an existing gateway and JVM, otherwise a new JVM + will be instatiated. + @param sparkContext: L{SparkContext} object. + @param duration: A L{Duration} object for SparkStreaming. + + """ + + if sparkContext is None: + # Create the Python Sparkcontext + self._sc = SparkContext(master=master, appName=appName, sparkHome=sparkHome, + pyFiles=pyFiles, environment=environment, batchSize=batchSize, + serializer=serializer, conf=conf, gateway=gateway) + else: + self._sc = sparkContext + + # Start py4j callback server. + # Callback sever is need only by SparkStreming; therefore the callback sever + # is started in StreamingContext. + SparkContext._gateway.restart_callback_server() + self._set_clean_up_trigger() + self._jvm = self._sc._jvm + self._jssc = self._initialize_context(self._sc._jsc, duration._jduration) + + # Initialize StremaingContext in function to allow subclass specific initialization + def _initialize_context(self, jspark_context, jduration): + return self._jvm.JavaStreamingContext(jspark_context, jduration) + + def _set_clean_up_trigger(self): + """Kill py4j callback server properly using signal lib""" + + def clean_up_handler(*args): + # Make sure stop callback server. + SparkContext._gateway.shutdown() + sys.exit(0) + + for sig in (SIGINT, SIGTERM): + signal(sig, clean_up_handler) + + def start(self): + """ + Start the execution of the streams. + """ + self._jssc.start() + + def awaitTermination(self, timeout=None): + """ + Wait for the execution to stop. + @param timeout: time to wait in milliseconds + """ + if timeout is None: + self._jssc.awaitTermination() + else: + self._jssc.awaitTermination(timeout) + + # TODO: add storageLevel + def socketTextStream(self, hostname, port): + """ + Create an input from TCP source hostname:port. Data is received using + a TCP socket and receive byte is interpreted as UTF8 encoded '\n' delimited + lines. + """ + return DStream(self._jssc.socketTextStream(hostname, port), self, UTF8Deserializer()) + + def textFileStream(self, directory): + """ + Create an input stream that monitors a Hadoop-compatible file system + for new files and reads them as text files. Files must be wrriten to the + monitored directory by "moving" them from another location within the same + file system. File names starting with . are ignored. + """ + return DStream(self._jssc.textFileStream(directory), self, UTF8Deserializer()) + + def stop(self, stopSparkContext=True, stopGraceFully=False): + """ + Stop the execution of the streams immediately (does not wait for all received data + to be processed). + """ + try: + self._jssc.stop(stopSparkContext, stopGraceFully) + finally: + SparkContext._gateway.shutdown() + + def _testInputStream(self, test_inputs, numSlices=None): + """ + This function is only for unittest. + It requires a list as input, and returns the i_th element at the i_th batch + under manual clock. + """ + test_rdds = list() + test_rdd_deserializers = list() + for test_input in test_inputs: + test_rdd = self._sc.parallelize(test_input, numSlices) + test_rdds.append(test_rdd._jrdd) + test_rdd_deserializers.append(test_rdd._jrdd_deserializer) + # All deserializers have to be the same. + # TODO: add deserializer validation + jtest_rdds = ListConverter().convert(test_rdds, SparkContext._gateway._gateway_client) + jinput_stream = self._jvm.PythonTestInputStream(self._jssc, jtest_rdds).asJavaDStream() + + return DStream(jinput_stream, self, test_rdd_deserializers[0]) diff --git a/python/pyspark/streaming/dstream.py b/python/pyspark/streaming/dstream.py new file mode 100644 index 0000000000000..696de900b0a69 --- /dev/null +++ b/python/pyspark/streaming/dstream.py @@ -0,0 +1,485 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from collections import defaultdict +from itertools import chain, ifilter, imap +import operator + +from pyspark.serializers import NoOpSerializer,\ + BatchedSerializer, CloudPickleSerializer, pack_long,\ + CompressedSerializer +from pyspark.rdd import _JavaStackTrace +from pyspark.storagelevel import StorageLevel +from pyspark.resultiterable import ResultIterable +from pyspark.streaming.util import rddToFileName, RDDFunction + + +from py4j.java_collections import ListConverter, MapConverter + +__all__ = ["DStream"] + + +class DStream(object): + def __init__(self, jdstream, ssc, jrdd_deserializer): + self._jdstream = jdstream + self._ssc = ssc + self.ctx = ssc._sc + self._jrdd_deserializer = jrdd_deserializer + self.is_cached = False + self.is_checkpointed = False + + def context(self): + """ + Return the StreamingContext associated with this DStream + """ + return self._ssc + + def count(self): + """ + Return a new DStream which contains the number of elements in this DStream. + """ + return self.mapPartitions(lambda i: [sum(1 for _ in i)])._sum() + + def _sum(self): + """ + Add up the elements in this DStream. + """ + return self.mapPartitions(lambda x: [sum(x)]).reduce(operator.add) + + def print_(self, label=None): + """ + Since print is reserved name for python, we cannot define a "print" method function. + This function prints serialized data in RDD in DStream because Scala and Java cannot + deserialized pickled python object. Please use DStream.pyprint() to print results. + + Call DStream.print() and this function will print byte array in the DStream + """ + # a hack to call print function in DStream + getattr(self._jdstream, "print")(label) + + def filter(self, f): + """ + Return a new DStream containing only the elements that satisfy predicate. + """ + def func(iterator): + return ifilter(f, iterator) + return self.mapPartitions(func) + + def flatMap(self, f, preservesPartitioning=False): + """ + Pass each value in the key-value pair DStream through flatMap function + without changing the keys: this also retains the original RDD's partition. + """ + def func(s, iterator): + return chain.from_iterable(imap(f, iterator)) + return self._mapPartitionsWithIndex(func, preservesPartitioning) + + def map(self, f, preservesPartitioning=False): + """ + Return a new DStream by applying a function to each element of DStream. + """ + def func(iterator): + return imap(f, iterator) + return self.mapPartitions(func, preservesPartitioning) + + def mapPartitions(self, f, preservesPartitioning=False): + """ + Return a new DStream by applying a function to each partition of this DStream. + """ + def func(s, iterator): + return f(iterator) + return self._mapPartitionsWithIndex(func, preservesPartitioning) + + def _mapPartitionsWithIndex(self, f, preservesPartitioning=False): + """ + Return a new DStream by applying a function to each partition of this DStream, + while tracking the index of the original partition. + """ + return PipelinedDStream(self, f, preservesPartitioning) + + def reduce(self, func): + """ + Return a new DStream by reduceing the elements of this RDD using the specified + commutative and associative binary operator. + """ + return self.map(lambda x: (None, x)).reduceByKey(func, 1).map(lambda x: x[1]) + + def reduceByKey(self, func, numPartitions=None): + """ + Merge the value for each key using an associative reduce function. + + This will also perform the merging locally on each mapper before + sending results to reducer, similarly to a "combiner" in MapReduce. + + Output will be hash-partitioned with C{numPartitions} partitions, or + the default parallelism level if C{numPartitions} is not specified. + """ + return self.combineByKey(lambda x: x, func, func, numPartitions) + + def combineByKey(self, createCombiner, mergeValue, mergeCombiners, + numPartitions=None): + """ + Count the number of elements for each key, and return the result to the + master as a dictionary + """ + if numPartitions is None: + numPartitions = self._defaultReducePartitions() + + def combineLocally(iterator): + combiners = {} + for x in iterator: + (k, v) = x + if k not in combiners: + combiners[k] = createCombiner(v) + else: + combiners[k] = mergeValue(combiners[k], v) + return combiners.iteritems() + locally_combined = self.mapPartitions(combineLocally) + shuffled = locally_combined.partitionBy(numPartitions) + + def _mergeCombiners(iterator): + combiners = {} + for (k, v) in iterator: + if k not in combiners: + combiners[k] = v + else: + combiners[k] = mergeCombiners(combiners[k], v) + return combiners.iteritems() + + return shuffled.mapPartitions(_mergeCombiners) + + def partitionBy(self, numPartitions, partitionFunc=None): + """ + Return a copy of the DStream partitioned using the specified partitioner. + """ + if numPartitions is None: + numPartitions = self.ctx._defaultReducePartitions() + + if partitionFunc is None: + partitionFunc = lambda x: 0 if x is None else hash(x) + + # Transferring O(n) objects to Java is too expensive. Instead, we'll + # form the hash buckets in Python, transferring O(numPartitions) objects + # to Java. Each object is a (splitNumber, [objects]) pair. + outputSerializer = self.ctx._unbatched_serializer + + def add_shuffle_key(split, iterator): + buckets = defaultdict(list) + + for (k, v) in iterator: + buckets[partitionFunc(k) % numPartitions].append((k, v)) + for (split, items) in buckets.iteritems(): + yield pack_long(split) + yield outputSerializer.dumps(items) + keyed = PipelinedDStream(self, add_shuffle_key) + keyed._bypass_serializer = True + with _JavaStackTrace(self.ctx) as st: + partitioner = self.ctx._jvm.PythonPartitioner(numPartitions, + id(partitionFunc)) + jdstream = self.ctx._jvm.PythonPairwiseDStream(keyed._jdstream.dstream(), + partitioner).asJavaDStream() + dstream = DStream(jdstream, self._ssc, BatchedSerializer(outputSerializer)) + # This is required so that id(partitionFunc) remains unique, even if + # partitionFunc is a lambda: + dstream._partitionFunc = partitionFunc + return dstream + + def _defaultReducePartitions(self): + """ + Returns the default number of partitions to use during reduce tasks (e.g., groupBy). + If spark.default.parallelism is set, then we'll use the value from SparkContext + defaultParallelism, otherwise we'll use the number of partitions in this RDD + + This mirrors the behavior of the Scala Partitioner#defaultPartitioner, intended to reduce + the likelihood of OOMs. Once PySpark adopts Partitioner-based APIs, this behavior will + be inherent. + """ + if self.ctx._conf.contains("spark.default.parallelism"): + return self.ctx.defaultParallelism + else: + return self.getNumPartitions() + + def getNumPartitions(self): + """ + Return the number of partitions in RDD + """ + # TODO: remove hard coding. RDD has NumPartitions. How do we get the number of partition + # through DStream? + return 2 + + def foreachRDD(self, func): + """ + Apply userdefined function to all RDD in a DStream. + This python implementation could be expensive because it uses callback server + in order to apply function to RDD in DStream. + This is an output operator, so this DStream will be registered as an output + stream and there materialized. + """ + wrapped_func = RDDFunction(self.ctx, self._jrdd_deserializer, func) + self.ctx._jvm.PythonForeachDStream(self._jdstream.dstream(), wrapped_func) + + def pyprint(self): + """ + Print the first ten elements of each RDD generated in this DStream. This is an output + operator, so this DStream will be registered as an output stream and there materialized. + """ + def takeAndPrint(rdd, time): + """ + Closure to take element from RDD and print first 10 elements. + This closure is called by py4j callback server. + """ + taken = rdd.take(11) + print "-------------------------------------------" + print "Time: %s" % (str(time)) + print "-------------------------------------------" + for record in taken[:10]: + print record + if len(taken) > 10: + print "..." + print + + self.foreachRDD(takeAndPrint) + + def mapValues(self, f): + """ + Pass each value in the key-value pair RDD through a map function + without changing the keys; this also retains the original RDD's + partitioning. + """ + map_values_fn = lambda (k, v): (k, f(v)) + return self.map(map_values_fn, preservesPartitioning=True) + + def flatMapValues(self, f): + """ + Pass each value in the key-value pair RDD through a flatMap function + without changing the keys; this also retains the original RDD's + partitioning. + """ + flat_map_fn = lambda (k, v): ((k, x) for x in f(v)) + return self.flatMap(flat_map_fn, preservesPartitioning=True) + + def glom(self): + """ + Return a new DStream in which RDD is generated by applying glom() to RDD of + this DStream. Applying glom() to an RDD coalesces all elements within each partition into + an list. + """ + def func(iterator): + yield list(iterator) + return self.mapPartitions(func) + + def cache(self): + """ + Persist this DStream with the default storage level (C{MEMORY_ONLY_SER}). + """ + self.is_cached = True + self.persist(StorageLevel.MEMORY_ONLY_SER) + return self + + def persist(self, storageLevel): + """ + Set this DStream's storage level to persist its values across operations + after the first time it is computed. This can only be used to assign + a new storage level if the DStream does not have a storage level set yet. + """ + self.is_cached = True + javaStorageLevel = self.ctx._getJavaStorageLevel(storageLevel) + self._jdstream.persist(javaStorageLevel) + return self + + def checkpoint(self, interval): + """ + Mark this DStream for checkpointing. It will be saved to a file inside the + checkpoint directory set with L{SparkContext.setCheckpointDir()} + + @param interval: Time interval after which generated RDD will be checkpointed + interval has to be pyspark.streaming.duration.Duration + """ + self.is_checkpointed = True + self._jdstream.checkpoint(interval._jduration) + return self + + def groupByKey(self, numPartitions=None): + """ + Return a new DStream which contains group the values for each key in the + DStream into a single sequence. + Hash-partitions the resulting RDD with into numPartitions partitions in + the DStream. + + Note: If you are grouping in order to perform an aggregation (such as a + sum or average) over each key, using reduceByKey will provide much + better performance. + + """ + def createCombiner(x): + return [x] + + def mergeValue(xs, x): + xs.append(x) + return xs + + def mergeCombiners(a, b): + a.extend(b) + return a + + return self.combineByKey(createCombiner, mergeValue, mergeCombiners, + numPartitions).mapValues(lambda x: ResultIterable(x)) + + def countByValue(self): + """ + Return new DStream which contains the count of each unique value in this + DStreeam as a (value, count) pairs. + """ + def countPartition(iterator): + counts = defaultdict(int) + for obj in iterator: + counts[obj] += 1 + yield counts + + def mergeMaps(m1, m2): + for (k, v) in m2.iteritems(): + m1[k] += v + return m1 + + return self.mapPartitions(countPartition).reduce(mergeMaps).flatMap(lambda x: x.items()) + + def saveAsTextFiles(self, prefix, suffix=None): + """ + Save this DStream as a text file, using string representations of elements. + """ + + def saveAsTextFile(rdd, time): + """ + Closure to save element in RDD in DStream as Pickled data in file. + This closure is called by py4j callback server. + """ + path = rddToFileName(prefix, suffix, time) + rdd.saveAsTextFile(path) + + return self.foreachRDD(saveAsTextFile) + + def saveAsPickleFiles(self, prefix, suffix=None): + """ + Save this DStream as a SequenceFile of serialized objects. The serializer + used is L{pyspark.serializers.PickleSerializer}, default batch size + is 10. + """ + + def saveAsPickleFile(rdd, time): + """ + Closure to save element in RDD in the DStream as Pickled data in file. + This closure is called by py4j callback server. + """ + path = rddToFileName(prefix, suffix, time) + rdd.saveAsPickleFile(path) + + return self.foreachRDD(saveAsPickleFile) + + def _test_output(self, result): + """ + This function is only for test case. + Store data in a DStream to result to verify the result in test case + """ + def get_output(rdd, time): + collected = rdd.collect() + result.append(collected) + + self.foreachRDD(get_output) + + +# TODO: implement updateStateByKey +# TODO: implement slice + +# Window Operations +# TODO: implement window +# TODO: implement groupByKeyAndWindow +# TODO: implement reduceByKeyAndWindow +# TODO: implement countByValueAndWindow +# TODO: implement countByWindow +# TODO: implement reduceByWindow + +# Transform Operation +# TODO: implement transform +# TODO: implement transformWith +# Following operation has dependency with transform +# TODO: implement union +# TODO: implement repertitions +# TODO: implement cogroup +# TODO: implement join +# TODO: implement leftOuterJoin +# TODO: implement rightOuterJoin + + +class PipelinedDStream(DStream): + def __init__(self, prev, func, preservesPartitioning=False): + if not isinstance(prev, PipelinedDStream) or not prev._is_pipelinable(): + # This transformation is the first in its stage: + self.func = func + self.preservesPartitioning = preservesPartitioning + self._prev_jdstream = prev._jdstream + self._prev_jrdd_deserializer = prev._jrdd_deserializer + else: + prev_func = prev.func + + def pipeline_func(split, iterator): + return func(split, prev_func(split, iterator)) + self.func = pipeline_func + self.preservesPartitioning = \ + prev.preservesPartitioning and preservesPartitioning + self._prev_jdstream = prev._prev_jdstream # maintain the pipeline + self._prev_jrdd_deserializer = prev._prev_jrdd_deserializer + self.is_cached = False + self.is_checkpointed = False + self._ssc = prev._ssc + self.ctx = prev.ctx + self.prev = prev + self._jdstream_val = None + self._jrdd_deserializer = self.ctx.serializer + self._bypass_serializer = False + + @property + def _jdstream(self): + if self._jdstream_val: + return self._jdstream_val + if self._bypass_serializer: + serializer = NoOpSerializer() + else: + serializer = self.ctx.serializer + + command = (self.func, self._prev_jrdd_deserializer, serializer) + ser = CompressedSerializer(CloudPickleSerializer()) + pickled_command = ser.dumps(command) + broadcast_vars = ListConverter().convert( + [x._jbroadcast for x in self.ctx._pickled_broadcast_vars], + self.ctx._gateway._gateway_client) + self.ctx._pickled_broadcast_vars.clear() + class_tag = self._prev_jdstream.classTag() + env = MapConverter().convert(self.ctx.environment, + self.ctx._gateway._gateway_client) + includes = ListConverter().convert(self.ctx._python_includes, + self.ctx._gateway._gateway_client) + python_dstream = self.ctx._jvm.PythonDStream(self._prev_jdstream.dstream(), + bytearray(pickled_command), + env, includes, self.preservesPartitioning, + self.ctx.pythonExec, + broadcast_vars, self.ctx._javaAccumulator, + class_tag) + self._jdstream_val = python_dstream.asJavaDStream() + return self._jdstream_val + + def _is_pipelinable(self): + return not (self.is_cached or self.is_checkpointed) diff --git a/python/pyspark/streaming/duration.py b/python/pyspark/streaming/duration.py new file mode 100644 index 0000000000000..495ac2edff198 --- /dev/null +++ b/python/pyspark/streaming/duration.py @@ -0,0 +1,376 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from pyspark.streaming import util + + +class Duration(object): + """ + Duration for Spark Streaming application. Used to set duration + + Most of the time, you would create a Duration object with + C{Duration()}, which will load values from C{spark.streaming.*} Java system + properties as well. In this case, any parameters you set directly on + the C{Duration} object take priority over system properties. + + """ + def __init__(self, millis, _jvm=None): + """ + Create new Duration. + + @param millis: milisecond + + """ + self._millis = millis + + from pyspark.context import SparkContext + SparkContext._ensure_initialized() + _jvm = _jvm or SparkContext._jvm + self._jduration = _jvm.Duration(millis) + + def toString(self): + """ + Return duration as string + + >>> d_10 = Duration(10) + >>> d_10.toString() + '10 ms' + """ + return str(self._millis) + " ms" + + def isZero(self): + """ + Check if millis is zero + + >>> d_10 = Duration(10) + >>> d_10.isZero() + False + >>> d_0 = Duration(0) + >>> d_0.isZero() + True + """ + return self._millis == 0 + + def prettyPrint(self): + """ + Return a human-readable string representing a duration + + >>> d_10 = Duration(10) + >>> d_10.prettyPrint() + '10 ms' + >>> d_1sec = Duration(1000) + >>> d_1sec.prettyPrint() + '1.0 s' + >>> d_1min = Duration(60 * 1000) + >>> d_1min.prettyPrint() + '1.0 m' + >>> d_1hour = Duration(60 * 60 * 1000) + >>> d_1hour.prettyPrint() + '1.00 h' + """ + return util.msDurationToString(self._millis) + + def milliseconds(self): + """ + Return millisecond + + >>> d_10 = Duration(10) + >>> d_10.milliseconds() + 10 + + """ + return self._millis + + def toFormattedString(self): + """ + Return millisecond + + >>> d_10 = Duration(10) + >>> d_10.toFormattedString() + '10' + + """ + return str(self._millis) + + def max(self, other): + """ + Return higher Duration + + >>> d_10 = Duration(10) + >>> d_100 = Duration(100) + >>> d_max = d_10.max(d_100) + >>> print d_max + 100 ms + + """ + Duration._is_duration(other) + if self > other: + return self + else: + return other + + def min(self, other): + """ + Return lower Durattion + + >>> d_10 = Duration(10) + >>> d_100 = Duration(100) + >>> d_min = d_10.min(d_100) + >>> print d_min + 10 ms + + """ + Duration._is_duration(other) + if self < other: + return self + else: + return other + + def __str__(self): + """ + >>> d_10 = Duration(10) + >>> str(d_10) + '10 ms' + + """ + return self.toString() + + def __add__(self, other): + """ + Add Duration and Duration + + >>> d_10 = Duration(10) + >>> d_100 = Duration(100) + >>> d_110 = d_10 + d_100 + >>> print d_110 + 110 ms + """ + Duration._is_duration(other) + return Duration(self._millis + other._millis) + + def __sub__(self, other): + """ + Subtract Duration by Duration + + >>> d_10 = Duration(10) + >>> d_100 = Duration(100) + >>> d_90 = d_100 - d_10 + >>> print d_90 + 90 ms + + """ + Duration._is_duration(other) + return Duration(self._millis - other._millis) + + def __mul__(self, other): + """ + Multiple Duration by Duration + + >>> d_10 = Duration(10) + >>> d_100 = Duration(100) + >>> d_1000 = d_10 * d_100 + >>> print d_1000 + 1000 ms + + """ + Duration._is_duration(other) + return Duration(self._millis * other._millis) + + def __div__(self, other): + """ + Divide Duration by Duration + for Python 2.X + + >>> d_10 = Duration(10) + >>> d_20 = Duration(20) + >>> d_2 = d_20 / d_10 + >>> print d_2 + 2 ms + + """ + Duration._is_duration(other) + return Duration(self._millis / other._millis) + + def __truediv__(self, other): + """ + Divide Duration by Duration + for Python 3.0 + + >>> d_10 = Duration(10) + >>> d_20 = Duration(20) + >>> d_2 = d_20 / d_10 + >>> print d_2 + 2 ms + + """ + Duration._is_duration(other) + return Duration(self._millis / other._millis) + + def __floordiv__(self, other): + """ + Divide Duration by Duration + + >>> d_10 = Duration(10) + >>> d_3 = Duration(3) + >>> d_3 = d_10 // d_3 + >>> print d_3 + 3 ms + + """ + Duration._is_duration(other) + return Duration(self._millis // other._millis) + + def __lt__(self, other): + """ + Duration < Duration + + >>> d_10 = Duration(10) + >>> d_20 = Duration(20) + >>> d_10 < d_20 + True + >>> d_20 < d_10 + False + + """ + Duration._is_duration(other) + return self._millis < other._millis + + def __le__(self, other): + """ + Duration <= Duration + + >>> d_10 = Duration(10) + >>> d_20 = Duration(20) + >>> d_10 <= d_20 + True + >>> d_20 <= d_10 + False + + """ + Duration._is_duration(other) + return self._millis <= other._millis + + def __eq__(self, other): + """ + Duration == Duration + + >>> d_10 = Duration(10) + >>> d_20 = Duration(20) + >>> d_10 == d_20 + False + >>> other_d_10 = Duration(10) + >>> d_10 == other_d_10 + True + + """ + Duration._is_duration(other) + return self._millis == other._millis + + def __ne__(self, other): + """ + Duration != Duration + + >>> d_10 = Duration(10) + >>> d_20 = Duration(20) + >>> d_10 != d_20 + True + >>> other_d_10 = Duration(10) + >>> d_10 != other_d_10 + False + + """ + Duration._is_duration(other) + return self._millis != other._millis + + def __gt__(self, other): + """ + Duration > Duration + + >>> d_10 = Duration(10) + >>> d_20 = Duration(20) + >>> d_10 > d_20 + False + >>> d_20 > d_10 + True + + """ + Duration._is_duration(other) + return self._millis > other._millis + + def __ge__(self, other): + """ + Duration >= Duration + + >>> d_10 = Duration(10) + >>> d_20 = Duration(20) + >>> d_10 < d_20 + True + >>> d_20 < d_10 + False + + + """ + Duration._is_duration(other) + return self._millis >= other._millis + + @classmethod + def _is_duration(self, instance): + """ is instance Duration """ + if not isinstance(instance, Duration): + raise TypeError("This should be Duration") + + +def Milliseconds(milliseconds): + """ + Helper function that creates instance of [[pysparkstreaming.duration]] representing + a given number of milliseconds. + + >>> milliseconds = Milliseconds(1) + >>> d_1 = Duration(1) + >>> milliseconds == d_1 + True + + """ + return Duration(milliseconds) + + +def Seconds(seconds): + """ + Helper function that creates instance of [[pysparkstreaming.duration]] representing + a given number of seconds. + + >>> seconds = Seconds(1) + >>> d_1sec = Duration(1000) + >>> seconds == d_1sec + True + + """ + return Duration(seconds * 1000) + + +def Minutes(minutes): + """ + Helper function that creates instance of [[pysparkstreaming.duration]] representing + a given number of minutes. + + >>> minutes = Minutes(1) + >>> d_1min = Duration(60 * 1000) + >>> minutes == d_1min + True + + """ + return Duration(minutes * 60 * 1000) diff --git a/python/pyspark/streaming/jtime.py b/python/pyspark/streaming/jtime.py new file mode 100644 index 0000000000000..02e0e0cf2d17e --- /dev/null +++ b/python/pyspark/streaming/jtime.py @@ -0,0 +1,138 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from pyspark.streaming.duration import Duration + +""" +The name of this file, time is not a good naming for python +because if we do import time when we want to use native python time package, it does +not import python time package. +""" +# TODO: add doctest + + +class Time(object): + """ + Time for Spark Streaming application. Used to set Time + + Most of the time, you would create a Duration object with + C{Time()}, which will load values from C{spark.streaming.*} Java system + properties as well. In this case, any parameters you set directly on + the C{Time} object take priority over system properties. + + """ + def __init__(self, millis, _jvm=None): + """ + Create new Time. + + @param millis: milisecond + + @param _jvm: internal parameter used to pass a handle to the + Java VM; does not need to be set by users + + """ + self._millis = millis + + from pyspark.context import StreamingContext + StreamingContext._ensure_initialized() + _jvm = _jvm or StreamingContext._jvm + self._jtime = _jvm.Time(millis) + + def toString(self): + """ Return time as string """ + return str(self._millis) + " ms" + + def milliseconds(self): + """ Return millisecond """ + return self._millis + + def max(self, other): + """ Return higher Time """ + Time._is_time(other) + if self > other: + return self + else: + return other + + def min(self, other): + """ Return lower Time """ + Time._is_time(other) + if self < other: + return self + else: + return other + + def __add__(self, other): + """ Add Time and Time """ + Duration._is_duration(other) + return Time(self._millis + other._millis) + + def __sub__(self, other): + """ Subtract Time by Duration or Time """ + if isinstance(other, Duration): + return Time(self._millis - other._millis) + elif isinstance(other, Time): + return Duration(self._millis, other._millis) + else: + raise TypeError + + def __lt__(self, other): + """ Time < Time """ + Time._is_time(other) + return self._millis < other._millis + + def __le__(self, other): + """ Time <= Time """ + Time._is_time(other) + return self._millis <= other._millis + + def __eq__(self, other): + """ Time == Time """ + Time._is_time(other) + return self._millis == other._millis + + def __ne__(self, other): + """ Time != Time """ + Time._is_time(other) + return self._millis != other._millis + + def __gt__(self, other): + """ Time > Time """ + Time._is_time(other) + return self._millis > other._millis + + def __ge__(self, other): + """ Time >= Time """ + Time._is_time(other) + return self._millis >= other._millis + + def isMultipbleOf(self, duration): + """ is multiple by Duration """ + Duration._is_duration(duration) + return self._millis % duration._millis == 0 + + def until(time, interval): + raise NotImplementedError + + def to(time, interval): + raise NotImplementedError + + @classmethod + def _is_time(self, instance): + """ is instance Time """ + if not isinstance(instance, Time): + raise TypeError diff --git a/python/pyspark/streaming/tests.py b/python/pyspark/streaming/tests.py new file mode 100644 index 0000000000000..2ed099b1004c3 --- /dev/null +++ b/python/pyspark/streaming/tests.py @@ -0,0 +1,464 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +""" +Unit tests for PySpark; additional tests are implemented as doctests in +individual modules. + +This file would be merged to tests.py after all functions are ready. +Since python API for streaming is beta, this file is separated. + +Callback server is sometimes unstable sometimes, which cause error in test case. +But this is very rare case. + +""" +from itertools import chain +import time +import operator +import sys + +if sys.version_info[:2] <= (2, 6): + import unittest2 as unittest +else: + import unittest + +from pyspark.context import SparkContext +from pyspark.streaming.context import StreamingContext +from pyspark.streaming.duration import * + + +class PySparkStreamingTestCase(unittest.TestCase): + def setUp(self): + class_name = self.__class__.__name__ + self.ssc = StreamingContext(appName=class_name, duration=Seconds(1)) + + def tearDown(self): + # Do not call pyspark.streaming.context.StreamingContext.stop directly because + # we do not wait to shutdown py4j client. + self.ssc._jssc.stop() + self.ssc._sc.stop() + # Why does it long time to terminate StremaingContext and SparkContext? + # Should we change the sleep time if this depends on machine spec? + time.sleep(1) + + @classmethod + def tearDownClass(cls): + # Make sure tp shutdown the callback server + SparkContext._gateway._shutdown_callback_server() + + +class TestBasicOperationsSuite(PySparkStreamingTestCase): + """ + 2 tests for each function for batach deserializer and unbatch deserilizer because + the deserializer is not changed dunamically after streaming process starts. + Default numInputPartitions is 2. + If the number of input element is over 3, that DStream use batach deserializer. + If not, that DStream use unbatch deserializer. + + All tests input should have list of lists(3 lists are default). This list represents stream. + Every batch interval, the first object of list are chosen to make DStream. + e.g The first list in the list is input of the first batch. + Please see the BasicTestSuits in Scala which is close to this implementation. + """ + def setUp(self): + PySparkStreamingTestCase.setUp(self) + self.timeout = 10 # seconds + self.numInputPartitions = 2 + + def tearDown(self): + PySparkStreamingTestCase.tearDown(self) + + @classmethod + def tearDownClass(cls): + PySparkStreamingTestCase.tearDownClass() + + def test_map_batch(self): + """Basic operation test for DStream.map with batch deserializer.""" + test_input = [range(1, 5), range(5, 9), range(9, 13)] + + def test_func(dstream): + return dstream.map(lambda x: str(x)) + expected_output = map(lambda x: map(lambda y: str(y), x), test_input) + output = self._run_stream(test_input, test_func, expected_output) + self.assertEqual(expected_output, output) + + def test_map_unbatach(self): + """Basic operation test for DStream.map with unbatch deserializer.""" + test_input = [range(1, 4), range(4, 7), range(7, 10)] + + def test_func(dstream): + return dstream.map(lambda x: str(x)) + expected_output = map(lambda x: map(lambda y: str(y), x), test_input) + output = self._run_stream(test_input, test_func, expected_output) + self.assertEqual(expected_output, output) + + def test_flatMap_batch(self): + """Basic operation test for DStream.faltMap with batch deserializer.""" + test_input = [range(1, 5), range(5, 9), range(9, 13)] + + def test_func(dstream): + return dstream.flatMap(lambda x: (x, x * 2)) + expected_output = map(lambda x: list(chain.from_iterable((map(lambda y: [y, y * 2], x)))), + test_input) + output = self._run_stream(test_input, test_func, expected_output) + self.assertEqual(expected_output, output) + + def test_flatMap_unbatch(self): + """Basic operation test for DStream.faltMap with unbatch deserializer.""" + test_input = [range(1, 4), range(4, 7), range(7, 10)] + + def test_func(dstream): + return dstream.flatMap(lambda x: (x, x * 2)) + expected_output = map(lambda x: list(chain.from_iterable((map(lambda y: [y, y * 2], x)))), + test_input) + output = self._run_stream(test_input, test_func, expected_output) + self.assertEqual(expected_output, output) + + def test_filter_batch(self): + """Basic operation test for DStream.filter with batch deserializer.""" + test_input = [range(1, 5), range(5, 9), range(9, 13)] + + def test_func(dstream): + return dstream.filter(lambda x: x % 2 == 0) + expected_output = map(lambda x: filter(lambda y: y % 2 == 0, x), test_input) + output = self._run_stream(test_input, test_func, expected_output) + self.assertEqual(expected_output, output) + + def test_filter_unbatch(self): + """Basic operation test for DStream.filter with unbatch deserializer.""" + test_input = [range(1, 4), range(4, 7), range(7, 10)] + + def test_func(dstream): + return dstream.filter(lambda x: x % 2 == 0) + expected_output = map(lambda x: filter(lambda y: y % 2 == 0, x), test_input) + output = self._run_stream(test_input, test_func, expected_output) + self.assertEqual(expected_output, output) + + def test_count_batch(self): + """Basic operation test for DStream.count with batch deserializer.""" + test_input = [range(1, 5), range(1, 10), range(1, 20)] + + def test_func(dstream): + return dstream.count() + expected_output = map(lambda x: [len(x)], test_input) + output = self._run_stream(test_input, test_func, expected_output) + self.assertEqual(expected_output, output) + + def test_count_unbatch(self): + """Basic operation test for DStream.count with unbatch deserializer.""" + test_input = [[], [1], range(1, 3), range(1, 4)] + + def test_func(dstream): + return dstream.count() + expected_output = map(lambda x: [len(x)], test_input) + output = self._run_stream(test_input, test_func, expected_output) + self.assertEqual(expected_output, output) + + def test_reduce_batch(self): + """Basic operation test for DStream.reduce with batch deserializer.""" + test_input = [range(1, 5), range(5, 9), range(9, 13)] + + def test_func(dstream): + return dstream.reduce(operator.add) + expected_output = map(lambda x: [reduce(operator.add, x)], test_input) + output = self._run_stream(test_input, test_func, expected_output) + self.assertEqual(expected_output, output) + + def test_reduce_unbatch(self): + """Basic operation test for DStream.reduce with unbatch deserializer.""" + test_input = [[1], range(1, 3), range(1, 4)] + + def test_func(dstream): + return dstream.reduce(operator.add) + expected_output = map(lambda x: [reduce(operator.add, x)], test_input) + output = self._run_stream(test_input, test_func, expected_output) + self.assertEqual(expected_output, output) + + def test_reduceByKey_batch(self): + """Basic operation test for DStream.reduceByKey with batch deserializer.""" + test_input = [[("a", 1), ("a", 1), ("b", 1), ("b", 1)], + [("", 1), ("", 1), ("", 1), ("", 1)], + [(1, 1), (1, 1), (2, 1), (2, 1), (3, 1)]] + + def test_func(dstream): + return dstream.reduceByKey(operator.add) + expected_output = [[("a", 2), ("b", 2)], [("", 4)], [(1, 2), (2, 2), (3, 1)]] + output = self._run_stream(test_input, test_func, expected_output) + for result in (output, expected_output): + self._sort_result_based_on_key(result) + self.assertEqual(expected_output, output) + + def test_reduceByKey_unbatch(self): + """Basic operation test for DStream.reduceByKey with unbatch deserializer.""" + test_input = [[("a", 1), ("a", 1), ("b", 1)], [("", 1), ("", 1)], []] + + def test_func(dstream): + return dstream.reduceByKey(operator.add) + expected_output = [[("a", 2), ("b", 1)], [("", 2)], []] + output = self._run_stream(test_input, test_func, expected_output) + for result in (output, expected_output): + self._sort_result_based_on_key(result) + self.assertEqual(expected_output, output) + + def test_mapValues_batch(self): + """Basic operation test for DStream.mapValues with batch deserializer.""" + test_input = [[("a", 2), ("b", 2), ("c", 1), ("d", 1)], + [("", 4), (1, 1), (2, 2), (3, 3)], + [(1, 1), (2, 1), (3, 1), (4, 1)]] + + def test_func(dstream): + return dstream.mapValues(lambda x: x + 10) + expected_output = [[("a", 12), ("b", 12), ("c", 11), ("d", 11)], + [("", 14), (1, 11), (2, 12), (3, 13)], + [(1, 11), (2, 11), (3, 11), (4, 11)]] + output = self._run_stream(test_input, test_func, expected_output) + for result in (output, expected_output): + self._sort_result_based_on_key(result) + self.assertEqual(expected_output, output) + + def test_mapValues_unbatch(self): + """Basic operation test for DStream.mapValues with unbatch deserializer.""" + test_input = [[("a", 2), ("b", 1)], [("", 2)], [], [(1, 1), (2, 2)]] + + def test_func(dstream): + return dstream.mapValues(lambda x: x + 10) + expected_output = [[("a", 12), ("b", 11)], [("", 12)], [], [(1, 11), (2, 12)]] + output = self._run_stream(test_input, test_func, expected_output) + for result in (output, expected_output): + self._sort_result_based_on_key(result) + self.assertEqual(expected_output, output) + + def test_flatMapValues_batch(self): + """Basic operation test for DStream.flatMapValues with batch deserializer.""" + test_input = [[("a", 2), ("b", 2), ("c", 1), ("d", 1)], + [("", 4), (1, 1), (2, 1), (3, 1)], + [(1, 1), (2, 1), (3, 1), (4, 1)]] + + def test_func(dstream): + return dstream.flatMapValues(lambda x: (x, x + 10)) + expected_output = [[("a", 2), ("a", 12), ("b", 2), ("b", 12), + ("c", 1), ("c", 11), ("d", 1), ("d", 11)], + [("", 4), ("", 14), (1, 1), (1, 11), (2, 1), (2, 11), (3, 1), (3, 11)], + [(1, 1), (1, 11), (2, 1), (2, 11), (3, 1), (3, 11), (4, 1), (4, 11)]] + output = self._run_stream(test_input, test_func, expected_output) + self.assertEqual(expected_output, output) + + def test_flatMapValues_unbatch(self): + """Basic operation test for DStream.flatMapValues with unbatch deserializer.""" + test_input = [[("a", 2), ("b", 1)], [("", 2)], []] + + def test_func(dstream): + return dstream.flatMapValues(lambda x: (x, x + 10)) + expected_output = [[("a", 2), ("a", 12), ("b", 1), ("b", 11)], [("", 2), ("", 12)], []] + output = self._run_stream(test_input, test_func, expected_output) + self.assertEqual(expected_output, output) + + def test_glom_batch(self): + """Basic operation test for DStream.glom with batch deserializer.""" + test_input = [range(1, 5), range(5, 9), range(9, 13)] + numSlices = 2 + + def test_func(dstream): + return dstream.glom() + expected_output = [[[1, 2], [3, 4]], [[5, 6], [7, 8]], [[9, 10], [11, 12]]] + output = self._run_stream(test_input, test_func, expected_output, numSlices) + self.assertEqual(expected_output, output) + + def test_glom_unbatach(self): + """Basic operation test for DStream.glom with unbatch deserializer.""" + test_input = [range(1, 4), range(4, 7), range(7, 10)] + numSlices = 2 + + def test_func(dstream): + return dstream.glom() + expected_output = [[[1], [2, 3]], [[4], [5, 6]], [[7], [8, 9]]] + output = self._run_stream(test_input, test_func, expected_output, numSlices) + self.assertEqual(expected_output, output) + + def test_mapPartitions_batch(self): + """Basic operation test for DStream.mapPartitions with batch deserializer.""" + test_input = [range(1, 5), range(5, 9), range(9, 13)] + numSlices = 2 + + def test_func(dstream): + def f(iterator): + yield sum(iterator) + return dstream.mapPartitions(f) + expected_output = [[3, 7], [11, 15], [19, 23]] + output = self._run_stream(test_input, test_func, expected_output, numSlices) + self.assertEqual(expected_output, output) + + def test_mapPartitions_unbatch(self): + """Basic operation test for DStream.mapPartitions with unbatch deserializer.""" + test_input = [range(1, 4), range(4, 7), range(7, 10)] + numSlices = 2 + + def test_func(dstream): + def f(iterator): + yield sum(iterator) + return dstream.mapPartitions(f) + expected_output = [[1, 5], [4, 11], [7, 17]] + output = self._run_stream(test_input, test_func, expected_output, numSlices) + self.assertEqual(expected_output, output) + + def test_countByValue_batch(self): + """Basic operation test for DStream.countByValue with batch deserializer.""" + test_input = [range(1, 5) * 2, range(5, 7) + range(5, 9), ["a", "a", "b", ""]] + + def test_func(dstream): + return dstream.countByValue() + expected_output = [[(1, 2), (2, 2), (3, 2), (4, 2)], + [(5, 2), (6, 2), (7, 1), (8, 1)], + [("a", 2), ("b", 1), ("", 1)]] + output = self._run_stream(test_input, test_func, expected_output) + for result in (output, expected_output): + self._sort_result_based_on_key(result) + self.assertEqual(expected_output, output) + + def test_countByValue_unbatch(self): + """Basic operation test for DStream.countByValue with unbatch deserializer.""" + test_input = [range(1, 4), [1, 1, ""], ["a", "a", "b"]] + + def test_func(dstream): + return dstream.countByValue() + expected_output = [[(1, 1), (2, 1), (3, 1)], + [(1, 2), ("", 1)], + [("a", 2), ("b", 1)]] + output = self._run_stream(test_input, test_func, expected_output) + for result in (output, expected_output): + self._sort_result_based_on_key(result) + self.assertEqual(expected_output, output) + + def test_groupByKey_batch(self): + """Basic operation test for DStream.groupByKey with batch deserializer.""" + test_input = [[(1, 1), (2, 1), (3, 1), (4, 1)], + [(1, 1), (1, 1), (1, 1), (2, 1), (2, 1), (3, 1)], + [("a", 1), ("a", 1), ("b", 1), ("", 1), ("", 1), ("", 1)]] + + def test_func(dstream): + return dstream.groupByKey() + expected_output = [[(1, [1]), (2, [1]), (3, [1]), (4, [1])], + [(1, [1, 1, 1]), (2, [1, 1]), (3, [1])], + [("a", [1, 1]), ("b", [1]), ("", [1, 1, 1])]] + scattered_output = self._run_stream(test_input, test_func, expected_output) + output = self._convert_iter_value_to_list(scattered_output) + for result in (output, expected_output): + self._sort_result_based_on_key(result) + self.assertEqual(expected_output, output) + + def test_groupByKey_unbatch(self): + """Basic operation test for DStream.groupByKey with unbatch deserializer.""" + test_input = [[(1, 1), (2, 1), (3, 1)], + [(1, 1), (1, 1), ("", 1)], + [("a", 1), ("a", 1), ("b", 1)]] + + def test_func(dstream): + return dstream.groupByKey() + expected_output = [[(1, [1]), (2, [1]), (3, [1])], + [(1, [1, 1]), ("", [1])], + [("a", [1, 1]), ("b", [1])]] + scattered_output = self._run_stream(test_input, test_func, expected_output) + output = self._convert_iter_value_to_list(scattered_output) + for result in (output, expected_output): + self._sort_result_based_on_key(result) + self.assertEqual(expected_output, output) + + def test_combineByKey_batch(self): + """Basic operation test for DStream.combineByKey with batch deserializer.""" + test_input = [[(1, 1), (2, 1), (3, 1), (4, 1)], + [(1, 1), (1, 1), (1, 1), (2, 1), (2, 1), (3, 1)], + [("a", 1), ("a", 1), ("b", 1), ("", 1), ("", 1), ("", 1)]] + + def test_func(dstream): + def add(a, b): + return a + str(b) + return dstream.combineByKey(str, add, add) + expected_output = [[(1, "1"), (2, "1"), (3, "1"), (4, "1")], + [(1, "111"), (2, "11"), (3, "1")], + [("a", "11"), ("b", "1"), ("", "111")]] + output = self._run_stream(test_input, test_func, expected_output) + for result in (output, expected_output): + self._sort_result_based_on_key(result) + self.assertEqual(expected_output, output) + + def test_combineByKey_unbatch(self): + """Basic operation test for DStream.combineByKey with unbatch deserializer.""" + test_input = [[(1, 1), (2, 1), (3, 1)], + [(1, 1), (1, 1), ("", 1)], + [("a", 1), ("a", 1), ("b", 1)]] + + def test_func(dstream): + def add(a, b): + return a + str(b) + return dstream.combineByKey(str, add, add) + expected_output = [[(1, "1"), (2, "1"), (3, "1")], + [(1, "11"), ("", "1")], + [("a", "11"), ("b", "1")]] + output = self._run_stream(test_input, test_func, expected_output) + for result in (output, expected_output): + self._sort_result_based_on_key(result) + self.assertEqual(expected_output, output) + + def _convert_iter_value_to_list(self, outputs): + """Return key value pair list. Value is converted to iterator to list.""" + result = list() + for output in outputs: + result.append(map(lambda (x, y): (x, list(y)), output)) + return result + + def _sort_result_based_on_key(self, outputs): + """Sort the list base onf first value.""" + for output in outputs: + output.sort(key=lambda x: x[0]) + + def _run_stream(self, test_input, test_func, expected_output, numSlices=None): + """ + Start stream and return the result. + @param test_input: dataset for the test. This should be list of lists. + @param test_func: wrapped test_function. This function should return PythonDStream object. + @param expected_output: expected output for this testcase. + @param numSlices: the number of slices in the rdd in the dstream. + """ + # Generate input stream with user-defined input. + numSlices = numSlices or self.numInputPartitions + test_input_stream = self.ssc._testInputStream(test_input, numSlices) + # Apply test function to stream. + test_stream = test_func(test_input_stream) + # Add job to get output from stream. + result = list() + test_stream._test_output(result) + self.ssc.start() + + start_time = time.time() + # Loop until get the expected the number of the result from the stream. + while True: + current_time = time.time() + # Check time out. + if (current_time - start_time) > self.timeout: + break + # StreamingContext.awaitTermination is not used to wait because + # if py4j server is called every 50 milliseconds, it gets an error. + time.sleep(0.05) + # Check if the output is the same length of expected output. + if len(expected_output) == len(result): + break + + return result + +if __name__ == "__main__": + unittest.main() + SparkContext._gateway._shutdown_callback_server() diff --git a/python/pyspark/streaming/util.py b/python/pyspark/streaming/util.py new file mode 100644 index 0000000000000..cf90952543fc0 --- /dev/null +++ b/python/pyspark/streaming/util.py @@ -0,0 +1,84 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from pyspark.rdd import RDD + + +class RDDFunction(): + """ + This class is for py4j callback. This class is related with + org.apache.spark.streaming.api.python.PythonRDDFunction. + """ + def __init__(self, ctx, jrdd_deserializer, func): + self.ctx = ctx + self.deserializer = jrdd_deserializer + self.func = func + + def call(self, jrdd, time): + # Wrap JavaRDD into python's RDD class + rdd = RDD(jrdd, self.ctx, self.deserializer) + # Call user defined RDD function + self.func(rdd, time) + + def __str__(self): + return "%s, %s" % (str(self.deserializer), str(self.func)) + + class Java: + implements = ['org.apache.spark.streaming.api.python.PythonRDDFunction'] + + +def msDurationToString(ms): + """ + Returns a human-readable string representing a duration such as "35ms" + + >> msDurationToString(10) + '10 ms' + >>> msDurationToString(1000) + '1.0 s' + >>> msDurationToString(60000) + '1.0 m' + >>> msDurationToString(3600000) + '1.00 h' + """ + second = 1000 + minute = 60 * second + hour = 60 * minute + + if ms < second: + return "%d ms" % ms + elif ms < minute: + return "%.1f s" % (float(ms) / second) + elif ms < hour: + return "%.1f m" % (float(ms) / minute) + else: + return "%.2f h" % (float(ms) / hour) + + +def rddToFileName(prefix, suffix, time): + """ + Return string prefix-time(.suffix) + + >>> rddToFileName("spark", None, 12345678910) + 'spark-12345678910' + >>> rddToFileName("spark", "tmp", 12345678910) + 'spark-12345678910.tmp' + + """ + if suffix is None: + return prefix + "-" + str(time) + else: + return prefix + "-" + str(time) + "." + suffix diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index c15bb457759ed..f1fece998cd54 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -19,6 +19,7 @@ Unit tests for PySpark; additional tests are implemented as doctests in individual modules. """ +from array import array from fileinput import input from glob import glob import os @@ -28,36 +29,117 @@ import sys import tempfile import time -import unittest import zipfile +if sys.version_info[:2] <= (2, 6): + import unittest2 as unittest +else: + import unittest + + from pyspark.context import SparkContext from pyspark.files import SparkFiles from pyspark.serializers import read_int +from pyspark.shuffle import Aggregator, InMemoryMerger, ExternalMerger _have_scipy = False +_have_numpy = False try: import scipy.sparse _have_scipy = True except: # No SciPy, but that's okay, we'll skip those tests pass +try: + import numpy as np + _have_numpy = True +except: + # No NumPy, but that's okay, we'll skip those tests + pass SPARK_HOME = os.environ["SPARK_HOME"] +class TestMerger(unittest.TestCase): + + def setUp(self): + self.N = 1 << 16 + self.l = [i for i in xrange(self.N)] + self.data = zip(self.l, self.l) + self.agg = Aggregator(lambda x: [x], + lambda x, y: x.append(y) or x, + lambda x, y: x.extend(y) or x) + + def test_in_memory(self): + m = InMemoryMerger(self.agg) + m.mergeValues(self.data) + self.assertEqual(sum(sum(v) for k, v in m.iteritems()), + sum(xrange(self.N))) + + m = InMemoryMerger(self.agg) + m.mergeCombiners(map(lambda (x, y): (x, [y]), self.data)) + self.assertEqual(sum(sum(v) for k, v in m.iteritems()), + sum(xrange(self.N))) + + def test_small_dataset(self): + m = ExternalMerger(self.agg, 1000) + m.mergeValues(self.data) + self.assertEqual(m.spills, 0) + self.assertEqual(sum(sum(v) for k, v in m.iteritems()), + sum(xrange(self.N))) + + m = ExternalMerger(self.agg, 1000) + m.mergeCombiners(map(lambda (x, y): (x, [y]), self.data)) + self.assertEqual(m.spills, 0) + self.assertEqual(sum(sum(v) for k, v in m.iteritems()), + sum(xrange(self.N))) + + def test_medium_dataset(self): + m = ExternalMerger(self.agg, 10) + m.mergeValues(self.data) + self.assertTrue(m.spills >= 1) + self.assertEqual(sum(sum(v) for k, v in m.iteritems()), + sum(xrange(self.N))) + + m = ExternalMerger(self.agg, 10) + m.mergeCombiners(map(lambda (x, y): (x, [y]), self.data * 3)) + self.assertTrue(m.spills >= 1) + self.assertEqual(sum(sum(v) for k, v in m.iteritems()), + sum(xrange(self.N)) * 3) + + def test_huge_dataset(self): + m = ExternalMerger(self.agg, 10) + m.mergeCombiners(map(lambda (k, v): (k, [str(v)]), self.data * 10)) + self.assertTrue(m.spills >= 1) + self.assertEqual(sum(len(v) for k, v in m._recursive_merged_items(0)), + self.N * 10) + m._cleanup() + + +class SerializationTestCase(unittest.TestCase): + + def test_namedtuple(self): + from collections import namedtuple + from cPickle import dumps, loads + P = namedtuple("P", "x y") + p1 = P(1, 3) + p2 = loads(dumps(p1, 2)) + self.assertEquals(p1, p2) + + class PySparkTestCase(unittest.TestCase): def setUp(self): self._old_sys_path = list(sys.path) class_name = self.__class__.__name__ - self.sc = SparkContext('local[4]', class_name , batchSize=2) + self.sc = SparkContext('local[4]', class_name, batchSize=2) def tearDown(self): self.sc.stop() sys.path = self._old_sys_path + class TestCheckpoint(PySparkTestCase): def setUp(self): @@ -107,11 +189,18 @@ class TestAddFile(PySparkTestCase): def test_add_py_file(self): # To ensure that we're actually testing addPyFile's effects, check that # this job fails due to `userlibrary` not being on the Python path: + # disable logging in log4j temporarily + log4j = self.sc._jvm.org.apache.log4j + old_level = log4j.LogManager.getRootLogger().getLevel() + log4j.LogManager.getRootLogger().setLevel(log4j.Level.FATAL) + def func(x): from userlibrary import UserClass return UserClass().hello() self.assertRaises(Exception, self.sc.parallelize(range(2)).map(func).first) + log4j.LogManager.getRootLogger().setLevel(old_level) + # Add the file, so the job should now succeed: path = os.path.join(SPARK_HOME, "python/test_support/userlibrary.py") self.sc.addPyFile(path) @@ -151,6 +240,12 @@ def func(): class TestRDDFunctions(PySparkTestCase): + def test_failed_sparkcontext_creation(self): + # Regression test for SPARK-1550 + self.sc.stop() + self.assertRaises(Exception, lambda: SparkContext("an-invalid-master-name")) + self.sc = SparkContext("local") + def test_save_as_textfile_with_unicode(self): # Regression test for SPARK-970 x = u"\u00A1Hola, mundo!" @@ -168,6 +263,15 @@ def test_transforming_cartesian_result(self): cart = rdd1.cartesian(rdd2) result = cart.map(lambda (x, y): x + y).collect() + def test_transforming_pickle_file(self): + # Regression test for SPARK-2601 + data = self.sc.parallelize(["Hello", "World!"]) + tempFile = tempfile.NamedTemporaryFile(delete=True) + tempFile.close() + data.saveAsPickleFile(tempFile.name) + pickled_file = self.sc.pickleFile(tempFile.name) + pickled_file.map(lambda x: x).collect() + def test_cartesian_on_textfile(self): # Regression test for path = os.path.join(SPARK_HOME, "python/test_support/hello.txt") @@ -190,6 +294,7 @@ def test_deleting_input_files(self): def testAggregateByKey(self): data = self.sc.parallelize([(1, 1), (1, 1), (3, 2), (5, 1), (5, 3)], 2) + def seqOp(x, y): x.add(y) return x @@ -197,17 +302,40 @@ def seqOp(x, y): def combOp(x, y): x |= y return x - + sets = dict(data.aggregateByKey(set(), seqOp, combOp).collect()) self.assertEqual(3, len(sets)) self.assertEqual(set([1]), sets[1]) self.assertEqual(set([2]), sets[3]) self.assertEqual(set([1, 3]), sets[5]) + def test_itemgetter(self): + rdd = self.sc.parallelize([range(10)]) + from operator import itemgetter + self.assertEqual([1], rdd.map(itemgetter(1)).collect()) + self.assertEqual([(2, 3)], rdd.map(itemgetter(2, 3)).collect()) + + def test_namedtuple_in_rdd(self): + from collections import namedtuple + Person = namedtuple("Person", "id firstName lastName") + jon = Person(1, "Jon", "Doe") + jane = Person(2, "Jane", "Doe") + theDoes = self.sc.parallelize([jon, jane]) + self.assertEquals([jon, jane], theDoes.collect()) + + def test_large_broadcast(self): + N = 100000 + data = [[float(i) for i in range(300)] for i in range(N)] + bdata = self.sc.broadcast(data) # 270MB + m = self.sc.parallelize(range(1), 1).map(lambda x: len(bdata.value)).sum() + self.assertEquals(N, m) + + class TestIO(PySparkTestCase): def test_stdout_redirection(self): import subprocess + def func(x): subprocess.check_call('ls', shell=True) self.sc.parallelize([1]).foreach(func) @@ -239,6 +367,17 @@ def test_sequencefiles(self): ed = [(1.0, u'aa'), (1.0, u'aa'), (2.0, u'aa'), (2.0, u'bb'), (2.0, u'bb'), (3.0, u'cc')] self.assertEqual(doubles, ed) + bytes = sorted(self.sc.sequenceFile(basepath + "/sftestdata/sfbytes/", + "org.apache.hadoop.io.IntWritable", + "org.apache.hadoop.io.BytesWritable").collect()) + ebs = [(1, bytearray('aa', 'utf-8')), + (1, bytearray('aa', 'utf-8')), + (2, bytearray('aa', 'utf-8')), + (2, bytearray('bb', 'utf-8')), + (2, bytearray('bb', 'utf-8')), + (3, bytearray('cc', 'utf-8'))] + self.assertEqual(bytes, ebs) + text = sorted(self.sc.sequenceFile(basepath + "/sftestdata/sftext/", "org.apache.hadoop.io.Text", "org.apache.hadoop.io.Text").collect()) @@ -265,14 +404,34 @@ def test_sequencefiles(self): maps = sorted(self.sc.sequenceFile(basepath + "/sftestdata/sfmap/", "org.apache.hadoop.io.IntWritable", "org.apache.hadoop.io.MapWritable").collect()) - em = [(1, {2.0: u'aa'}), + em = [(1, {}), (1, {3.0: u'bb'}), (2, {1.0: u'aa'}), (2, {1.0: u'cc'}), - (2, {3.0: u'bb'}), (3, {2.0: u'dd'})] self.assertEqual(maps, em) + # arrays get pickled to tuples by default + tuples = sorted(self.sc.sequenceFile( + basepath + "/sftestdata/sfarray/", + "org.apache.hadoop.io.IntWritable", + "org.apache.spark.api.python.DoubleArrayWritable").collect()) + et = [(1, ()), + (2, (3.0, 4.0, 5.0)), + (3, (4.0, 5.0, 6.0))] + self.assertEqual(tuples, et) + + # with custom converters, primitive arrays can stay as arrays + arrays = sorted(self.sc.sequenceFile( + basepath + "/sftestdata/sfarray/", + "org.apache.hadoop.io.IntWritable", + "org.apache.spark.api.python.DoubleArrayWritable", + valueConverter="org.apache.spark.api.python.WritableToDoubleArrayConverter").collect()) + ea = [(1, array('d')), + (2, array('d', [3.0, 4.0, 5.0])), + (3, array('d', [4.0, 5.0, 6.0]))] + self.assertEqual(arrays, ea) + clazz = sorted(self.sc.sequenceFile(basepath + "/sftestdata/sfclass/", "org.apache.hadoop.io.Text", "org.apache.spark.api.python.TestWritable").collect()) @@ -281,6 +440,12 @@ def test_sequencefiles(self): u'double': 54.0, u'int': 123, u'str': u'test1'}) self.assertEqual(clazz[0], ec) + unbatched_clazz = sorted(self.sc.sequenceFile(basepath + "/sftestdata/sfclass/", + "org.apache.hadoop.io.Text", + "org.apache.spark.api.python.TestWritable", + batchSize=1).collect()) + self.assertEqual(unbatched_clazz[0], ec) + def test_oldhadoop(self): basepath = self.tempdir.name ints = sorted(self.sc.hadoopFile(basepath + "/sftestdata/sfint/", @@ -291,10 +456,11 @@ def test_oldhadoop(self): self.assertEqual(ints, ei) hellopath = os.path.join(SPARK_HOME, "python/test_support/hello.txt") - hello = self.sc.hadoopFile(hellopath, - "org.apache.hadoop.mapred.TextInputFormat", - "org.apache.hadoop.io.LongWritable", - "org.apache.hadoop.io.Text").collect() + oldconf = {"mapred.input.dir": hellopath} + hello = self.sc.hadoopRDD("org.apache.hadoop.mapred.TextInputFormat", + "org.apache.hadoop.io.LongWritable", + "org.apache.hadoop.io.Text", + conf=oldconf).collect() result = [(0, u'Hello World!')] self.assertEqual(hello, result) @@ -309,10 +475,11 @@ def test_newhadoop(self): self.assertEqual(ints, ei) hellopath = os.path.join(SPARK_HOME, "python/test_support/hello.txt") - hello = self.sc.newAPIHadoopFile(hellopath, - "org.apache.hadoop.mapreduce.lib.input.TextInputFormat", - "org.apache.hadoop.io.LongWritable", - "org.apache.hadoop.io.Text").collect() + newconf = {"mapred.input.dir": hellopath} + hello = self.sc.newAPIHadoopRDD("org.apache.hadoop.mapreduce.lib.input.TextInputFormat", + "org.apache.hadoop.io.LongWritable", + "org.apache.hadoop.io.Text", + conf=newconf).collect() result = [(0, u'Hello World!')] self.assertEqual(hello, result) @@ -347,18 +514,282 @@ def test_bad_inputs(self): "org.apache.hadoop.io.IntWritable", "org.apache.hadoop.io.Text")) - def test_converter(self): + def test_converters(self): + # use of custom converters basepath = self.tempdir.name maps = sorted(self.sc.sequenceFile( basepath + "/sftestdata/sfmap/", "org.apache.hadoop.io.IntWritable", "org.apache.hadoop.io.MapWritable", - valueConverter="org.apache.spark.api.python.TestConverter").collect()) - em = [(1, [2.0]), (1, [3.0]), (2, [1.0]), (2, [1.0]), (2, [3.0]), (3, [2.0])] + keyConverter="org.apache.spark.api.python.TestInputKeyConverter", + valueConverter="org.apache.spark.api.python.TestInputValueConverter").collect()) + em = [(u'\x01', []), + (u'\x01', [3.0]), + (u'\x02', [1.0]), + (u'\x02', [1.0]), + (u'\x03', [2.0])] self.assertEqual(maps, em) +class TestOutputFormat(PySparkTestCase): + + def setUp(self): + PySparkTestCase.setUp(self) + self.tempdir = tempfile.NamedTemporaryFile(delete=False) + os.unlink(self.tempdir.name) + + def tearDown(self): + PySparkTestCase.tearDown(self) + shutil.rmtree(self.tempdir.name, ignore_errors=True) + + def test_sequencefiles(self): + basepath = self.tempdir.name + ei = [(1, u'aa'), (1, u'aa'), (2, u'aa'), (2, u'bb'), (2, u'bb'), (3, u'cc')] + self.sc.parallelize(ei).saveAsSequenceFile(basepath + "/sfint/") + ints = sorted(self.sc.sequenceFile(basepath + "/sfint/").collect()) + self.assertEqual(ints, ei) + + ed = [(1.0, u'aa'), (1.0, u'aa'), (2.0, u'aa'), (2.0, u'bb'), (2.0, u'bb'), (3.0, u'cc')] + self.sc.parallelize(ed).saveAsSequenceFile(basepath + "/sfdouble/") + doubles = sorted(self.sc.sequenceFile(basepath + "/sfdouble/").collect()) + self.assertEqual(doubles, ed) + + ebs = [(1, bytearray(b'\x00\x07spam\x08')), (2, bytearray(b'\x00\x07spam\x08'))] + self.sc.parallelize(ebs).saveAsSequenceFile(basepath + "/sfbytes/") + bytes = sorted(self.sc.sequenceFile(basepath + "/sfbytes/").collect()) + self.assertEqual(bytes, ebs) + + et = [(u'1', u'aa'), + (u'2', u'bb'), + (u'3', u'cc')] + self.sc.parallelize(et).saveAsSequenceFile(basepath + "/sftext/") + text = sorted(self.sc.sequenceFile(basepath + "/sftext/").collect()) + self.assertEqual(text, et) + + eb = [(1, False), (1, True), (2, False), (2, False), (2, True), (3, True)] + self.sc.parallelize(eb).saveAsSequenceFile(basepath + "/sfbool/") + bools = sorted(self.sc.sequenceFile(basepath + "/sfbool/").collect()) + self.assertEqual(bools, eb) + + en = [(1, None), (1, None), (2, None), (2, None), (2, None), (3, None)] + self.sc.parallelize(en).saveAsSequenceFile(basepath + "/sfnull/") + nulls = sorted(self.sc.sequenceFile(basepath + "/sfnull/").collect()) + self.assertEqual(nulls, en) + + em = [(1, {}), + (1, {3.0: u'bb'}), + (2, {1.0: u'aa'}), + (2, {1.0: u'cc'}), + (3, {2.0: u'dd'})] + self.sc.parallelize(em).saveAsSequenceFile(basepath + "/sfmap/") + maps = sorted(self.sc.sequenceFile(basepath + "/sfmap/").collect()) + self.assertEqual(maps, em) + + def test_oldhadoop(self): + basepath = self.tempdir.name + dict_data = [(1, {}), + (1, {"row1": 1.0}), + (2, {"row2": 2.0})] + self.sc.parallelize(dict_data).saveAsHadoopFile( + basepath + "/oldhadoop/", + "org.apache.hadoop.mapred.SequenceFileOutputFormat", + "org.apache.hadoop.io.IntWritable", + "org.apache.hadoop.io.MapWritable") + result = sorted(self.sc.hadoopFile( + basepath + "/oldhadoop/", + "org.apache.hadoop.mapred.SequenceFileInputFormat", + "org.apache.hadoop.io.IntWritable", + "org.apache.hadoop.io.MapWritable").collect()) + self.assertEqual(result, dict_data) + + conf = { + "mapred.output.format.class": "org.apache.hadoop.mapred.SequenceFileOutputFormat", + "mapred.output.key.class": "org.apache.hadoop.io.IntWritable", + "mapred.output.value.class": "org.apache.hadoop.io.MapWritable", + "mapred.output.dir": basepath + "/olddataset/" + } + self.sc.parallelize(dict_data).saveAsHadoopDataset(conf) + input_conf = {"mapred.input.dir": basepath + "/olddataset/"} + old_dataset = sorted(self.sc.hadoopRDD( + "org.apache.hadoop.mapred.SequenceFileInputFormat", + "org.apache.hadoop.io.IntWritable", + "org.apache.hadoop.io.MapWritable", + conf=input_conf).collect()) + self.assertEqual(old_dataset, dict_data) + + @unittest.skipIf(sys.version_info[:2] <= (2, 6), "Skipped on 2.6 until SPARK-2951 is fixed") + def test_newhadoop(self): + basepath = self.tempdir.name + # use custom ArrayWritable types and converters to handle arrays + array_data = [(1, array('d')), + (1, array('d', [1.0, 2.0, 3.0])), + (2, array('d', [3.0, 4.0, 5.0]))] + self.sc.parallelize(array_data).saveAsNewAPIHadoopFile( + basepath + "/newhadoop/", + "org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat", + "org.apache.hadoop.io.IntWritable", + "org.apache.spark.api.python.DoubleArrayWritable", + valueConverter="org.apache.spark.api.python.DoubleArrayToWritableConverter") + result = sorted(self.sc.newAPIHadoopFile( + basepath + "/newhadoop/", + "org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat", + "org.apache.hadoop.io.IntWritable", + "org.apache.spark.api.python.DoubleArrayWritable", + valueConverter="org.apache.spark.api.python.WritableToDoubleArrayConverter").collect()) + self.assertEqual(result, array_data) + + conf = { + "mapreduce.outputformat.class": + "org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat", + "mapred.output.key.class": "org.apache.hadoop.io.IntWritable", + "mapred.output.value.class": "org.apache.spark.api.python.DoubleArrayWritable", + "mapred.output.dir": basepath + "/newdataset/" + } + self.sc.parallelize(array_data).saveAsNewAPIHadoopDataset( + conf, + valueConverter="org.apache.spark.api.python.DoubleArrayToWritableConverter") + input_conf = {"mapred.input.dir": basepath + "/newdataset/"} + new_dataset = sorted(self.sc.newAPIHadoopRDD( + "org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat", + "org.apache.hadoop.io.IntWritable", + "org.apache.spark.api.python.DoubleArrayWritable", + valueConverter="org.apache.spark.api.python.WritableToDoubleArrayConverter", + conf=input_conf).collect()) + self.assertEqual(new_dataset, array_data) + + def test_newolderror(self): + basepath = self.tempdir.name + rdd = self.sc.parallelize(range(1, 4)).map(lambda x: (x, "a" * x)) + self.assertRaises(Exception, lambda: rdd.saveAsHadoopFile( + basepath + "/newolderror/saveAsHadoopFile/", + "org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat")) + self.assertRaises(Exception, lambda: rdd.saveAsNewAPIHadoopFile( + basepath + "/newolderror/saveAsNewAPIHadoopFile/", + "org.apache.hadoop.mapred.SequenceFileOutputFormat")) + + def test_bad_inputs(self): + basepath = self.tempdir.name + rdd = self.sc.parallelize(range(1, 4)).map(lambda x: (x, "a" * x)) + self.assertRaises(Exception, lambda: rdd.saveAsHadoopFile( + basepath + "/badinputs/saveAsHadoopFile/", + "org.apache.hadoop.mapred.NotValidOutputFormat")) + self.assertRaises(Exception, lambda: rdd.saveAsNewAPIHadoopFile( + basepath + "/badinputs/saveAsNewAPIHadoopFile/", + "org.apache.hadoop.mapreduce.lib.output.NotValidOutputFormat")) + + def test_converters(self): + # use of custom converters + basepath = self.tempdir.name + data = [(1, {3.0: u'bb'}), + (2, {1.0: u'aa'}), + (3, {2.0: u'dd'})] + self.sc.parallelize(data).saveAsNewAPIHadoopFile( + basepath + "/converters/", + "org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat", + keyConverter="org.apache.spark.api.python.TestOutputKeyConverter", + valueConverter="org.apache.spark.api.python.TestOutputValueConverter") + converted = sorted(self.sc.sequenceFile(basepath + "/converters/").collect()) + expected = [(u'1', 3.0), + (u'2', 1.0), + (u'3', 2.0)] + self.assertEqual(converted, expected) + + def test_reserialization(self): + basepath = self.tempdir.name + x = range(1, 5) + y = range(1001, 1005) + data = zip(x, y) + rdd = self.sc.parallelize(x).zip(self.sc.parallelize(y)) + rdd.saveAsSequenceFile(basepath + "/reserialize/sequence") + result1 = sorted(self.sc.sequenceFile(basepath + "/reserialize/sequence").collect()) + self.assertEqual(result1, data) + + rdd.saveAsHadoopFile( + basepath + "/reserialize/hadoop", + "org.apache.hadoop.mapred.SequenceFileOutputFormat") + result2 = sorted(self.sc.sequenceFile(basepath + "/reserialize/hadoop").collect()) + self.assertEqual(result2, data) + + rdd.saveAsNewAPIHadoopFile( + basepath + "/reserialize/newhadoop", + "org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat") + result3 = sorted(self.sc.sequenceFile(basepath + "/reserialize/newhadoop").collect()) + self.assertEqual(result3, data) + + conf4 = { + "mapred.output.format.class": "org.apache.hadoop.mapred.SequenceFileOutputFormat", + "mapred.output.key.class": "org.apache.hadoop.io.IntWritable", + "mapred.output.value.class": "org.apache.hadoop.io.IntWritable", + "mapred.output.dir": basepath + "/reserialize/dataset"} + rdd.saveAsHadoopDataset(conf4) + result4 = sorted(self.sc.sequenceFile(basepath + "/reserialize/dataset").collect()) + self.assertEqual(result4, data) + + conf5 = {"mapreduce.outputformat.class": + "org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat", + "mapred.output.key.class": "org.apache.hadoop.io.IntWritable", + "mapred.output.value.class": "org.apache.hadoop.io.IntWritable", + "mapred.output.dir": basepath + "/reserialize/newdataset"} + rdd.saveAsNewAPIHadoopDataset(conf5) + result5 = sorted(self.sc.sequenceFile(basepath + "/reserialize/newdataset").collect()) + self.assertEqual(result5, data) + + def test_unbatched_save_and_read(self): + basepath = self.tempdir.name + ei = [(1, u'aa'), (1, u'aa'), (2, u'aa'), (2, u'bb'), (2, u'bb'), (3, u'cc')] + self.sc.parallelize(ei, numSlices=len(ei)).saveAsSequenceFile( + basepath + "/unbatched/") + + unbatched_sequence = sorted(self.sc.sequenceFile( + basepath + "/unbatched/", + batchSize=1).collect()) + self.assertEqual(unbatched_sequence, ei) + + unbatched_hadoopFile = sorted(self.sc.hadoopFile( + basepath + "/unbatched/", + "org.apache.hadoop.mapred.SequenceFileInputFormat", + "org.apache.hadoop.io.IntWritable", + "org.apache.hadoop.io.Text", + batchSize=1).collect()) + self.assertEqual(unbatched_hadoopFile, ei) + + unbatched_newAPIHadoopFile = sorted(self.sc.newAPIHadoopFile( + basepath + "/unbatched/", + "org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat", + "org.apache.hadoop.io.IntWritable", + "org.apache.hadoop.io.Text", + batchSize=1).collect()) + self.assertEqual(unbatched_newAPIHadoopFile, ei) + + oldconf = {"mapred.input.dir": basepath + "/unbatched/"} + unbatched_hadoopRDD = sorted(self.sc.hadoopRDD( + "org.apache.hadoop.mapred.SequenceFileInputFormat", + "org.apache.hadoop.io.IntWritable", + "org.apache.hadoop.io.Text", + conf=oldconf, + batchSize=1).collect()) + self.assertEqual(unbatched_hadoopRDD, ei) + + newconf = {"mapred.input.dir": basepath + "/unbatched/"} + unbatched_newAPIHadoopRDD = sorted(self.sc.newAPIHadoopRDD( + "org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat", + "org.apache.hadoop.io.IntWritable", + "org.apache.hadoop.io.Text", + conf=newconf, + batchSize=1).collect()) + self.assertEqual(unbatched_newAPIHadoopRDD, ei) + + def test_malformed_RDD(self): + basepath = self.tempdir.name + # non-batch-serialized RDD[[(K, V)]] should be rejected + data = [[(1, "a")], [(2, "aa")], [(3, "aaa")]] + rdd = self.sc.parallelize(data, numSlices=len(data)) + self.assertRaises(Exception, lambda: rdd.saveAsSequenceFile( + basepath + "/malformed/sequence")) + + class TestDaemon(unittest.TestCase): + def connect(self, port): from socket import socket, AF_INET, SOCK_STREAM sock = socket(AF_INET, SOCK_STREAM) @@ -404,7 +835,62 @@ def test_termination_sigterm(self): self.do_termination_test(lambda daemon: os.kill(daemon.pid, SIGTERM)) +class TestWorker(PySparkTestCase): + + def test_cancel_task(self): + temp = tempfile.NamedTemporaryFile(delete=True) + temp.close() + path = temp.name + + def sleep(x): + import os + import time + with open(path, 'w') as f: + f.write("%d %d" % (os.getppid(), os.getpid())) + time.sleep(100) + + # start job in background thread + def run(): + self.sc.parallelize(range(1)).foreach(sleep) + import threading + t = threading.Thread(target=run) + t.daemon = True + t.start() + + daemon_pid, worker_pid = 0, 0 + while True: + if os.path.exists(path): + data = open(path).read().split(' ') + daemon_pid, worker_pid = map(int, data) + break + time.sleep(0.1) + + # cancel jobs + self.sc.cancelAllJobs() + t.join() + + for i in range(50): + try: + os.kill(worker_pid, 0) + time.sleep(0.1) + except OSError: + break # worker was killed + else: + self.fail("worker has not been killed after 5 seconds") + + try: + os.kill(daemon_pid, 0) + except OSError: + self.fail("daemon had been killed") + + def test_fd_leak(self): + N = 1100 # fd limit is 1024 by default + rdd = self.sc.parallelize(range(N), N) + self.assertEquals(N, rdd.count()) + + class TestSparkSubmit(unittest.TestCase): + def setUp(self): self.programDir = tempfile.mkdtemp() self.sparkSubmit = os.path.join(os.environ.get("SPARK_HOME"), "bin", "spark-submit") @@ -432,8 +918,9 @@ def createFileInZip(self, name, content): pattern = re.compile(r'^ *\|', re.MULTILINE) content = re.sub(pattern, '', content.strip()) path = os.path.join(self.programDir, name + ".zip") - with zipfile.ZipFile(path, 'w') as zip: - zip.writestr(name, content) + zip = zipfile.ZipFile(path, 'w') + zip.writestr(name, content) + zip.close() return path def test_single_script(self): @@ -479,7 +966,7 @@ def test_module_dependency(self): | return x + 1 """) proc = subprocess.Popen([self.sparkSubmit, "--py-files", zip, script], - stdout=subprocess.PIPE) + stdout=subprocess.PIPE) out, err = proc.communicate() self.assertEqual(0, proc.returncode) self.assertIn("[2, 3, 4]", out) @@ -497,9 +984,9 @@ def test_module_dependency_on_cluster(self): |def myfunc(x): | return x + 1 """) - proc = subprocess.Popen( - [self.sparkSubmit, "--py-files", zip, "--master", "local-cluster[1,1,512]", script], - stdout=subprocess.PIPE) + proc = subprocess.Popen([self.sparkSubmit, "--py-files", zip, "--master", + "local-cluster[1,1,512]", script], + stdout=subprocess.PIPE) out, err = proc.communicate() self.assertEqual(0, proc.returncode) self.assertIn("[2, 3, 4]", out) @@ -525,6 +1012,7 @@ def test_single_script_on_cluster(self): @unittest.skipIf(not _have_scipy, "SciPy not installed") class SciPyTests(PySparkTestCase): + """General PySpark tests that depend on scipy """ def test_serialize(self): @@ -535,9 +1023,27 @@ def test_serialize(self): self.assertEqual(expected, observed) +@unittest.skipIf(not _have_numpy, "NumPy not installed") +class NumPyTests(PySparkTestCase): + + """General PySpark tests that depend on numpy """ + + def test_statcounter_array(self): + x = self.sc.parallelize([np.array([1.0, 1.0]), np.array([2.0, 2.0]), np.array([3.0, 3.0])]) + s = x.stats() + self.assertSequenceEqual([2.0, 2.0], s.mean().tolist()) + self.assertSequenceEqual([1.0, 1.0], s.min().tolist()) + self.assertSequenceEqual([3.0, 3.0], s.max().tolist()) + self.assertSequenceEqual([1.0, 1.0], s.sampleStdev().tolist()) + + if __name__ == "__main__": if not _have_scipy: print "NOTE: Skipping SciPy tests as it does not seem to be installed" + if not _have_numpy: + print "NOTE: Skipping NumPy tests as it does not seem to be installed" unittest.main() if not _have_scipy: print "NOTE: SciPy tests were skipped as it does not seem to be installed" + if not _have_numpy: + print "NOTE: NumPy tests were skipped as it does not seem to be installed" diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index f43210c6c0301..77a9c4a0e0677 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -30,7 +30,8 @@ from pyspark.cloudpickle import CloudPickler from pyspark.files import SparkFiles from pyspark.serializers import write_with_length, write_int, read_long, \ - write_long, read_int, SpecialLengths, UTF8Deserializer, PickleSerializer + write_long, read_int, SpecialLengths, UTF8Deserializer, PickleSerializer, \ + CompressedSerializer pickleSer = PickleSerializer() @@ -57,32 +58,38 @@ def main(infile, outfile): SparkFiles._is_running_on_worker = True # fetch names of includes (*.zip and *.egg files) and construct PYTHONPATH - sys.path.append(spark_files_dir) # *.py files that were added will be copied here - num_python_includes = read_int(infile) + sys.path.append(spark_files_dir) # *.py files that were added will be copied here + num_python_includes = read_int(infile) for _ in range(num_python_includes): filename = utf8_deserializer.loads(infile) sys.path.append(os.path.join(spark_files_dir, filename)) # fetch names and values of broadcast variables num_broadcast_variables = read_int(infile) + ser = CompressedSerializer(pickleSer) for _ in range(num_broadcast_variables): bid = read_long(infile) - value = pickleSer._read_with_length(infile) + value = ser._read_with_length(infile) _broadcastRegistry[bid] = Broadcast(bid, value) - command = pickleSer._read_with_length(infile) + command = ser._read_with_length(infile) (func, deserializer, serializer) = command init_time = time.time() iterator = deserializer.load_stream(infile) serializer.dump_stream(func(split_index, iterator), outfile) - except Exception as e: - # Write the error to stderr in addition to trying to pass it back to - # Java, in case it happened while serializing a record - print >> sys.stderr, "PySpark worker failed with exception:" - print >> sys.stderr, traceback.format_exc() - write_int(SpecialLengths.PYTHON_EXCEPTION_THROWN, outfile) - write_with_length(traceback.format_exc(), outfile) - sys.exit(-1) + except Exception: + try: + write_int(SpecialLengths.PYTHON_EXCEPTION_THROWN, outfile) + write_with_length(traceback.format_exc(), outfile) + outfile.flush() + except IOError: + # JVM close the socket + pass + except Exception: + # Write the error to stderr if it happened while serializing + print >> sys.stderr, "PySpark worker failed with exception:" + print >> sys.stderr, traceback.format_exc() + exit(-1) finish_time = time.time() report_times(outfile, boot_time, init_time, finish_time) # Mark the beginning of the accumulators section of the output diff --git a/python/run-tests b/python/run-tests index 9282aa47e8375..d5560dad69dc4 100755 --- a/python/run-tests +++ b/python/run-tests @@ -48,6 +48,14 @@ function run_test() { echo "Running PySpark tests. Output is in python/unit-tests.log." +# Try to test with Python 2.6, since that's the minimum version that we support: +if [ $(which python2.6) ]; then + export PYSPARK_PYTHON="python2.6" +fi + +echo "Testing with Python version:" +$PYSPARK_PYTHON --version + run_test "pyspark/rdd.py" run_test "pyspark/context.py" run_test "pyspark/conf.py" @@ -60,15 +68,23 @@ export PYSPARK_DOC_TEST=1 run_test "pyspark/broadcast.py" run_test "pyspark/accumulators.py" run_test "pyspark/serializers.py" +run_test "pyspark/streaming/duration.py" +run_test "pyspark/streaming/util.py" unset PYSPARK_DOC_TEST +run_test "pyspark/shuffle.py" run_test "pyspark/tests.py" run_test "pyspark/mllib/_common.py" run_test "pyspark/mllib/classification.py" run_test "pyspark/mllib/clustering.py" run_test "pyspark/mllib/linalg.py" +run_test "pyspark/mllib/random.py" run_test "pyspark/mllib/recommendation.py" run_test "pyspark/mllib/regression.py" run_test "pyspark/mllib/tests.py" +run_test "pyspark/mllib/util.py" +if [ -n "$_RUN_STREAMING_TESTS" ]; then + run_test "pyspark/streaming/tests.py" +fi if [[ $FAILED == 0 ]]; then echo -en "\033[32m" # Green diff --git a/python/test_support/userlibrary.py b/python/test_support/userlibrary.py index 8e4a6292bc17c..73fd26e71f10d 100755 --- a/python/test_support/userlibrary.py +++ b/python/test_support/userlibrary.py @@ -19,6 +19,8 @@ Used to test shipping of code depenencies with SparkContext.addPyFile(). """ + class UserClass(object): + def hello(self): return "Hello World!" diff --git a/repl/pom.xml b/repl/pom.xml index 4a66408ef3d2d..68f4504450778 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -32,6 +32,7 @@ http://spark.apache.org/ + repl /usr/share/spark root @@ -54,6 +55,12 @@ ${project.version} runtime + + org.apache.spark + spark-sql_${scala.binary.version} + ${project.version} + test + org.eclipse.jetty jetty-server diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala index e1db4d5395ab9..65788f4646d91 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -230,6 +230,20 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, case xs => xs find (_.name == cmd) } } + private var fallbackMode = false + + private def toggleFallbackMode() { + val old = fallbackMode + fallbackMode = !old + System.setProperty("spark.repl.fallback", fallbackMode.toString) + echo(s""" + |Switched ${if (old) "off" else "on"} fallback mode without restarting. + | If you have defined classes in the repl, it would + |be good to redefine them incase you plan to use them. If you still run + |into issues it would be good to restart the repl and turn on `:fallback` + |mode as first command. + """.stripMargin) + } /** Show the history */ lazy val historyCommand = new LoopCommand("history", "show the history (optional num is commands to show)") { @@ -299,6 +313,9 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, nullary("reset", "reset the repl to its initial state, forgetting all session entries", resetCommand), shCommand, nullary("silent", "disable/enable automatic printing of results", verbosity), + nullary("fallback", """ + |disable/enable advanced repl changes, these fix some issues but may introduce others. + |This mode will be removed once these fixes stablize""".stripMargin, toggleFallbackMode), cmd("type", "[-v] ", "display the type of an expression without evaluating it", typeCommand), nullary("warnings", "show the suppressed warnings from the most recent line which had any", warningsCommand) ) @@ -557,29 +574,27 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, if (isReplPower) powerCommands else Nil )*/ - val replayQuestionMessage = + private val replayQuestionMessage = """|That entry seems to have slain the compiler. Shall I replay |your session? I can re-run each line except the last one. |[y/n] """.trim.stripMargin - private val crashRecovery: PartialFunction[Throwable, Boolean] = { - case ex: Throwable => - echo(intp.global.throwableAsString(ex)) - - ex match { - case _: NoSuchMethodError | _: NoClassDefFoundError => - echo("\nUnrecoverable error.") - throw ex - case _ => - def fn(): Boolean = - try in.readYesOrNo(replayQuestionMessage, { echo("\nYou must enter y or n.") ; fn() }) - catch { case _: RuntimeException => false } - - if (fn()) replay() - else echo("\nAbandoning crashed session.") - } - true + private def crashRecovery(ex: Throwable): Boolean = { + echo(ex.toString) + ex match { + case _: NoSuchMethodError | _: NoClassDefFoundError => + echo("\nUnrecoverable error.") + throw ex + case _ => + def fn(): Boolean = + try in.readYesOrNo(replayQuestionMessage, { echo("\nYou must enter y or n.") ; fn() }) + catch { case _: RuntimeException => false } + + if (fn()) replay() + else echo("\nAbandoning crashed session.") + } + true } /** The main read-eval-print loop for the repl. It calls @@ -605,7 +620,10 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, } } def innerLoop() { - if ( try processLine(readOneLine()) catch crashRecovery ) + val shouldContinue = try { + processLine(readOneLine()) + } catch {case t: Throwable => crashRecovery(t)} + if (shouldContinue) innerLoop() } innerLoop() @@ -951,9 +969,6 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, if (execUri != null) { conf.set("spark.executor.uri", execUri) } - if (System.getenv("SPARK_HOME") != null) { - conf.setSparkHome(System.getenv("SPARK_HOME")) - } sparkContext = new SparkContext(conf) logInfo("Created spark context..") sparkContext diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala b/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala index 3842c291d0b7b..84b57cd2dc1af 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala @@ -102,7 +102,8 @@ import org.apache.spark.util.Utils val virtualDirectory = new PlainFile(outputDir) // "directory" for classfiles /** Jetty server that will serve our classes to worker nodes */ - val classServer = new HttpServer(outputDir, new SecurityManager(conf)) + val classServerPort = conf.getInt("spark.replClassServer.port", 0) + val classServer = new HttpServer(outputDir, new SecurityManager(conf), classServerPort, "HTTP class server") private var currentSettings: Settings = initialSettings var printResults = true // whether to print result lines var totalSilence = false // whether to print anything @@ -892,11 +893,16 @@ import org.apache.spark.util.Utils def definedTypeSymbol(name: String) = definedSymbols(newTypeName(name)) def definedTermSymbol(name: String) = definedSymbols(newTermName(name)) + val definedClasses = handlers.exists { + case _: ClassHandler => true + case _ => false + } + /** Code to import bound names from previous lines - accessPath is code to * append to objectName to access anything bound by request. */ val SparkComputedImports(importsPreamble, importsTrailer, accessPath) = - importsCode(referencedNames.toSet) + importsCode(referencedNames.toSet, definedClasses) /** Code to access a variable with the specified name */ def fullPath(vname: String) = { diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkImports.scala b/repl/src/main/scala/org/apache/spark/repl/SparkImports.scala index bce5c74b9d0da..193a42dcded12 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkImports.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkImports.scala @@ -108,8 +108,9 @@ trait SparkImports { * last one imported is actually usable. */ case class SparkComputedImports(prepend: String, append: String, access: String) + def fallback = System.getProperty("spark.repl.fallback", "false").toBoolean - protected def importsCode(wanted: Set[Name]): SparkComputedImports = { + protected def importsCode(wanted: Set[Name], definedClass: Boolean): SparkComputedImports = { /** Narrow down the list of requests from which imports * should be taken. Removes requests which cannot contribute * useful imports for the specified set of wanted names. @@ -124,8 +125,14 @@ trait SparkImports { // Single symbol imports might be implicits! See bug #1752. Rather than // try to finesse this, we will mimic all imports for now. def keepHandler(handler: MemberHandler) = handler match { - case _: ImportHandler => true - case x => x.definesImplicit || (x.definedNames exists wanted) + /* This case clause tries to "precisely" import only what is required. And in this + * it may miss out on some implicits, because implicits are not known in `wanted`. Thus + * it is suitable for defining classes. AFAIK while defining classes implicits are not + * needed.*/ + case h: ImportHandler if definedClass && !fallback => + h.importedNames.exists(x => wanted.contains(x)) + case _: ImportHandler => true + case x => x.definesImplicit || (x.definedNames exists wanted) } reqs match { @@ -182,7 +189,7 @@ trait SparkImports { // ambiguity errors will not be generated. Also, quote // the name of the variable, so that we don't need to // handle quoting keywords separately. - case x: ClassHandler => + case x: ClassHandler if !fallback => // I am trying to guess if the import is a defined class // This is an ugly hack, I am not 100% sure of the consequences. // Here we, let everything but "defined classes" use the import with val. @@ -197,7 +204,7 @@ trait SparkImports { for (imv <- x.definedNames) { if (currentImps contains imv) addWrapper() val objName = req.lineRep.readPath - val valName = "$VAL" + req.lineRep.lineId + val valName = "$VAL" + newValId() if(!code.toString.endsWith(".`" + imv + "`;\n")) { // Which means already imported code.append("val " + valName + " = " + objName + ".INSTANCE;\n") @@ -222,4 +229,10 @@ trait SparkImports { private def membersAtPickler(sym: Symbol): List[Symbol] = beforePickler(sym.info.nonPrivateMembers.toList) + private var curValId = 0 + + private def newValId(): Int = { + curValId += 1 + curValId + } } diff --git a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala index f2aa42dbcb4fc..c8763eb277052 100644 --- a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala +++ b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala @@ -235,7 +235,7 @@ class ReplSuite extends FunSuite { assertContains("res4: Array[Int] = Array(0, 0, 0, 0, 0)", output) } - test("SPARK-1199-simple-reproduce") { + test("SPARK-1199 two instances of same class don't type check.") { val output = runInterpreter("local-cluster[1,1,512]", """ |case class Sum(exp: String, exp2: String) @@ -247,6 +247,42 @@ class ReplSuite extends FunSuite { assertDoesNotContain("Exception", output) } + test("SPARK-2452 compound statements.") { + val output = runInterpreter("local", + """ + |val x = 4 ; def f() = x + |f() + """.stripMargin) + assertDoesNotContain("error:", output) + assertDoesNotContain("Exception", output) + } + + test("SPARK-2576 importing SQLContext.createSchemaRDD.") { + // We need to use local-cluster to test this case. + val output = runInterpreter("local-cluster[1,1,512]", + """ + |val sqlContext = new org.apache.spark.sql.SQLContext(sc) + |import sqlContext.createSchemaRDD + |case class TestCaseClass(value: Int) + |sc.parallelize(1 to 10).map(x => TestCaseClass(x)).toSchemaRDD.collect + """.stripMargin) + assertDoesNotContain("error:", output) + assertDoesNotContain("Exception", output) + } + + test("SPARK-2632 importing a method from non serializable class and not using it.") { + val output = runInterpreter("local", + """ + |class TestClass() { def testMethod = 3 } + |val t = new TestClass + |import t.testMethod + |case class TestCaseClass(value: Int) + |sc.parallelize(1 to 10).map(x => TestCaseClass(x)).collect + """.stripMargin) + assertDoesNotContain("error:", output) + assertDoesNotContain("Exception", output) + } + if (System.getenv("MESOS_NATIVE_LIBRARY") != null) { test("running on Mesos") { val output = runInterpreter("localquiet", diff --git a/sbin/spark-config.sh b/sbin/spark-config.sh index 147b506dd5ca3..5c87da5815b64 100755 --- a/sbin/spark-config.sh +++ b/sbin/spark-config.sh @@ -36,4 +36,4 @@ export SPARK_HOME=${SPARK_PREFIX} export SPARK_CONF_DIR="$SPARK_HOME/conf" # Add the PySpark classes to the PYTHONPATH: export PYTHONPATH=$SPARK_HOME/python:$PYTHONPATH -export PYTHONPATH=$SPARK_HOME/python/lib/py4j-0.8.1-src.zip:$PYTHONPATH +export PYTHONPATH=$SPARK_HOME/python/lib/py4j-0.8.2.1-src.zip:$PYTHONPATH diff --git a/sbin/spark-executor b/sbin/spark-executor index 336549f29c9ce..3621321a9bc8d 100755 --- a/sbin/spark-executor +++ b/sbin/spark-executor @@ -20,7 +20,7 @@ FWDIR="$(cd `dirname $0`/..; pwd)" export PYTHONPATH=$FWDIR/python:$PYTHONPATH -export PYTHONPATH=$FWDIR/python/lib/py4j-0.8.1-src.zip:$PYTHONPATH +export PYTHONPATH=$FWDIR/python/lib/py4j-0.8.2.1-src.zip:$PYTHONPATH echo "Running spark-executor with framework dir = $FWDIR" exec $FWDIR/bin/spark-class org.apache.spark.executor.MesosExecutorBackend diff --git a/sbin/start-thriftserver.sh b/sbin/start-thriftserver.sh new file mode 100755 index 0000000000000..2c4452473ccbc --- /dev/null +++ b/sbin/start-thriftserver.sh @@ -0,0 +1,78 @@ +#!/usr/bin/env bash + +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# +# Shell script for starting the Spark SQL Thrift server + +# Enter posix mode for bash +set -o posix + +# Figure out where Spark is installed +FWDIR="$(cd `dirname $0`/..; pwd)" + +CLASS="org.apache.spark.sql.hive.thriftserver.HiveThriftServer2" + +function usage { + echo "Usage: ./sbin/start-thriftserver [options] [thrift server options]" + pattern="usage" + pattern+="\|Spark assembly has been built with Hive" + pattern+="\|NOTE: SPARK_PREPEND_CLASSES is set" + pattern+="\|Spark Command: " + pattern+="\|=======" + pattern+="\|--help" + + $FWDIR/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 + echo + echo "Thrift server options:" + $FWDIR/bin/spark-class $CLASS --help 2>&1 | grep -v "$pattern" 1>&2 +} + +function ensure_arg_number { + arg_number=$1 + at_least=$2 + + if [[ $arg_number -lt $at_least ]]; then + usage + exit 1 + fi +} + +if [[ "$@" = --help ]] || [[ "$@" = -h ]]; then + usage + exit 0 +fi + +THRIFT_SERVER_ARGS=() +SUBMISSION_ARGS=() + +while (($#)); do + case $1 in + --hiveconf) + ensure_arg_number $# 2 + THRIFT_SERVER_ARGS+=("$1"); shift + THRIFT_SERVER_ARGS+=("$1"); shift + ;; + + *) + SUBMISSION_ARGS+=("$1"); shift + ;; + esac +done + +exec "$FWDIR"/bin/spark-submit --class $CLASS "${SUBMISSION_ARGS[@]}" spark-internal "${THRIFT_SERVER_ARGS[@]}" diff --git a/sbt/sbt b/sbt/sbt index 9de265bd07dcb..1b1aa1483a829 100755 --- a/sbt/sbt +++ b/sbt/sbt @@ -72,6 +72,7 @@ Usage: $script_name [options] -J-X pass option -X directly to the java runtime (-J is stripped) -S-X add -X to sbt's scalacOptions (-J is stripped) + -PmavenProfiles Enable a maven profile for the build. In the case of duplicated or conflicting options, the order above shows precedence: JAVA_OPTS lowest, command line options highest. diff --git a/sbt/sbt-launch-lib.bash b/sbt/sbt-launch-lib.bash index 64e40a88206be..c91fecf024ad4 100755 --- a/sbt/sbt-launch-lib.bash +++ b/sbt/sbt-launch-lib.bash @@ -16,6 +16,7 @@ declare -a residual_args declare -a java_args declare -a scalac_args declare -a sbt_commands +declare -a maven_profiles if test -x "$JAVA_HOME/bin/java"; then echo -e "Using $JAVA_HOME as default JAVA_HOME." @@ -87,6 +88,13 @@ addJava () { dlog "[addJava] arg = '$1'" java_args=( "${java_args[@]}" "$1" ) } + +enableProfile () { + dlog "[enableProfile] arg = '$1'" + maven_profiles=( "${maven_profiles[@]}" "$1" ) + export SBT_MAVEN_PROFILES="${maven_profiles[@]}" +} + addSbt () { dlog "[addSbt] arg = '$1'" sbt_commands=( "${sbt_commands[@]}" "$1" ) @@ -141,7 +149,8 @@ process_args () { -java-home) require_arg path "$1" "$2" && java_cmd="$2/bin/java" && export JAVA_HOME=$2 && shift 2 ;; -D*) addJava "$1" && shift ;; - -J*) addJava "${1:2}" && shift ;; + -J*) addJava "${1:2}" && shift ;; + -P*) enableProfile "$1" && shift ;; *) addResidual "$1" && shift ;; esac done diff --git a/sql/README.md b/sql/README.md index 14d5555f0c713..31f9152344086 100644 --- a/sql/README.md +++ b/sql/README.md @@ -3,10 +3,11 @@ Spark SQL This module provides support for executing relational queries expressed in either SQL or a LINQ-like Scala DSL. -Spark SQL is broken up into three subprojects: +Spark SQL is broken up into four subprojects: - Catalyst (sql/catalyst) - An implementation-agnostic framework for manipulating trees of relational operators and expressions. - Execution (sql/core) - A query planner / execution engine for translating Catalyst’s logical query plans into Spark RDDs. This component also includes a new public interface, SQLContext, that allows users to execute SQL or LINQ statements against existing RDDs and Parquet files. - Hive Support (sql/hive) - Includes an extension of SQLContext called HiveContext that allows users to write queries using a subset of HiveQL and access data from a Hive Metastore using Hive SerDes. There are also wrappers that allows users to run queries that include Hive UDFs, UDAFs, and UDTFs. + - HiveServer and CLI support (sql/hive-thriftserver) - Includes support for the SQL CLI (bin/spark-sql) and a HiveServer2 (for JDBC/ODBC) compatible server. Other dependencies for developers diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 01d7b569080ea..830711a46a35b 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -31,22 +31,29 @@ jar Spark Project Catalyst http://spark.apache.org/ + + catalyst + + + org.scala-lang + scala-compiler + org.scala-lang scala-reflect + + org.scalamacros + quasiquotes_${scala.binary.version} + ${scala.macros.version} + org.apache.spark spark-core_${scala.binary.version} ${project.version} - - com.typesafe - scalalogging-slf4j_${scala.binary.version} - 1.0.1 - org.scalatest scalatest_${scala.binary.version} @@ -70,28 +77,28 @@ org.apache.maven.plugins maven-jar-plugin - - - test-jar - - - - test-jar-on-compile - compile - - test-jar - - + + + test-jar + + + + test-jar-on-test-compile + test-compile + + test-jar + + diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index 5a55be1e51558..0d26b52a84695 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala @@ -85,6 +85,26 @@ object ScalaReflection { case t if t <:< definitions.BooleanTpe => Schema(BooleanType, nullable = false) } + def typeOfObject: PartialFunction[Any, DataType] = { + // The data type can be determined without ambiguity. + case obj: BooleanType.JvmType => BooleanType + case obj: BinaryType.JvmType => BinaryType + case obj: StringType.JvmType => StringType + case obj: ByteType.JvmType => ByteType + case obj: ShortType.JvmType => ShortType + case obj: IntegerType.JvmType => IntegerType + case obj: LongType.JvmType => LongType + case obj: FloatType.JvmType => FloatType + case obj: DoubleType.JvmType => DoubleType + case obj: DecimalType.JvmType => DecimalType + case obj: TimestampType.JvmType => TimestampType + case null => NullType + // For other cases, there is no obvious mapping from the type of the given object to a + // Catalyst data type. A user should provide his/her specific rules + // (in a user-defined PartialFunction) to infer the Catalyst data type for other types of + // objects and then compose the user-defined PartialFunction with this one. + } + implicit class CaseClassRelation[A <: Product : TypeTag](data: Seq[A]) { /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index e5653c5b14ac1..2c73a80f64ebf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -120,7 +120,8 @@ class SqlParser extends StandardTokenParsers with PackratParsers { protected val WHERE = Keyword("WHERE") protected val INTERSECT = Keyword("INTERSECT") protected val EXCEPT = Keyword("EXCEPT") - + protected val SUBSTR = Keyword("SUBSTR") + protected val SUBSTRING = Keyword("SUBSTRING") // Use reflection to find the reserved words defined in this class. protected val reservedWords = @@ -209,21 +210,21 @@ class SqlParser extends StandardTokenParsers with PackratParsers { } | "(" ~> query ~ ")" ~ opt(AS) ~ ident ^^ { case s ~ _ ~ _ ~ a => Subquery(a, s) } - protected lazy val joinedRelation: Parser[LogicalPlan] = - relationFactor ~ opt(joinType) ~ JOIN ~ relationFactor ~ opt(joinConditions) ^^ { + protected lazy val joinedRelation: Parser[LogicalPlan] = + relationFactor ~ opt(joinType) ~ JOIN ~ relationFactor ~ opt(joinConditions) ^^ { case r1 ~ jt ~ _ ~ r2 ~ cond => Join(r1, r2, joinType = jt.getOrElse(Inner), cond) - } + } - protected lazy val joinConditions: Parser[Expression] = - ON ~> expression + protected lazy val joinConditions: Parser[Expression] = + ON ~> expression - protected lazy val joinType: Parser[JoinType] = - INNER ^^^ Inner | - LEFT ~ SEMI ^^^ LeftSemi | - LEFT ~ opt(OUTER) ^^^ LeftOuter | - RIGHT ~ opt(OUTER) ^^^ RightOuter | - FULL ~ opt(OUTER) ^^^ FullOuter + protected lazy val joinType: Parser[JoinType] = + INNER ^^^ Inner | + LEFT ~ SEMI ^^^ LeftSemi | + LEFT ~ opt(OUTER) ^^^ LeftOuter | + RIGHT ~ opt(OUTER) ^^^ RightOuter | + FULL ~ opt(OUTER) ^^^ FullOuter protected lazy val filter: Parser[Expression] = WHERE ~ expression ^^ { case _ ~ e => e } @@ -316,6 +317,12 @@ class SqlParser extends StandardTokenParsers with PackratParsers { IF ~> "(" ~> expression ~ "," ~ expression ~ "," ~ expression <~ ")" ^^ { case c ~ "," ~ t ~ "," ~ f => If(c,t,f) } | + (SUBSTR | SUBSTRING) ~> "(" ~> expression ~ "," ~ expression <~ ")" ^^ { + case s ~ "," ~ p => Substring(s,p,Literal(Integer.MAX_VALUE)) + } | + (SUBSTR | SUBSTRING) ~> "(" ~> expression ~ "," ~ expression ~ "," ~ expression <~ ")" ^^ { + case s ~ "," ~ p ~ "," ~ l => Substring(s,p,l) + } | ident ~ "(" ~ repsep(expression, ",") <~ ")" ^^ { case udfName ~ _ ~ exprs => UnresolvedFunction(udfName, exprs) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index c7188469bfb86..c18d7858f0a43 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -22,7 +22,6 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ - /** * A trivial [[Analyzer]] with an [[EmptyCatalog]] and [[EmptyFunctionRegistry]]. Used for testing * when all relations are already filled in and the analyser needs only to resolve attribute @@ -49,11 +48,13 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool Batch("Resolution", fixedPoint, ResolveReferences :: ResolveRelations :: + ResolveSortReferences :: NewRelationInstances :: ImplicitGenerate :: StarExpansion :: ResolveFunctions :: GlobalAggregates :: + UnresolvedHavingClauseAttributes :: typeCoercionRules :_*), Batch("Check Analysis", Once, CheckResolution), @@ -109,17 +110,62 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool object ResolveReferences extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { case q: LogicalPlan if q.childrenResolved => - logger.trace(s"Attempting to resolve ${q.simpleString}") + logTrace(s"Attempting to resolve ${q.simpleString}") q transformExpressions { case u @ UnresolvedAttribute(name) => // Leave unchanged if resolution fails. Hopefully will be resolved next round. - val result = q.resolve(name).getOrElse(u) - logger.debug(s"Resolving $u to $result") + val result = q.resolveChildren(name).getOrElse(u) + logDebug(s"Resolving $u to $result") result } } } + /** + * In many dialects of SQL is it valid to sort by attributes that are not present in the SELECT + * clause. This rule detects such queries and adds the required attributes to the original + * projection, so that they will be available during sorting. Another projection is added to + * remove these attributes after sorting. + */ + object ResolveSortReferences extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { + case s @ Sort(ordering, p @ Project(projectList, child)) if !s.resolved && p.resolved => + val unresolved = ordering.flatMap(_.collect { case UnresolvedAttribute(name) => name }) + val resolved = unresolved.flatMap(child.resolveChildren) + val requiredAttributes = resolved.collect { case a: Attribute => a }.toSet + + val missingInProject = requiredAttributes -- p.output + if (missingInProject.nonEmpty) { + // Add missing attributes and then project them away after the sort. + Project(projectList, + Sort(ordering, + Project(projectList ++ missingInProject, child))) + } else { + s // Nothing we can do here. Return original plan. + } + case s @ Sort(ordering, a @ Aggregate(grouping, aggs, child)) if !s.resolved && a.resolved => + val unresolved = ordering.flatMap(_.collect { case UnresolvedAttribute(name) => name }) + // A small hack to create an object that will allow us to resolve any references that + // refer to named expressions that are present in the grouping expressions. + val groupingRelation = LocalRelation( + grouping.collect { case ne: NamedExpression => ne.toAttribute } + ) + + logDebug(s"Grouping expressions: $groupingRelation") + val resolved = unresolved.flatMap(groupingRelation.resolve).toSet + val missingInAggs = resolved -- a.outputSet + logDebug(s"Resolved: $resolved Missing in aggs: $missingInAggs") + if (missingInAggs.nonEmpty) { + // Add missing grouping exprs and then project them away after the sort. + Project(a.output, + Sort(ordering, + Aggregate(grouping, aggs ++ missingInAggs, child))) + } else { + s // Nothing we can do here. Return original plan. + } + } + } + /** * Replaces [[UnresolvedFunction]]s with concrete [[catalyst.expressions.Expression Expressions]]. */ @@ -151,6 +197,31 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool } } + /** + * This rule finds expressions in HAVING clause filters that depend on + * unresolved attributes. It pushes these expressions down to the underlying + * aggregates and then projects them away above the filter. + */ + object UnresolvedHavingClauseAttributes extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { + case filter @ Filter(havingCondition, aggregate @ Aggregate(_, originalAggExprs, _)) + if aggregate.resolved && containsAggregate(havingCondition) => { + val evaluatedCondition = Alias(havingCondition, "havingCondition")() + val aggExprsWithHaving = evaluatedCondition +: originalAggExprs + + Project(aggregate.output, + Filter(evaluatedCondition.toAttribute, + aggregate.copy(aggregateExpressions = aggExprsWithHaving))) + } + + } + + protected def containsAggregate(condition: Expression): Boolean = + condition + .collect { case ae: AggregateExpression => ae } + .nonEmpty + } + /** * When a SELECT clause has only a single expression and that expression is a * [[catalyst.expressions.Generator Generator]] we convert the diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala index 0d05d9808b407..616f1e2ecb60f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala @@ -113,11 +113,12 @@ trait OverrideCatalog extends Catalog { alias: Option[String] = None): LogicalPlan = { val (dbName, tblName) = processDatabaseAndTableName(databaseName, tableName) val overriddenTable = overrides.get((dbName, tblName)) + val tableWithQualifers = overriddenTable.map(r => Subquery(tblName, r)) // If an alias was specified by the lookup, wrap the plan in a subquery so that attributes are // properly qualified with this alias. val withAlias = - overriddenTable.map(r => alias.map(a => Subquery(a, r)).getOrElse(r)) + tableWithQualifers.map(r => alias.map(a => Subquery(a, r)).getOrElse(r)) withAlias.getOrElse(super.lookupRelation(dbName, tblName, alias)) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index c0255701b7ba5..760c49fbca4a5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -18,17 +18,49 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.catalyst.expressions.Expression +import scala.collection.mutable /** A catalog for looking up user defined functions, used by an [[Analyzer]]. */ trait FunctionRegistry { + type FunctionBuilder = Seq[Expression] => Expression + + def registerFunction(name: String, builder: FunctionBuilder): Unit + def lookupFunction(name: String, children: Seq[Expression]): Expression } +trait OverrideFunctionRegistry extends FunctionRegistry { + + val functionBuilders = new mutable.HashMap[String, FunctionBuilder]() + + def registerFunction(name: String, builder: FunctionBuilder) = { + functionBuilders.put(name, builder) + } + + abstract override def lookupFunction(name: String, children: Seq[Expression]): Expression = { + functionBuilders.get(name).map(_(children)).getOrElse(super.lookupFunction(name,children)) + } +} + +class SimpleFunctionRegistry extends FunctionRegistry { + val functionBuilders = new mutable.HashMap[String, FunctionBuilder]() + + def registerFunction(name: String, builder: FunctionBuilder) = { + functionBuilders.put(name, builder) + } + + override def lookupFunction(name: String, children: Seq[Expression]): Expression = { + functionBuilders(name)(children) + } +} + /** * A trivial catalog that returns an error when a function is requested. Used for testing when all * functions are already filled in and the analyser needs only to resolve attribute references. */ object EmptyFunctionRegistry extends FunctionRegistry { + def registerFunction(name: String, builder: FunctionBuilder) = ??? + def lookupFunction(name: String, children: Seq[Expression]): Expression = { throw new UnsupportedOperationException } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala index 76ddeba9cb312..15eb5982a4a91 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala @@ -49,9 +49,21 @@ trait HiveTypeCoercion { BooleanCasts :: StringToIntegralCasts :: FunctionArgumentConversion :: - CastNulls :: + CaseWhenCoercion :: + Division :: Nil + trait TypeWidening { + def findTightestCommonType(t1: DataType, t2: DataType): Option[DataType] = { + // Try and find a promotion rule that contains both types in question. + val applicableConversion = + HiveTypeCoercion.allPromotions.find(p => p.contains(t1) && p.contains(t2)) + + // If found return the widest common type, otherwise None + applicableConversion.map(_.filter(t => t == t1 || t == t2).last) + } + } + /** * Applies any changes to [[AttributeReference]] data types that are made by other rules to * instances higher in the query tree. @@ -74,7 +86,7 @@ trait HiveTypeCoercion { // Leave the same if the dataTypes match. case Some(newType) if a.dataType == newType.dataType => a case Some(newType) => - logger.debug(s"Promoting $a to $newType in ${q.simpleString}}") + logDebug(s"Promoting $a to $newType in ${q.simpleString}}") newType } } @@ -132,16 +144,7 @@ trait HiveTypeCoercion { * - LongType to FloatType * - LongType to DoubleType */ - object WidenTypes extends Rule[LogicalPlan] { - - def findTightestCommonType(t1: DataType, t2: DataType): Option[DataType] = { - // Try and find a promotion rule that contains both types in question. - val applicableConversion = - HiveTypeCoercion.allPromotions.find(p => p.contains(t1) && p.contains(t2)) - - // If found return the widest common type, otherwise None - applicableConversion.map(_.filter(t => t == t1 || t == t2).last) - } + object WidenTypes extends Rule[LogicalPlan] with TypeWidening { def apply(plan: LogicalPlan): LogicalPlan = plan transform { case u @ Union(left, right) if u.childrenResolved && !u.resolved => @@ -153,7 +156,7 @@ trait HiveTypeCoercion { (Alias(Cast(l, StringType), l.name)(), r) case (l, r) if l.dataType != r.dataType => - logger.debug(s"Resolving mismatched union input ${l.dataType}, ${r.dataType}") + logDebug(s"Resolving mismatched union input ${l.dataType}, ${r.dataType}") findTightestCommonType(l.dataType, r.dataType).map { widestType => val newLeft = if (l.dataType == widestType) l else Alias(Cast(l, widestType), l.name)() @@ -169,7 +172,7 @@ trait HiveTypeCoercion { val newLeft = if (castedLeft.map(_.dataType) != left.output.map(_.dataType)) { - logger.debug(s"Widening numeric types in union $castedLeft ${left.output}") + logDebug(s"Widening numeric types in union $castedLeft ${left.output}") Project(castedLeft, left) } else { left @@ -177,7 +180,7 @@ trait HiveTypeCoercion { val newRight = if (castedRight.map(_.dataType) != right.output.map(_.dataType)) { - logger.debug(s"Widening numeric types in union $castedRight ${right.output}") + logDebug(s"Widening numeric types in union $castedRight ${right.output}") Project(castedRight, right) } else { right @@ -231,11 +234,23 @@ trait HiveTypeCoercion { * Changes Boolean values to Bytes so that expressions like true < false can be Evaluated. */ object BooleanComparisons extends Rule[LogicalPlan] { + val trueValues = Seq(1, 1L, 1.toByte, 1.toShort, BigDecimal(1)).map(Literal(_)) + val falseValues = Seq(0, 0L, 0.toByte, 0.toShort, BigDecimal(0)).map(Literal(_)) + def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { // Skip nodes who's children have not been resolved yet. case e if !e.childrenResolved => e - // No need to change EqualTo operators as that actually makes sense for boolean types. + + // Hive treats (true = 1) as true and (false = 0) as true. + case EqualTo(l @ BooleanType(), r) if trueValues.contains(r) => l + case EqualTo(l, r @ BooleanType()) if trueValues.contains(l) => r + case EqualTo(l @ BooleanType(), r) if falseValues.contains(r) => Not(l) + case EqualTo(l, r @ BooleanType()) if falseValues.contains(l) => Not(r) + + // No need to change other EqualTo operators as that actually makes sense for boolean types. case e: EqualTo => e + // No need to change the EqualNullSafe operators, too + case e: EqualNullSafe => e // Otherwise turn them to Byte types so that there exists and ordering. case p: BinaryComparison if p.left.dataType == BooleanType && p.right.dataType == BooleanType => @@ -306,28 +321,51 @@ trait HiveTypeCoercion { } /** - * Ensures that NullType gets casted to some other types under certain circumstances. + * Hive only performs integral division with the DIV operator. The arguments to / are always + * converted to fractional types. */ - object CastNulls extends Rule[LogicalPlan] { + object Division extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { - case cw @ CaseWhen(branches) => + // Skip nodes who's children have not been resolved yet. + case e if !e.childrenResolved => e + + // Decimal and Double remain the same + case d: Divide if d.dataType == DoubleType => d + case d: Divide if d.dataType == DecimalType => d + + case Divide(l, r) => Divide(Cast(l, DoubleType), Cast(r, DoubleType)) + } + } + + /** + * Coerces the type of different branches of a CASE WHEN statement to a common type. + */ + object CaseWhenCoercion extends Rule[LogicalPlan] with TypeWidening { + def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { + case cw @ CaseWhen(branches) if !cw.resolved && !branches.exists(!_.resolved) => val valueTypes = branches.sliding(2, 2).map { - case Seq(_, value) if value.resolved => Some(value.dataType) - case Seq(elseVal) if elseVal.resolved => Some(elseVal.dataType) - case _ => None + case Seq(_, value) => value.dataType + case Seq(elseVal) => elseVal.dataType }.toSeq - if (valueTypes.distinct.size == 2 && valueTypes.exists(_ == Some(NullType))) { - val otherType = valueTypes.filterNot(_ == Some(NullType))(0).get + + logDebug(s"Input values for null casting ${valueTypes.mkString(",")}") + + if (valueTypes.distinct.size > 1) { + val commonType = valueTypes.reduce { (v1, v2) => + findTightestCommonType(v1, v2) + .getOrElse(sys.error( + s"Types in CASE WHEN must be the same or coercible to a common type: $v1 != $v2")) + } val transformedBranches = branches.sliding(2, 2).map { - case Seq(cond, value) if value.resolved && value.dataType == NullType => - Seq(cond, Cast(value, otherType)) - case Seq(elseVal) if elseVal.resolved && elseVal.dataType == NullType => - Seq(Cast(elseVal, otherType)) + case Seq(cond, value) if value.dataType != commonType => + Seq(cond, Cast(value, commonType)) + case Seq(elseVal) if elseVal.dataType != commonType => + Seq(Cast(elseVal, commonType)) case s => s }.reduce(_ ++ _) CaseWhen(transformedBranches) } else { - // It is possible to have more types due to the possibility of short-circuiting. + // Types match up. Hopefully some other rule fixes whatever is wrong with resolution. cw } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/MultiInstanceRelation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/MultiInstanceRelation.scala index a6ce90854dcb4..22941edef2d46 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/MultiInstanceRelation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/MultiInstanceRelation.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan * of itself with globally unique expression ids. */ trait MultiInstanceRelation { - def newInstance: this.type + def newInstance(): this.type } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index 7abeb032964e1..a0e25775da6dd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.catalyst.{errors, trees} import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.logical.BaseRelation +import org.apache.spark.sql.catalyst.plans.logical.LeafNode import org.apache.spark.sql.catalyst.trees.TreeNode /** @@ -36,7 +36,7 @@ class UnresolvedException[TreeType <: TreeNode[_]](tree: TreeType, function: Str case class UnresolvedRelation( databaseName: Option[String], tableName: String, - alias: Option[String] = None) extends BaseRelation { + alias: Option[String] = None) extends LeafNode { override def output = Nil override lazy val resolved = false } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala index 1b503b957d146..f44521d6381c9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -77,10 +77,27 @@ package object dsl { def > (other: Expression) = GreaterThan(expr, other) def >= (other: Expression) = GreaterThanOrEqual(expr, other) def === (other: Expression) = EqualTo(expr, other) + def <=> (other: Expression) = EqualNullSafe(expr, other) def !== (other: Expression) = Not(EqualTo(expr, other)) + def in(list: Expression*) = In(expr, list) + def like(other: Expression) = Like(expr, other) def rlike(other: Expression) = RLike(expr, other) + def contains(other: Expression) = Contains(expr, other) + def startsWith(other: Expression) = StartsWith(expr, other) + def endsWith(other: Expression) = EndsWith(expr, other) + def substr(pos: Expression, len: Expression = Literal(Int.MaxValue)) = + Substring(expr, pos, len) + def substring(pos: Expression, len: Expression = Literal(Int.MaxValue)) = + Substring(expr, pos, len) + + def isNull = IsNull(expr) + def isNotNull = IsNotNull(expr) + + def getItem(ordinal: Expression) = GetItem(expr, ordinal) + def getField(fieldName: String) = GetField(expr, fieldName) + def cast(to: DataType) = Cast(expr, to) def asc = SortOrder(expr, Ascending) @@ -112,6 +129,7 @@ package object dsl { def sumDistinct(e: Expression) = SumDistinct(e) def count(e: Expression) = Count(e) def countDistinct(e: Expression*) = CountDistinct(e) + def approxCountDistinct(e: Expression, rsd: Double = 0.05) = ApproxCountDistinct(e, rsd) def avg(e: Expression) = Average(e) def first(e: Expression) = First(e) def min(e: Expression) = Min(e) @@ -163,6 +181,18 @@ package object dsl { /** Creates a new AttributeReference of type binary */ def binary = AttributeReference(s, BinaryType, nullable = true)() + + /** Creates a new AttributeReference of type array */ + def array(dataType: DataType) = AttributeReference(s, ArrayType(dataType), nullable = true)() + + /** Creates a new AttributeReference of type map */ + def map(keyType: DataType, valueType: DataType): AttributeReference = + map(MapType(keyType, valueType)) + def map(mapType: MapType) = AttributeReference(s, mapType, nullable = true)() + + /** Creates a new AttributeReference of type struct */ + def struct(fields: StructField*): AttributeReference = struct(StructType(fields)) + def struct(structType: StructType) = AttributeReference(s, structType, nullable = true)() } implicit class DslAttribute(a: AttributeReference) { @@ -172,7 +202,7 @@ package object dsl { // Protobuf terminology def required = a.withNullability(false) - def at(ordinal: Int) = BoundReference(ordinal, a) + def at(ordinal: Int) = BoundReference(ordinal, a.dataType, a.nullable) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala index 9ce1f01056462..0913f15888780 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala @@ -17,72 +17,37 @@ package org.apache.spark.sql.catalyst.expressions -import org.apache.spark.sql.catalyst.trees +import org.apache.spark.Logging import org.apache.spark.sql.catalyst.errors.attachTree -import org.apache.spark.sql.catalyst.plans.QueryPlan -import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.Logging +import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.catalyst.trees /** * A bound reference points to a specific slot in the input tuple, allowing the actual value * to be retrieved more efficiently. However, since operations like column pruning can change * the layout of intermediate tuples, BindReferences should be run after all such transformations. */ -case class BoundReference(ordinal: Int, baseReference: Attribute) - extends Attribute with trees.LeafNode[Expression] { +case class BoundReference(ordinal: Int, dataType: DataType, nullable: Boolean) + extends Expression with trees.LeafNode[Expression] { type EvaluatedType = Any - override def nullable = baseReference.nullable - override def dataType = baseReference.dataType - override def exprId = baseReference.exprId - override def qualifiers = baseReference.qualifiers - override def name = baseReference.name - - override def newInstance = BoundReference(ordinal, baseReference.newInstance) - override def withNullability(newNullability: Boolean) = - BoundReference(ordinal, baseReference.withNullability(newNullability)) - override def withQualifiers(newQualifiers: Seq[String]) = - BoundReference(ordinal, baseReference.withQualifiers(newQualifiers)) + override def references = Set.empty - override def toString = s"$baseReference:$ordinal" + override def toString = s"input[$ordinal]" override def eval(input: Row): Any = input(ordinal) } -/** - * Used to denote operators that do their own binding of attributes internally. - */ -trait NoBind { self: trees.TreeNode[_] => } - -class BindReferences[TreeNode <: QueryPlan[TreeNode]] extends Rule[TreeNode] { - import BindReferences._ - - def apply(plan: TreeNode): TreeNode = { - plan.transform { - case n: NoBind => n.asInstanceOf[TreeNode] - case leafNode if leafNode.children.isEmpty => leafNode - case unaryNode if unaryNode.children.size == 1 => unaryNode.transformExpressions { case e => - bindReference(e, unaryNode.children.head.output) - } - } - } -} - object BindReferences extends Logging { def bindReference[A <: Expression](expression: A, input: Seq[Attribute]): A = { expression.transform { case a: AttributeReference => attachTree(a, "Binding attribute") { val ordinal = input.indexWhere(_.exprId == a.exprId) if (ordinal == -1) { - // TODO: This fallback is required because some operators (such as ScriptTransform) - // produce new attributes that can't be bound. Likely the right thing to do is remove - // this rule and require all operators to explicitly bind to the input schema that - // they specify. - logger.debug(s"Couldn't find $a in ${input.mkString("[", ",", "]")}") - a + sys.error(s"Couldn't find $a in ${input.mkString("[", ",", "]")}") } else { - BoundReference(ordinal, a) + BoundReference(ordinal, a.dataType, a.nullable) } } }.asInstanceOf[A] // Kind of a hack, but safe. TODO: Tighten return type when possible. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index 1f9716e385e9e..0ad2b30cf9c1f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.expressions import java.sql.Timestamp +import java.text.{DateFormat, SimpleDateFormat} import org.apache.spark.sql.catalyst.types._ @@ -41,6 +42,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { // UDFToString private[this] def castToString: Any => Any = child.dataType match { case BinaryType => buildCast[Array[Byte]](_, new String(_, "UTF-8")) + case TimestampType => buildCast[Timestamp](_, timestampToString) case _ => buildCast[Any](_, _.toString) } @@ -126,6 +128,18 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { ts.getTime / 1000 + ts.getNanos.toDouble / 1000000000 } + // Converts Timestamp to string according to Hive TimestampWritable convention + private[this] def timestampToString(ts: Timestamp): String = { + val timestampString = ts.toString + val formatted = Cast.threadLocalDateFormat.get.format(ts) + + if (timestampString.length > 19 && timestampString.substring(19) != ".0") { + formatted + timestampString.substring(19) + } else { + formatted + } + } + private[this] def castToLong: Any => Any = child.dataType match { case StringType => buildCast[String](_, s => try s.toLong catch { @@ -249,3 +263,12 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { if (evaluated == null) null else cast(evaluated) } } + +object Cast { + // `SimpleDateFormat` is not thread-safe. + private[sql] val threadLocalDateFormat = new ThreadLocal[DateFormat] { + override def initialValue() = { + new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") + } + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala index 2c71d2c7b3563..8fc5896974438 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala @@ -17,12 +17,13 @@ package org.apache.spark.sql.catalyst.expressions + /** - * Converts a [[Row]] to another Row given a sequence of expression that define each column of the - * new row. If the schema of the input row is specified, then the given expression will be bound to - * that schema. + * A [[Projection]] that is calculated by calling the `eval` of each of the specified expressions. + * @param expressions a sequence of expressions that determine the value of each column of the + * output row. */ -class Projection(expressions: Seq[Expression]) extends (Row => Row) { +class InterpretedProjection(expressions: Seq[Expression]) extends Projection { def this(expressions: Seq[Expression], inputSchema: Seq[Attribute]) = this(expressions.map(BindReferences.bindReference(_, inputSchema))) @@ -40,25 +41,25 @@ class Projection(expressions: Seq[Expression]) extends (Row => Row) { } /** - * Converts a [[Row]] to another Row given a sequence of expression that define each column of th - * new row. If the schema of the input row is specified, then the given expression will be bound to - * that schema. - * - * In contrast to a normal projection, a MutableProjection reuses the same underlying row object - * each time an input row is added. This significantly reduces the cost of calculating the - * projection, but means that it is not safe to hold on to a reference to a [[Row]] after `next()` - * has been called on the [[Iterator]] that produced it. Instead, the user must call `Row.copy()` - * and hold on to the returned [[Row]] before calling `next()`. + * A [[MutableProjection]] that is calculated by calling `eval` on each of the specified + * expressions. + * @param expressions a sequence of expressions that determine the value of each column of the + * output row. */ -case class MutableProjection(expressions: Seq[Expression]) extends (Row => Row) { +case class InterpretedMutableProjection(expressions: Seq[Expression]) extends MutableProjection { def this(expressions: Seq[Expression], inputSchema: Seq[Attribute]) = this(expressions.map(BindReferences.bindReference(_, inputSchema))) private[this] val exprArray = expressions.toArray - private[this] val mutableRow = new GenericMutableRow(exprArray.size) + private[this] var mutableRow: MutableRow = new GenericMutableRow(exprArray.size) def currentValue: Row = mutableRow - def apply(input: Row): Row = { + override def target(row: MutableRow): MutableProjection = { + mutableRow = row + this + } + + override def apply(input: Row): Row = { var i = 0 while (i < exprArray.length) { mutableRow(i) = exprArray(i).eval(input) @@ -76,6 +77,12 @@ class JoinedRow extends Row { private[this] var row1: Row = _ private[this] var row2: Row = _ + def this(left: Row, right: Row) = { + this() + row1 = left + row2 = right + } + /** Updates this JoinedRow to used point at two new base rows. Returns itself. */ def apply(r1: Row, r2: Row): Row = { row1 = r1 diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala index 74ae723686cfe..c9a63e201ef60 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala @@ -32,6 +32,16 @@ object Row { * }}} */ def unapplySeq(row: Row): Some[Seq[Any]] = Some(row) + + /** + * This method can be used to construct a [[Row]] with the given values. + */ + def apply(values: Any*): Row = new GenericRow(values.toArray) + + /** + * This method can be used to construct a [[Row]] from a [[Seq]] of values. + */ + def fromSeq(values: Seq[Any]): Row = new GenericRow(values.toArray) } /** @@ -88,15 +98,6 @@ trait MutableRow extends Row { def setByte(ordinal: Int, value: Byte) def setFloat(ordinal: Int, value: Float) def setString(ordinal: Int, value: String) - - /** - * Experimental - * - * Returns a mutable string builder for the specified column. A given row should return the - * result of any mutations made to the returned buffer next time getString is called for the same - * column. - */ - def getStringBuilder(ordinal: Int): StringBuilder } /** @@ -180,6 +181,35 @@ class GenericRow(protected[catalyst] val values: Array[Any]) extends Row { values(i).asInstanceOf[String] } + // Custom hashCode function that matches the efficient code generated version. + override def hashCode(): Int = { + var result: Int = 37 + + var i = 0 + while (i < values.length) { + val update: Int = + if (isNullAt(i)) { + 0 + } else { + apply(i) match { + case b: Boolean => if (b) 0 else 1 + case b: Byte => b.toInt + case s: Short => s.toInt + case i: Int => i + case l: Long => (l ^ (l >>> 32)).toInt + case f: Float => java.lang.Float.floatToIntBits(f) + case d: Double => + val b = java.lang.Double.doubleToLongBits(d) + (b ^ (b >>> 32)).toInt + case other => other.hashCode() + } + } + result = 37 * result + update + i += 1 + } + result + } + def copy() = this } @@ -187,8 +217,6 @@ class GenericMutableRow(size: Int) extends GenericRow(size) with MutableRow { /** No-arg constructor for serialization. */ def this() = this(0) - def getStringBuilder(ordinal: Int): StringBuilder = ??? - override def setBoolean(ordinal: Int,value: Boolean): Unit = { values(ordinal) = value } override def setByte(ordinal: Int,value: Byte): Unit = { values(ordinal) = value } override def setDouble(ordinal: Int,value: Double): Unit = { values(ordinal) = value } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala index 5e089f7618e0a..95633dd0c9870 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala @@ -27,13 +27,321 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi def references = children.flatMap(_.references).toSet def nullable = true + /** This method has been generated by this script + + (1 to 22).map { x => + val anys = (1 to x).map(x => "Any").reduce(_ + ", " + _) + val evals = (0 to x - 1).map(x => s"children($x).eval(input)").reduce(_ + ",\n " + _) + + s""" + case $x => + function.asInstanceOf[($anys) => Any]( + $evals) + """ + } + + */ + + // scalastyle:off override def eval(input: Row): Any = { children.size match { + case 0 => function.asInstanceOf[() => Any]() case 1 => function.asInstanceOf[(Any) => Any](children(0).eval(input)) case 2 => function.asInstanceOf[(Any, Any) => Any]( children(0).eval(input), children(1).eval(input)) + case 3 => + function.asInstanceOf[(Any, Any, Any) => Any]( + children(0).eval(input), + children(1).eval(input), + children(2).eval(input)) + case 4 => + function.asInstanceOf[(Any, Any, Any, Any) => Any]( + children(0).eval(input), + children(1).eval(input), + children(2).eval(input), + children(3).eval(input)) + case 5 => + function.asInstanceOf[(Any, Any, Any, Any, Any) => Any]( + children(0).eval(input), + children(1).eval(input), + children(2).eval(input), + children(3).eval(input), + children(4).eval(input)) + case 6 => + function.asInstanceOf[(Any, Any, Any, Any, Any, Any) => Any]( + children(0).eval(input), + children(1).eval(input), + children(2).eval(input), + children(3).eval(input), + children(4).eval(input), + children(5).eval(input)) + case 7 => + function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any) => Any]( + children(0).eval(input), + children(1).eval(input), + children(2).eval(input), + children(3).eval(input), + children(4).eval(input), + children(5).eval(input), + children(6).eval(input)) + case 8 => + function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any) => Any]( + children(0).eval(input), + children(1).eval(input), + children(2).eval(input), + children(3).eval(input), + children(4).eval(input), + children(5).eval(input), + children(6).eval(input), + children(7).eval(input)) + case 9 => + function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( + children(0).eval(input), + children(1).eval(input), + children(2).eval(input), + children(3).eval(input), + children(4).eval(input), + children(5).eval(input), + children(6).eval(input), + children(7).eval(input), + children(8).eval(input)) + case 10 => + function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( + children(0).eval(input), + children(1).eval(input), + children(2).eval(input), + children(3).eval(input), + children(4).eval(input), + children(5).eval(input), + children(6).eval(input), + children(7).eval(input), + children(8).eval(input), + children(9).eval(input)) + case 11 => + function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( + children(0).eval(input), + children(1).eval(input), + children(2).eval(input), + children(3).eval(input), + children(4).eval(input), + children(5).eval(input), + children(6).eval(input), + children(7).eval(input), + children(8).eval(input), + children(9).eval(input), + children(10).eval(input)) + case 12 => + function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( + children(0).eval(input), + children(1).eval(input), + children(2).eval(input), + children(3).eval(input), + children(4).eval(input), + children(5).eval(input), + children(6).eval(input), + children(7).eval(input), + children(8).eval(input), + children(9).eval(input), + children(10).eval(input), + children(11).eval(input)) + case 13 => + function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( + children(0).eval(input), + children(1).eval(input), + children(2).eval(input), + children(3).eval(input), + children(4).eval(input), + children(5).eval(input), + children(6).eval(input), + children(7).eval(input), + children(8).eval(input), + children(9).eval(input), + children(10).eval(input), + children(11).eval(input), + children(12).eval(input)) + case 14 => + function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( + children(0).eval(input), + children(1).eval(input), + children(2).eval(input), + children(3).eval(input), + children(4).eval(input), + children(5).eval(input), + children(6).eval(input), + children(7).eval(input), + children(8).eval(input), + children(9).eval(input), + children(10).eval(input), + children(11).eval(input), + children(12).eval(input), + children(13).eval(input)) + case 15 => + function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( + children(0).eval(input), + children(1).eval(input), + children(2).eval(input), + children(3).eval(input), + children(4).eval(input), + children(5).eval(input), + children(6).eval(input), + children(7).eval(input), + children(8).eval(input), + children(9).eval(input), + children(10).eval(input), + children(11).eval(input), + children(12).eval(input), + children(13).eval(input), + children(14).eval(input)) + case 16 => + function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( + children(0).eval(input), + children(1).eval(input), + children(2).eval(input), + children(3).eval(input), + children(4).eval(input), + children(5).eval(input), + children(6).eval(input), + children(7).eval(input), + children(8).eval(input), + children(9).eval(input), + children(10).eval(input), + children(11).eval(input), + children(12).eval(input), + children(13).eval(input), + children(14).eval(input), + children(15).eval(input)) + case 17 => + function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( + children(0).eval(input), + children(1).eval(input), + children(2).eval(input), + children(3).eval(input), + children(4).eval(input), + children(5).eval(input), + children(6).eval(input), + children(7).eval(input), + children(8).eval(input), + children(9).eval(input), + children(10).eval(input), + children(11).eval(input), + children(12).eval(input), + children(13).eval(input), + children(14).eval(input), + children(15).eval(input), + children(16).eval(input)) + case 18 => + function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( + children(0).eval(input), + children(1).eval(input), + children(2).eval(input), + children(3).eval(input), + children(4).eval(input), + children(5).eval(input), + children(6).eval(input), + children(7).eval(input), + children(8).eval(input), + children(9).eval(input), + children(10).eval(input), + children(11).eval(input), + children(12).eval(input), + children(13).eval(input), + children(14).eval(input), + children(15).eval(input), + children(16).eval(input), + children(17).eval(input)) + case 19 => + function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( + children(0).eval(input), + children(1).eval(input), + children(2).eval(input), + children(3).eval(input), + children(4).eval(input), + children(5).eval(input), + children(6).eval(input), + children(7).eval(input), + children(8).eval(input), + children(9).eval(input), + children(10).eval(input), + children(11).eval(input), + children(12).eval(input), + children(13).eval(input), + children(14).eval(input), + children(15).eval(input), + children(16).eval(input), + children(17).eval(input), + children(18).eval(input)) + case 20 => + function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( + children(0).eval(input), + children(1).eval(input), + children(2).eval(input), + children(3).eval(input), + children(4).eval(input), + children(5).eval(input), + children(6).eval(input), + children(7).eval(input), + children(8).eval(input), + children(9).eval(input), + children(10).eval(input), + children(11).eval(input), + children(12).eval(input), + children(13).eval(input), + children(14).eval(input), + children(15).eval(input), + children(16).eval(input), + children(17).eval(input), + children(18).eval(input), + children(19).eval(input)) + case 21 => + function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( + children(0).eval(input), + children(1).eval(input), + children(2).eval(input), + children(3).eval(input), + children(4).eval(input), + children(5).eval(input), + children(6).eval(input), + children(7).eval(input), + children(8).eval(input), + children(9).eval(input), + children(10).eval(input), + children(11).eval(input), + children(12).eval(input), + children(13).eval(input), + children(14).eval(input), + children(15).eval(input), + children(16).eval(input), + children(17).eval(input), + children(18).eval(input), + children(19).eval(input), + children(20).eval(input)) + case 22 => + function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( + children(0).eval(input), + children(1).eval(input), + children(2).eval(input), + children(3).eval(input), + children(4).eval(input), + children(5).eval(input), + children(6).eval(input), + children(7).eval(input), + children(8).eval(input), + children(9).eval(input), + children(10).eval(input), + children(11).eval(input), + children(12).eval(input), + children(13).eval(input), + children(14).eval(input), + children(15).eval(input), + children(16).eval(input), + children(17).eval(input), + children(18).eval(input), + children(19).eval(input), + children(20).eval(input), + children(21).eval(input)) } + // scalastyle:on } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala index e787c59e75723..eb8898900d6a5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala @@ -21,8 +21,16 @@ import scala.language.dynamics import org.apache.spark.sql.catalyst.types.DataType -case object DynamicType extends DataType +/** + * The data type representing [[DynamicRow]] values. + */ +case object DynamicType extends DataType { + def simpleString: String = "dynamic" +} +/** + * Wrap a [[Row]] as a [[DynamicRow]]. + */ case class WrapDynamic(children: Seq[Attribute]) extends Expression { type EvaluatedType = DynamicRow @@ -37,6 +45,11 @@ case class WrapDynamic(children: Seq[Attribute]) extends Expression { } } +/** + * DynamicRows use scala's Dynamic trait to emulate an ORM of in a dynamically typed language. + * Since the type of the column is not known at compile time, all attributes are converted to + * strings before being passed to the function. + */ class DynamicRow(val schema: Seq[Attribute], values: Array[Any]) extends GenericRow(values) with Dynamic { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala new file mode 100644 index 0000000000000..de2d67ce82ff1 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -0,0 +1,485 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions.codegen + +import com.google.common.cache.{CacheLoader, CacheBuilder} + +import scala.language.existentials + +import org.apache.spark.Logging +import org.apache.spark.sql.catalyst.expressions +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.types._ + +/** + * A base class for generators of byte code to perform expression evaluation. Includes a set of + * helpers for referring to Catalyst types and building trees that perform evaluation of individual + * expressions. + */ +abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Logging { + import scala.reflect.runtime.{universe => ru} + import scala.reflect.runtime.universe._ + + import scala.tools.reflect.ToolBox + + protected val toolBox = runtimeMirror(getClass.getClassLoader).mkToolBox() + + protected val rowType = typeOf[Row] + protected val mutableRowType = typeOf[MutableRow] + protected val genericRowType = typeOf[GenericRow] + protected val genericMutableRowType = typeOf[GenericMutableRow] + + protected val projectionType = typeOf[Projection] + protected val mutableProjectionType = typeOf[MutableProjection] + + private val curId = new java.util.concurrent.atomic.AtomicInteger() + private val javaSeparator = "$" + + /** + * Generates a class for a given input expression. Called when there is not cached code + * already available. + */ + protected def create(in: InType): OutType + + /** + * Canonicalizes an input expression. Used to avoid double caching expressions that differ only + * cosmetically. + */ + protected def canonicalize(in: InType): InType + + /** Binds an input expression to a given input schema */ + protected def bind(in: InType, inputSchema: Seq[Attribute]): InType + + /** + * A cache of generated classes. + * + * From the Guava Docs: A Cache is similar to ConcurrentMap, but not quite the same. The most + * fundamental difference is that a ConcurrentMap persists all elements that are added to it until + * they are explicitly removed. A Cache on the other hand is generally configured to evict entries + * automatically, in order to constrain its memory footprint + */ + protected val cache = CacheBuilder.newBuilder() + .maximumSize(1000) + .build( + new CacheLoader[InType, OutType]() { + override def load(in: InType): OutType = globalLock.synchronized { + val startTime = System.nanoTime() + val result = create(in) + val endTime = System.nanoTime() + def timeMs = (endTime - startTime).toDouble / 1000000 + logInfo(s"Code generated expression $in in $timeMs ms") + result + } + }) + + /** Generates the requested evaluator binding the given expression(s) to the inputSchema. */ + def apply(expressions: InType, inputSchema: Seq[Attribute]): OutType = + apply(bind(expressions, inputSchema)) + + /** Generates the requested evaluator given already bound expression(s). */ + def apply(expressions: InType): OutType = cache.get(canonicalize(expressions)) + + /** + * Returns a term name that is unique within this instance of a `CodeGenerator`. + * + * (Since we aren't in a macro context we do not seem to have access to the built in `freshName` + * function.) + */ + protected def freshName(prefix: String): TermName = { + newTermName(s"$prefix$javaSeparator${curId.getAndIncrement}") + } + + /** + * Scala ASTs for evaluating an [[Expression]] given a [[Row]] of input. + * + * @param code The sequence of statements required to evaluate the expression. + * @param nullTerm A term that holds a boolean value representing whether the expression evaluated + * to null. + * @param primitiveTerm A term for a possible primitive value of the result of the evaluation. Not + * valid if `nullTerm` is set to `false`. + * @param objectTerm A possibly boxed version of the result of evaluating this expression. + */ + protected case class EvaluatedExpression( + code: Seq[Tree], + nullTerm: TermName, + primitiveTerm: TermName, + objectTerm: TermName) + + /** + * Given an expression tree returns an [[EvaluatedExpression]], which contains Scala trees that + * can be used to determine the result of evaluating the expression on an input row. + */ + def expressionEvaluator(e: Expression): EvaluatedExpression = { + val primitiveTerm = freshName("primitiveTerm") + val nullTerm = freshName("nullTerm") + val objectTerm = freshName("objectTerm") + + implicit class Evaluate1(e: Expression) { + def castOrNull(f: TermName => Tree, dataType: DataType): Seq[Tree] = { + val eval = expressionEvaluator(e) + eval.code ++ + q""" + val $nullTerm = ${eval.nullTerm} + val $primitiveTerm = + if($nullTerm) + ${defaultPrimitive(dataType)} + else + ${f(eval.primitiveTerm)} + """.children + } + } + + implicit class Evaluate2(expressions: (Expression, Expression)) { + + /** + * Short hand for generating binary evaluation code, which depends on two sub-evaluations of + * the same type. If either of the sub-expressions is null, the result of this computation + * is assumed to be null. + * + * @param f a function from two primitive term names to a tree that evaluates them. + */ + def evaluate(f: (TermName, TermName) => Tree): Seq[Tree] = + evaluateAs(expressions._1.dataType)(f) + + def evaluateAs(resultType: DataType)(f: (TermName, TermName) => Tree): Seq[Tree] = { + // TODO: Right now some timestamp tests fail if we enforce this... + if (expressions._1.dataType != expressions._2.dataType) { + log.warn(s"${expressions._1.dataType} != ${expressions._2.dataType}") + } + + val eval1 = expressionEvaluator(expressions._1) + val eval2 = expressionEvaluator(expressions._2) + val resultCode = f(eval1.primitiveTerm, eval2.primitiveTerm) + + eval1.code ++ eval2.code ++ + q""" + val $nullTerm = ${eval1.nullTerm} || ${eval2.nullTerm} + val $primitiveTerm: ${termForType(resultType)} = + if($nullTerm) { + ${defaultPrimitive(resultType)} + } else { + $resultCode.asInstanceOf[${termForType(resultType)}] + } + """.children : Seq[Tree] + } + } + + val inputTuple = newTermName(s"i") + + // TODO: Skip generation of null handling code when expression are not nullable. + val primitiveEvaluation: PartialFunction[Expression, Seq[Tree]] = { + case b @ BoundReference(ordinal, dataType, nullable) => + val nullValue = q"$inputTuple.isNullAt($ordinal)" + q""" + val $nullTerm: Boolean = $nullValue + val $primitiveTerm: ${termForType(dataType)} = + if($nullTerm) + ${defaultPrimitive(dataType)} + else + ${getColumn(inputTuple, dataType, ordinal)} + """.children + + case expressions.Literal(null, dataType) => + q""" + val $nullTerm = true + val $primitiveTerm: ${termForType(dataType)} = null.asInstanceOf[${termForType(dataType)}] + """.children + + case expressions.Literal(value: Boolean, dataType) => + q""" + val $nullTerm = ${value == null} + val $primitiveTerm: ${termForType(dataType)} = $value + """.children + + case expressions.Literal(value: String, dataType) => + q""" + val $nullTerm = ${value == null} + val $primitiveTerm: ${termForType(dataType)} = $value + """.children + + case expressions.Literal(value: Int, dataType) => + q""" + val $nullTerm = ${value == null} + val $primitiveTerm: ${termForType(dataType)} = $value + """.children + + case expressions.Literal(value: Long, dataType) => + q""" + val $nullTerm = ${value == null} + val $primitiveTerm: ${termForType(dataType)} = $value + """.children + + case Cast(e @ BinaryType(), StringType) => + val eval = expressionEvaluator(e) + eval.code ++ + q""" + val $nullTerm = ${eval.nullTerm} + val $primitiveTerm = + if($nullTerm) + ${defaultPrimitive(StringType)} + else + new String(${eval.primitiveTerm}.asInstanceOf[Array[Byte]]) + """.children + + case Cast(child @ NumericType(), IntegerType) => + child.castOrNull(c => q"$c.toInt", IntegerType) + + case Cast(child @ NumericType(), LongType) => + child.castOrNull(c => q"$c.toLong", LongType) + + case Cast(child @ NumericType(), DoubleType) => + child.castOrNull(c => q"$c.toDouble", DoubleType) + + case Cast(child @ NumericType(), FloatType) => + child.castOrNull(c => q"$c.toFloat", IntegerType) + + // Special handling required for timestamps in hive test cases since the toString function + // does not match the expected output. + case Cast(e, StringType) if e.dataType != TimestampType => + val eval = expressionEvaluator(e) + eval.code ++ + q""" + val $nullTerm = ${eval.nullTerm} + val $primitiveTerm = + if($nullTerm) + ${defaultPrimitive(StringType)} + else + ${eval.primitiveTerm}.toString + """.children + + case EqualTo(e1, e2) => + (e1, e2).evaluateAs (BooleanType) { case (eval1, eval2) => q"$eval1 == $eval2" } + + /* TODO: Fix null semantics. + case In(e1, list) if !list.exists(!_.isInstanceOf[expressions.Literal]) => + val eval = expressionEvaluator(e1) + + val checks = list.map { + case expressions.Literal(v: String, dataType) => + q"if(${eval.primitiveTerm} == $v) return true" + case expressions.Literal(v: Int, dataType) => + q"if(${eval.primitiveTerm} == $v) return true" + } + + val funcName = newTermName(s"isIn${curId.getAndIncrement()}") + + q""" + def $funcName: Boolean = { + ..${eval.code} + if(${eval.nullTerm}) return false + ..$checks + return false + } + val $nullTerm = false + val $primitiveTerm = $funcName + """.children + */ + + case GreaterThan(e1 @ NumericType(), e2 @ NumericType()) => + (e1, e2).evaluateAs (BooleanType) { case (eval1, eval2) => q"$eval1 > $eval2" } + case GreaterThanOrEqual(e1 @ NumericType(), e2 @ NumericType()) => + (e1, e2).evaluateAs (BooleanType) { case (eval1, eval2) => q"$eval1 >= $eval2" } + case LessThan(e1 @ NumericType(), e2 @ NumericType()) => + (e1, e2).evaluateAs (BooleanType) { case (eval1, eval2) => q"$eval1 < $eval2" } + case LessThanOrEqual(e1 @ NumericType(), e2 @ NumericType()) => + (e1, e2).evaluateAs (BooleanType) { case (eval1, eval2) => q"$eval1 <= $eval2" } + + case And(e1, e2) => + val eval1 = expressionEvaluator(e1) + val eval2 = expressionEvaluator(e2) + + eval1.code ++ eval2.code ++ + q""" + var $nullTerm = false + var $primitiveTerm: ${termForType(BooleanType)} = false + + if ((!${eval1.nullTerm} && !${eval1.primitiveTerm}) || + (!${eval2.nullTerm} && !${eval2.primitiveTerm})) { + $nullTerm = false + $primitiveTerm = false + } else if (${eval1.nullTerm} || ${eval2.nullTerm} ) { + $nullTerm = true + } else { + $nullTerm = false + $primitiveTerm = true + } + """.children + + case Or(e1, e2) => + val eval1 = expressionEvaluator(e1) + val eval2 = expressionEvaluator(e2) + + eval1.code ++ eval2.code ++ + q""" + var $nullTerm = false + var $primitiveTerm: ${termForType(BooleanType)} = false + + if ((!${eval1.nullTerm} && ${eval1.primitiveTerm}) || + (!${eval2.nullTerm} && ${eval2.primitiveTerm})) { + $nullTerm = false + $primitiveTerm = true + } else if (${eval1.nullTerm} || ${eval2.nullTerm} ) { + $nullTerm = true + } else { + $nullTerm = false + $primitiveTerm = false + } + """.children + + case Not(child) => + // Uh, bad function name... + child.castOrNull(c => q"!$c", BooleanType) + + case Add(e1, e2) => (e1, e2) evaluate { case (eval1, eval2) => q"$eval1 + $eval2" } + case Subtract(e1, e2) => (e1, e2) evaluate { case (eval1, eval2) => q"$eval1 - $eval2" } + case Multiply(e1, e2) => (e1, e2) evaluate { case (eval1, eval2) => q"$eval1 * $eval2" } + case Divide(e1, e2) => (e1, e2) evaluate { case (eval1, eval2) => q"$eval1 / $eval2" } + + case IsNotNull(e) => + val eval = expressionEvaluator(e) + q""" + ..${eval.code} + var $nullTerm = false + var $primitiveTerm: ${termForType(BooleanType)} = !${eval.nullTerm} + """.children + + case IsNull(e) => + val eval = expressionEvaluator(e) + q""" + ..${eval.code} + var $nullTerm = false + var $primitiveTerm: ${termForType(BooleanType)} = ${eval.nullTerm} + """.children + + case c @ Coalesce(children) => + q""" + var $nullTerm = true + var $primitiveTerm: ${termForType(c.dataType)} = ${defaultPrimitive(c.dataType)} + """.children ++ + children.map { c => + val eval = expressionEvaluator(c) + q""" + if($nullTerm) { + ..${eval.code} + if(!${eval.nullTerm}) { + $nullTerm = false + $primitiveTerm = ${eval.primitiveTerm} + } + } + """ + } + + case i @ expressions.If(condition, trueValue, falseValue) => + val condEval = expressionEvaluator(condition) + val trueEval = expressionEvaluator(trueValue) + val falseEval = expressionEvaluator(falseValue) + + q""" + var $nullTerm = false + var $primitiveTerm: ${termForType(i.dataType)} = ${defaultPrimitive(i.dataType)} + ..${condEval.code} + if(!${condEval.nullTerm} && ${condEval.primitiveTerm}) { + ..${trueEval.code} + $nullTerm = ${trueEval.nullTerm} + $primitiveTerm = ${trueEval.primitiveTerm} + } else { + ..${falseEval.code} + $nullTerm = ${falseEval.nullTerm} + $primitiveTerm = ${falseEval.primitiveTerm} + } + """.children + } + + // If there was no match in the partial function above, we fall back on calling the interpreted + // expression evaluator. + val code: Seq[Tree] = + primitiveEvaluation.lift.apply(e).getOrElse { + log.debug(s"No rules to generate $e") + val tree = reify { e } + q""" + val $objectTerm = $tree.eval(i) + val $nullTerm = $objectTerm == null + val $primitiveTerm = $objectTerm.asInstanceOf[${termForType(e.dataType)}] + """.children + } + + // Only inject debugging code if debugging is turned on. + val debugCode = + if (log.isDebugEnabled) { + val localLogger = log + val localLoggerTree = reify { localLogger } + q""" + $localLoggerTree.debug(${e.toString} + ": " + (if($nullTerm) "null" else $primitiveTerm)) + """ :: Nil + } else { + Nil + } + + EvaluatedExpression(code ++ debugCode, nullTerm, primitiveTerm, objectTerm) + } + + protected def getColumn(inputRow: TermName, dataType: DataType, ordinal: Int) = { + dataType match { + case dt @ NativeType() => q"$inputRow.${accessorForType(dt)}($ordinal)" + case _ => q"$inputRow.apply($ordinal).asInstanceOf[${termForType(dataType)}]" + } + } + + protected def setColumn( + destinationRow: TermName, + dataType: DataType, + ordinal: Int, + value: TermName) = { + dataType match { + case dt @ NativeType() => q"$destinationRow.${mutatorForType(dt)}($ordinal, $value)" + case _ => q"$destinationRow.update($ordinal, $value)" + } + } + + protected def accessorForType(dt: DataType) = newTermName(s"get${primitiveForType(dt)}") + protected def mutatorForType(dt: DataType) = newTermName(s"set${primitiveForType(dt)}") + + protected def primitiveForType(dt: DataType) = dt match { + case IntegerType => "Int" + case LongType => "Long" + case ShortType => "Short" + case ByteType => "Byte" + case DoubleType => "Double" + case FloatType => "Float" + case BooleanType => "Boolean" + case StringType => "String" + } + + protected def defaultPrimitive(dt: DataType) = dt match { + case BooleanType => ru.Literal(Constant(false)) + case FloatType => ru.Literal(Constant(-1.0.toFloat)) + case StringType => ru.Literal(Constant("")) + case ShortType => ru.Literal(Constant(-1.toShort)) + case LongType => ru.Literal(Constant(1L)) + case ByteType => ru.Literal(Constant(-1.toByte)) + case DoubleType => ru.Literal(Constant(-1.toDouble)) + case DecimalType => ru.Literal(Constant(-1)) // Will get implicity converted as needed. + case IntegerType => ru.Literal(Constant(-1)) + case _ => ru.Literal(Constant(null)) + } + + protected def termForType(dt: DataType) = dt match { + case n: NativeType => n.tag + case _ => typeTag[Any] + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala new file mode 100644 index 0000000000000..a419fd7ecb39b --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions.codegen + +import org.apache.spark.sql.catalyst.expressions._ + +/** + * Generates byte code that produces a [[MutableRow]] object that can update itself based on a new + * input [[Row]] for a fixed set of [[Expression Expressions]]. + */ +object GenerateMutableProjection extends CodeGenerator[Seq[Expression], () => MutableProjection] { + import scala.reflect.runtime.{universe => ru} + import scala.reflect.runtime.universe._ + + val mutableRowName = newTermName("mutableRow") + + protected def canonicalize(in: Seq[Expression]): Seq[Expression] = + in.map(ExpressionCanonicalizer(_)) + + protected def bind(in: Seq[Expression], inputSchema: Seq[Attribute]): Seq[Expression] = + in.map(BindReferences.bindReference(_, inputSchema)) + + protected def create(expressions: Seq[Expression]): (() => MutableProjection) = { + val projectionCode = expressions.zipWithIndex.flatMap { case (e, i) => + val evaluationCode = expressionEvaluator(e) + + evaluationCode.code :+ + q""" + if(${evaluationCode.nullTerm}) + mutableRow.setNullAt($i) + else + ${setColumn(mutableRowName, e.dataType, i, evaluationCode.primitiveTerm)} + """ + } + + val code = + q""" + () => { new $mutableProjectionType { + + private[this] var $mutableRowName: $mutableRowType = + new $genericMutableRowType(${expressions.size}) + + def target(row: $mutableRowType): $mutableProjectionType = { + $mutableRowName = row + this + } + + /* Provide immutable access to the last projected row. */ + def currentValue: $rowType = mutableRow + + def apply(i: $rowType): $rowType = { + ..$projectionCode + mutableRow + } + } } + """ + + log.debug(s"code for ${expressions.mkString(",")}:\n$code") + toolBox.eval(code).asInstanceOf[() => MutableProjection] + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala new file mode 100644 index 0000000000000..094ff14552283 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions.codegen + +import org.apache.spark.Logging +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.types.{StringType, NumericType} + +/** + * Generates bytecode for an [[Ordering]] of [[Row Rows]] for a given set of + * [[Expression Expressions]]. + */ +object GenerateOrdering extends CodeGenerator[Seq[SortOrder], Ordering[Row]] with Logging { + import scala.reflect.runtime.{universe => ru} + import scala.reflect.runtime.universe._ + + protected def canonicalize(in: Seq[SortOrder]): Seq[SortOrder] = + in.map(ExpressionCanonicalizer(_).asInstanceOf[SortOrder]) + + protected def bind(in: Seq[SortOrder], inputSchema: Seq[Attribute]): Seq[SortOrder] = + in.map(BindReferences.bindReference(_, inputSchema)) + + protected def create(ordering: Seq[SortOrder]): Ordering[Row] = { + val a = newTermName("a") + val b = newTermName("b") + val comparisons = ordering.zipWithIndex.map { case (order, i) => + val evalA = expressionEvaluator(order.child) + val evalB = expressionEvaluator(order.child) + + val compare = order.child.dataType match { + case _: NumericType => + q""" + val comp = ${evalA.primitiveTerm} - ${evalB.primitiveTerm} + if(comp != 0) { + return ${if (order.direction == Ascending) q"comp.toInt" else q"-comp.toInt"} + } + """ + case StringType => + if (order.direction == Ascending) { + q"""return ${evalA.primitiveTerm}.compare(${evalB.primitiveTerm})""" + } else { + q"""return ${evalB.primitiveTerm}.compare(${evalA.primitiveTerm})""" + } + } + + q""" + i = $a + ..${evalA.code} + i = $b + ..${evalB.code} + if (${evalA.nullTerm} && ${evalB.nullTerm}) { + // Nothing + } else if (${evalA.nullTerm}) { + return ${if (order.direction == Ascending) q"-1" else q"1"} + } else if (${evalB.nullTerm}) { + return ${if (order.direction == Ascending) q"1" else q"-1"} + } else { + $compare + } + """ + } + + val q"class $orderingName extends $orderingType { ..$body }" = reify { + class SpecificOrdering extends Ordering[Row] { + val o = ordering + } + }.tree.children.head + + val code = q""" + class $orderingName extends $orderingType { + ..$body + def compare(a: $rowType, b: $rowType): Int = { + var i: $rowType = null // Holds current row being evaluated. + ..$comparisons + return 0 + } + } + new $orderingName() + """ + logDebug(s"Generated Ordering: $code") + toolBox.eval(code).asInstanceOf[Ordering[Row]] + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala new file mode 100644 index 0000000000000..2a0935c790cf3 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions.codegen + +import org.apache.spark.sql.catalyst.expressions._ + +/** + * Generates bytecode that evaluates a boolean [[Expression]] on a given input [[Row]]. + */ +object GeneratePredicate extends CodeGenerator[Expression, (Row) => Boolean] { + import scala.reflect.runtime.{universe => ru} + import scala.reflect.runtime.universe._ + + protected def canonicalize(in: Expression): Expression = ExpressionCanonicalizer(in) + + protected def bind(in: Expression, inputSchema: Seq[Attribute]): Expression = + BindReferences.bindReference(in, inputSchema) + + protected def create(predicate: Expression): ((Row) => Boolean) = { + val cEval = expressionEvaluator(predicate) + + val code = + q""" + (i: $rowType) => { + ..${cEval.code} + if (${cEval.nullTerm}) false else ${cEval.primitiveTerm} + } + """ + + log.debug(s"Generated predicate '$predicate':\n$code") + toolBox.eval(code).asInstanceOf[Row => Boolean] + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateProjection.scala new file mode 100644 index 0000000000000..77fa02c13de30 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateProjection.scala @@ -0,0 +1,219 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions.codegen + +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.types._ + + +/** + * Generates bytecode that produces a new [[Row]] object based on a fixed set of input + * [[Expression Expressions]] and a given input [[Row]]. The returned [[Row]] object is custom + * generated based on the output types of the [[Expression]] to avoid boxing of primitive values. + */ +object GenerateProjection extends CodeGenerator[Seq[Expression], Projection] { + import scala.reflect.runtime.{universe => ru} + import scala.reflect.runtime.universe._ + + protected def canonicalize(in: Seq[Expression]): Seq[Expression] = + in.map(ExpressionCanonicalizer(_)) + + protected def bind(in: Seq[Expression], inputSchema: Seq[Attribute]): Seq[Expression] = + in.map(BindReferences.bindReference(_, inputSchema)) + + // Make Mutablility optional... + protected def create(expressions: Seq[Expression]): Projection = { + val tupleLength = ru.Literal(Constant(expressions.length)) + val lengthDef = q"final val length = $tupleLength" + + /* TODO: Configurable... + val nullFunctions = + q""" + private final val nullSet = new org.apache.spark.util.collection.BitSet(length) + final def setNullAt(i: Int) = nullSet.set(i) + final def isNullAt(i: Int) = nullSet.get(i) + """ + */ + + val nullFunctions = + q""" + private[this] var nullBits = new Array[Boolean](${expressions.size}) + final def setNullAt(i: Int) = { nullBits(i) = true } + final def isNullAt(i: Int) = nullBits(i) + """.children + + val tupleElements = expressions.zipWithIndex.flatMap { + case (e, i) => + val elementName = newTermName(s"c$i") + val evaluatedExpression = expressionEvaluator(e) + val iLit = ru.Literal(Constant(i)) + + q""" + var ${newTermName(s"c$i")}: ${termForType(e.dataType)} = _ + { + ..${evaluatedExpression.code} + if(${evaluatedExpression.nullTerm}) + setNullAt($iLit) + else + $elementName = ${evaluatedExpression.primitiveTerm} + } + """.children : Seq[Tree] + } + + val iteratorFunction = { + val allColumns = (0 until expressions.size).map { i => + val iLit = ru.Literal(Constant(i)) + q"if(isNullAt($iLit)) { null } else { ${newTermName(s"c$i")} }" + } + q"final def iterator = Iterator[Any](..$allColumns)" + } + + val accessorFailure = q"""scala.sys.error("Invalid ordinal:" + i)""" + val applyFunction = { + val cases = (0 until expressions.size).map { i => + val ordinal = ru.Literal(Constant(i)) + val elementName = newTermName(s"c$i") + val iLit = ru.Literal(Constant(i)) + + q"if(i == $ordinal) { if(isNullAt($i)) return null else return $elementName }" + } + q"final def apply(i: Int): Any = { ..$cases; $accessorFailure }" + } + + val updateFunction = { + val cases = expressions.zipWithIndex.map {case (e, i) => + val ordinal = ru.Literal(Constant(i)) + val elementName = newTermName(s"c$i") + val iLit = ru.Literal(Constant(i)) + + q""" + if(i == $ordinal) { + if(value == null) { + setNullAt(i) + } else { + $elementName = value.asInstanceOf[${termForType(e.dataType)}] + return + } + }""" + } + q"final def update(i: Int, value: Any): Unit = { ..$cases; $accessorFailure }" + } + + val specificAccessorFunctions = NativeType.all.map { dataType => + val ifStatements = expressions.zipWithIndex.flatMap { + case (e, i) if e.dataType == dataType => + val elementName = newTermName(s"c$i") + // TODO: The string of ifs gets pretty inefficient as the row grows in size. + // TODO: Optional null checks? + q"if(i == $i) return $elementName" :: Nil + case _ => Nil + } + + q""" + final def ${accessorForType(dataType)}(i: Int):${termForType(dataType)} = { + ..$ifStatements; + $accessorFailure + }""" + } + + val specificMutatorFunctions = NativeType.all.map { dataType => + val ifStatements = expressions.zipWithIndex.flatMap { + case (e, i) if e.dataType == dataType => + val elementName = newTermName(s"c$i") + // TODO: The string of ifs gets pretty inefficient as the row grows in size. + // TODO: Optional null checks? + q"if(i == $i) { $elementName = value; return }" :: Nil + case _ => Nil + } + + q""" + final def ${mutatorForType(dataType)}(i: Int, value: ${termForType(dataType)}): Unit = { + ..$ifStatements; + $accessorFailure + }""" + } + + val hashValues = expressions.zipWithIndex.map { case (e,i) => + val elementName = newTermName(s"c$i") + val nonNull = e.dataType match { + case BooleanType => q"if ($elementName) 0 else 1" + case ByteType | ShortType | IntegerType => q"$elementName.toInt" + case LongType => q"($elementName ^ ($elementName >>> 32)).toInt" + case FloatType => q"java.lang.Float.floatToIntBits($elementName)" + case DoubleType => + q"{ val b = java.lang.Double.doubleToLongBits($elementName); (b ^ (b >>>32)).toInt }" + case _ => q"$elementName.hashCode" + } + q"if (isNullAt($i)) 0 else $nonNull" + } + + val hashUpdates: Seq[Tree] = hashValues.map(v => q"""result = 37 * result + $v""": Tree) + + val hashCodeFunction = + q""" + override def hashCode(): Int = { + var result: Int = 37 + ..$hashUpdates + result + } + """ + + val columnChecks = (0 until expressions.size).map { i => + val elementName = newTermName(s"c$i") + q"if (this.$elementName != specificType.$elementName) return false" + } + + val equalsFunction = + q""" + override def equals(other: Any): Boolean = other match { + case specificType: SpecificRow => + ..$columnChecks + return true + case other => super.equals(other) + } + """ + + val copyFunction = + q""" + final def copy() = new $genericRowType(this.toArray) + """ + + val classBody = + nullFunctions ++ ( + lengthDef +: + iteratorFunction +: + applyFunction +: + updateFunction +: + equalsFunction +: + hashCodeFunction +: + copyFunction +: + (tupleElements ++ specificAccessorFunctions ++ specificMutatorFunctions)) + + val code = q""" + final class SpecificRow(i: $rowType) extends $mutableRowType { + ..$classBody + } + + new $projectionType { def apply(r: $rowType) = new SpecificRow(r) } + """ + + log.debug( + s"MutableRow, initExprs: ${expressions.mkString(",")} code:\n${toolBox.typeCheck(code)}") + toolBox.eval(code).asInstanceOf[Projection] + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/package.scala new file mode 100644 index 0000000000000..80c7dfd376c96 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/package.scala @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.sql.catalyst.rules +import org.apache.spark.sql.catalyst.util + +/** + * A collection of generators that build custom bytecode at runtime for performing the evaluation + * of catalyst expression. + */ +package object codegen { + + /** + * A lock to protect invoking the scala compiler at runtime, since it is not thread safe in Scala + * 2.10. + */ + protected[codegen] val globalLock = org.apache.spark.sql.catalyst.ScalaReflectionLock + + /** Canonicalizes an expression so those that differ only by names can reuse the same code. */ + object ExpressionCanonicalizer extends rules.RuleExecutor[Expression] { + val batches = + Batch("CleanExpressions", FixedPoint(20), CleanExpressions) :: Nil + + object CleanExpressions extends rules.Rule[Expression] { + def apply(e: Expression): Expression = e transform { + case Alias(c, _) => c + } + } + } + + /** + * :: DeveloperApi :: + * Dumps the bytecode from a class to the screen using javap. + */ + @DeveloperApi + object DumpByteCode { + import scala.sys.process._ + val dumpDirectory = util.getTempFilePath("sparkSqlByteCode") + dumpDirectory.mkdir() + + def apply(obj: Any): Unit = { + val generatedClass = obj.getClass + val classLoader = + generatedClass + .getClassLoader + .asInstanceOf[scala.tools.nsc.interpreter.AbstractFileClassLoader] + val generatedBytes = classLoader.classBytes(generatedClass.getName) + + val packageDir = new java.io.File(dumpDirectory, generatedClass.getPackage.getName) + if (!packageDir.exists()) { packageDir.mkdir() } + + val classFile = + new java.io.File(packageDir, generatedClass.getName.split("\\.").last + ".class") + + val outfile = new java.io.FileOutputStream(classFile) + outfile.write(generatedBytes) + outfile.close() + + println( + s"javap -p -v -classpath ${dumpDirectory.getCanonicalPath} ${generatedClass.getName}".!!) + } + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala index 5d3bb25ad568c..c1154eb81c319 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.expressions +import scala.collection.Map + import org.apache.spark.sql.catalyst.types._ /** @@ -31,8 +33,8 @@ case class GetItem(child: Expression, ordinal: Expression) extends Expression { override def foldable = child.foldable && ordinal.foldable override def references = children.flatMap(_.references).toSet def dataType = child.dataType match { - case ArrayType(dt) => dt - case MapType(_, vt) => vt + case ArrayType(dt, _) => dt + case MapType(_, vt, _) => vt } override lazy val resolved = childrenResolved && @@ -61,7 +63,6 @@ case class GetItem(child: Expression, ordinal: Expression) extends Expression { } } else { val baseValue = value.asInstanceOf[Map[Any, _]] - val key = ordinal.eval(input) baseValue.get(key).orNull } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala index dd78614754e12..e99c5b452d183 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.expressions +import scala.collection.Map + import org.apache.spark.sql.catalyst.trees import org.apache.spark.sql.catalyst.types._ @@ -84,28 +86,28 @@ case class Explode(attributeNames: Seq[String], child: Expression) (child.dataType.isInstanceOf[ArrayType] || child.dataType.isInstanceOf[MapType]) private lazy val elementTypes = child.dataType match { - case ArrayType(et) => et :: Nil - case MapType(kt,vt) => kt :: vt :: Nil + case ArrayType(et, containsNull) => (et, containsNull) :: Nil + case MapType(kt, vt, valueContainsNull) => (kt, false) :: (vt, valueContainsNull) :: Nil } // TODO: Move this pattern into Generator. protected def makeOutput() = if (attributeNames.size == elementTypes.size) { attributeNames.zip(elementTypes).map { - case (n, t) => AttributeReference(n, t, nullable = true)() + case (n, (t, nullable)) => AttributeReference(n, t, nullable)() } } else { elementTypes.zipWithIndex.map { - case (t, i) => AttributeReference(s"c_$i", t, nullable = true)() + case ((t, nullable), i) => AttributeReference(s"c_$i", t, nullable)() } } override def eval(input: Row): TraversableOnce[Row] = { child.dataType match { - case ArrayType(_) => + case ArrayType(_, _) => val inputArray = child.eval(input).asInstanceOf[Seq[Any]] if (inputArray == null) Nil else inputArray.map(v => new GenericRow(Array(v))) - case MapType(_, _) => + case MapType(_, _, _) => val inputMap = child.eval(input).asInstanceOf[Map[Any,Any]] if (inputMap == null) Nil else inputMap.map { case (k,v) => new GenericRow(Array(k,v)) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index 934bad8c27294..02d04762629f5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -28,8 +28,8 @@ object NamedExpression { } /** - * A globally (within this JVM) id for a given named expression. - * Used to identify with attribute output by a relation is being + * A globally unique (within this JVM) id for a given named expression. + * Used to identify which attribute output by a relation is being * referenced in a subsequent computation. */ case class ExprId(id: Long) @@ -134,7 +134,7 @@ case class AttributeReference(name: String, dataType: DataType, nullable: Boolea /** * Returns a copy of this [[AttributeReference]] with changed nullability. */ - override def withNullability(newNullability: Boolean) = { + override def withNullability(newNullability: Boolean): AttributeReference = { if (nullable == newNullability) { this } else { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala index ce6d99c911ab3..e88c5d4fa178a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala @@ -60,6 +60,8 @@ case class IsNull(child: Expression) extends Predicate with trees.UnaryNode[Expr override def eval(input: Row): Any = { child.eval(input) == null } + + override def toString = s"IS NULL $child" } case class IsNotNull(child: Expression) extends Predicate with trees.UnaryNode[Expression] { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala index b6f2451b52e1f..55d95991c5f11 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala @@ -47,4 +47,30 @@ package org.apache.spark.sql.catalyst * ==Evaluation== * The result of expressions can be evaluated using the `Expression.apply(Row)` method. */ -package object expressions +package object expressions { + + /** + * Converts a [[Row]] to another Row given a sequence of expression that define each column of the + * new row. If the schema of the input row is specified, then the given expression will be bound + * to that schema. + */ + abstract class Projection extends (Row => Row) + + /** + * Converts a [[Row]] to another Row given a sequence of expression that define each column of the + * new row. If the schema of the input row is specified, then the given expression will be bound + * to that schema. + * + * In contrast to a normal projection, a MutableProjection reuses the same underlying row object + * each time an input row is added. This significantly reduces the cost of calculating the + * projection, but means that it is not safe to hold on to a reference to a [[Row]] after `next()` + * has been called on the [[Iterator]] that produced it. Instead, the user must call `Row.copy()` + * and hold on to the returned [[Row]] before calling `next()`. + */ + abstract class MutableProjection extends Projection { + def currentValue: Row + + /** Uses the given row to store the output of the projection. */ + def target(row: MutableRow): MutableProjection + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index b63406b94a4a3..5976b0ddf3e03 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -23,6 +23,9 @@ import org.apache.spark.sql.catalyst.types.BooleanType object InterpretedPredicate { + def apply(expression: Expression, inputSchema: Seq[Attribute]): (Row => Boolean) = + apply(BindReferences.bindReference(expression, inputSchema)) + def apply(expression: Expression): (Row => Boolean) = { (r: Row) => expression.eval(r).asInstanceOf[Boolean] } @@ -153,6 +156,22 @@ case class EqualTo(left: Expression, right: Expression) extends BinaryComparison } } +case class EqualNullSafe(left: Expression, right: Expression) extends BinaryComparison { + def symbol = "<=>" + override def nullable = false + override def eval(input: Row): Any = { + val l = left.eval(input) + val r = right.eval(input) + if (l == null && r == null) { + true + } else if (l == null || r == null) { + false + } else { + l == r + } + } +} + case class LessThan(left: Expression, right: Expression) extends BinaryComparison { def symbol = "<" override def eval(input: Row): Any = c2(input, left, right, _.lt(_, _)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala index 347471cebdc7e..97fc3a3b14b88 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala @@ -19,11 +19,12 @@ package org.apache.spark.sql.catalyst.expressions import java.util.regex.Pattern -import org.apache.spark.sql.catalyst.types.DataType -import org.apache.spark.sql.catalyst.types.StringType -import org.apache.spark.sql.catalyst.types.BooleanType +import scala.collection.IndexedSeqOptimized +import org.apache.spark.sql.catalyst.analysis.UnresolvedException +import org.apache.spark.sql.catalyst.types.{BinaryType, BooleanType, DataType, StringType} + trait StringRegexExpression { self: BinaryExpression => @@ -32,7 +33,7 @@ trait StringRegexExpression { def escape(v: String): String def matches(regex: Pattern, str: String): Boolean - def nullable: Boolean = true + def nullable: Boolean = left.nullable || right.nullable def dataType: DataType = BooleanType // try cache the pattern for Literal @@ -157,19 +158,13 @@ case class Lower(child: Expression) extends UnaryExpression with CaseConversionE override def toString() = s"Lower($child)" } -/** A base class for functions that compare two strings, returning a boolean. */ -abstract class StringComparison extends Expression { - self: Product => +/** A base trait for functions that compare two strings, returning a boolean. */ +trait StringComparison { + self: BinaryExpression => type EvaluatedType = Any - def left: Expression - def right: Expression - - override def references = children.flatMap(_.references).toSet - override def children = left :: right :: Nil - - override def nullable: Boolean = true + def nullable: Boolean = left.nullable || right.nullable override def dataType: DataType = BooleanType def compare(l: String, r: String): Boolean @@ -184,26 +179,102 @@ abstract class StringComparison extends Expression { } } + def symbol: String = nodeName + override def toString() = s"$nodeName($left, $right)" } /** * A function that returns true if the string `left` contains the string `right`. */ -case class Contains(left: Expression, right: Expression) extends StringComparison { +case class Contains(left: Expression, right: Expression) + extends BinaryExpression with StringComparison { override def compare(l: String, r: String) = l.contains(r) } /** * A function that returns true if the string `left` starts with the string `right`. */ -case class StartsWith(left: Expression, right: Expression) extends StringComparison { +case class StartsWith(left: Expression, right: Expression) + extends BinaryExpression with StringComparison { def compare(l: String, r: String) = l.startsWith(r) } /** * A function that returns true if the string `left` ends with the string `right`. */ -case class EndsWith(left: Expression, right: Expression) extends StringComparison { +case class EndsWith(left: Expression, right: Expression) + extends BinaryExpression with StringComparison { def compare(l: String, r: String) = l.endsWith(r) } + +/** + * A function that takes a substring of its first argument starting at a given position. + * Defined for String and Binary types. + */ +case class Substring(str: Expression, pos: Expression, len: Expression) extends Expression { + + type EvaluatedType = Any + + override def foldable = str.foldable && pos.foldable && len.foldable + + def nullable: Boolean = str.nullable || pos.nullable || len.nullable + def dataType: DataType = { + if (!resolved) { + throw new UnresolvedException(this, s"Cannot resolve since $children are not resolved") + } + if (str.dataType == BinaryType) str.dataType else StringType + } + + def references = children.flatMap(_.references).toSet + + override def children = str :: pos :: len :: Nil + + @inline + def slice[T, C <: Any](str: C, startPos: Int, sliceLen: Int) + (implicit ev: (C=>IndexedSeqOptimized[T,_])): Any = { + val len = str.length + // Hive and SQL use one-based indexing for SUBSTR arguments but also accept zero and + // negative indices for start positions. If a start index i is greater than 0, it + // refers to element i-1 in the sequence. If a start index i is less than 0, it refers + // to the -ith element before the end of the sequence. If a start index i is 0, it + // refers to the first element. + + val start = startPos match { + case pos if pos > 0 => pos - 1 + case neg if neg < 0 => len + neg + case _ => 0 + } + + val end = sliceLen match { + case max if max == Integer.MAX_VALUE => max + case x => start + x + } + + str.slice(start, end) + } + + override def eval(input: Row): Any = { + val string = str.eval(input) + + val po = pos.eval(input) + val ln = len.eval(input) + + if ((string == null) || (po == null) || (ln == null)) { + null + } else { + val start = po.asInstanceOf[Int] + val length = ln.asInstanceOf[Int] + + string match { + case ba: Array[Byte] => slice(ba, start, length) + case other => slice(other.toString, start, length) + } + } + } + + override def toString = len match { + case max if max == Integer.MAX_VALUE => s"SUBSTR($str, $pos)" + case _ => s"SUBSTR($str, $pos, $len)" + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index f0904f59d028f..5f86d6047cb9c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -123,6 +123,7 @@ object LikeSimplification extends Rule[LogicalPlan] { val startsWith = "([^_%]+)%".r val endsWith = "%([^_%]+)".r val contains = "%([^_%]+)%".r + val equalTo = "([^_%]*)".r def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { case Like(l, Literal(startsWith(pattern), StringType)) if !pattern.endsWith("\\") => @@ -131,6 +132,8 @@ object LikeSimplification extends Rule[LogicalPlan] { EndsWith(l, Literal(pattern)) case Like(l, Literal(contains(pattern), StringType)) if !pattern.endsWith("\\") => Contains(l, Literal(pattern)) + case Like(l, Literal(equalTo(pattern), StringType)) => + EqualTo(l, Literal(pattern)) } } @@ -150,11 +153,15 @@ object NullPropagation extends Rule[LogicalPlan] { case e @ GetItem(Literal(null, _), _) => Literal(null, e.dataType) case e @ GetItem(_, Literal(null, _)) => Literal(null, e.dataType) case e @ GetField(Literal(null, _), _) => Literal(null, e.dataType) - case e @ Coalesce(children) => { - val newChildren = children.filter(c => c match { + case e @ EqualNullSafe(Literal(null, _), r) => IsNull(r) + case e @ EqualNullSafe(l, Literal(null, _)) => IsNull(l) + + // For Coalesce, remove null literals. + case e @ Coalesce(children) => + val newChildren = children.filter { case Literal(null, _) => false case _ => true - }) + } if (newChildren.length == 0) { Literal(null, e.dataType) } else if (newChildren.length == 1) { @@ -162,12 +169,11 @@ object NullPropagation extends Rule[LogicalPlan] { } else { Coalesce(newChildren) } - } - case e @ If(Literal(v, _), trueValue, falseValue) => if (v == true) trueValue else falseValue - case e @ In(Literal(v, _), list) if (list.exists(c => c match { - case Literal(candidate, _) if candidate == v => true - case _ => false - })) => Literal(true, BooleanType) + + case e @ Substring(Literal(null, _), _, _) => Literal(null, e.dataType) + case e @ Substring(_, Literal(null, _), _) => Literal(null, e.dataType) + case e @ Substring(_, _, Literal(null, _)) => Literal(null, e.dataType) + // Put exceptional cases above if any case e: BinaryArithmetic => e.children match { case Literal(null, _) :: right :: Nil => Literal(null, e.dataType) @@ -184,6 +190,11 @@ object NullPropagation extends Rule[LogicalPlan] { case left :: Literal(null, _) :: Nil => Literal(null, e.dataType) case _ => e } + case e: StringComparison => e.children match { + case Literal(null, _) :: right :: Nil => Literal(null, e.dataType) + case left :: Literal(null, _) :: Nil => Literal(null, e.dataType) + case _ => e + } } } } @@ -195,9 +206,19 @@ object NullPropagation extends Rule[LogicalPlan] { object ConstantFolding extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { case q: LogicalPlan => q transformExpressionsDown { - // Skip redundant folding of literals. + // Skip redundant folding of literals. This rule is technically not necessary. Placing this + // here avoids running the next rule for Literal values, which would create a new Literal + // object and running eval unnecessarily. case l: Literal => l + + // Fold expressions that are foldable. case e if e.foldable => Literal(e.eval(null), e.dataType) + + // Fold "literal in (item1, item2, ..., literal, ...)" into true directly. + case In(Literal(v, _), list) if list.exists { + case Literal(candidate, _) if candidate == v => true + case _ => false + } => Literal(true, BooleanType) } } } @@ -227,6 +248,9 @@ object BooleanSimplification extends Rule[LogicalPlan] { case (l, Literal(false, BooleanType)) => l case (_, _) => or } + + // Turn "if (true) a else b" into "a", and if (false) a else b" into "b". + case e @ If(Literal(v, _), trueValue, falseValue) => if (v == true) trueValue else falseValue } } } @@ -248,12 +272,12 @@ object CombineFilters extends Rule[LogicalPlan] { */ object SimplifyFilters extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case Filter(Literal(true, BooleanType), child) => - child - case Filter(Literal(null, _), child) => - LocalRelation(child.output) - case Filter(Literal(false, BooleanType), child) => - LocalRelation(child.output) + // If the filter condition always evaluate to true, remove the filter. + case Filter(Literal(true, BooleanType), child) => child + // If the filter condition always evaluate to null or false, + // replace the input with an empty relation. + case Filter(Literal(null, _), child) => LocalRelation(child.output, data = Seq.empty) + case Filter(Literal(false, BooleanType), child) => LocalRelation(child.output, data = Seq.empty) } } @@ -295,7 +319,7 @@ object PushPredicateThroughJoin extends Rule[LogicalPlan] with PredicateHelper { /** * Splits join condition expressions into three categories based on the attributes required * to evaluate them. - * @returns (canEvaluateInLeft, canEvaluateInRight, haveToEvaluateInBoth) + * @return (canEvaluateInLeft, canEvaluateInRight, haveToEvaluateInBoth) */ private def split(condition: Seq[Expression], left: LogicalPlan, right: LogicalPlan) = { val (leftEvaluateCondition, rest) = diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/package.scala new file mode 100644 index 0000000000000..bdd07bbeb2230 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/package.scala @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +package object catalyst { + /** + * A JVM-global lock that should be used to prevent thread safety issues when using things in + * scala.reflect.*. Note that Scala Reflection API is made thread-safe in 2.11, but not yet for + * 2.10.* builds. See SI-6240 for more details. + */ + protected[catalyst] object ScalaReflectionLock + +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala index 67833664b35ae..5839c9f7c43ef 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.planning -import org.apache.spark.sql.Logging +import org.apache.spark.Logging import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.trees.TreeNode diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala index 026692abe067d..90923fe31a063 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala @@ -19,9 +19,8 @@ package org.apache.spark.sql.catalyst.planning import scala.annotation.tailrec -import org.apache.spark.sql.Logging - import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.Logging import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ @@ -104,6 +103,77 @@ object PhysicalOperation extends PredicateHelper { } } +/** + * Matches a logical aggregation that can be performed on distributed data in two steps. The first + * operates on the data in each partition performing partial aggregation for each group. The second + * occurs after the shuffle and completes the aggregation. + * + * This pattern will only match if all aggregate expressions can be computed partially and will + * return the rewritten aggregation expressions for both phases. + * + * The returned values for this match are as follows: + * - Grouping attributes for the final aggregation. + * - Aggregates for the final aggregation. + * - Grouping expressions for the partial aggregation. + * - Partial aggregate expressions. + * - Input to the aggregation. + */ +object PartialAggregation { + type ReturnType = + (Seq[Attribute], Seq[NamedExpression], Seq[Expression], Seq[NamedExpression], LogicalPlan) + + def unapply(plan: LogicalPlan): Option[ReturnType] = plan match { + case logical.Aggregate(groupingExpressions, aggregateExpressions, child) => + // Collect all aggregate expressions. + val allAggregates = + aggregateExpressions.flatMap(_ collect { case a: AggregateExpression => a}) + // Collect all aggregate expressions that can be computed partially. + val partialAggregates = + aggregateExpressions.flatMap(_ collect { case p: PartialAggregate => p}) + + // Only do partial aggregation if supported by all aggregate expressions. + if (allAggregates.size == partialAggregates.size) { + // Create a map of expressions to their partial evaluations for all aggregate expressions. + val partialEvaluations: Map[Long, SplitEvaluation] = + partialAggregates.map(a => (a.id, a.asPartial)).toMap + + // We need to pass all grouping expressions though so the grouping can happen a second + // time. However some of them might be unnamed so we alias them allowing them to be + // referenced in the second aggregation. + val namedGroupingExpressions: Map[Expression, NamedExpression] = groupingExpressions.map { + case n: NamedExpression => (n, n) + case other => (other, Alias(other, "PartialGroup")()) + }.toMap + + // Replace aggregations with a new expression that computes the result from the already + // computed partial evaluations and grouping values. + val rewrittenAggregateExpressions = aggregateExpressions.map(_.transformUp { + case e: Expression if partialEvaluations.contains(e.id) => + partialEvaluations(e.id).finalEvaluation + case e: Expression if namedGroupingExpressions.contains(e) => + namedGroupingExpressions(e).toAttribute + }).asInstanceOf[Seq[NamedExpression]] + + val partialComputation = + (namedGroupingExpressions.values ++ + partialEvaluations.values.flatMap(_.partialEvaluations)).toSeq + + val namedGroupingAttributes = namedGroupingExpressions.values.map(_.toAttribute).toSeq + + Some( + (namedGroupingAttributes, + rewrittenAggregateExpressions, + groupingExpressions, + partialComputation, + child)) + } else { + None + } + case _ => None + } +} + + /** * A pattern that finds joins with equality conditions that can be evaluated using equi-join. */ @@ -114,7 +184,7 @@ object ExtractEquiJoinKeys extends Logging with PredicateHelper { def unapply(plan: LogicalPlan): Option[ReturnType] = plan match { case join @ Join(left, right, joinType, condition) => - logger.debug(s"Considering join on: $condition") + logDebug(s"Considering join on: $condition") // Find equi-join predicates that can be evaluated before the join, and thus can be used // as join keys. val (joinPredicates, otherPredicates) = @@ -132,7 +202,7 @@ object ExtractEquiJoinKeys extends Logging with PredicateHelper { val rightKeys = joinKeys.map(_._2) if (joinKeys.nonEmpty) { - logger.debug(s"leftKeys:${leftKeys} | rightKeys:${rightKeys}") + logDebug(s"leftKeys:${leftKeys} | rightKeys:${rightKeys}") Some((joinType, leftKeys, rightKeys, otherPredicates.reduceOption(And), left, right)) } else { None diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala index 7b82e19b2e714..0988b0c6d990c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala @@ -125,51 +125,10 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy }.toSeq } - protected def generateSchemaString(schema: Seq[Attribute]): String = { - val builder = new StringBuilder - builder.append("root\n") - val prefix = " |" - schema.foreach { attribute => - val name = attribute.name - val dataType = attribute.dataType - dataType match { - case fields: StructType => - builder.append(s"$prefix-- $name: $StructType\n") - generateSchemaString(fields, s"$prefix |", builder) - case ArrayType(fields: StructType) => - builder.append(s"$prefix-- $name: $ArrayType[$StructType]\n") - generateSchemaString(fields, s"$prefix |", builder) - case ArrayType(elementType: DataType) => - builder.append(s"$prefix-- $name: $ArrayType[$elementType]\n") - case _ => builder.append(s"$prefix-- $name: $dataType\n") - } - } - - builder.toString() - } - - protected def generateSchemaString( - schema: StructType, - prefix: String, - builder: StringBuilder): StringBuilder = { - schema.fields.foreach { - case StructField(name, fields: StructType, _) => - builder.append(s"$prefix-- $name: $StructType\n") - generateSchemaString(fields, s"$prefix |", builder) - case StructField(name, ArrayType(fields: StructType), _) => - builder.append(s"$prefix-- $name: $ArrayType[$StructType]\n") - generateSchemaString(fields, s"$prefix |", builder) - case StructField(name, ArrayType(elementType: DataType), _) => - builder.append(s"$prefix-- $name: $ArrayType[$elementType]\n") - case StructField(name, fieldType: DataType, _) => - builder.append(s"$prefix-- $name: $fieldType\n") - } - - builder - } + def schema: StructType = StructType.fromAttributes(output) /** Returns the output schema in the tree format. */ - def schemaString: String = generateSchemaString(output) + def schemaString: String = schema.treeString /** Prints out the schema in the tree format */ def printSchema(): Unit = println(schemaString) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index edc37e3877c0e..278569f0cb14a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -26,6 +26,25 @@ import org.apache.spark.sql.catalyst.trees abstract class LogicalPlan extends QueryPlan[LogicalPlan] { self: Product => + /** + * Estimates of various statistics. The default estimation logic simply lazily multiplies the + * corresponding statistic produced by the children. To override this behavior, override + * `statistics` and assign it an overriden version of `Statistics`. + * + * '''NOTE''': concrete and/or overriden versions of statistics fields should pay attention to the + * performance of the implementations. The reason is that estimations might get triggered in + * performance-critical processes, such as query plan planning. + * + * @param sizeInBytes Physical size in bytes. For leaf operators this defaults to 1, otherwise it + * defaults to the product of children's `sizeInBytes`. + */ + case class Statistics( + sizeInBytes: BigInt + ) + lazy val statistics: Statistics = Statistics( + sizeInBytes = children.map(_.statistics).map(_.sizeInBytes).product + ) + /** * Returns the set of attributes that are referenced by this node * during evaluation. @@ -53,16 +72,29 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] { def childrenResolved: Boolean = !children.exists(!_.resolved) /** - * Optionally resolves the given string to a [[NamedExpression]]. The attribute is expressed as + * Optionally resolves the given string to a [[NamedExpression]] using the input from all child + * nodes of this LogicalPlan. The attribute is expressed as * as string in the following form: `[scope].AttributeName.[nested].[fields]...`. */ - def resolve(name: String): Option[NamedExpression] = { + def resolveChildren(name: String): Option[NamedExpression] = + resolve(name, children.flatMap(_.output)) + + /** + * Optionally resolves the given string to a [[NamedExpression]] based on the output of this + * LogicalPlan. The attribute is expressed as string in the following form: + * `[scope].AttributeName.[nested].[fields]...`. + */ + def resolve(name: String): Option[NamedExpression] = + resolve(name, output) + + /** Performs attribute resolution given a name and a sequence of possible attributes. */ + protected def resolve(name: String, input: Seq[Attribute]): Option[NamedExpression] = { val parts = name.split("\\.") // Collect all attributes that are output by this nodes children where either the first part // matches the name or where the first part matches the scope and the second part matches the // name. Return these matches along with any remaining parts, which represent dotted access to // struct fields. - val options = children.flatMap(_.output).flatMap { option => + val options = input.flatMap { option => // If the first part of the desired name matches a qualifier for this possible match, drop it. val remainingParts = if (option.qualifiers.contains(parts.head) && parts.size > 1) parts.drop(1) else parts @@ -70,15 +102,15 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] { } options.distinct match { - case (a, Nil) :: Nil => Some(a) // One match, no nested fields, use it. + case Seq((a, Nil)) => Some(a) // One match, no nested fields, use it. // One match, but we also need to extract the requested nested field. - case (a, nestedFields) :: Nil => + case Seq((a, nestedFields)) => a.dataType match { case StructType(fields) => Some(Alias(nestedFields.foldLeft(a: Expression)(GetField), nestedFields.last)()) case _ => None // Don't know how to resolve these field references } - case Nil => None // No matches. + case Seq() => None // No matches. case ambiguousReferences => throw new TreeNodeException( this, s"Ambiguous references to $name: ${ambiguousReferences.mkString(",")}") @@ -92,6 +124,9 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] { abstract class LeafNode extends LogicalPlan with trees.LeafNode[LogicalPlan] { self: Product => + override lazy val statistics: Statistics = + throw new UnsupportedOperationException(s"LeafNode $nodeName must implement statistics.") + // Leaf nodes by definition cannot reference any input attributes. override def references = Set.empty } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index 1537de259c5b4..3cb407217c4c3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -177,7 +177,7 @@ case class LowerCaseSchema(child: LogicalPlan) extends UnaryNode { case StructType(fields) => StructType(fields.map(f => StructField(f.name.toLowerCase(), lowerCaseSchema(f.dataType), f.nullable))) - case ArrayType(elemType) => ArrayType(lowerCaseSchema(elemType)) + case ArrayType(elemType, containsNull) => ArrayType(lowerCaseSchema(elemType), containsNull) case otherType => otherType } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala index 1d5f033f0d274..481a5a4f212b2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala @@ -35,7 +35,7 @@ abstract class Command extends LeafNode { */ case class NativeCommand(cmd: String) extends Command { override def output = - Seq(BoundReference(0, AttributeReference("result", StringType, nullable = false)())) + Seq(AttributeReference("result", StringType, nullable = false)()) } /** @@ -43,8 +43,7 @@ case class NativeCommand(cmd: String) extends Command { */ case class SetCommand(key: Option[String], value: Option[String]) extends Command { override def output = Seq( - BoundReference(0, AttributeReference("key", StringType, nullable = false)()), - BoundReference(1, AttributeReference("value", StringType, nullable = false)())) + AttributeReference("", StringType, nullable = false)()) } /** @@ -53,7 +52,7 @@ case class SetCommand(key: Option[String], value: Option[String]) extends Comman */ case class ExplainCommand(plan: LogicalPlan) extends Command { override def output = - Seq(BoundReference(0, AttributeReference("plan", StringType, nullable = false)())) + Seq(AttributeReference("plan", StringType, nullable = false)()) } /** @@ -72,7 +71,7 @@ case class DescribeCommand( isExtended: Boolean) extends Command { override def output = Seq( // Column names are based on Hive. - BoundReference(0, AttributeReference("col_name", StringType, nullable = false)()), - BoundReference(1, AttributeReference("data_type", StringType, nullable = false)()), - BoundReference(2, AttributeReference("comment", StringType, nullable = false)())) + AttributeReference("col_name", StringType, nullable = false)(), + AttributeReference("data_type", StringType, nullable = false)(), + AttributeReference("comment", StringType, nullable = false)()) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/Rule.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/Rule.scala index 1076537bc7602..03414b2301e81 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/Rule.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/Rule.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.rules -import org.apache.spark.sql.Logging +import org.apache.spark.Logging import org.apache.spark.sql.catalyst.trees.TreeNode abstract class Rule[TreeType <: TreeNode[_]] extends Logging { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala index e32adb76fe146..d192b151ac1c3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala @@ -15,10 +15,9 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package rules +package org.apache.spark.sql.catalyst.rules +import org.apache.spark.Logging import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.catalyst.util.sideBySide @@ -61,7 +60,7 @@ abstract class RuleExecutor[TreeType <: TreeNode[_]] extends Logging { case (plan, rule) => val result = rule(plan) if (!result.fastEquals(plan)) { - logger.trace( + logTrace( s""" |=== Applying Rule ${rule.ruleName} === |${sideBySide(plan.treeString, result.treeString).mkString("\n")} @@ -72,25 +71,28 @@ abstract class RuleExecutor[TreeType <: TreeNode[_]] extends Logging { } iteration += 1 if (iteration > batch.strategy.maxIterations) { - logger.info(s"Max iterations ($iteration) reached for batch ${batch.name}") + // Only log if this is a rule that is supposed to run more than once. + if (iteration != 2) { + logInfo(s"Max iterations (${iteration - 1}) reached for batch ${batch.name}") + } continue = false } if (curPlan.fastEquals(lastPlan)) { - logger.trace(s"Fixed point reached for batch ${batch.name} after $iteration iterations.") + logTrace(s"Fixed point reached for batch ${batch.name} after $iteration iterations.") continue = false } lastPlan = curPlan } if (!batchStartPlan.fastEquals(curPlan)) { - logger.debug( + logDebug( s""" |=== Result of Batch ${batch.name} === |${sideBySide(plan.treeString, curPlan.treeString).mkString("\n")} """.stripMargin) } else { - logger.trace(s"Batch ${batch.name} has no effect.") + logTrace(s"Batch ${batch.name} has no effect.") } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala index d159ecdd5d781..d725a92c06f7b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst -import org.apache.spark.sql.Logger +import org.apache.spark.Logging /** * A library for easily manipulating trees of operators. Operators that extend TreeNode are @@ -33,7 +33,8 @@ import org.apache.spark.sql.Logger *
  • debugging support - pretty printing, easy splicing of trees, etc.
  • * */ -package object trees { +package object trees extends Logging { // Since we want tree nodes to be lightweight, we create one logger for all treenode instances. - protected val logger = Logger("catalyst.trees") + protected override def logName = "catalyst.trees" + } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala index bb77bccf86176..b52ee6d3378a3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala @@ -19,16 +19,16 @@ package org.apache.spark.sql.catalyst.types import java.sql.Timestamp -import scala.util.parsing.combinator.RegexParsers - import scala.reflect.ClassTag import scala.reflect.runtime.universe.{typeTag, TypeTag, runtimeMirror} +import scala.util.parsing.combinator.RegexParsers +import org.apache.spark.sql.catalyst.ScalaReflectionLock import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Expression} import org.apache.spark.util.Utils /** - * + * Utility functions for working with DataTypes. */ object DataType extends RegexParsers { protected lazy val primitiveType: Parser[DataType] = @@ -45,11 +45,13 @@ object DataType extends RegexParsers { "TimestampType" ^^^ TimestampType protected lazy val arrayType: Parser[DataType] = - "ArrayType" ~> "(" ~> dataType <~ ")" ^^ ArrayType + "ArrayType" ~> "(" ~> dataType ~ "," ~ boolVal <~ ")" ^^ { + case tpe ~ _ ~ containsNull => ArrayType(tpe, containsNull) + } protected lazy val mapType: Parser[DataType] = - "MapType" ~> "(" ~> dataType ~ "," ~ dataType <~ ")" ^^ { - case t1 ~ _ ~ t2 => MapType(t1, t2) + "MapType" ~> "(" ~> dataType ~ "," ~ dataType ~ "," ~ boolVal <~ ")" ^^ { + case t1 ~ _ ~ t2 ~ _ ~ valueContainsNull => MapType(t1, t2, valueContainsNull) } protected lazy val structField: Parser[StructField] = @@ -62,7 +64,6 @@ object DataType extends RegexParsers { "true" ^^^ true | "false" ^^^ false - protected lazy val structType: Parser[DataType] = "StructType\\([A-zA-z]*\\(".r ~> repsep(structField, ",") <~ "))" ^^ { case fields => new StructType(fields) @@ -83,6 +84,21 @@ object DataType extends RegexParsers { case Success(result, _) => result case failure: NoSuccess => sys.error(s"Unsupported dataType: $asString, $failure") } + + protected[types] def buildFormattedString( + dataType: DataType, + prefix: String, + builder: StringBuilder): Unit = { + dataType match { + case array: ArrayType => + array.buildFormattedString(prefix, builder) + case struct: StructType => + struct.buildFormattedString(prefix, builder) + case map: MapType => + map.buildFormattedString(prefix, builder) + case _ => + } + } } abstract class DataType { @@ -93,47 +109,65 @@ abstract class DataType { } def isPrimitive: Boolean = false + + def simpleString: String +} + +case object NullType extends DataType { + def simpleString: String = "null" } -case object NullType extends DataType +object NativeType { + def all = Seq( + IntegerType, BooleanType, LongType, DoubleType, FloatType, ShortType, ByteType, StringType) + + def unapply(dt: DataType): Boolean = all.contains(dt) +} trait PrimitiveType extends DataType { override def isPrimitive = true } abstract class NativeType extends DataType { - type JvmType - @transient val tag: TypeTag[JvmType] - val ordering: Ordering[JvmType] + private[sql] type JvmType + @transient private[sql] val tag: TypeTag[JvmType] + private[sql] val ordering: Ordering[JvmType] - @transient val classTag = { + @transient private[sql] val classTag = ScalaReflectionLock.synchronized { val mirror = runtimeMirror(Utils.getSparkClassLoader) ClassTag[JvmType](mirror.runtimeClass(tag.tpe)) } } case object StringType extends NativeType with PrimitiveType { - type JvmType = String - @transient lazy val tag = typeTag[JvmType] - val ordering = implicitly[Ordering[JvmType]] + private[sql] type JvmType = String + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } + private[sql] val ordering = implicitly[Ordering[JvmType]] + def simpleString: String = "string" } + case object BinaryType extends DataType with PrimitiveType { - type JvmType = Array[Byte] + private[sql] type JvmType = Array[Byte] + def simpleString: String = "binary" } + case object BooleanType extends NativeType with PrimitiveType { - type JvmType = Boolean - @transient lazy val tag = typeTag[JvmType] - val ordering = implicitly[Ordering[JvmType]] + private[sql] type JvmType = Boolean + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } + private[sql] val ordering = implicitly[Ordering[JvmType]] + def simpleString: String = "boolean" } case object TimestampType extends NativeType { - type JvmType = Timestamp + private[sql] type JvmType = Timestamp - @transient lazy val tag = typeTag[JvmType] + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } - val ordering = new Ordering[JvmType] { + private[sql] val ordering = new Ordering[JvmType] { def compare(x: Timestamp, y: Timestamp) = x.compareTo(y) } + + def simpleString: String = "timestamp" } abstract class NumericType extends NativeType with PrimitiveType { @@ -142,7 +176,11 @@ abstract class NumericType extends NativeType with PrimitiveType { // type parameter and and add a numeric annotation (i.e., [JvmType : Numeric]). This gets // desugared by the compiler into an argument to the objects constructor. This means there is no // longer an no argument constructor and thus the JVM cannot serialize the object anymore. - val numeric: Numeric[JvmType] + private[sql] val numeric: Numeric[JvmType] +} + +object NumericType { + def unapply(e: Expression): Boolean = e.dataType.isInstanceOf[NumericType] } /** Matcher for any expressions that evaluate to [[IntegralType]]s */ @@ -154,39 +192,43 @@ object IntegralType { } abstract class IntegralType extends NumericType { - val integral: Integral[JvmType] + private[sql] val integral: Integral[JvmType] } case object LongType extends IntegralType { - type JvmType = Long - @transient lazy val tag = typeTag[JvmType] - val numeric = implicitly[Numeric[Long]] - val integral = implicitly[Integral[Long]] - val ordering = implicitly[Ordering[JvmType]] + private[sql] type JvmType = Long + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } + private[sql] val numeric = implicitly[Numeric[Long]] + private[sql] val integral = implicitly[Integral[Long]] + private[sql] val ordering = implicitly[Ordering[JvmType]] + def simpleString: String = "long" } case object IntegerType extends IntegralType { - type JvmType = Int - @transient lazy val tag = typeTag[JvmType] - val numeric = implicitly[Numeric[Int]] - val integral = implicitly[Integral[Int]] - val ordering = implicitly[Ordering[JvmType]] + private[sql] type JvmType = Int + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } + private[sql] val numeric = implicitly[Numeric[Int]] + private[sql] val integral = implicitly[Integral[Int]] + private[sql] val ordering = implicitly[Ordering[JvmType]] + def simpleString: String = "integer" } case object ShortType extends IntegralType { - type JvmType = Short - @transient lazy val tag = typeTag[JvmType] - val numeric = implicitly[Numeric[Short]] - val integral = implicitly[Integral[Short]] - val ordering = implicitly[Ordering[JvmType]] + private[sql] type JvmType = Short + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } + private[sql] val numeric = implicitly[Numeric[Short]] + private[sql] val integral = implicitly[Integral[Short]] + private[sql] val ordering = implicitly[Ordering[JvmType]] + def simpleString: String = "short" } case object ByteType extends IntegralType { - type JvmType = Byte - @transient lazy val tag = typeTag[JvmType] - val numeric = implicitly[Numeric[Byte]] - val integral = implicitly[Integral[Byte]] - val ordering = implicitly[Ordering[JvmType]] + private[sql] type JvmType = Byte + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } + private[sql] val numeric = implicitly[Numeric[Byte]] + private[sql] val integral = implicitly[Integral[Byte]] + private[sql] val ordering = implicitly[Ordering[JvmType]] + def simpleString: String = "byte" } /** Matcher for any expressions that evaluate to [[FractionalType]]s */ @@ -197,47 +239,159 @@ object FractionalType { } } abstract class FractionalType extends NumericType { - val fractional: Fractional[JvmType] + private[sql] val fractional: Fractional[JvmType] } case object DecimalType extends FractionalType { - type JvmType = BigDecimal - @transient lazy val tag = typeTag[JvmType] - val numeric = implicitly[Numeric[BigDecimal]] - val fractional = implicitly[Fractional[BigDecimal]] - val ordering = implicitly[Ordering[JvmType]] + private[sql] type JvmType = BigDecimal + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } + private[sql] val numeric = implicitly[Numeric[BigDecimal]] + private[sql] val fractional = implicitly[Fractional[BigDecimal]] + private[sql] val ordering = implicitly[Ordering[JvmType]] + def simpleString: String = "decimal" } case object DoubleType extends FractionalType { - type JvmType = Double - @transient lazy val tag = typeTag[JvmType] - val numeric = implicitly[Numeric[Double]] - val fractional = implicitly[Fractional[Double]] - val ordering = implicitly[Ordering[JvmType]] + private[sql] type JvmType = Double + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } + private[sql] val numeric = implicitly[Numeric[Double]] + private[sql] val fractional = implicitly[Fractional[Double]] + private[sql] val ordering = implicitly[Ordering[JvmType]] + def simpleString: String = "double" } case object FloatType extends FractionalType { - type JvmType = Float - @transient lazy val tag = typeTag[JvmType] - val numeric = implicitly[Numeric[Float]] - val fractional = implicitly[Fractional[Float]] - val ordering = implicitly[Ordering[JvmType]] + private[sql] type JvmType = Float + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } + private[sql] val numeric = implicitly[Numeric[Float]] + private[sql] val fractional = implicitly[Fractional[Float]] + private[sql] val ordering = implicitly[Ordering[JvmType]] + def simpleString: String = "float" } -case class ArrayType(elementType: DataType) extends DataType +object ArrayType { + /** Construct a [[ArrayType]] object with the given element type. The `containsNull` is false. */ + def apply(elementType: DataType): ArrayType = ArrayType(elementType, false) +} -case class StructField(name: String, dataType: DataType, nullable: Boolean) +/** + * The data type for collections of multiple values. + * Internally these are represented as columns that contain a ``scala.collection.Seq``. + * + * @param elementType The data type of values. + * @param containsNull Indicates if values have `null` values + */ +case class ArrayType(elementType: DataType, containsNull: Boolean) extends DataType { + private[sql] def buildFormattedString(prefix: String, builder: StringBuilder): Unit = { + builder.append( + s"${prefix}-- element: ${elementType.simpleString} (containsNull = ${containsNull})\n") + DataType.buildFormattedString(elementType, s"$prefix |", builder) + } + + def simpleString: String = "array" +} + +/** + * A field inside a StructType. + * @param name The name of this field. + * @param dataType The data type of this field. + * @param nullable Indicates if values of this field can be `null` values. + */ +case class StructField(name: String, dataType: DataType, nullable: Boolean) { + + private[sql] def buildFormattedString(prefix: String, builder: StringBuilder): Unit = { + builder.append(s"${prefix}-- ${name}: ${dataType.simpleString} (nullable = ${nullable})\n") + DataType.buildFormattedString(dataType, s"$prefix |", builder) + } +} object StructType { - def fromAttributes(attributes: Seq[Attribute]): StructType = { + protected[sql] def fromAttributes(attributes: Seq[Attribute]): StructType = StructType(attributes.map(a => StructField(a.name, a.dataType, a.nullable))) - } - // def apply(fields: Seq[StructField]) = new StructType(fields.toIndexedSeq) + private def validateFields(fields: Seq[StructField]): Boolean = + fields.map(field => field.name).distinct.size == fields.size } case class StructType(fields: Seq[StructField]) extends DataType { - def toAttributes = fields.map(f => AttributeReference(f.name, f.dataType, f.nullable)()) + require(StructType.validateFields(fields), "Found fields with the same name.") + + /** + * Returns all field names in a [[Seq]]. + */ + lazy val fieldNames: Seq[String] = fields.map(_.name) + private lazy val fieldNamesSet: Set[String] = fieldNames.toSet + private lazy val nameToField: Map[String, StructField] = fields.map(f => f.name -> f).toMap + /** + * Extracts a [[StructField]] of the given name. If the [[StructType]] object does not + * have a name matching the given name, `null` will be returned. + */ + def apply(name: String): StructField = { + nameToField.get(name).getOrElse( + throw new IllegalArgumentException(s"Field ${name} does not exist.")) + } + + /** + * Returns a [[StructType]] containing [[StructField]]s of the given names. + * Those names which do not have matching fields will be ignored. + */ + def apply(names: Set[String]): StructType = { + val nonExistFields = names -- fieldNamesSet + if (!nonExistFields.isEmpty) { + throw new IllegalArgumentException( + s"Field ${nonExistFields.mkString(",")} does not exist.") + } + // Preserve the original order of fields. + StructType(fields.filter(f => names.contains(f.name))) + } + + protected[sql] def toAttributes = + fields.map(f => AttributeReference(f.name, f.dataType, f.nullable)()) + + def treeString: String = { + val builder = new StringBuilder + builder.append("root\n") + val prefix = " |" + fields.foreach(field => field.buildFormattedString(prefix, builder)) + + builder.toString() + } + + def printTreeString(): Unit = println(treeString) + + private[sql] def buildFormattedString(prefix: String, builder: StringBuilder): Unit = { + fields.foreach(field => field.buildFormattedString(prefix, builder)) + } + + def simpleString: String = "struct" } -case class MapType(keyType: DataType, valueType: DataType) extends DataType +object MapType { + /** + * Construct a [[MapType]] object with the given key type and value type. + * The `valueContainsNull` is true. + */ + def apply(keyType: DataType, valueType: DataType): MapType = + MapType(keyType: DataType, valueType: DataType, true) +} + +/** + * The data type for Maps. Keys in a map are not allowed to have `null` values. + * @param keyType The data type of map keys. + * @param valueType The data type of map values. + * @param valueContainsNull Indicates if map values have `null` values. + */ +case class MapType( + keyType: DataType, + valueType: DataType, + valueContainsNull: Boolean) extends DataType { + private[sql] def buildFormattedString(prefix: String, builder: StringBuilder): Unit = { + builder.append(s"${prefix}-- key: ${keyType.simpleString}\n") + builder.append(s"${prefix}-- value: ${valueType.simpleString} " + + s"(valueContainsNull = ${valueContainsNull})\n") + DataType.buildFormattedString(keyType, s"$prefix |", builder) + DataType.buildFormattedString(valueType, s"$prefix |", builder) + } + + def simpleString: String = "map" +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala index c0438dbe52a47..e75373d5a74a7 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala @@ -17,11 +17,11 @@ package org.apache.spark.sql.catalyst +import java.math.BigInteger import java.sql.Timestamp import org.scalatest.FunSuite -import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.types._ case class PrimitiveData( @@ -148,4 +148,68 @@ class ScalaReflectionSuite extends FunSuite { StructField("_2", StringType, nullable = true))), nullable = true)) } + + test("get data type of a value") { + // BooleanType + assert(BooleanType === typeOfObject(true)) + assert(BooleanType === typeOfObject(false)) + + // BinaryType + assert(BinaryType === typeOfObject("string".getBytes)) + + // StringType + assert(StringType === typeOfObject("string")) + + // ByteType + assert(ByteType === typeOfObject(127.toByte)) + + // ShortType + assert(ShortType === typeOfObject(32767.toShort)) + + // IntegerType + assert(IntegerType === typeOfObject(2147483647)) + + // LongType + assert(LongType === typeOfObject(9223372036854775807L)) + + // FloatType + assert(FloatType === typeOfObject(3.4028235E38.toFloat)) + + // DoubleType + assert(DoubleType === typeOfObject(1.7976931348623157E308)) + + // DecimalType + assert(DecimalType === typeOfObject(BigDecimal("1.7976931348623157E318"))) + + // TimestampType + assert(TimestampType === typeOfObject(java.sql.Timestamp.valueOf("2014-07-25 10:26:00"))) + + // NullType + assert(NullType === typeOfObject(null)) + + def typeOfObject1: PartialFunction[Any, DataType] = typeOfObject orElse { + case value: java.math.BigInteger => DecimalType + case value: java.math.BigDecimal => DecimalType + case _ => StringType + } + + assert(DecimalType === typeOfObject1( + new BigInteger("92233720368547758070"))) + assert(DecimalType === typeOfObject1( + new java.math.BigDecimal("1.7976931348623157E318"))) + assert(StringType === typeOfObject1(BigInt("92233720368547758070"))) + + def typeOfObject2: PartialFunction[Any, DataType] = typeOfObject orElse { + case value: java.math.BigInteger => DecimalType + } + + intercept[MatchError](typeOfObject2(BigInt("92233720368547758070"))) + + def typeOfObject3: PartialFunction[Any, DataType] = typeOfObject orElse { + case c: Seq[_] => ArrayType(typeOfObject3(c.head)) + } + + assert(ArrayType(IntegerType) === typeOfObject3(Seq(1, 2, 3))) + assert(ArrayType(ArrayType(IntegerType)) === typeOfObject3(Seq(Seq(1,2,3)))) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala index 84d72814778ba..999c9fff38d60 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala @@ -29,7 +29,11 @@ import org.apache.spark.sql.catalyst.dsl.expressions._ class ExpressionEvaluationSuite extends FunSuite { test("literals") { - assert((Literal(1) + Literal(1)).eval(null) === 2) + checkEvaluation(Literal(1), 1) + checkEvaluation(Literal(true), true) + checkEvaluation(Literal(0L), 0L) + checkEvaluation(Literal("test"), "test") + checkEvaluation(Literal(1) + Literal(1), 2) } /** @@ -61,10 +65,8 @@ class ExpressionEvaluationSuite extends FunSuite { test("3VL Not") { notTrueTable.foreach { case (v, answer) => - val expr = ! Literal(v, BooleanType) - val result = expr.eval(null) - if (result != answer) - fail(s"$expr should not evaluate to $result, expected: $answer") } + checkEvaluation(!Literal(v, BooleanType), answer) + } } booleanLogicTest("AND", _ && _, @@ -127,6 +129,13 @@ class ExpressionEvaluationSuite extends FunSuite { } } + test("IN") { + checkEvaluation(In(Literal(1), Seq(Literal(1), Literal(2))), true) + checkEvaluation(In(Literal(2), Seq(Literal(1), Literal(2))), true) + checkEvaluation(In(Literal(3), Seq(Literal(1), Literal(2))), false) + checkEvaluation(In(Literal(1), Seq(Literal(1), Literal(2))) && In(Literal(2), Seq(Literal(1), Literal(2))), true) + } + test("LIKE literal Regular Expression") { checkEvaluation(Literal(null, StringType).like("a"), null) checkEvaluation(Literal("a", StringType).like(Literal(null, StringType)), null) @@ -158,7 +167,7 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation("abc" like regEx, true, new GenericRow(Array[Any]("a%"))) checkEvaluation("abc" like regEx, false, new GenericRow(Array[Any]("b%"))) checkEvaluation("abc" like regEx, false, new GenericRow(Array[Any]("bc%"))) - + checkEvaluation(Literal(null, StringType) like regEx, null, new GenericRow(Array[Any]("bc%"))) } @@ -203,7 +212,7 @@ class ExpressionEvaluationSuite extends FunSuite { test("data type casting") { - val sts = "1970-01-01 00:00:01.0" + val sts = "1970-01-01 00:00:01.1" val ts = Timestamp.valueOf(sts) checkEvaluation("abdef" cast StringType, "abdef") @@ -232,21 +241,21 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(Literal(false) cast IntegerType, 0) checkEvaluation(Cast(Literal(1) cast BooleanType, IntegerType), 1) checkEvaluation(Cast(Literal(0) cast BooleanType, IntegerType), 0) - checkEvaluation("23" cast DoubleType, 23) + checkEvaluation("23" cast DoubleType, 23d) checkEvaluation("23" cast IntegerType, 23) - checkEvaluation("23" cast FloatType, 23) - checkEvaluation("23" cast DecimalType, 23) - checkEvaluation("23" cast ByteType, 23) - checkEvaluation("23" cast ShortType, 23) + checkEvaluation("23" cast FloatType, 23f) + checkEvaluation("23" cast DecimalType, 23: BigDecimal) + checkEvaluation("23" cast ByteType, 23.toByte) + checkEvaluation("23" cast ShortType, 23.toShort) checkEvaluation("2012-12-11" cast DoubleType, null) checkEvaluation(Literal(123) cast IntegerType, 123) - checkEvaluation(Literal(23d) + Cast(true, DoubleType), 24) + checkEvaluation(Literal(23d) + Cast(true, DoubleType), 24d) checkEvaluation(Literal(23) + Cast(true, IntegerType), 24) - checkEvaluation(Literal(23f) + Cast(true, FloatType), 24) - checkEvaluation(Literal(BigDecimal(23)) + Cast(true, DecimalType), 24) - checkEvaluation(Literal(23.toByte) + Cast(true, ByteType), 24) - checkEvaluation(Literal(23.toShort) + Cast(true, ShortType), 24) + checkEvaluation(Literal(23f) + Cast(true, FloatType), 24f) + checkEvaluation(Literal(BigDecimal(23)) + Cast(true, DecimalType), 24: BigDecimal) + checkEvaluation(Literal(23.toByte) + Cast(true, ByteType), 24.toByte) + checkEvaluation(Literal(23.toShort) + Cast(true, ShortType), 24.toShort) intercept[Exception] {evaluate(Literal(1) cast BinaryType, null)} @@ -293,7 +302,7 @@ class ExpressionEvaluationSuite extends FunSuite { // A test for higher precision than millis checkEvaluation(Cast(Cast(0.00000001, TimestampType), DoubleType), 0.00000001) } - + test("null checking") { val row = new GenericRow(Array[Any]("^Ba*n", null, true, null)) val c1 = 'a.string.at(0) @@ -301,18 +310,18 @@ class ExpressionEvaluationSuite extends FunSuite { val c3 = 'a.boolean.at(2) val c4 = 'a.boolean.at(3) - checkEvaluation(IsNull(c1), false, row) - checkEvaluation(IsNotNull(c1), true, row) + checkEvaluation(c1.isNull, false, row) + checkEvaluation(c1.isNotNull, true, row) - checkEvaluation(IsNull(c2), true, row) - checkEvaluation(IsNotNull(c2), false, row) + checkEvaluation(c2.isNull, true, row) + checkEvaluation(c2.isNotNull, false, row) - checkEvaluation(IsNull(Literal(1, ShortType)), false) - checkEvaluation(IsNotNull(Literal(1, ShortType)), true) + checkEvaluation(Literal(1, ShortType).isNull, false) + checkEvaluation(Literal(1, ShortType).isNotNull, true) + + checkEvaluation(Literal(null, ShortType).isNull, true) + checkEvaluation(Literal(null, ShortType).isNotNull, false) - checkEvaluation(IsNull(Literal(null, ShortType)), true) - checkEvaluation(IsNotNull(Literal(null, ShortType)), false) - checkEvaluation(Coalesce(c1 :: c2 :: Nil), "^Ba*n", row) checkEvaluation(Coalesce(Literal(null, StringType) :: Nil), null, row) checkEvaluation(Coalesce(Literal(null, StringType) :: c1 :: c2 :: Nil), "^Ba*n", row) @@ -323,14 +332,14 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(If(Literal(null, BooleanType), c2, c1), "^Ba*n", row) checkEvaluation(If(Literal(true, BooleanType), c1, c2), "^Ba*n", row) checkEvaluation(If(Literal(false, BooleanType), c2, c1), "^Ba*n", row) - checkEvaluation(If(Literal(false, BooleanType), + checkEvaluation(If(Literal(false, BooleanType), Literal("a", StringType), Literal("b", StringType)), "b", row) - checkEvaluation(In(c1, c1 :: c2 :: Nil), true, row) - checkEvaluation(In(Literal("^Ba*n", StringType), - Literal("^Ba*n", StringType) :: Nil), true, row) - checkEvaluation(In(Literal("^Ba*n", StringType), - Literal("^Ba*n", StringType) :: c2 :: Nil), true, row) + checkEvaluation(c1 in (c1, c2), true, row) + checkEvaluation( + Literal("^Ba*n", StringType) in (Literal("^Ba*n", StringType)), true, row) + checkEvaluation( + Literal("^Ba*n", StringType) in (Literal("^Ba*n", StringType), c2), true, row) } test("case when") { @@ -378,7 +387,7 @@ class ExpressionEvaluationSuite extends FunSuite { test("complex type") { val row = new GenericRow(Array[Any]( - "^Ba*n", // 0 + "^Ba*n", // 0 null.asInstanceOf[String], // 1 new GenericRow(Array[Any]("aa", "bb")), // 2 Map("aa"->"bb"), // 3 @@ -391,21 +400,21 @@ class ExpressionEvaluationSuite extends FunSuite { val typeMap = MapType(StringType, StringType) val typeArray = ArrayType(StringType) - checkEvaluation(GetItem(BoundReference(3, AttributeReference("c", typeMap)()), + checkEvaluation(GetItem(BoundReference(3, typeMap, true), Literal("aa")), "bb", row) checkEvaluation(GetItem(Literal(null, typeMap), Literal("aa")), null, row) checkEvaluation(GetItem(Literal(null, typeMap), Literal(null, StringType)), null, row) - checkEvaluation(GetItem(BoundReference(3, AttributeReference("c", typeMap)()), + checkEvaluation(GetItem(BoundReference(3, typeMap, true), Literal(null, StringType)), null, row) - checkEvaluation(GetItem(BoundReference(4, AttributeReference("c", typeArray)()), + checkEvaluation(GetItem(BoundReference(4, typeArray, true), Literal(1)), "bb", row) checkEvaluation(GetItem(Literal(null, typeArray), Literal(1)), null, row) checkEvaluation(GetItem(Literal(null, typeArray), Literal(null, IntegerType)), null, row) - checkEvaluation(GetItem(BoundReference(4, AttributeReference("c", typeArray)()), + checkEvaluation(GetItem(BoundReference(4, typeArray, true), Literal(null, IntegerType)), null, row) - checkEvaluation(GetField(BoundReference(2, AttributeReference("c", typeS)()), "a"), "aa", row) + checkEvaluation(GetField(BoundReference(2, typeS, nullable = true), "a"), "aa", row) checkEvaluation(GetField(Literal(null, typeS), "a"), null, row) val typeS_notNullable = StructType( @@ -413,13 +422,15 @@ class ExpressionEvaluationSuite extends FunSuite { :: StructField("b", StringType, nullable = false) :: Nil ) - assert(GetField(BoundReference(2, - AttributeReference("c", typeS)()), "a").nullable === true) - assert(GetField(BoundReference(2, - AttributeReference("c", typeS_notNullable, nullable = false)()), "a").nullable === false) + assert(GetField(BoundReference(2,typeS, nullable = true), "a").nullable === true) + assert(GetField(BoundReference(2, typeS_notNullable, nullable = false), "a").nullable === false) assert(GetField(Literal(null, typeS), "a").nullable === true) assert(GetField(Literal(null, typeS_notNullable), "a").nullable === true) + + checkEvaluation('c.map(typeMap).at(3).getItem("aa"), "bb", row) + checkEvaluation('c.array(typeArray.elementType).at(4).getItem(1), "bb", row) + checkEvaluation('c.struct(typeS).at(2).getField("a"), "aa", row) } test("arithmetic") { @@ -447,11 +458,13 @@ class ExpressionEvaluationSuite extends FunSuite { } test("BinaryComparison") { - val row = new GenericRow(Array[Any](1, 2, 3, null)) + val row = new GenericRow(Array[Any](1, 2, 3, null, 3, null)) val c1 = 'a.int.at(0) val c2 = 'a.int.at(1) val c3 = 'a.int.at(2) val c4 = 'a.int.at(3) + val c5 = 'a.int.at(4) + val c6 = 'a.int.at(5) checkEvaluation(LessThan(c1, c4), null, row) checkEvaluation(LessThan(c1, c2), true, row) @@ -465,6 +478,93 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(c1 >= c2, false, row) checkEvaluation(c1 === c2, false, row) checkEvaluation(c1 !== c2, true, row) + checkEvaluation(c4 <=> c1, false, row) + checkEvaluation(c1 <=> c4, false, row) + checkEvaluation(c4 <=> c6, true, row) + checkEvaluation(c3 <=> c5, true, row) + checkEvaluation(Literal(true) <=> Literal(null, BooleanType), false, row) + checkEvaluation(Literal(null, BooleanType) <=> Literal(true), false, row) } -} + test("StringComparison") { + val row = new GenericRow(Array[Any]("abc", null)) + val c1 = 'a.string.at(0) + val c2 = 'a.string.at(1) + + checkEvaluation(c1 contains "b", true, row) + checkEvaluation(c1 contains "x", false, row) + checkEvaluation(c2 contains "b", null, row) + checkEvaluation(c1 contains Literal(null, StringType), null, row) + + checkEvaluation(c1 startsWith "a", true, row) + checkEvaluation(c1 startsWith "b", false, row) + checkEvaluation(c2 startsWith "a", null, row) + checkEvaluation(c1 startsWith Literal(null, StringType), null, row) + + checkEvaluation(c1 endsWith "c", true, row) + checkEvaluation(c1 endsWith "b", false, row) + checkEvaluation(c2 endsWith "b", null, row) + checkEvaluation(c1 endsWith Literal(null, StringType), null, row) + } + + test("Substring") { + val row = new GenericRow(Array[Any]("example", "example".toArray.map(_.toByte))) + + val s = 'a.string.at(0) + + // substring from zero position with less-than-full length + checkEvaluation(Substring(s, Literal(0, IntegerType), Literal(2, IntegerType)), "ex", row) + checkEvaluation(Substring(s, Literal(1, IntegerType), Literal(2, IntegerType)), "ex", row) + + // substring from zero position with full length + checkEvaluation(Substring(s, Literal(0, IntegerType), Literal(7, IntegerType)), "example", row) + checkEvaluation(Substring(s, Literal(1, IntegerType), Literal(7, IntegerType)), "example", row) + + // substring from zero position with greater-than-full length + checkEvaluation(Substring(s, Literal(0, IntegerType), Literal(100, IntegerType)), "example", row) + checkEvaluation(Substring(s, Literal(1, IntegerType), Literal(100, IntegerType)), "example", row) + + // substring from nonzero position with less-than-full length + checkEvaluation(Substring(s, Literal(2, IntegerType), Literal(2, IntegerType)), "xa", row) + + // substring from nonzero position with full length + checkEvaluation(Substring(s, Literal(2, IntegerType), Literal(6, IntegerType)), "xample", row) + + // substring from nonzero position with greater-than-full length + checkEvaluation(Substring(s, Literal(2, IntegerType), Literal(100, IntegerType)), "xample", row) + + // zero-length substring (within string bounds) + checkEvaluation(Substring(s, Literal(0, IntegerType), Literal(0, IntegerType)), "", row) + + // zero-length substring (beyond string bounds) + checkEvaluation(Substring(s, Literal(100, IntegerType), Literal(4, IntegerType)), "", row) + + // substring(null, _, _) -> null + checkEvaluation(Substring(s, Literal(100, IntegerType), Literal(4, IntegerType)), null, new GenericRow(Array[Any](null))) + + // substring(_, null, _) -> null + checkEvaluation(Substring(s, Literal(null, IntegerType), Literal(4, IntegerType)), null, row) + + // substring(_, _, null) -> null + checkEvaluation(Substring(s, Literal(100, IntegerType), Literal(null, IntegerType)), null, row) + + // 2-arg substring from zero position + checkEvaluation(Substring(s, Literal(0, IntegerType), Literal(Integer.MAX_VALUE, IntegerType)), "example", row) + checkEvaluation(Substring(s, Literal(1, IntegerType), Literal(Integer.MAX_VALUE, IntegerType)), "example", row) + + // 2-arg substring from nonzero position + checkEvaluation(Substring(s, Literal(2, IntegerType), Literal(Integer.MAX_VALUE, IntegerType)), "xample", row) + + val s_notNull = 'a.string.notNull.at(0) + + assert(Substring(s, Literal(0, IntegerType), Literal(2, IntegerType)).nullable === true) + assert(Substring(s_notNull, Literal(0, IntegerType), Literal(2, IntegerType)).nullable === false) + assert(Substring(s_notNull, Literal(null, IntegerType), Literal(2, IntegerType)).nullable === true) + assert(Substring(s_notNull, Literal(0, IntegerType), Literal(null, IntegerType)).nullable === true) + + checkEvaluation(s.substr(0, 2), "ex", row) + checkEvaluation(s.substr(0), "example", row) + checkEvaluation(s.substring(0, 2), "ex", row) + checkEvaluation(s.substring(0), "example", row) + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedEvaluationSuite.scala new file mode 100644 index 0000000000000..245a2e148030c --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedEvaluationSuite.scala @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.optimizer + +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.expressions.codegen._ + +/** + * Overrides our expression evaluation tests to use code generation for evaluation. + */ +class GeneratedEvaluationSuite extends ExpressionEvaluationSuite { + override def checkEvaluation( + expression: Expression, + expected: Any, + inputRow: Row = EmptyRow): Unit = { + val plan = try { + GenerateMutableProjection(Alias(expression, s"Optimized($expression)")() :: Nil)() + } catch { + case e: Throwable => + val evaluated = GenerateProjection.expressionEvaluator(expression) + fail( + s""" + |Code generation of $expression failed: + |${evaluated.code.mkString("\n")} + |$e + """.stripMargin) + } + + val actual = plan(inputRow).apply(0) + if(actual != expected) { + val input = if(inputRow == EmptyRow) "" else s", input: $inputRow" + fail(s"Incorrect Evaluation: $expression, actual: $actual, expected: $expected$input") + } + } + + + test("multithreaded eval") { + import scala.concurrent._ + import ExecutionContext.Implicits.global + import scala.concurrent.duration._ + + val futures = (1 to 20).map { _ => + future { + GeneratePredicate(EqualTo(Literal(1), Literal(1))) + GenerateProjection(EqualTo(Literal(1), Literal(1)) :: Nil) + GenerateMutableProjection(EqualTo(Literal(1), Literal(1)) :: Nil) + GenerateOrdering(Add(Literal(1), Literal(1)).asc :: Nil) + } + } + + futures.foreach(Await.result(_, 10.seconds)) + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedMutableEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedMutableEvaluationSuite.scala new file mode 100644 index 0000000000000..887aabb1d5fb4 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedMutableEvaluationSuite.scala @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.optimizer + +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.expressions.codegen._ + +/** + * Overrides our expression evaluation tests to use generated code on mutable rows. + */ +class GeneratedMutableEvaluationSuite extends ExpressionEvaluationSuite { + override def checkEvaluation( + expression: Expression, + expected: Any, + inputRow: Row = EmptyRow): Unit = { + lazy val evaluated = GenerateProjection.expressionEvaluator(expression) + + val plan = try { + GenerateProjection(Alias(expression, s"Optimized($expression)")() :: Nil) + } catch { + case e: Throwable => + fail( + s""" + |Code generation of $expression failed: + |${evaluated.code.mkString("\n")} + |$e + """.stripMargin) + } + + val actual = plan(inputRow) + val expectedRow = new GenericRow(Array[Any](expected)) + if (actual.hashCode() != expectedRow.hashCode()) { + fail( + s""" + |Mismatched hashCodes for values: $actual, $expectedRow + |Hash Codes: ${actual.hashCode()} != ${expectedRow.hashCode()} + |${evaluated.code.mkString("\n")} + """.stripMargin) + } + if (actual != expectedRow) { + val input = if(inputRow == EmptyRow) "" else s", input: $inputRow" + fail(s"Incorrect Evaluation: $expression, actual: $actual, expected: $expected$input") + } + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CombiningLimitsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CombiningLimitsSuite.scala index 4896f1b955f01..e2ae0d25db1a5 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CombiningLimitsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CombiningLimitsSuite.scala @@ -27,9 +27,9 @@ class CombiningLimitsSuite extends PlanTest { object Optimize extends RuleExecutor[LogicalPlan] { val batches = - Batch("Combine Limit", FixedPoint(2), + Batch("Combine Limit", FixedPoint(10), CombineLimits) :: - Batch("Constant Folding", FixedPoint(3), + Batch("Constant Folding", FixedPoint(10), NullPropagation, ConstantFolding, BooleanSimplification) :: Nil diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala index 0ff82064012a8..0a27cce337482 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala @@ -83,7 +83,7 @@ class ConstantFoldingSuite extends PlanTest { Literal(10) as Symbol("2*3+4"), Literal(14) as Symbol("2*(3+4)")) .where(Literal(true)) - .groupBy(Literal(3))(Literal(3) as Symbol("9/3")) + .groupBy(Literal(3.0))(Literal(3.0) as Symbol("9/3")) .analyze comparePlans(optimized, correctAnswer) @@ -201,7 +201,15 @@ class ConstantFoldingSuite extends PlanTest { Like(Literal(null, StringType), "abc") as 'c13, Like("abc", Literal(null, StringType)) as 'c14, - Upper(Literal(null, StringType)) as 'c15) + Upper(Literal(null, StringType)) as 'c15, + + Substring(Literal(null, StringType), 0, 1) as 'c16, + Substring("abc", Literal(null, IntegerType), 1) as 'c17, + Substring("abc", 0, Literal(null, IntegerType)) as 'c18, + + Contains(Literal(null, StringType), "abc") as 'c19, + Contains("abc", Literal(null, StringType)) as 'c20 + ) val optimized = Optimize(originalQuery.analyze) @@ -228,8 +236,15 @@ class ConstantFoldingSuite extends PlanTest { Literal(null, BooleanType) as 'c13, Literal(null, BooleanType) as 'c14, - Literal(null, StringType) as 'c15) - .analyze + Literal(null, StringType) as 'c15, + + Literal(null, StringType) as 'c16, + Literal(null, StringType) as 'c17, + Literal(null, StringType) as 'c18, + + Literal(null, BooleanType) as 'c19, + Literal(null, BooleanType) as 'c20 + ).analyze comparePlans(optimized, correctAnswer) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LikeSimplificationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LikeSimplificationSuite.scala new file mode 100644 index 0000000000000..b10577c8001e2 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LikeSimplificationSuite.scala @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.optimizer + +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.plans.PlanTest +import org.apache.spark.sql.catalyst.rules._ + +/* Implicit conversions */ +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.dsl.plans._ + +class LikeSimplificationSuite extends PlanTest { + + object Optimize extends RuleExecutor[LogicalPlan] { + val batches = + Batch("Like Simplification", Once, + LikeSimplification) :: Nil + } + + val testRelation = LocalRelation('a.string) + + test("simplify Like into StartsWith") { + val originalQuery = + testRelation + .where(('a like "abc%") || ('a like "abc\\%")) + + val optimized = Optimize(originalQuery.analyze) + val correctAnswer = testRelation + .where(StartsWith('a, "abc") || ('a like "abc\\%")) + .analyze + + comparePlans(optimized, correctAnswer) + } + + test("simplify Like into EndsWith") { + val originalQuery = + testRelation + .where('a like "%xyz") + + val optimized = Optimize(originalQuery.analyze) + val correctAnswer = testRelation + .where(EndsWith('a, "xyz")) + .analyze + + comparePlans(optimized, correctAnswer) + } + + test("simplify Like into Contains") { + val originalQuery = + testRelation + .where(('a like "%mn%") || ('a like "%mn\\%")) + + val optimized = Optimize(originalQuery.analyze) + val correctAnswer = testRelation + .where(Contains('a, "mn") || ('a like "%mn\\%")) + .analyze + + comparePlans(optimized, correctAnswer) + } + + test("simplify Like into EqualTo") { + val originalQuery = + testRelation + .where(('a like "") || ('a like "abc")) + + val optimized = Optimize(originalQuery.analyze) + val correctAnswer = testRelation + .where(('a === "") || ('a === "abc")) + .analyze + + comparePlans(optimized, correctAnswer) + } +} diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 8210fd1f210d1..c8016e41256d5 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -31,6 +31,9 @@ jar Spark Project SQL http://spark.apache.org/ + + sql + @@ -65,6 +68,11 @@ jackson-databind 2.3.0 + + junit + junit + test + org.scalatest scalatest_${scala.binary.version} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/ArrayType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/ArrayType.java new file mode 100644 index 0000000000000..b73a371e93001 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/ArrayType.java @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +/** + * The data type representing Lists. + * An ArrayType object comprises two fields, {@code DataType elementType} and + * {@code boolean containsNull}. The field of {@code elementType} is used to specify the type of + * array elements. The field of {@code containsNull} is used to specify if the array has + * {@code null} values. + * + * To create an {@link ArrayType}, + * {@link DataType#createArrayType(DataType)} or + * {@link DataType#createArrayType(DataType, boolean)} + * should be used. + */ +public class ArrayType extends DataType { + private DataType elementType; + private boolean containsNull; + + protected ArrayType(DataType elementType, boolean containsNull) { + this.elementType = elementType; + this.containsNull = containsNull; + } + + public DataType getElementType() { + return elementType; + } + + public boolean isContainsNull() { + return containsNull; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + ArrayType arrayType = (ArrayType) o; + + if (containsNull != arrayType.containsNull) return false; + if (!elementType.equals(arrayType.elementType)) return false; + + return true; + } + + @Override + public int hashCode() { + int result = elementType.hashCode(); + result = 31 * result + (containsNull ? 1 : 0); + return result; + } +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/BinaryType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/BinaryType.java new file mode 100644 index 0000000000000..7daad60f62a0b --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/BinaryType.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +/** + * The data type representing byte[] values. + * + * {@code BinaryType} is represented by the singleton object {@link DataType#BinaryType}. + */ +public class BinaryType extends DataType { + protected BinaryType() {} +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/BooleanType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/BooleanType.java new file mode 100644 index 0000000000000..5a1f52725631b --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/BooleanType.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +/** + * The data type representing boolean and Boolean values. + * + * {@code BooleanType} is represented by the singleton object {@link DataType#BooleanType}. + */ +public class BooleanType extends DataType { + protected BooleanType() {} +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/ByteType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/ByteType.java new file mode 100644 index 0000000000000..e5cdf06b21bbe --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/ByteType.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +/** + * The data type representing byte and Byte values. + * + * {@code ByteType} is represented by the singleton object {@link DataType#ByteType}. + */ +public class ByteType extends DataType { + protected ByteType() {} +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/DataType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/DataType.java new file mode 100644 index 0000000000000..3eccddef88134 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/DataType.java @@ -0,0 +1,190 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +/** + * The base type of all Spark SQL data types. + * + * To get/create specific data type, users should use singleton objects and factory methods + * provided by this class. + */ +public abstract class DataType { + + /** + * Gets the StringType object. + */ + public static final StringType StringType = new StringType(); + + /** + * Gets the BinaryType object. + */ + public static final BinaryType BinaryType = new BinaryType(); + + /** + * Gets the BooleanType object. + */ + public static final BooleanType BooleanType = new BooleanType(); + + /** + * Gets the TimestampType object. + */ + public static final TimestampType TimestampType = new TimestampType(); + + /** + * Gets the DecimalType object. + */ + public static final DecimalType DecimalType = new DecimalType(); + + /** + * Gets the DoubleType object. + */ + public static final DoubleType DoubleType = new DoubleType(); + + /** + * Gets the FloatType object. + */ + public static final FloatType FloatType = new FloatType(); + + /** + * Gets the ByteType object. + */ + public static final ByteType ByteType = new ByteType(); + + /** + * Gets the IntegerType object. + */ + public static final IntegerType IntegerType = new IntegerType(); + + /** + * Gets the LongType object. + */ + public static final LongType LongType = new LongType(); + + /** + * Gets the ShortType object. + */ + public static final ShortType ShortType = new ShortType(); + + /** + * Creates an ArrayType by specifying the data type of elements ({@code elementType}). + * The field of {@code containsNull} is set to {@code false}. + */ + public static ArrayType createArrayType(DataType elementType) { + if (elementType == null) { + throw new IllegalArgumentException("elementType should not be null."); + } + + return new ArrayType(elementType, false); + } + + /** + * Creates an ArrayType by specifying the data type of elements ({@code elementType}) and + * whether the array contains null values ({@code containsNull}). + */ + public static ArrayType createArrayType(DataType elementType, boolean containsNull) { + if (elementType == null) { + throw new IllegalArgumentException("elementType should not be null."); + } + + return new ArrayType(elementType, containsNull); + } + + /** + * Creates a MapType by specifying the data type of keys ({@code keyType}) and values + * ({@code keyType}). The field of {@code valueContainsNull} is set to {@code true}. + */ + public static MapType createMapType(DataType keyType, DataType valueType) { + if (keyType == null) { + throw new IllegalArgumentException("keyType should not be null."); + } + if (valueType == null) { + throw new IllegalArgumentException("valueType should not be null."); + } + + return new MapType(keyType, valueType, true); + } + + /** + * Creates a MapType by specifying the data type of keys ({@code keyType}), the data type of + * values ({@code keyType}), and whether values contain any null value + * ({@code valueContainsNull}). + */ + public static MapType createMapType( + DataType keyType, + DataType valueType, + boolean valueContainsNull) { + if (keyType == null) { + throw new IllegalArgumentException("keyType should not be null."); + } + if (valueType == null) { + throw new IllegalArgumentException("valueType should not be null."); + } + + return new MapType(keyType, valueType, valueContainsNull); + } + + /** + * Creates a StructField by specifying the name ({@code name}), data type ({@code dataType}) and + * whether values of this field can be null values ({@code nullable}). + */ + public static StructField createStructField(String name, DataType dataType, boolean nullable) { + if (name == null) { + throw new IllegalArgumentException("name should not be null."); + } + if (dataType == null) { + throw new IllegalArgumentException("dataType should not be null."); + } + + return new StructField(name, dataType, nullable); + } + + /** + * Creates a StructType with the given list of StructFields ({@code fields}). + */ + public static StructType createStructType(List fields) { + return createStructType(fields.toArray(new StructField[0])); + } + + /** + * Creates a StructType with the given StructField array ({@code fields}). + */ + public static StructType createStructType(StructField[] fields) { + if (fields == null) { + throw new IllegalArgumentException("fields should not be null."); + } + Set distinctNames = new HashSet(); + for (StructField field: fields) { + if (field == null) { + throw new IllegalArgumentException( + "fields should not contain any null."); + } + + distinctNames.add(field.getName()); + } + if (distinctNames.size() != fields.length) { + throw new IllegalArgumentException("fields should have distinct names."); + } + + return new StructType(fields); + } + +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/DecimalType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/DecimalType.java new file mode 100644 index 0000000000000..bc54c078d7a4e --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/DecimalType.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +/** + * The data type representing java.math.BigDecimal values. + * + * {@code DecimalType} is represented by the singleton object {@link DataType#DecimalType}. + */ +public class DecimalType extends DataType { + protected DecimalType() {} +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/DoubleType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/DoubleType.java new file mode 100644 index 0000000000000..f0060d0bcf9f5 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/DoubleType.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +/** + * The data type representing double and Double values. + * + * {@code DoubleType} is represented by the singleton object {@link DataType#DoubleType}. + */ +public class DoubleType extends DataType { + protected DoubleType() {} +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/FloatType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/FloatType.java new file mode 100644 index 0000000000000..4a6a37f69176a --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/FloatType.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +/** + * The data type representing float and Float values. + * + * {@code FloatType} is represented by the singleton object {@link DataType#FloatType}. + */ +public class FloatType extends DataType { + protected FloatType() {} +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/IntegerType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/IntegerType.java new file mode 100644 index 0000000000000..bfd70490bbbbb --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/IntegerType.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +/** + * The data type representing int and Integer values. + * + * {@code IntegerType} is represented by the singleton object {@link DataType#IntegerType}. + */ +public class IntegerType extends DataType { + protected IntegerType() {} +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/LongType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/LongType.java new file mode 100644 index 0000000000000..af13a46eb165c --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/LongType.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +/** + * The data type representing long and Long values. + * + * {@code LongType} is represented by the singleton object {@link DataType#LongType}. + */ +public class LongType extends DataType { + protected LongType() {} +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/MapType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/MapType.java new file mode 100644 index 0000000000000..063e6b34abc48 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/MapType.java @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +/** + * The data type representing Maps. A MapType object comprises two fields, + * {@code DataType keyType}, {@code DataType valueType}, and {@code boolean valueContainsNull}. + * The field of {@code keyType} is used to specify the type of keys in the map. + * The field of {@code valueType} is used to specify the type of values in the map. + * The field of {@code valueContainsNull} is used to specify if map values have + * {@code null} values. + * For values of a MapType column, keys are not allowed to have {@code null} values. + * + * To create a {@link MapType}, + * {@link DataType#createMapType(DataType, DataType)} or + * {@link DataType#createMapType(DataType, DataType, boolean)} + * should be used. + */ +public class MapType extends DataType { + private DataType keyType; + private DataType valueType; + private boolean valueContainsNull; + + protected MapType(DataType keyType, DataType valueType, boolean valueContainsNull) { + this.keyType = keyType; + this.valueType = valueType; + this.valueContainsNull = valueContainsNull; + } + + public DataType getKeyType() { + return keyType; + } + + public DataType getValueType() { + return valueType; + } + + public boolean isValueContainsNull() { + return valueContainsNull; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + MapType mapType = (MapType) o; + + if (valueContainsNull != mapType.valueContainsNull) return false; + if (!keyType.equals(mapType.keyType)) return false; + if (!valueType.equals(mapType.valueType)) return false; + + return true; + } + + @Override + public int hashCode() { + int result = keyType.hashCode(); + result = 31 * result + valueType.hashCode(); + result = 31 * result + (valueContainsNull ? 1 : 0); + return result; + } +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/ShortType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/ShortType.java new file mode 100644 index 0000000000000..7d7604b4e3d2d --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/ShortType.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +/** + * The data type representing short and Short values. + * + * {@code ShortType} is represented by the singleton object {@link DataType#ShortType}. + */ +public class ShortType extends DataType { + protected ShortType() {} +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/StringType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/StringType.java new file mode 100644 index 0000000000000..f4ba0c07c9c6e --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/StringType.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +/** + * The data type representing String values. + * + * {@code StringType} is represented by the singleton object {@link DataType#StringType}. + */ +public class StringType extends DataType { + protected StringType() {} +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/StructField.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/StructField.java new file mode 100644 index 0000000000000..b48e2a2c5f953 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/StructField.java @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +/** + * A StructField object represents a field in a StructType object. + * A StructField object comprises three fields, {@code String name}, {@code DataType dataType}, + * and {@code boolean nullable}. The field of {@code name} is the name of a StructField. + * The field of {@code dataType} specifies the data type of a StructField. + * The field of {@code nullable} specifies if values of a StructField can contain {@code null} + * values. + * + * To create a {@link StructField}, + * {@link DataType#createStructField(String, DataType, boolean)} + * should be used. + */ +public class StructField { + private String name; + private DataType dataType; + private boolean nullable; + + protected StructField(String name, DataType dataType, boolean nullable) { + this.name = name; + this.dataType = dataType; + this.nullable = nullable; + } + + public String getName() { + return name; + } + + public DataType getDataType() { + return dataType; + } + + public boolean isNullable() { + return nullable; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + StructField that = (StructField) o; + + if (nullable != that.nullable) return false; + if (!dataType.equals(that.dataType)) return false; + if (!name.equals(that.name)) return false; + + return true; + } + + @Override + public int hashCode() { + int result = name.hashCode(); + result = 31 * result + dataType.hashCode(); + result = 31 * result + (nullable ? 1 : 0); + return result; + } +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/StructType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/StructType.java new file mode 100644 index 0000000000000..a4b501efd9a10 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/StructType.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.util.Arrays; + +/** + * The data type representing Rows. + * A StructType object comprises an array of StructFields. + * + * To create an {@link StructType}, + * {@link DataType#createStructType(java.util.List)} or + * {@link DataType#createStructType(StructField[])} + * should be used. + */ +public class StructType extends DataType { + private StructField[] fields; + + protected StructType(StructField[] fields) { + this.fields = fields; + } + + public StructField[] getFields() { + return fields; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + StructType that = (StructType) o; + + if (!Arrays.equals(fields, that.fields)) return false; + + return true; + } + + @Override + public int hashCode() { + return Arrays.hashCode(fields); + } +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/TimestampType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/TimestampType.java new file mode 100644 index 0000000000000..06d44c731cdfe --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/TimestampType.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +/** + * The data type representing java.sql.Timestamp values. + * + * {@code TimestampType} is represented by the singleton object {@link DataType#TimestampType}. + */ +public class TimestampType extends DataType { + protected TimestampType() {} +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF1.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF1.java new file mode 100644 index 0000000000000..ef959e35e1027 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF1.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 1 arguments. + */ +public interface UDF1 extends Serializable { + public R call(T1 t1) throws Exception; +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF10.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF10.java new file mode 100644 index 0000000000000..96ab3a96c3d5e --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF10.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 10 arguments. + */ +public interface UDF10 extends Serializable { + public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10) throws Exception; +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF11.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF11.java new file mode 100644 index 0000000000000..58ae8edd6d817 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF11.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 11 arguments. + */ +public interface UDF11 extends Serializable { + public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11) throws Exception; +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF12.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF12.java new file mode 100644 index 0000000000000..d9da0f6eddd94 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF12.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 12 arguments. + */ +public interface UDF12 extends Serializable { + public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12) throws Exception; +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF13.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF13.java new file mode 100644 index 0000000000000..095fc1a8076b5 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF13.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 13 arguments. + */ +public interface UDF13 extends Serializable { + public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13) throws Exception; +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF14.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF14.java new file mode 100644 index 0000000000000..eb27eaa180086 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF14.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 14 arguments. + */ +public interface UDF14 extends Serializable { + public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14) throws Exception; +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF15.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF15.java new file mode 100644 index 0000000000000..1fbcff56332b6 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF15.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 15 arguments. + */ +public interface UDF15 extends Serializable { + public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15) throws Exception; +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF16.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF16.java new file mode 100644 index 0000000000000..1133561787a69 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF16.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 16 arguments. + */ +public interface UDF16 extends Serializable { + public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15, T16 t16) throws Exception; +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF17.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF17.java new file mode 100644 index 0000000000000..dfae7922c9b63 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF17.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 17 arguments. + */ +public interface UDF17 extends Serializable { + public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15, T16 t16, T17 t17) throws Exception; +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF18.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF18.java new file mode 100644 index 0000000000000..e9d1c6d52d4ea --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF18.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 18 arguments. + */ +public interface UDF18 extends Serializable { + public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15, T16 t16, T17 t17, T18 t18) throws Exception; +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF19.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF19.java new file mode 100644 index 0000000000000..46b9d2d3c9457 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF19.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 19 arguments. + */ +public interface UDF19 extends Serializable { + public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15, T16 t16, T17 t17, T18 t18, T19 t19) throws Exception; +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF2.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF2.java new file mode 100644 index 0000000000000..cd3fde8da419e --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF2.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 2 arguments. + */ +public interface UDF2 extends Serializable { + public R call(T1 t1, T2 t2) throws Exception; +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF20.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF20.java new file mode 100644 index 0000000000000..113d3d26be4a7 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF20.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 20 arguments. + */ +public interface UDF20 extends Serializable { + public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15, T16 t16, T17 t17, T18 t18, T19 t19, T20 t20) throws Exception; +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF21.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF21.java new file mode 100644 index 0000000000000..74118f2cf8da7 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF21.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 21 arguments. + */ +public interface UDF21 extends Serializable { + public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15, T16 t16, T17 t17, T18 t18, T19 t19, T20 t20, T21 t21) throws Exception; +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF22.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF22.java new file mode 100644 index 0000000000000..0e7cc40be45ec --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF22.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 22 arguments. + */ +public interface UDF22 extends Serializable { + public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15, T16 t16, T17 t17, T18 t18, T19 t19, T20 t20, T21 t21, T22 t22) throws Exception; +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF3.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF3.java new file mode 100644 index 0000000000000..6a880f16be47a --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF3.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 3 arguments. + */ +public interface UDF3 extends Serializable { + public R call(T1 t1, T2 t2, T3 t3) throws Exception; +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF4.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF4.java new file mode 100644 index 0000000000000..fcad2febb18e6 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF4.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 4 arguments. + */ +public interface UDF4 extends Serializable { + public R call(T1 t1, T2 t2, T3 t3, T4 t4) throws Exception; +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF5.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF5.java new file mode 100644 index 0000000000000..ce0cef43a2144 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF5.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 5 arguments. + */ +public interface UDF5 extends Serializable { + public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5) throws Exception; +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF6.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF6.java new file mode 100644 index 0000000000000..f56b806684e61 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF6.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 6 arguments. + */ +public interface UDF6 extends Serializable { + public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6) throws Exception; +} diff --git a/examples/src/main/scala/org/apache/spark/examples/pythonconverters/HBaseConverter.scala b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF7.java similarity index 63% rename from examples/src/main/scala/org/apache/spark/examples/pythonconverters/HBaseConverter.scala rename to sql/core/src/main/java/org/apache/spark/sql/api/java/UDF7.java index 42ae960bd64a1..25bd6d3241bd4 100644 --- a/examples/src/main/scala/org/apache/spark/examples/pythonconverters/HBaseConverter.scala +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF7.java @@ -15,19 +15,18 @@ * limitations under the License. */ -package org.apache.spark.examples.pythonconverters +package org.apache.spark.sql.api.java; -import org.apache.spark.api.python.Converter -import org.apache.hadoop.hbase.client.Result -import org.apache.hadoop.hbase.util.Bytes +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** /** - * Implementation of [[org.apache.spark.api.python.Converter]] that converts a HBase Result - * to a String + * A Spark SQL UDF that has 7 arguments. */ -class HBaseConverter extends Converter[Any, String] { - override def convert(obj: Any): String = { - val result = obj.asInstanceOf[Result] - Bytes.toStringBinary(result.value()) - } +public interface UDF7 extends Serializable { + public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF8.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF8.java new file mode 100644 index 0000000000000..a3b7ac5f94ce7 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF8.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 8 arguments. + */ +public interface UDF8 extends Serializable { + public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8) throws Exception; +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF9.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF9.java new file mode 100644 index 0000000000000..205e72a1522fc --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF9.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 9 arguments. + */ +public interface UDF9 extends Serializable { + public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9) throws Exception; +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/package-info.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/package-info.java similarity index 95% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/package-info.java rename to sql/core/src/main/java/org/apache/spark/sql/api/java/package-info.java index 53603614518f5..67007a9f0d1a3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/package-info.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/package-info.java @@ -18,4 +18,4 @@ /** * Allows the execution of relational queries, including those expressed in SQL using Spark. */ -package org.apache.spark.sql; \ No newline at end of file +package org.apache.spark.sql.api.java; diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala index b6fb46a3acc03..90de11182e605 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala @@ -17,76 +17,138 @@ package org.apache.spark.sql +import scala.collection.immutable +import scala.collection.JavaConversions._ + import java.util.Properties -import scala.collection.JavaConverters._ + +private[spark] object SQLConf { + val COMPRESS_CACHED = "spark.sql.inMemoryColumnarStorage.compressed" + val COLUMN_BATCH_SIZE = "spark.sql.inMemoryColumnarStorage.batchSize" + val AUTO_BROADCASTJOIN_THRESHOLD = "spark.sql.autoBroadcastJoinThreshold" + val DEFAULT_SIZE_IN_BYTES = "spark.sql.defaultSizeInBytes" + val SHUFFLE_PARTITIONS = "spark.sql.shuffle.partitions" + val CODEGEN_ENABLED = "spark.sql.codegen" + val DIALECT = "spark.sql.dialect" + val PARQUET_BINARY_AS_STRING = "spark.sql.parquet.binaryAsString" + + object Deprecated { + val MAPRED_REDUCE_TASKS = "mapred.reduce.tasks" + } +} /** - * SQLConf holds mutable config parameters and hints. These can be set and - * queried either by passing SET commands into Spark SQL's DSL - * functions (sql(), hql(), etc.), or by programmatically using setters and - * getters of this class. + * A trait that enables the setting and getting of mutable config parameters/hints. * - * SQLConf is thread-safe (internally synchronized so safe to be used in multiple threads). + * In the presence of a SQLContext, these can be set and queried by passing SET commands + * into Spark SQL's query functions (i.e. sql()). Otherwise, users of this trait can + * modify the hints by programmatically calling the setters and getters of this trait. + * + * SQLConf is thread-safe (internally synchronized, so safe to be used in multiple threads). */ trait SQLConf { + import SQLConf._ + + @transient protected[spark] val settings = java.util.Collections.synchronizedMap( + new java.util.HashMap[String, String]()) /** ************************ Spark SQL Params/Hints ******************* */ // TODO: refactor so that these hints accessors don't pollute the name space of SQLContext? + /** + * The SQL dialect that is used when parsing queries. This defaults to 'sql' which uses + * a simple SQL parser provided by Spark SQL. This is currently the only option for users of + * SQLContext. + * + * When using a HiveContext, this value defaults to 'hiveql', which uses the Hive 0.12.0 HiveQL + * parser. Users can change this to 'sql' if they want to run queries that aren't supported by + * HiveQL (e.g., SELECT 1). + * + * Note that the choice of dialect does not affect things like what tables are available or + * how query execution is performed. + */ + private[spark] def dialect: String = getConf(DIALECT, "sql") + + /** When true tables cached using the in-memory columnar caching will be compressed. */ + private[spark] def useCompression: Boolean = getConf(COMPRESS_CACHED, "false").toBoolean + + /** The number of rows that will be */ + private[spark] def columnBatchSize: Int = getConf(COLUMN_BATCH_SIZE, "1000").toInt + /** Number of partitions to use for shuffle operators. */ - private[spark] def numShufflePartitions: Int = get("spark.sql.shuffle.partitions", "200").toInt + private[spark] def numShufflePartitions: Int = getConf(SHUFFLE_PARTITIONS, "200").toInt + + /** + * When set to true, Spark SQL will use the Scala compiler at runtime to generate custom bytecode + * that evaluates expressions found in queries. In general this custom code runs much faster + * than interpreted evaluation, but there are significant start-up costs due to compilation. + * As a result codegen is only benificial when queries run for a long time, or when the same + * expressions are used multiple times. + * + * Defaults to false as this feature is currently experimental. + */ + private[spark] def codegenEnabled: Boolean = getConf(CODEGEN_ENABLED, "false").toBoolean /** * Upper bound on the sizes (in bytes) of the tables qualified for the auto conversion to - * a broadcast value during the physical executions of join operations. Setting this to 0 + * a broadcast value during the physical executions of join operations. Setting this to -1 * effectively disables auto conversion. - * Hive setting: hive.auto.convert.join.noconditionaltask.size. + * + * Hive setting: hive.auto.convert.join.noconditionaltask.size, whose default value is also 10000. */ - private[spark] def autoConvertJoinSize: Int = - get("spark.sql.auto.convert.join.size", "10000").toInt + private[spark] def autoBroadcastJoinThreshold: Int = + getConf(AUTO_BROADCASTJOIN_THRESHOLD, "10000").toInt - /** A comma-separated list of table names marked to be broadcasted during joins. */ - private[spark] def joinBroadcastTables: String = get("spark.sql.join.broadcastTables", "") + /** + * The default size in bytes to assign to a logical operator's estimation statistics. By default, + * it is set to a larger value than `autoConvertJoinSize`, hence any logical operator without a + * properly implemented estimation of this statistic will not be incorrectly broadcasted in joins. + */ + private[spark] def defaultSizeInBytes: Long = + getConf(DEFAULT_SIZE_IN_BYTES, (autoBroadcastJoinThreshold + 1).toString).toLong - /** ********************** SQLConf functionality methods ************ */ + /** + * When set to true, we always treat byte arrays in Parquet files as strings. + */ + private[spark] def isParquetBinaryAsString: Boolean = + getConf(PARQUET_BINARY_AS_STRING, "false").toBoolean - @transient - protected[sql] val settings = java.util.Collections.synchronizedMap( - new java.util.HashMap[String, String]()) + /** ********************** SQLConf functionality methods ************ */ - def set(props: Properties): Unit = { - props.asScala.foreach { case (k, v) => this.settings.put(k, v) } + /** Set Spark SQL configuration properties. */ + def setConf(props: Properties): Unit = settings.synchronized { + props.foreach { case (k, v) => settings.put(k, v) } } - def set(key: String, value: String): Unit = { + /** Set the given Spark SQL configuration property. */ + def setConf(key: String, value: String): Unit = { require(key != null, "key cannot be null") - require(value != null, s"value cannot be null for ${key}") + require(value != null, s"value cannot be null for key: $key") settings.put(key, value) } - def get(key: String): String = { + /** Return the value of Spark SQL configuration property for the given key. */ + def getConf(key: String): String = { Option(settings.get(key)).getOrElse(throw new NoSuchElementException(key)) } - def get(key: String, defaultValue: String): String = { + /** + * Return the value of Spark SQL configuration property for the given key. If the key is not set + * yet, return `defaultValue`. + */ + def getConf(key: String, defaultValue: String): String = { Option(settings.get(key)).getOrElse(defaultValue) } - def getAll: Array[(String, String)] = settings.synchronized { settings.asScala.toArray } - - def getOption(key: String): Option[String] = Option(settings.get(key)) - - def contains(key: String): Boolean = settings.containsKey(key) - - def toDebugString: String = { - settings.synchronized { - settings.asScala.toArray.sorted.map{ case (k, v) => s"$k=$v" }.mkString("\n") - } - } + /** + * Return all the configuration properties that have been set (i.e. not the default). + * This creates a new copy of the config properties in the form of a Map. + */ + def getAllConfs: immutable.Map[String, String] = settings.synchronized { settings.toMap } private[spark] def clear() { settings.clear() } - } + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 4abd89955bd27..af9f7c62a1d25 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -24,11 +24,10 @@ import org.apache.hadoop.conf.Configuration import org.apache.spark.annotation.{AlphaComponent, DeveloperApi, Experimental} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.ScalaReflection -import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.dsl.ExpressionConversions -import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.optimizer.Optimizer import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.RuleExecutor @@ -37,7 +36,7 @@ import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.SparkStrategies import org.apache.spark.sql.json._ import org.apache.spark.sql.parquet.ParquetRelation -import org.apache.spark.SparkContext +import org.apache.spark.{Logging, SparkContext} /** * :: AlphaComponent :: @@ -49,18 +48,23 @@ import org.apache.spark.SparkContext */ @AlphaComponent class SQLContext(@transient val sparkContext: SparkContext) - extends Logging + extends org.apache.spark.Logging with SQLConf with ExpressionConversions + with UDFRegistration with Serializable { self => @transient protected[sql] lazy val catalog: Catalog = new SimpleCatalog(true) + + @transient + protected[sql] lazy val functionRegistry: FunctionRegistry = new SimpleFunctionRegistry + @transient protected[sql] lazy val analyzer: Analyzer = - new Analyzer(catalog, EmptyFunctionRegistry, caseSensitive = true) + new Analyzer(catalog, functionRegistry, caseSensitive = true) @transient protected[sql] val optimizer = Optimizer @transient @@ -86,7 +90,45 @@ class SQLContext(@transient val sparkContext: SparkContext) * @group userf */ implicit def createSchemaRDD[A <: Product: TypeTag](rdd: RDD[A]) = - new SchemaRDD(this, SparkLogicalPlan(ExistingRdd.fromProductRdd(rdd))) + new SchemaRDD(this, SparkLogicalPlan(ExistingRdd.fromProductRdd(rdd))(self)) + + /** + * :: DeveloperApi :: + * Creates a [[SchemaRDD]] from an [[RDD]] containing [[Row]]s by applying a schema to this RDD. + * It is important to make sure that the structure of every [[Row]] of the provided RDD matches + * the provided schema. Otherwise, there will be runtime exception. + * Example: + * {{{ + * import org.apache.spark.sql._ + * val sqlContext = new org.apache.spark.sql.SQLContext(sc) + * + * val schema = + * StructType( + * StructField("name", StringType, false) :: + * StructField("age", IntegerType, true) :: Nil) + * + * val people = + * sc.textFile("examples/src/main/resources/people.txt").map( + * _.split(",")).map(p => Row(p(0), p(1).trim.toInt)) + * val peopleSchemaRDD = sqlContext. applySchema(people, schema) + * peopleSchemaRDD.printSchema + * // root + * // |-- name: string (nullable = false) + * // |-- age: integer (nullable = true) + * + * peopleSchemaRDD.registerTempTable("people") + * sqlContext.sql("select name from people").collect.foreach(println) + * }}} + * + * @group userf + */ + @DeveloperApi + def applySchema(rowRDD: RDD[Row], schema: StructType): SchemaRDD = { + // TODO: use MutableProjection when rowRDD is another SchemaRDD and the applied + // schema differs from the existing schema on any field data type. + val logicalPlan = SparkLogicalPlan(ExistingRdd(schema.toAttributes, rowRDD))(self) + new SchemaRDD(this, logicalPlan) + } /** * Loads a Parquet file, returning the result as a [[SchemaRDD]]. @@ -94,7 +136,7 @@ class SQLContext(@transient val sparkContext: SparkContext) * @group userf */ def parquetFile(path: String): SchemaRDD = - new SchemaRDD(this, parquet.ParquetRelation(path, Some(sparkContext.hadoopConfiguration))) + new SchemaRDD(this, parquet.ParquetRelation(path, Some(sparkContext.hadoopConfiguration), this)) /** * Loads a JSON file (one object per line), returning the result as a [[SchemaRDD]]. @@ -104,6 +146,19 @@ class SQLContext(@transient val sparkContext: SparkContext) */ def jsonFile(path: String): SchemaRDD = jsonFile(path, 1.0) + /** + * :: Experimental :: + * Loads a JSON file (one object per line) and applies the given schema, + * returning the result as a [[SchemaRDD]]. + * + * @group userf + */ + @Experimental + def jsonFile(path: String, schema: StructType): SchemaRDD = { + val json = sparkContext.textFile(path) + jsonRDD(json, schema) + } + /** * :: Experimental :: */ @@ -122,12 +177,30 @@ class SQLContext(@transient val sparkContext: SparkContext) */ def jsonRDD(json: RDD[String]): SchemaRDD = jsonRDD(json, 1.0) + /** + * :: Experimental :: + * Loads an RDD[String] storing JSON objects (one object per record) and applies the given schema, + * returning the result as a [[SchemaRDD]]. + * + * @group userf + */ + @Experimental + def jsonRDD(json: RDD[String], schema: StructType): SchemaRDD = { + val appliedSchema = + Option(schema).getOrElse(JsonRDD.nullTypeToStringType(JsonRDD.inferSchema(json, 1.0))) + val rowRDD = JsonRDD.jsonStringToRow(json, appliedSchema) + applySchema(rowRDD, appliedSchema) + } + /** * :: Experimental :: */ @Experimental - def jsonRDD(json: RDD[String], samplingRatio: Double): SchemaRDD = - new SchemaRDD(this, JsonRDD.inferSchema(json, samplingRatio)) + def jsonRDD(json: RDD[String], samplingRatio: Double): SchemaRDD = { + val appliedSchema = JsonRDD.nullTypeToStringType(JsonRDD.inferSchema(json, samplingRatio)) + val rowRDD = JsonRDD.jsonStringToRow(json, appliedSchema) + applySchema(rowRDD, appliedSchema) + } /** * :: Experimental :: @@ -139,7 +212,7 @@ class SQLContext(@transient val sparkContext: SparkContext) * import sqlContext._ * * case class Person(name: String, age: Int) - * createParquetFile[Person]("path/to/file.parquet").registerAsTable("people") + * createParquetFile[Person]("path/to/file.parquet").registerTempTable("people") * sql("INSERT INTO people SELECT 'michael', 29") * }}} * @@ -160,7 +233,8 @@ class SQLContext(@transient val sparkContext: SparkContext) conf: Configuration = new Configuration()): SchemaRDD = { new SchemaRDD( this, - ParquetRelation.createEmpty(path, ScalaReflection.attributesFor[A], allowExisting, conf)) + ParquetRelation.createEmpty( + path, ScalaReflection.attributesFor[A], allowExisting, conf, this)) } /** @@ -170,19 +244,22 @@ class SQLContext(@transient val sparkContext: SparkContext) * @group userf */ def registerRDDAsTable(rdd: SchemaRDD, tableName: String): Unit = { - val name = tableName - val newPlan = rdd.logicalPlan transform { - case s @ SparkLogicalPlan(ExistingRdd(_, _), _) => s.copy(tableName = name) - } - catalog.registerTable(None, tableName, newPlan) + catalog.registerTable(None, tableName, rdd.logicalPlan) } /** - * Executes a SQL query using Spark, returning the result as a SchemaRDD. + * Executes a SQL query using Spark, returning the result as a SchemaRDD. The dialect that is + * used for SQL parsing can be configured with 'spark.sql.dialect'. * * @group userf */ - def sql(sqlText: String): SchemaRDD = new SchemaRDD(this, parseSql(sqlText)) + def sql(sqlText: String): SchemaRDD = { + if (dialect == "sql") { + new SchemaRDD(this, parseSql(sqlText)) + } else { + sys.error(s"Unsupported SQL dialect: $dialect") + } + } /** Returns the specified table as a SchemaRDD */ def table(tableName: String): SchemaRDD = @@ -196,9 +273,7 @@ class SQLContext(@transient val sparkContext: SparkContext) currentTable.logicalPlan case _ => - val useCompression = - sparkContext.conf.getBoolean("spark.sql.inMemoryColumnarStorage.compressed", false) - InMemoryRelation(useCompression, executePlan(currentTable).executedPlan) + InMemoryRelation(useCompression, columnBatchSize, executePlan(currentTable).executedPlan) } catalog.registerTable(None, tableName, asInMemoryRelation) @@ -209,10 +284,10 @@ class SQLContext(@transient val sparkContext: SparkContext) table(tableName).queryExecution.analyzed match { // This is kind of a hack to make sure that if this was just an RDD registered as a table, // we reregister the RDD as a table. - case inMem @ InMemoryRelation(_, _, e: ExistingRdd) => + case inMem @ InMemoryRelation(_, _, _, e: ExistingRdd) => inMem.cachedColumnBuffers.unpersist() catalog.unregisterTable(None, tableName) - catalog.registerTable(None, tableName, SparkLogicalPlan(e)) + catalog.registerTable(None, tableName, SparkLogicalPlan(e)(self)) case inMem: InMemoryRelation => inMem.cachedColumnBuffers.unpersist() catalog.unregisterTable(None, tableName) @@ -234,12 +309,14 @@ class SQLContext(@transient val sparkContext: SparkContext) val sqlContext: SQLContext = self + def codegenEnabled = self.codegenEnabled + def numPartitions = self.numShufflePartitions val strategies: Seq[Strategy] = CommandStrategy(self) :: TakeOrdered :: - PartialAggregation :: + HashAggregation :: LeftSemiJoin :: HashJoin :: InMemoryScans :: @@ -297,27 +374,30 @@ class SQLContext(@transient val sparkContext: SparkContext) protected[sql] lazy val emptyResult = sparkContext.parallelize(Seq.empty[Row], 1) /** - * Prepares a planned SparkPlan for execution by binding references to specific ordinals, and - * inserting shuffle operations as needed. + * Prepares a planned SparkPlan for execution by inserting shuffle operations as needed. */ @transient protected[sql] val prepareForExecution = new RuleExecutor[SparkPlan] { val batches = - Batch("Add exchange", Once, AddExchange(self)) :: - Batch("Prepare Expressions", Once, new BindReferences[SparkPlan]) :: Nil + Batch("Add exchange", Once, AddExchange(self)) :: Nil } /** + * :: DeveloperApi :: * The primary workflow for executing relational queries using Spark. Designed to allow easy * access to the intermediate phases of query execution for developers. */ + @DeveloperApi protected abstract class QueryExecution { def logical: LogicalPlan - lazy val analyzed = analyzer(logical) + lazy val analyzed = ExtractPythonUdfs(analyzer(logical)) lazy val optimizedPlan = optimizer(analyzed) // TODO: Don't just pick the first one... - lazy val sparkPlan = planner(optimizedPlan).next() + lazy val sparkPlan = { + SparkPlan.currentContext.set(self) + planner(optimizedPlan).next() + } // executedPlan should not be used to initialize any SparkPlan. It should be // only used for execution. lazy val executedPlan: SparkPlan = prepareForExecution(sparkPlan) @@ -337,41 +417,102 @@ class SQLContext(@transient val sparkContext: SparkContext) |${stringOrError(optimizedPlan)} |== Physical Plan == |${stringOrError(executedPlan)} + |Code Generation: ${executedPlan.codegenEnabled} + |== RDD == + |${stringOrError(toRdd.toDebugString)} """.stripMargin.trim } /** - * Peek at the first row of the RDD and infer its schema. - * TODO: consolidate this with the type system developed in SPARK-2060. + * Parses the data type in our internal string representation. The data type string should + * have the same format as the one generated by `toString` in scala. + * It is only used by PySpark. */ - private[sql] def inferSchema(rdd: RDD[Map[String, _]]): SchemaRDD = { + private[sql] def parseDataType(dataTypeString: String): DataType = { + val parser = org.apache.spark.sql.catalyst.types.DataType + parser(dataTypeString) + } + + /** + * Apply a schema defined by the schemaString to an RDD. It is only used by PySpark. + */ + private[sql] def applySchemaToPythonRDD( + rdd: RDD[Array[Any]], + schemaString: String): SchemaRDD = { + val schema = parseDataType(schemaString).asInstanceOf[StructType] + applySchemaToPythonRDD(rdd, schema) + } + + /** + * Apply a schema defined by the schema to an RDD. It is only used by PySpark. + */ + private[sql] def applySchemaToPythonRDD( + rdd: RDD[Array[Any]], + schema: StructType): SchemaRDD = { import scala.collection.JavaConversions._ - def typeFor(obj: Any): DataType = obj match { - case c: java.lang.String => StringType - case c: java.lang.Integer => IntegerType - case c: java.lang.Long => LongType - case c: java.lang.Double => DoubleType - case c: java.lang.Boolean => BooleanType - case c: java.util.List[_] => ArrayType(typeFor(c.head)) - case c: java.util.Set[_] => ArrayType(typeFor(c.head)) - case c: java.util.Map[_, _] => - val (key, value) = c.head - MapType(typeFor(key), typeFor(value)) - case c if c.getClass.isArray => - val elem = c.asInstanceOf[Array[_]].head - ArrayType(typeFor(elem)) - case c => throw new Exception(s"Object of type $c cannot be used") + import scala.collection.convert.Wrappers.{JListWrapper, JMapWrapper} + + def needsConversion(dataType: DataType): Boolean = dataType match { + case ByteType => true + case ShortType => true + case FloatType => true + case TimestampType => true + case ArrayType(_, _) => true + case MapType(_, _, _) => true + case StructType(_) => true + case other => false } - val schema = rdd.first().map { case (fieldName, obj) => - AttributeReference(fieldName, typeFor(obj), true)() - }.toSeq - val rowRdd = rdd.mapPartitions { iter => - iter.map { map => - new GenericRow(map.values.toArray.asInstanceOf[Array[Any]]): Row - } + // Converts value to the type specified by the data type. + // Because Python does not have data types for TimestampType, FloatType, ShortType, and + // ByteType, we need to explicitly convert values in columns of these data types to the desired + // JVM data types. + def convert(obj: Any, dataType: DataType): Any = (obj, dataType) match { + // TODO: We should check nullable + case (null, _) => null + + case (c: java.util.List[_], ArrayType(elementType, _)) => + val converted = c.map { e => convert(e, elementType)} + JListWrapper(converted) + + case (c, ArrayType(elementType, _)) if c.getClass.isArray => + c.asInstanceOf[Array[_]].map(e => convert(e, elementType)): Seq[Any] + + case (c: java.util.Map[_, _], MapType(keyType, valueType, _)) => c.map { + case (key, value) => (convert(key, keyType), convert(value, valueType)) + }.toMap + + case (c, StructType(fields)) if c.getClass.isArray => + new GenericRow(c.asInstanceOf[Array[_]].zip(fields).map { + case (e, f) => convert(e, f.dataType) + }): Row + + case (c: java.util.Calendar, TimestampType) => + new java.sql.Timestamp(c.getTime().getTime()) + + case (c: Int, ByteType) => c.toByte + case (c: Long, ByteType) => c.toByte + case (c: Int, ShortType) => c.toShort + case (c: Long, ShortType) => c.toShort + case (c: Long, IntegerType) => c.toInt + case (c: Double, FloatType) => c.toFloat + case (c, StringType) if !c.isInstanceOf[String] => c.toString + + case (c, _) => c + } + + val convertedRdd = if (schema.fields.exists(f => needsConversion(f.dataType))) { + rdd.map(m => m.zip(schema.fields).map { + case (value, field) => convert(value, field.dataType) + }) + } else { + rdd } - new SchemaRDD(this, SparkLogicalPlan(ExistingRdd(schema, rowRdd))) - } + val rowRdd = convertedRdd.mapPartitions { iter => + iter.map { m => new GenericRow(m): Row} + } + + new SchemaRDD(this, SparkLogicalPlan(ExistingRdd(schema.toAttributes, rowRdd))(self)) + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index 8bcfc7c064c2f..33b2ed1b3a399 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql -import java.util.{Map => JMap, List => JList, Set => JSet} +import java.util.{Map => JMap, List => JList} import scala.collection.JavaConversions._ import scala.collection.JavaConverters._ @@ -32,7 +32,6 @@ import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.{Inner, JoinType} -import org.apache.spark.sql.catalyst.types.{ArrayType, BooleanType, StructType} import org.apache.spark.sql.execution.{ExistingRdd, SparkLogicalPlan} import org.apache.spark.api.java.JavaRDD @@ -68,7 +67,7 @@ import org.apache.spark.api.java.JavaRDD * val rdd = sc.parallelize((1 to 100).map(i => Record(i, s"val_$i"))) * // Any RDD containing case classes can be registered as a table. The schema of the table is * // automatically inferred using scala reflection. - * rdd.registerAsTable("records") + * rdd.registerTempTable("records") * * val results: SchemaRDD = sql("SELECT * FROM records") * }}} @@ -120,6 +119,11 @@ class SchemaRDD( override protected def getDependencies: Seq[Dependency[_]] = List(new OneToOneDependency(queryExecution.toRdd)) + /** Returns the schema of this SchemaRDD (represented by a [[StructType]]). + * + * @group schema + */ + def schema: StructType = queryExecution.analyzed.schema // ======================================================================= // Query DSL @@ -256,6 +260,26 @@ class SchemaRDD( def unionAll(otherPlan: SchemaRDD) = new SchemaRDD(sqlContext, Union(logicalPlan, otherPlan.logicalPlan)) + /** + * Performs a relational except on two SchemaRDDs + * + * @param otherPlan the [[SchemaRDD]] that should be excepted from this one. + * + * @group Query + */ + def except(otherPlan: SchemaRDD): SchemaRDD = + new SchemaRDD(sqlContext, Except(logicalPlan, otherPlan.logicalPlan)) + + /** + * Performs a relational intersect on two SchemaRDDs + * + * @param otherPlan the [[SchemaRDD]] that should be intersected with this one. + * + * @group Query + */ + def intersect(otherPlan: SchemaRDD): SchemaRDD = + new SchemaRDD(sqlContext, Intersect(logicalPlan, otherPlan.logicalPlan)) + /** * Filters tuples using a function over the value of the specified column. * @@ -356,48 +380,41 @@ class SchemaRDD( * Converts a JavaRDD to a PythonRDD. It is used by pyspark. */ private[sql] def javaToPython: JavaRDD[Array[Byte]] = { - def rowToMap(row: Row, structType: StructType): JMap[String, Any] = { - val fields = structType.fields.map(field => (field.name, field.dataType)) - val map: JMap[String, Any] = new java.util.HashMap - row.zip(fields).foreach { - case (obj, (name, dataType)) => - dataType match { - case struct: StructType => map.put(name, rowToMap(obj.asInstanceOf[Row], struct)) - case array @ ArrayType(struct: StructType) => - val arrayValues = obj match { - case seq: Seq[Any] => - seq.map(element => rowToMap(element.asInstanceOf[Row], struct)).asJava - case list: JList[Any] => - list.map(element => rowToMap(element.asInstanceOf[Row], struct)) - case set: JSet[Any] => - set.map(element => rowToMap(element.asInstanceOf[Row], struct)) - case array if array != null && array.getClass.isArray => - array.asInstanceOf[Array[Any]].map { - element => rowToMap(element.asInstanceOf[Row], struct) - } - case other => other - } - map.put(name, arrayValues) - case array: ArrayType => { - val arrayValues = obj match { - case seq: Seq[Any] => seq.asJava - case other => other - } - map.put(name, arrayValues) - } - case other => map.put(name, obj) - } - } - - map + import scala.collection.Map + + def toJava(obj: Any, dataType: DataType): Any = (obj, dataType) match { + case (null, _) => null + + case (obj: Row, struct: StructType) => rowToArray(obj, struct) + + case (seq: Seq[Any], array: ArrayType) => + seq.map(x => toJava(x, array.elementType)).asJava + case (list: JList[_], array: ArrayType) => + list.map(x => toJava(x, array.elementType)).asJava + case (arr, array: ArrayType) if arr.getClass.isArray => + arr.asInstanceOf[Array[Any]].map(x => toJava(x, array.elementType)) + + case (obj: Map[_, _], mt: MapType) => obj.map { + case (k, v) => (k, toJava(v, mt.valueType)) // key should be primitive type + }.asJava + + // Pyrolite can handle Timestamp + case (other, _) => other + } + + def rowToArray(row: Row, structType: StructType): Array[Any] = { + val fields = structType.fields.map(field => field.dataType) + row.zip(fields).map { + case (obj, dataType) => toJava(obj, dataType) + }.toArray } val rowSchema = StructType.fromAttributes(this.queryExecution.analyzed.output) this.mapPartitions { iter => val pickle = new Pickler iter.map { row => - rowToMap(row, rowSchema) - }.grouped(10).map(batched => pickle.dumps(batched.toArray)) + rowToArray(row, rowSchema) + }.grouped(100).map(batched => pickle.dumps(batched.toArray)) } } @@ -410,7 +427,8 @@ class SchemaRDD( * @group schema */ private def applySchema(rdd: RDD[Row]): SchemaRDD = { - new SchemaRDD(sqlContext, SparkLogicalPlan(ExistingRdd(logicalPlan.output, rdd))) + new SchemaRDD(sqlContext, + SparkLogicalPlan(ExistingRdd(queryExecution.analyzed.output, rdd))(sqlContext)) } // ======================================================================= diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala index fe81721943202..2f3033a5f94f0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala @@ -56,7 +56,7 @@ private[sql] trait SchemaRDDLike { // happen right away to let these side effects take place eagerly. case _: Command | _: InsertIntoTable | _: InsertIntoCreatedTable | _: WriteToFile => queryExecution.toRdd - SparkLogicalPlan(queryExecution.executedPlan) + SparkLogicalPlan(queryExecution.executedPlan)(sqlContext) case _ => baseLogicalPlan } @@ -83,10 +83,13 @@ private[sql] trait SchemaRDDLike { * * @group schema */ - def registerAsTable(tableName: String): Unit = { + def registerTempTable(tableName: String): Unit = { sqlContext.registerRDDAsTable(baseSchemaRDD, tableName) } + @deprecated("Use registerTempTable instead of registerAsTable.", "1.1") + def registerAsTable(tableName: String): Unit = registerTempTable(tableName) + /** * :: Experimental :: * Adds the rows from this RDD to the specified table, optionally overwriting the existing data. @@ -123,9 +126,15 @@ private[sql] trait SchemaRDDLike { def saveAsTable(tableName: String): Unit = sqlContext.executePlan(InsertIntoCreatedTable(None, tableName, logicalPlan)).toRdd - /** Returns the output schema in the tree format. */ - def schemaString: String = queryExecution.analyzed.schemaString + /** Returns the schema as a string in the tree format. + * + * @group schema + */ + def schemaString: String = baseSchemaRDD.schema.treeString - /** Prints out the schema in the tree format. */ + /** Prints out the schema. + * + * @group schema + */ def printSchema(): Unit = println(schemaString) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala b/sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala new file mode 100644 index 0000000000000..0b48e9e659faa --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala @@ -0,0 +1,196 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import java.util.{List => JList, Map => JMap} + +import org.apache.spark.Accumulator +import org.apache.spark.sql.catalyst.ScalaReflection +import org.apache.spark.sql.catalyst.expressions.{Expression, ScalaUdf} +import org.apache.spark.sql.execution.PythonUDF + +import scala.reflect.runtime.universe.{TypeTag, typeTag} + +/** + * Functions for registering scala lambda functions as UDFs in a SQLContext. + */ +protected[sql] trait UDFRegistration { + self: SQLContext => + + private[spark] def registerPython( + name: String, + command: Array[Byte], + envVars: JMap[String, String], + pythonIncludes: JList[String], + pythonExec: String, + accumulator: Accumulator[JList[Array[Byte]]], + stringDataType: String): Unit = { + log.debug( + s""" + | Registering new PythonUDF: + | name: $name + | command: ${command.toSeq} + | envVars: $envVars + | pythonIncludes: $pythonIncludes + | pythonExec: $pythonExec + | dataType: $stringDataType + """.stripMargin) + + + val dataType = parseDataType(stringDataType) + + def builder(e: Seq[Expression]) = + PythonUDF( + name, + command, + envVars, + pythonIncludes, + pythonExec, + accumulator, + dataType, + e) + + functionRegistry.registerFunction(name, builder) + } + + /** registerFunction 1-22 were generated by this script + + (1 to 22).map { x => + val types = (1 to x).map(x => "_").reduce(_ + ", " + _) + s""" + def registerFunction[T: TypeTag](name: String, func: Function$x[$types, T]): Unit = { + def builder(e: Seq[Expression]) = + ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + """ + } + */ + + // scalastyle:off + def registerFunction[T: TypeTag](name: String, func: Function1[_, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + + def registerFunction[T: TypeTag](name: String, func: Function2[_, _, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + + def registerFunction[T: TypeTag](name: String, func: Function3[_, _, _, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + + def registerFunction[T: TypeTag](name: String, func: Function4[_, _, _, _, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + + def registerFunction[T: TypeTag](name: String, func: Function5[_, _, _, _, _, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + + def registerFunction[T: TypeTag](name: String, func: Function6[_, _, _, _, _, _, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + + def registerFunction[T: TypeTag](name: String, func: Function7[_, _, _, _, _, _, _, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + + def registerFunction[T: TypeTag](name: String, func: Function8[_, _, _, _, _, _, _, _, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + + def registerFunction[T: TypeTag](name: String, func: Function9[_, _, _, _, _, _, _, _, _, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + + def registerFunction[T: TypeTag](name: String, func: Function10[_, _, _, _, _, _, _, _, _, _, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + + def registerFunction[T: TypeTag](name: String, func: Function11[_, _, _, _, _, _, _, _, _, _, _, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + + def registerFunction[T: TypeTag](name: String, func: Function12[_, _, _, _, _, _, _, _, _, _, _, _, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + + def registerFunction[T: TypeTag](name: String, func: Function13[_, _, _, _, _, _, _, _, _, _, _, _, _, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + + def registerFunction[T: TypeTag](name: String, func: Function14[_, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + + def registerFunction[T: TypeTag](name: String, func: Function15[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + + def registerFunction[T: TypeTag](name: String, func: Function16[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + + def registerFunction[T: TypeTag](name: String, func: Function17[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + + def registerFunction[T: TypeTag](name: String, func: Function18[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + + def registerFunction[T: TypeTag](name: String, func: Function19[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + + def registerFunction[T: TypeTag](name: String, func: Function20[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + + def registerFunction[T: TypeTag](name: String, func: Function21[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + + def registerFunction[T: TypeTag](name: String, func: Function22[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + // scalastyle:on +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala index 790d9ef22cf16..150ff8a42063d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala @@ -21,28 +21,36 @@ import java.beans.Introspector import org.apache.hadoop.conf.Configuration -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} import org.apache.spark.sql.json.JsonRDD -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.{SQLContext, StructType => SStructType} import org.apache.spark.sql.catalyst.expressions.{AttributeReference, GenericRow, Row => ScalaRow} -import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.parquet.ParquetRelation import org.apache.spark.sql.execution.{ExistingRdd, SparkLogicalPlan} +import org.apache.spark.sql.types.util.DataTypeConversions.asScalaDataType import org.apache.spark.util.Utils /** * The entry point for executing Spark SQL queries from a Java program. */ -class JavaSQLContext(val sqlContext: SQLContext) { +class JavaSQLContext(val sqlContext: SQLContext) extends UDFRegistration { def this(sparkContext: JavaSparkContext) = this(new SQLContext(sparkContext.sc)) /** - * Executes a query expressed in SQL, returning the result as a JavaSchemaRDD + * Executes a SQL query using Spark, returning the result as a SchemaRDD. The dialect that is + * used for SQL parsing can be configured with 'spark.sql.dialect'. + * + * @group userf */ - def sql(sqlQuery: String): JavaSchemaRDD = - new JavaSchemaRDD(sqlContext, sqlContext.parseSql(sqlQuery)) + def sql(sqlText: String): JavaSchemaRDD = { + if (sqlContext.dialect == "sql") { + new JavaSchemaRDD(sqlContext, sqlContext.parseSql(sqlText)) + } else { + sys.error(s"Unsupported SQL dialect: $sqlContext.dialect") + } + } /** * :: Experimental :: @@ -52,7 +60,7 @@ class JavaSQLContext(val sqlContext: SQLContext) { * {{{ * JavaSQLContext sqlCtx = new JavaSQLContext(...) * - * sqlCtx.createParquetFile(Person.class, "path/to/file.parquet").registerAsTable("people") + * sqlCtx.createParquetFile(Person.class, "path/to/file.parquet").registerTempTable("people") * sqlCtx.sql("INSERT INTO people SELECT 'michael', 29") * }}} * @@ -72,7 +80,7 @@ class JavaSQLContext(val sqlContext: SQLContext) { conf: Configuration = new Configuration()): JavaSchemaRDD = { new JavaSchemaRDD( sqlContext, - ParquetRelation.createEmpty(path, getSchema(beanClass), allowExisting, conf)) + ParquetRelation.createEmpty(path, getSchema(beanClass), allowExisting, conf, sqlContext)) } /** @@ -92,7 +100,22 @@ class JavaSQLContext(val sqlContext: SQLContext) { new GenericRow(extractors.map(e => e.invoke(row)).toArray[Any]): ScalaRow } } - new JavaSchemaRDD(sqlContext, SparkLogicalPlan(ExistingRdd(schema, rowRdd))) + new JavaSchemaRDD(sqlContext, SparkLogicalPlan(ExistingRdd(schema, rowRdd))(sqlContext)) + } + + /** + * :: DeveloperApi :: + * Creates a JavaSchemaRDD from an RDD containing Rows by applying a schema to this RDD. + * It is important to make sure that the structure of every Row of the provided RDD matches the + * provided schema. Otherwise, there will be runtime exception. + */ + @DeveloperApi + def applySchema(rowRDD: JavaRDD[Row], schema: StructType): JavaSchemaRDD = { + val scalaRowRDD = rowRDD.rdd.map(r => r.row) + val scalaSchema = asScalaDataType(schema).asInstanceOf[SStructType] + val logicalPlan = + SparkLogicalPlan(ExistingRdd(scalaSchema.toAttributes, scalaRowRDD))(sqlContext) + new JavaSchemaRDD(sqlContext, logicalPlan) } /** @@ -101,26 +124,52 @@ class JavaSQLContext(val sqlContext: SQLContext) { def parquetFile(path: String): JavaSchemaRDD = new JavaSchemaRDD( sqlContext, - ParquetRelation(path, Some(sqlContext.sparkContext.hadoopConfiguration))) + ParquetRelation(path, Some(sqlContext.sparkContext.hadoopConfiguration), sqlContext)) /** - * Loads a JSON file (one object per line), returning the result as a [[JavaSchemaRDD]]. + * Loads a JSON file (one object per line), returning the result as a JavaSchemaRDD. * It goes through the entire dataset once to determine the schema. - * - * @group userf */ def jsonFile(path: String): JavaSchemaRDD = jsonRDD(sqlContext.sparkContext.textFile(path)) + /** + * :: Experimental :: + * Loads a JSON file (one object per line) and applies the given schema, + * returning the result as a JavaSchemaRDD. + */ + @Experimental + def jsonFile(path: String, schema: StructType): JavaSchemaRDD = + jsonRDD(sqlContext.sparkContext.textFile(path), schema) + /** * Loads an RDD[String] storing JSON objects (one object per record), returning the result as a - * [[JavaSchemaRDD]]. + * JavaSchemaRDD. * It goes through the entire dataset once to determine the schema. - * - * @group userf */ - def jsonRDD(json: JavaRDD[String]): JavaSchemaRDD = - new JavaSchemaRDD(sqlContext, JsonRDD.inferSchema(json, 1.0)) + def jsonRDD(json: JavaRDD[String]): JavaSchemaRDD = { + val appliedScalaSchema = JsonRDD.nullTypeToStringType(JsonRDD.inferSchema(json.rdd, 1.0)) + val scalaRowRDD = JsonRDD.jsonStringToRow(json.rdd, appliedScalaSchema) + val logicalPlan = + SparkLogicalPlan(ExistingRdd(appliedScalaSchema.toAttributes, scalaRowRDD))(sqlContext) + new JavaSchemaRDD(sqlContext, logicalPlan) + } + + /** + * :: Experimental :: + * Loads an RDD[String] storing JSON objects (one object per record) and applies the given schema, + * returning the result as a JavaSchemaRDD. + */ + @Experimental + def jsonRDD(json: JavaRDD[String], schema: StructType): JavaSchemaRDD = { + val appliedScalaSchema = + Option(asScalaDataType(schema)).getOrElse( + JsonRDD.nullTypeToStringType(JsonRDD.inferSchema(json.rdd, 1.0))).asInstanceOf[SStructType] + val scalaRowRDD = JsonRDD.jsonStringToRow(json.rdd, appliedScalaSchema) + val logicalPlan = + SparkLogicalPlan(ExistingRdd(appliedScalaSchema.toAttributes, scalaRowRDD))(sqlContext) + new JavaSchemaRDD(sqlContext, logicalPlan) + } /** * Registers the given RDD as a temporary table in the catalog. Temporary tables exist only @@ -138,22 +187,37 @@ class JavaSQLContext(val sqlContext: SQLContext) { val fields = beanInfo.getPropertyDescriptors.filterNot(_.getName == "class") fields.map { property => val (dataType, nullable) = property.getPropertyType match { - case c: Class[_] if c == classOf[java.lang.String] => (StringType, true) - case c: Class[_] if c == java.lang.Short.TYPE => (ShortType, false) - case c: Class[_] if c == java.lang.Integer.TYPE => (IntegerType, false) - case c: Class[_] if c == java.lang.Long.TYPE => (LongType, false) - case c: Class[_] if c == java.lang.Double.TYPE => (DoubleType, false) - case c: Class[_] if c == java.lang.Byte.TYPE => (ByteType, false) - case c: Class[_] if c == java.lang.Float.TYPE => (FloatType, false) - case c: Class[_] if c == java.lang.Boolean.TYPE => (BooleanType, false) - - case c: Class[_] if c == classOf[java.lang.Short] => (ShortType, true) - case c: Class[_] if c == classOf[java.lang.Integer] => (IntegerType, true) - case c: Class[_] if c == classOf[java.lang.Long] => (LongType, true) - case c: Class[_] if c == classOf[java.lang.Double] => (DoubleType, true) - case c: Class[_] if c == classOf[java.lang.Byte] => (ByteType, true) - case c: Class[_] if c == classOf[java.lang.Float] => (FloatType, true) - case c: Class[_] if c == classOf[java.lang.Boolean] => (BooleanType, true) + case c: Class[_] if c == classOf[java.lang.String] => + (org.apache.spark.sql.StringType, true) + case c: Class[_] if c == java.lang.Short.TYPE => + (org.apache.spark.sql.ShortType, false) + case c: Class[_] if c == java.lang.Integer.TYPE => + (org.apache.spark.sql.IntegerType, false) + case c: Class[_] if c == java.lang.Long.TYPE => + (org.apache.spark.sql.LongType, false) + case c: Class[_] if c == java.lang.Double.TYPE => + (org.apache.spark.sql.DoubleType, false) + case c: Class[_] if c == java.lang.Byte.TYPE => + (org.apache.spark.sql.ByteType, false) + case c: Class[_] if c == java.lang.Float.TYPE => + (org.apache.spark.sql.FloatType, false) + case c: Class[_] if c == java.lang.Boolean.TYPE => + (org.apache.spark.sql.BooleanType, false) + + case c: Class[_] if c == classOf[java.lang.Short] => + (org.apache.spark.sql.ShortType, true) + case c: Class[_] if c == classOf[java.lang.Integer] => + (org.apache.spark.sql.IntegerType, true) + case c: Class[_] if c == classOf[java.lang.Long] => + (org.apache.spark.sql.LongType, true) + case c: Class[_] if c == classOf[java.lang.Double] => + (org.apache.spark.sql.DoubleType, true) + case c: Class[_] if c == classOf[java.lang.Byte] => + (org.apache.spark.sql.ByteType, true) + case c: Class[_] if c == classOf[java.lang.Float] => + (org.apache.spark.sql.FloatType, true) + case c: Class[_] if c == classOf[java.lang.Boolean] => + (org.apache.spark.sql.BooleanType, true) } AttributeReference(property.getName, dataType, nullable)() } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala index aff6ffe9f3478..4d799b4038fdd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala @@ -17,11 +17,15 @@ package org.apache.spark.sql.api.java +import java.util.{List => JList} + import org.apache.spark.Partitioner import org.apache.spark.api.java.{JavaRDDLike, JavaRDD} import org.apache.spark.api.java.function.{Function => JFunction} +import org.apache.spark.sql.types.util.DataTypeConversions import org.apache.spark.sql.{SQLContext, SchemaRDD, SchemaRDDLike} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import DataTypeConversions._ import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel @@ -51,6 +55,10 @@ class JavaSchemaRDD( override def toString: String = baseSchemaRDD.toString + /** Returns the schema of this JavaSchemaRDD (represented by a StructType). */ + def schema: StructType = + asJavaDataType(baseSchemaRDD.schema).asInstanceOf[StructType] + // ======================================================================= // Base RDD functions that do NOT change schema // ======================================================================= @@ -96,6 +104,20 @@ class JavaSchemaRDD( this } + // Overridden actions from JavaRDDLike. + + override def collect(): JList[Row] = { + import scala.collection.JavaConversions._ + val arr: java.util.Collection[Row] = baseSchemaRDD.collect().toSeq.map(new Row(_)) + new java.util.ArrayList(arr) + } + + override def take(num: Int): JList[Row] = { + import scala.collection.JavaConversions._ + val arr: java.util.Collection[Row] = baseSchemaRDD.take(num).toSeq.map(new Row(_)) + new java.util.ArrayList(arr) + } + // Transformations (return a new RDD) /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/Row.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/Row.scala index 9b0dd2176149b..6c67934bda5b8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/Row.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/Row.scala @@ -17,6 +17,11 @@ package org.apache.spark.sql.api.java +import scala.annotation.varargs +import scala.collection.convert.Wrappers.{JListWrapper, JMapWrapper} +import scala.collection.JavaConversions +import scala.math.BigDecimal + import org.apache.spark.sql.catalyst.expressions.{Row => ScalaRow} /** @@ -29,7 +34,7 @@ class Row(private[spark] val row: ScalaRow) extends Serializable { /** Returns the value of column `i`. */ def get(i: Int): Any = - row(i) + Row.toJavaValue(row(i)) /** Returns true if value at column `i` is NULL. */ def isNullAt(i: Int) = get(i) == null @@ -89,5 +94,57 @@ class Row(private[spark] val row: ScalaRow) extends Serializable { */ def getString(i: Int): String = row.getString(i) + + def canEqual(other: Any): Boolean = other.isInstanceOf[Row] + + override def equals(other: Any): Boolean = other match { + case that: Row => + (that canEqual this) && + row == that.row + case _ => false + } + + override def hashCode(): Int = row.hashCode() } +object Row { + + private def toJavaValue(value: Any): Any = value match { + // For values of this ScalaRow, we will do the conversion when + // they are actually accessed. + case row: ScalaRow => new Row(row) + case map: scala.collection.Map[_, _] => + JavaConversions.mapAsJavaMap( + map.map { + case (key, value) => (toJavaValue(key), toJavaValue(value)) + } + ) + case seq: scala.collection.Seq[_] => + JavaConversions.seqAsJavaList(seq.map(toJavaValue)) + case decimal: BigDecimal => decimal.underlying() + case other => other + } + + // TODO: Consolidate the toScalaValue at here with the scalafy in JsonRDD? + private def toScalaValue(value: Any): Any = value match { + // Values of this row have been converted to Scala values. + case row: Row => row.row + case map: java.util.Map[_, _] => + JMapWrapper(map).map { + case (key, value) => (toScalaValue(key), toScalaValue(value)) + } + case list: java.util.List[_] => + JListWrapper(list).map(toScalaValue) + case decimal: java.math.BigDecimal => BigDecimal(decimal) + case other => other + } + + /** + * Creates a Row with the given values. + */ + @varargs def create(values: Any*): Row = { + // Right now, we cannot use @varargs to annotate the constructor of + // org.apache.spark.sql.api.java.Row. See https://issues.scala-lang.org/browse/SI-8383. + new Row(ScalaRow(values.map(toScalaValue):_*)) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/UDFRegistration.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/UDFRegistration.scala new file mode 100644 index 0000000000000..158f26e3d445f --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/UDFRegistration.scala @@ -0,0 +1,252 @@ +/* +* Licensed to the Apache Software Foundation (ASF) under one or more +* contributor license agreements. See the NOTICE file distributed with +* this work for additional information regarding copyright ownership. +* The ASF licenses this file to You under the Apache License, Version 2.0 +* (the "License"); you may not use this file except in compliance with +* the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ + +package org.apache.spark.sql.api.java + +import org.apache.spark.sql.catalyst.expressions.{Expression, ScalaUdf} +import org.apache.spark.sql.types.util.DataTypeConversions._ + +/** + * A collection of functions that allow Java users to register UDFs. In order to handle functions + * of varying airities with minimal boilerplate for our users, we generate classes and functions + * for each airity up to 22. The code for this generation can be found in comments in this trait. + */ +private[java] trait UDFRegistration { + self: JavaSQLContext => + + /* The following functions and required interfaces are generated with these code fragments: + + (1 to 22).foreach { i => + val extTypeArgs = (1 to i).map(_ => "_").mkString(", ") + val anyTypeArgs = (1 to i).map(_ => "Any").mkString(", ") + val anyCast = s".asInstanceOf[UDF$i[$anyTypeArgs, Any]]" + val anyParams = (1 to i).map(_ => "_: Any").mkString(", ") + println(s""" + |def registerFunction( + | name: String, f: UDF$i[$extTypeArgs, _], @transient dataType: DataType) = { + | val scalaType = asScalaDataType(dataType) + | sqlContext.functionRegistry.registerFunction( + | name, + | (e: Seq[Expression]) => ScalaUdf(f$anyCast.call($anyParams), scalaType, e)) + |} + """.stripMargin) + } + + import java.io.File + import org.apache.spark.sql.catalyst.util.stringToFile + val directory = new File("sql/core/src/main/java/org/apache/spark/sql/api/java/") + (1 to 22).foreach { i => + val typeArgs = (1 to i).map(i => s"T$i").mkString(", ") + val args = (1 to i).map(i => s"T$i t$i").mkString(", ") + + val contents = + s"""/* + | * Licensed to the Apache Software Foundation (ASF) under one or more + | * contributor license agreements. See the NOTICE file distributed with + | * this work for additional information regarding copyright ownership. + | * The ASF licenses this file to You under the Apache License, Version 2.0 + | * (the "License"); you may not use this file except in compliance with + | * the License. You may obtain a copy of the License at + | * + | * http://www.apache.org/licenses/LICENSE-2.0 + | * + | * Unless required by applicable law or agreed to in writing, software + | * distributed under the License is distributed on an "AS IS" BASIS, + | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + | * See the License for the specific language governing permissions and + | * limitations under the License. + | */ + | + |package org.apache.spark.sql.api.java; + | + |import java.io.Serializable; + | + |// ************************************************** + |// THIS FILE IS AUTOGENERATED BY CODE IN + |// org.apache.spark.sql.api.java.FunctionRegistration + |// ************************************************** + | + |/** + | * A Spark SQL UDF that has $i arguments. + | */ + |public interface UDF$i<$typeArgs, R> extends Serializable { + | public R call($args) throws Exception; + |} + |""".stripMargin + + stringToFile(new File(directory, s"UDF$i.java"), contents) + } + + */ + + // scalastyle:off + def registerFunction(name: String, f: UDF1[_, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF1[Any, Any]].call(_: Any), scalaType, e)) + } + + def registerFunction(name: String, f: UDF2[_, _, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF2[Any, Any, Any]].call(_: Any, _: Any), scalaType, e)) + } + + def registerFunction(name: String, f: UDF3[_, _, _, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF3[Any, Any, Any, Any]].call(_: Any, _: Any, _: Any), scalaType, e)) + } + + def registerFunction(name: String, f: UDF4[_, _, _, _, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF4[Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any), scalaType, e)) + } + + def registerFunction(name: String, f: UDF5[_, _, _, _, _, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF5[Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + } + + def registerFunction(name: String, f: UDF6[_, _, _, _, _, _, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF6[Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + } + + def registerFunction(name: String, f: UDF7[_, _, _, _, _, _, _, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF7[Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + } + + def registerFunction(name: String, f: UDF8[_, _, _, _, _, _, _, _, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF8[Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + } + + def registerFunction(name: String, f: UDF9[_, _, _, _, _, _, _, _, _, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF9[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + } + + def registerFunction(name: String, f: UDF10[_, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF10[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + } + + def registerFunction(name: String, f: UDF11[_, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF11[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + } + + def registerFunction(name: String, f: UDF12[_, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF12[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + } + + def registerFunction(name: String, f: UDF13[_, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF13[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + } + + def registerFunction(name: String, f: UDF14[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF14[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + } + + def registerFunction(name: String, f: UDF15[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF15[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + } + + def registerFunction(name: String, f: UDF16[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF16[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + } + + def registerFunction(name: String, f: UDF17[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF17[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + } + + def registerFunction(name: String, f: UDF18[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF18[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + } + + def registerFunction(name: String, f: UDF19[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF19[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + } + + def registerFunction(name: String, f: UDF20[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF20[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + } + + def registerFunction(name: String, f: UDF21[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF21[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + } + + def registerFunction(name: String, f: UDF22[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF22[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + } + + // scalastyle:on +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala index 3c39e1d350fa8..42a5a9a84f362 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala @@ -90,6 +90,9 @@ private[sql] class FloatColumnAccessor(buffer: ByteBuffer) private[sql] class StringColumnAccessor(buffer: ByteBuffer) extends NativeColumnAccessor(buffer, STRING) +private[sql] class TimestampColumnAccessor(buffer: ByteBuffer) + extends NativeColumnAccessor(buffer, TIMESTAMP) + private[sql] class BinaryColumnAccessor(buffer: ByteBuffer) extends BasicColumnAccessor[BinaryType.type, Array[Byte]](buffer, BINARY) with NullableColumnAccessor @@ -105,16 +108,17 @@ private[sql] object ColumnAccessor { val columnTypeId = dup.getInt() columnTypeId match { - case INT.typeId => new IntColumnAccessor(dup) - case LONG.typeId => new LongColumnAccessor(dup) - case FLOAT.typeId => new FloatColumnAccessor(dup) - case DOUBLE.typeId => new DoubleColumnAccessor(dup) - case BOOLEAN.typeId => new BooleanColumnAccessor(dup) - case BYTE.typeId => new ByteColumnAccessor(dup) - case SHORT.typeId => new ShortColumnAccessor(dup) - case STRING.typeId => new StringColumnAccessor(dup) - case BINARY.typeId => new BinaryColumnAccessor(dup) - case GENERIC.typeId => new GenericColumnAccessor(dup) + case INT.typeId => new IntColumnAccessor(dup) + case LONG.typeId => new LongColumnAccessor(dup) + case FLOAT.typeId => new FloatColumnAccessor(dup) + case DOUBLE.typeId => new DoubleColumnAccessor(dup) + case BOOLEAN.typeId => new BooleanColumnAccessor(dup) + case BYTE.typeId => new ByteColumnAccessor(dup) + case SHORT.typeId => new ShortColumnAccessor(dup) + case STRING.typeId => new StringColumnAccessor(dup) + case TIMESTAMP.typeId => new TimestampColumnAccessor(dup) + case BINARY.typeId => new BinaryColumnAccessor(dup) + case GENERIC.typeId => new GenericColumnAccessor(dup) } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala index 4be048cd742d6..7e7bb2859bbcd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala @@ -109,13 +109,16 @@ private[sql] class FloatColumnBuilder extends NativeColumnBuilder(new FloatColum private[sql] class StringColumnBuilder extends NativeColumnBuilder(new StringColumnStats, STRING) +private[sql] class TimestampColumnBuilder + extends NativeColumnBuilder(new TimestampColumnStats, TIMESTAMP) + private[sql] class BinaryColumnBuilder extends ComplexColumnBuilder(BINARY) // TODO (lian) Add support for array, struct and map private[sql] class GenericColumnBuilder extends ComplexColumnBuilder(GENERIC) private[sql] object ColumnBuilder { - val DEFAULT_INITIAL_BUFFER_SIZE = 10 * 1024 * 104 + val DEFAULT_INITIAL_BUFFER_SIZE = 1024 * 1024 private[columnar] def ensureFreeSpace(orig: ByteBuffer, size: Int) = { if (orig.remaining >= size) { @@ -151,6 +154,7 @@ private[sql] object ColumnBuilder { case STRING.typeId => new StringColumnBuilder case BINARY.typeId => new BinaryColumnBuilder case GENERIC.typeId => new GenericColumnBuilder + case TIMESTAMP.typeId => new TimestampColumnBuilder }).asInstanceOf[ColumnBuilder] builder.initialize(initialSize, columnName, useCompression) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala index 95602d321dc6f..6502110e903fe 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala @@ -344,21 +344,52 @@ private[sql] class StringColumnStats extends BasicColumnStats(STRING) { } override def contains(row: Row, ordinal: Int) = { - !(upperBound eq null) && { + (upperBound ne null) && { val field = columnType.getField(row, ordinal) lowerBound.compareTo(field) <= 0 && field.compareTo(upperBound) <= 0 } } override def isAbove(row: Row, ordinal: Int) = { - !(upperBound eq null) && { + (upperBound ne null) && { val field = columnType.getField(row, ordinal) field.compareTo(upperBound) < 0 } } override def isBelow(row: Row, ordinal: Int) = { - !(lowerBound eq null) && { + (lowerBound ne null) && { + val field = columnType.getField(row, ordinal) + lowerBound.compareTo(field) < 0 + } + } +} + +private[sql] class TimestampColumnStats extends BasicColumnStats(TIMESTAMP) { + override def initialBounds = (null, null) + + override def gatherStats(row: Row, ordinal: Int) { + val field = columnType.getField(row, ordinal) + if ((upperBound eq null) || field.compareTo(upperBound) > 0) _upper = field + if ((lowerBound eq null) || field.compareTo(lowerBound) < 0) _lower = field + } + + override def contains(row: Row, ordinal: Int) = { + (upperBound ne null) && { + val field = columnType.getField(row, ordinal) + lowerBound.compareTo(field) <= 0 && field.compareTo(upperBound) <= 0 + } + } + + override def isAbove(row: Row, ordinal: Int) = { + (lowerBound ne null) && { + val field = columnType.getField(row, ordinal) + field.compareTo(upperBound) < 0 + } + } + + override def isBelow(row: Row, ordinal: Int) = { + (lowerBound ne null) && { val field = columnType.getField(row, ordinal) lowerBound.compareTo(field) < 0 } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala index 4cd52d8288137..794bc60d0e315 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala @@ -21,6 +21,8 @@ import java.nio.ByteBuffer import scala.reflect.runtime.universe.TypeTag +import java.sql.Timestamp + import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions.MutableRow import org.apache.spark.sql.catalyst.types._ @@ -221,6 +223,26 @@ private[sql] object STRING extends NativeColumnType(StringType, 7, 8) { override def getField(row: Row, ordinal: Int) = row.getString(ordinal) } +private[sql] object TIMESTAMP extends NativeColumnType(TimestampType, 8, 12) { + override def extract(buffer: ByteBuffer) = { + val timestamp = new Timestamp(buffer.getLong()) + timestamp.setNanos(buffer.getInt()) + timestamp + } + + override def append(v: Timestamp, buffer: ByteBuffer) { + buffer.putLong(v.getTime).putInt(v.getNanos) + } + + override def getField(row: Row, ordinal: Int) = { + row(ordinal).asInstanceOf[Timestamp] + } + + override def setField(row: MutableRow, ordinal: Int, value: Timestamp) { + row(ordinal) = value + } +} + private[sql] sealed abstract class ByteArrayColumnType[T <: DataType]( typeId: Int, defaultSize: Int) @@ -240,7 +262,7 @@ private[sql] sealed abstract class ByteArrayColumnType[T <: DataType]( } } -private[sql] object BINARY extends ByteArrayColumnType[BinaryType.type](8, 16) { +private[sql] object BINARY extends ByteArrayColumnType[BinaryType.type](9, 16) { override def setField(row: MutableRow, ordinal: Int, value: Array[Byte]) { row(ordinal) = value } @@ -251,7 +273,7 @@ private[sql] object BINARY extends ByteArrayColumnType[BinaryType.type](8, 16) { // Used to process generic objects (all types other than those listed above). Objects should be // serialized first before appending to the column `ByteBuffer`, and is also extracted as serialized // byte array. -private[sql] object GENERIC extends ByteArrayColumnType[DataType](9, 16) { +private[sql] object GENERIC extends ByteArrayColumnType[DataType](10, 16) { override def setField(row: MutableRow, ordinal: Int, value: Array[Byte]) { row(ordinal) = SparkSqlSerializer.deserialize[Any](value) } @@ -262,16 +284,17 @@ private[sql] object GENERIC extends ByteArrayColumnType[DataType](9, 16) { private[sql] object ColumnType { def apply(dataType: DataType): ColumnType[_, _] = { dataType match { - case IntegerType => INT - case LongType => LONG - case FloatType => FLOAT - case DoubleType => DOUBLE - case BooleanType => BOOLEAN - case ByteType => BYTE - case ShortType => SHORT - case StringType => STRING - case BinaryType => BINARY - case _ => GENERIC + case IntegerType => INT + case LongType => LONG + case FloatType => FLOAT + case DoubleType => DOUBLE + case BooleanType => BOOLEAN + case ByteType => BYTE + case ShortType => SHORT + case StringType => STRING + case BinaryType => BINARY + case TimestampType => TIMESTAMP + case _ => GENERIC } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala index e1e4f24c6c66c..e63b4903041f6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala @@ -17,55 +17,80 @@ package org.apache.spark.sql.columnar +import java.nio.ByteBuffer + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation -import org.apache.spark.sql.catalyst.expressions.{GenericMutableRow, Attribute} +import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericMutableRow} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.execution.{SparkPlan, LeafNode} -import org.apache.spark.sql.Row -import org.apache.spark.SparkConf +import org.apache.spark.sql.execution.{LeafNode, SparkPlan} object InMemoryRelation { - def apply(useCompression: Boolean, child: SparkPlan): InMemoryRelation = - new InMemoryRelation(child.output, useCompression, child) + def apply(useCompression: Boolean, batchSize: Int, child: SparkPlan): InMemoryRelation = + new InMemoryRelation(child.output, useCompression, batchSize, child)() } private[sql] case class InMemoryRelation( output: Seq[Attribute], useCompression: Boolean, + batchSize: Int, child: SparkPlan) + (private var _cachedColumnBuffers: RDD[Array[ByteBuffer]] = null) extends LogicalPlan with MultiInstanceRelation { - override def children = Seq.empty - override def references = Set.empty - - override def newInstance() = - new InMemoryRelation(output.map(_.newInstance), useCompression, child).asInstanceOf[this.type] - - lazy val cachedColumnBuffers = { + // If the cached column buffers were not passed in, we calculate them in the constructor. + // As in Spark, the actual work of caching is lazy. + if (_cachedColumnBuffers == null) { val output = child.output - val cached = child.execute().mapPartitions { iterator => - val columnBuilders = output.map { attribute => - ColumnBuilder(ColumnType(attribute.dataType).typeId, 0, attribute.name, useCompression) - }.toArray - - var row: Row = null - while (iterator.hasNext) { - row = iterator.next() - var i = 0 - while (i < row.length) { - columnBuilders(i).appendFrom(row, i) - i += 1 + val cached = child.execute().mapPartitions { baseIterator => + new Iterator[Array[ByteBuffer]] { + def next() = { + val columnBuilders = output.map { attribute => + val columnType = ColumnType(attribute.dataType) + val initialBufferSize = columnType.defaultSize * batchSize + ColumnBuilder(columnType.typeId, initialBufferSize, attribute.name, useCompression) + }.toArray + + var row: Row = null + var rowCount = 0 + + while (baseIterator.hasNext && rowCount < batchSize) { + row = baseIterator.next() + var i = 0 + while (i < row.length) { + columnBuilders(i).appendFrom(row, i) + i += 1 + } + rowCount += 1 + } + + columnBuilders.map(_.build()) } - } - Iterator.single(columnBuilders.map(_.build())) + def hasNext = baseIterator.hasNext + } }.cache() cached.setName(child.toString) - // Force the materialization of the cached RDD. - cached.count() - cached + _cachedColumnBuffers = cached + } + + + override def children = Seq.empty + + override def references = Set.empty + + override def newInstance() = { + new InMemoryRelation( + output.map(_.newInstance), + useCompression, + batchSize, + child)( + _cachedColumnBuffers).asInstanceOf[this.type] } + + def cachedColumnBuffers = _cachedColumnBuffers } private[sql] case class InMemoryColumnarTableScan( @@ -77,18 +102,31 @@ private[sql] case class InMemoryColumnarTableScan( override def execute() = { relation.cachedColumnBuffers.mapPartitions { iterator => - val columnBuffers = iterator.next() - assert(!iterator.hasNext) + // Find the ordinals of the requested columns. If none are requested, use the first. + val requestedColumns = + if (attributes.isEmpty) { + Seq(0) + } else { + attributes.map(a => relation.output.indexWhere(_.exprId == a.exprId)) + } new Iterator[Row] { - // Find the ordinals of the requested columns. If none are requested, use the first. - val requestedColumns = - if (attributes.isEmpty) Seq(0) else attributes.map(relation.output.indexOf(_)) + private[this] var columnBuffers: Array[ByteBuffer] = null + private[this] var columnAccessors: Seq[ColumnAccessor] = null + nextBatch() + + private[this] val nextRow = new GenericMutableRow(columnAccessors.length) - val columnAccessors = requestedColumns.map(columnBuffers(_)).map(ColumnAccessor(_)) - val nextRow = new GenericMutableRow(columnAccessors.length) + def nextBatch() = { + columnBuffers = iterator.next() + columnAccessors = requestedColumns.map(columnBuffers(_)).map(ColumnAccessor(_)) + } override def next() = { + if (!columnAccessors.head.hasNext) { + nextBatch() + } + var i = 0 while (i < nextRow.length) { columnAccessors(i).extractTo(nextRow, i) @@ -97,7 +135,7 @@ private[sql] case class InMemoryColumnarTableScan( nextRow } - override def hasNext = columnAccessors.head.hasNext + override def hasNext = columnAccessors.head.hasNext || iterator.hasNext } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala index d008806eedbe1..f631ee76fcd78 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala @@ -36,9 +36,9 @@ import org.apache.spark.sql.Row * }}} */ private[sql] trait NullableColumnBuilder extends ColumnBuilder { - private var nulls: ByteBuffer = _ + protected var nulls: ByteBuffer = _ + protected var nullCount: Int = _ private var pos: Int = _ - private var nullCount: Int = _ abstract override def initialize(initialSize: Int, columnName: String, useCompression: Boolean) { nulls = ByteBuffer.allocate(1024) @@ -78,4 +78,9 @@ private[sql] trait NullableColumnBuilder extends ColumnBuilder { buffer.rewind() buffer } + + protected def buildNonNulls(): ByteBuffer = { + nulls.limit(nulls.position()).rewind() + super.build() + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala index 4c6675c3c87bf..a5826bb033e41 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala @@ -19,7 +19,8 @@ package org.apache.spark.sql.columnar.compression import java.nio.{ByteBuffer, ByteOrder} -import org.apache.spark.sql.{Logging, Row} +import org.apache.spark.Logging +import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.types.NativeType import org.apache.spark.sql.columnar.{ColumnBuilder, NativeColumnBuilder} @@ -45,8 +46,6 @@ private[sql] trait CompressibleColumnBuilder[T <: NativeType] this: NativeColumnBuilder[T] with WithCompressionSchemes => - import CompressionScheme._ - var compressionEncoders: Seq[Encoder[T]] = _ abstract override def initialize(initialSize: Int, columnName: String, useCompression: Boolean) { @@ -80,28 +79,32 @@ private[sql] trait CompressibleColumnBuilder[T <: NativeType] } } - abstract override def build() = { - val rawBuffer = super.build() + override def build() = { + val nonNullBuffer = buildNonNulls() + val typeId = nonNullBuffer.getInt() val encoder: Encoder[T] = { val candidate = compressionEncoders.minBy(_.compressionRatio) if (isWorthCompressing(candidate)) candidate else PassThrough.encoder } - val headerSize = columnHeaderSize(rawBuffer) + // Header = column type ID + null count + null positions + val headerSize = 4 + 4 + nulls.limit() val compressedSize = if (encoder.compressedSize == 0) { - rawBuffer.limit - headerSize + nonNullBuffer.remaining() } else { encoder.compressedSize } - // Reserves 4 bytes for compression scheme ID val compressedBuffer = ByteBuffer + // Reserves 4 bytes for compression scheme ID .allocate(headerSize + 4 + compressedSize) .order(ByteOrder.nativeOrder) + // Write the header + .putInt(typeId) + .putInt(nullCount) + .put(nulls) - copyColumnHeader(rawBuffer, compressedBuffer) - - logger.info(s"Compressor for [$columnName]: $encoder, ratio: ${encoder.compressionRatio}") - encoder.compress(rawBuffer, compressedBuffer, columnType) + logInfo(s"Compressor for [$columnName]: $encoder, ratio: ${encoder.compressionRatio}") + encoder.compress(nonNullBuffer, compressedBuffer, columnType) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala index ba1810dd2ae66..7797f75177893 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala @@ -67,22 +67,6 @@ private[sql] object CompressionScheme { s"Unrecognized compression scheme type ID: $typeId")) } - def copyColumnHeader(from: ByteBuffer, to: ByteBuffer) { - // Writes column type ID - to.putInt(from.getInt()) - - // Writes null count - val nullCount = from.getInt() - to.putInt(nullCount) - - // Writes null positions - var i = 0 - while (i < nullCount) { - to.putInt(from.getInt()) - i += 1 - } - } - def columnHeaderSize(columnBuffer: ByteBuffer): Int = { val header = columnBuffer.duplicate().order(ByteOrder.nativeOrder) val nullCount = header.getInt(4) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala index c1ced8bfa404a..463a1d32d7fd7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala @@ -42,8 +42,8 @@ case class Aggregate( partial: Boolean, groupingExpressions: Seq[Expression], aggregateExpressions: Seq[NamedExpression], - child: SparkPlan)(@transient sqlContext: SQLContext) - extends UnaryNode with NoBind { + child: SparkPlan) + extends UnaryNode { override def requiredChildDistribution = if (partial) { @@ -56,8 +56,6 @@ case class Aggregate( } } - override def otherCopyArgs = sqlContext :: Nil - // HACK: Generators don't correctly preserve their output through serializations so we grab // out child's output attributes statically here. private[this] val childOutput = child.output @@ -138,7 +136,7 @@ case class Aggregate( i += 1 } } - val resultProjection = new Projection(resultExpressions, computedSchema) + val resultProjection = new InterpretedProjection(resultExpressions, computedSchema) val aggregateResults = new GenericMutableRow(computedAggregates.length) var i = 0 @@ -152,7 +150,7 @@ case class Aggregate( } else { child.execute().mapPartitions { iter => val hashTable = new HashMap[Row, Array[AggregateFunction]] - val groupingProjection = new MutableProjection(groupingExpressions, childOutput) + val groupingProjection = new InterpretedMutableProjection(groupingExpressions, childOutput) var currentRow: Row = null while (iter.hasNext) { @@ -175,7 +173,8 @@ case class Aggregate( private[this] val hashTableIter = hashTable.entrySet().iterator() private[this] val aggregateResults = new GenericMutableRow(computedAggregates.length) private[this] val resultProjection = - new MutableProjection(resultExpressions, computedSchema ++ namedGroups.map(_._2)) + new InterpretedMutableProjection( + resultExpressions, computedSchema ++ namedGroups.map(_._2)) private[this] val joinedRow = new JoinedRow override final def hasNext: Boolean = hashTableIter.hasNext diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala index 00010ef6e798a..77dc2ad733215 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala @@ -22,7 +22,7 @@ import org.apache.spark.{HashPartitioner, RangePartitioner, SparkConf} import org.apache.spark.rdd.ShuffledRDD import org.apache.spark.sql.{SQLContext, Row} import org.apache.spark.sql.catalyst.errors.attachTree -import org.apache.spark.sql.catalyst.expressions.{NoBind, MutableProjection, RowOrdering} +import org.apache.spark.sql.catalyst.expressions.RowOrdering import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.util.MutablePair @@ -31,7 +31,7 @@ import org.apache.spark.util.MutablePair * :: DeveloperApi :: */ @DeveloperApi -case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends UnaryNode with NoBind { +case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends UnaryNode { override def outputPartitioning = newPartitioning @@ -42,12 +42,14 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una case HashPartitioning(expressions, numPartitions) => // TODO: Eliminate redundant expressions in grouping key and value. val rdd = child.execute().mapPartitions { iter => - val hashExpressions = new MutableProjection(expressions, child.output) + @transient val hashExpressions = + newMutableProjection(expressions, child.output)() + val mutablePair = new MutablePair[Row, Row]() iter.map(r => mutablePair.update(hashExpressions(r), r)) } val part = new HashPartitioner(numPartitions) - val shuffled = new ShuffledRDD[Row, Row, Row, MutablePair[Row, Row]](rdd, part) + val shuffled = new ShuffledRDD[Row, Row, Row](rdd, part) shuffled.setSerializer(new SparkSqlSerializer(new SparkConf(false))) shuffled.map(_._2) @@ -60,7 +62,7 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una iter.map(row => mutablePair.update(row, null)) } val part = new RangePartitioner(numPartitions, rdd, ascending = true) - val shuffled = new ShuffledRDD[Row, Null, Null, MutablePair[Row, Null]](rdd, part) + val shuffled = new ShuffledRDD[Row, Null, Null](rdd, part) shuffled.setSerializer(new SparkSqlSerializer(new SparkConf(false))) shuffled.map(_._1) @@ -71,7 +73,7 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una iter.map(r => mutablePair.update(null, r)) } val partitioner = new HashPartitioner(1) - val shuffled = new ShuffledRDD[Null, Row, Row, MutablePair[Null, Row]](rdd, partitioner) + val shuffled = new ShuffledRDD[Null, Row, Row](rdd, partitioner) shuffled.setSerializer(new SparkSqlSerializer(new SparkConf(false))) shuffled.map(_._2) @@ -99,7 +101,7 @@ private[sql] case class AddExchange(sqlContext: SQLContext) extends Rule[SparkPl !operator.requiredChildDistribution.zip(operator.children).map { case (required, child) => val valid = child.outputPartitioning.satisfies(required) - logger.debug( + logDebug( s"${if (valid) "Valid" else "Invalid"} distribution," + s"required: $required current: ${child.outputPartitioning}") valid diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala index 47b3d00262dbb..c386fd121c5de 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala @@ -47,23 +47,26 @@ case class Generate( } } - override def output = + // This must be a val since the generator output expr ids are not preserved by serialization. + override val output = if (join) child.output ++ generatorOutput else generatorOutput + val boundGenerator = BindReferences.bindReference(generator, child.output) + override def execute() = { if (join) { child.execute().mapPartitions { iter => val nullValues = Seq.fill(generator.output.size)(Literal(null)) // Used to produce rows with no matches when outer = true. val outerProjection = - new Projection(child.output ++ nullValues, child.output) + newProjection(child.output ++ nullValues, child.output) val joinProjection = - new Projection(child.output ++ generator.output, child.output ++ generator.output) + newProjection(child.output ++ generator.output, child.output ++ generator.output) val joinedRow = new JoinedRow iter.flatMap {row => - val outputRows = generator.eval(row) + val outputRows = boundGenerator.eval(row) if (outer && outputRows.isEmpty) { outerProjection(row) :: Nil } else { @@ -72,7 +75,7 @@ case class Generate( } } } else { - child.execute().mapPartitions(iter => iter.flatMap(row => generator.eval(row))) + child.execute().mapPartitions(iter => iter.flatMap(row => boundGenerator.eval(row))) } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala new file mode 100644 index 0000000000000..4a26934c49c93 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala @@ -0,0 +1,200 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.physical._ +import org.apache.spark.sql.catalyst.types._ + +case class AggregateEvaluation( + schema: Seq[Attribute], + initialValues: Seq[Expression], + update: Seq[Expression], + result: Expression) + +/** + * :: DeveloperApi :: + * Alternate version of aggregation that leverages projection and thus code generation. + * Aggregations are converted into a set of projections from a aggregation buffer tuple back onto + * itself. Currently only used for simple aggregations like SUM, COUNT, or AVERAGE are supported. + * + * @param partial if true then aggregation is done partially on local data without shuffling to + * ensure all values where `groupingExpressions` are equal are present. + * @param groupingExpressions expressions that are evaluated to determine grouping. + * @param aggregateExpressions expressions that are computed for each group. + * @param child the input data source. + */ +@DeveloperApi +case class GeneratedAggregate( + partial: Boolean, + groupingExpressions: Seq[Expression], + aggregateExpressions: Seq[NamedExpression], + child: SparkPlan) + extends UnaryNode { + + override def requiredChildDistribution = + if (partial) { + UnspecifiedDistribution :: Nil + } else { + if (groupingExpressions == Nil) { + AllTuples :: Nil + } else { + ClusteredDistribution(groupingExpressions) :: Nil + } + } + + override def output = aggregateExpressions.map(_.toAttribute) + + override def execute() = { + val aggregatesToCompute = aggregateExpressions.flatMap { a => + a.collect { case agg: AggregateExpression => agg} + } + + val computeFunctions = aggregatesToCompute.map { + case c @ Count(expr) => + val currentCount = AttributeReference("currentCount", LongType, nullable = false)() + val initialValue = Literal(0L) + val updateFunction = If(IsNotNull(expr), Add(currentCount, Literal(1L)), currentCount) + val result = currentCount + + AggregateEvaluation(currentCount :: Nil, initialValue :: Nil, updateFunction :: Nil, result) + + case Sum(expr) => + val currentSum = AttributeReference("currentSum", expr.dataType, nullable = false)() + val initialValue = Cast(Literal(0L), expr.dataType) + + // Coalasce avoids double calculation... + // but really, common sub expression elimination would be better.... + val updateFunction = Coalesce(Add(expr, currentSum) :: currentSum :: Nil) + val result = currentSum + + AggregateEvaluation(currentSum :: Nil, initialValue :: Nil, updateFunction :: Nil, result) + + case a @ Average(expr) => + val currentCount = AttributeReference("currentCount", LongType, nullable = false)() + val currentSum = AttributeReference("currentSum", expr.dataType, nullable = false)() + val initialCount = Literal(0L) + val initialSum = Cast(Literal(0L), expr.dataType) + val updateCount = If(IsNotNull(expr), Add(currentCount, Literal(1L)), currentCount) + val updateSum = Coalesce(Add(expr, currentSum) :: currentSum :: Nil) + + val result = Divide(Cast(currentSum, DoubleType), Cast(currentCount, DoubleType)) + + AggregateEvaluation( + currentCount :: currentSum :: Nil, + initialCount :: initialSum :: Nil, + updateCount :: updateSum :: Nil, + result + ) + } + + val computationSchema = computeFunctions.flatMap(_.schema) + + val resultMap: Map[Long, Expression] = aggregatesToCompute.zip(computeFunctions).map { + case (agg, func) => agg.id -> func.result + }.toMap + + val namedGroups = groupingExpressions.zipWithIndex.map { + case (ne: NamedExpression, _) => (ne, ne) + case (e, i) => (e, Alias(e, s"GroupingExpr$i")()) + } + + val groupMap: Map[Expression, Attribute] = + namedGroups.map { case (k, v) => k -> v.toAttribute}.toMap + + // The set of expressions that produce the final output given the aggregation buffer and the + // grouping expressions. + val resultExpressions = aggregateExpressions.map(_.transform { + case e: Expression if resultMap.contains(e.id) => resultMap(e.id) + case e: Expression if groupMap.contains(e) => groupMap(e) + }) + + child.execute().mapPartitions { iter => + // Builds a new custom class for holding the results of aggregation for a group. + val initialValues = computeFunctions.flatMap(_.initialValues) + val newAggregationBuffer = newProjection(initialValues, child.output) + log.info(s"Initial values: ${initialValues.mkString(",")}") + + // A projection that computes the group given an input tuple. + val groupProjection = newProjection(groupingExpressions, child.output) + log.info(s"Grouping Projection: ${groupingExpressions.mkString(",")}") + + // A projection that is used to update the aggregate values for a group given a new tuple. + // This projection should be targeted at the current values for the group and then applied + // to a joined row of the current values with the new input row. + val updateExpressions = computeFunctions.flatMap(_.update) + val updateSchema = computeFunctions.flatMap(_.schema) ++ child.output + val updateProjection = newMutableProjection(updateExpressions, updateSchema)() + log.info(s"Update Expressions: ${updateExpressions.mkString(",")}") + + // A projection that produces the final result, given a computation. + val resultProjectionBuilder = + newMutableProjection( + resultExpressions, + (namedGroups.map(_._2.toAttribute) ++ computationSchema).toSeq) + log.info(s"Result Projection: ${resultExpressions.mkString(",")}") + + val joinedRow = new JoinedRow + + if (groupingExpressions.isEmpty) { + // TODO: Codegening anything other than the updateProjection is probably over kill. + val buffer = newAggregationBuffer(EmptyRow).asInstanceOf[MutableRow] + var currentRow: Row = null + updateProjection.target(buffer) + + while (iter.hasNext) { + currentRow = iter.next() + updateProjection(joinedRow(buffer, currentRow)) + } + + val resultProjection = resultProjectionBuilder() + Iterator(resultProjection(buffer)) + } else { + val buffers = new java.util.HashMap[Row, MutableRow]() + + var currentRow: Row = null + while (iter.hasNext) { + currentRow = iter.next() + val currentGroup = groupProjection(currentRow) + var currentBuffer = buffers.get(currentGroup) + if (currentBuffer == null) { + currentBuffer = newAggregationBuffer(EmptyRow).asInstanceOf[MutableRow] + buffers.put(currentGroup, currentBuffer) + } + // Target the projection at the current aggregation buffer and then project the updated + // values. + updateProjection.target(currentBuffer)(joinedRow(currentBuffer, currentRow)) + } + + new Iterator[Row] { + private[this] val resultIterator = buffers.entrySet.iterator() + private[this] val resultProjection = resultProjectionBuilder() + + def hasNext = resultIterator.hasNext + + def next() = { + val currentGroup = resultIterator.next() + resultProjection(joinedRow(currentGroup.getKey, currentGroup.getValue)) + } + } + } + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala index 27dc091b85812..21cbbc9772a00 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala @@ -18,22 +18,55 @@ package org.apache.spark.sql.execution import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.Logging import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{Logging, Row} + + +import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.trees import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation -import org.apache.spark.sql.catalyst.expressions.GenericRow +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.plans.QueryPlan -import org.apache.spark.sql.catalyst.plans.logical.BaseRelation +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.plans.physical._ + +object SparkPlan { + protected[sql] val currentContext = new ThreadLocal[SQLContext]() +} + /** * :: DeveloperApi :: */ @DeveloperApi -abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging { +abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializable { self: Product => + /** + * A handle to the SQL Context that was used to create this plan. Since many operators need + * access to the sqlContext for RDD operations or configuration this field is automatically + * populated by the query planning infrastructure. + */ + @transient + protected val sqlContext = SparkPlan.currentContext.get() + + protected def sparkContext = sqlContext.sparkContext + + // sqlContext will be null when we are being deserialized on the slaves. In this instance + // the value of codegenEnabled will be set by the desserializer after the constructor has run. + val codegenEnabled: Boolean = if (sqlContext != null) { + sqlContext.codegenEnabled + } else { + false + } + + /** Overridden make copy also propogates sqlContext to copied plan. */ + override def makeCopy(newArgs: Array[AnyRef]): this.type = { + SparkPlan.currentContext.set(sqlContext) + super.makeCopy(newArgs) + } + // TODO: Move to `DistributedPlan` /** Specifies how data is partitioned across different nodes in the cluster. */ def outputPartitioning: Partitioning = UnknownPartitioning(0) // TODO: WRONG WIDTH! @@ -51,8 +84,46 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging { */ def executeCollect(): Array[Row] = execute().map(_.copy()).collect() - protected def buildRow(values: Seq[Any]): Row = - new GenericRow(values.toArray) + protected def newProjection( + expressions: Seq[Expression], inputSchema: Seq[Attribute]): Projection = { + log.debug( + s"Creating Projection: $expressions, inputSchema: $inputSchema, codegen:$codegenEnabled") + if (codegenEnabled) { + GenerateProjection(expressions, inputSchema) + } else { + new InterpretedProjection(expressions, inputSchema) + } + } + + protected def newMutableProjection( + expressions: Seq[Expression], + inputSchema: Seq[Attribute]): () => MutableProjection = { + log.debug( + s"Creating MutableProj: $expressions, inputSchema: $inputSchema, codegen:$codegenEnabled") + if(codegenEnabled) { + GenerateMutableProjection(expressions, inputSchema) + } else { + () => new InterpretedMutableProjection(expressions, inputSchema) + } + } + + + protected def newPredicate( + expression: Expression, inputSchema: Seq[Attribute]): (Row) => Boolean = { + if (codegenEnabled) { + GeneratePredicate(expression, inputSchema) + } else { + InterpretedPredicate(expression, inputSchema) + } + } + + protected def newOrdering(order: Seq[SortOrder], inputSchema: Seq[Attribute]): Ordering[Row] = { + if (codegenEnabled) { + GenerateOrdering(order, inputSchema) + } else { + new RowOrdering(order, inputSchema) + } + } } /** @@ -66,8 +137,8 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging { * linking. */ @DeveloperApi -case class SparkLogicalPlan(alreadyPlanned: SparkPlan, tableName: String = "SparkLogicalPlan") - extends BaseRelation with MultiInstanceRelation { +case class SparkLogicalPlan(alreadyPlanned: SparkPlan)(@transient sqlContext: SQLContext) + extends LogicalPlan with MultiInstanceRelation { def output = alreadyPlanned.output override def references = Set.empty @@ -78,9 +149,15 @@ case class SparkLogicalPlan(alreadyPlanned: SparkPlan, tableName: String = "Spar alreadyPlanned match { case ExistingRdd(output, rdd) => ExistingRdd(output.map(_.newInstance), rdd) case _ => sys.error("Multiple instance of the same relation detected.") - }, tableName) - .asInstanceOf[this.type] + })(sqlContext).asInstanceOf[this.type] } + + @transient override lazy val statistics = Statistics( + // TODO: Instead of returning a default value here, find a way to return a meaningful size + // estimate for RDDs. See PR 1238 for more discussions. + sizeInBytes = BigInt(sqlContext.defaultSizeInBytes) + ) + } private[sql] trait LeafNode extends SparkPlan with trees.LeafNode[SparkPlan] { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala index 34b355e906695..34654447a5f4b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala @@ -24,10 +24,10 @@ import scala.reflect.ClassTag import com.clearspring.analytics.stream.cardinality.HyperLogLog import com.esotericsoftware.kryo.io.{Input, Output} import com.esotericsoftware.kryo.{Serializer, Kryo} -import com.twitter.chill.AllScalaRegistrar +import com.twitter.chill.{AllScalaRegistrar, ResourcePool} import org.apache.spark.{SparkEnv, SparkConf} -import org.apache.spark.serializer.KryoSerializer +import org.apache.spark.serializer.{SerializerInstance, KryoSerializer} import org.apache.spark.util.MutablePair import org.apache.spark.util.Utils @@ -48,22 +48,41 @@ private[sql] class SparkSqlSerializer(conf: SparkConf) extends KryoSerializer(co } } -private[sql] object SparkSqlSerializer { - // TODO (lian) Using KryoSerializer here is workaround, needs further investigation - // Using SparkSqlSerializer here makes BasicQuerySuite to fail because of Kryo serialization - // related error. - @transient lazy val ser: KryoSerializer = { +private[execution] class KryoResourcePool(size: Int) + extends ResourcePool[SerializerInstance](size) { + + val ser: KryoSerializer = { val sparkConf = Option(SparkEnv.get).map(_.conf).getOrElse(new SparkConf()) + // TODO (lian) Using KryoSerializer here is workaround, needs further investigation + // Using SparkSqlSerializer here makes BasicQuerySuite to fail because of Kryo serialization + // related error. new KryoSerializer(sparkConf) } - def serialize[T: ClassTag](o: T): Array[Byte] = { - ser.newInstance().serialize(o).array() - } + def newInstance() = ser.newInstance() +} - def deserialize[T: ClassTag](bytes: Array[Byte]): T = { - ser.newInstance().deserialize[T](ByteBuffer.wrap(bytes)) +private[sql] object SparkSqlSerializer { + @transient lazy val resourcePool = new KryoResourcePool(30) + + private[this] def acquireRelease[O](fn: SerializerInstance => O): O = { + val kryo = resourcePool.borrow + try { + fn(kryo) + } finally { + resourcePool.release(kryo) + } } + + def serialize[T: ClassTag](o: T): Array[Byte] = + acquireRelease { k => + k.serialize(o).array() + } + + def deserialize[T: ClassTag](bytes: Array[Byte]): T = + acquireRelease { k => + k.deserialize[T](ByteBuffer.wrap(bytes)) + } } private[sql] class BigDecimalSerializer extends Serializer[BigDecimal] { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 7080074a69c07..f0c958fdb537f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -21,7 +21,7 @@ import org.apache.spark.sql.{SQLContext, execution} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning._ import org.apache.spark.sql.catalyst.plans._ -import org.apache.spark.sql.catalyst.plans.logical.{BaseRelation, LogicalPlan} +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.columnar.{InMemoryRelation, InMemoryColumnarTableScan} import org.apache.spark.sql.parquet._ @@ -39,7 +39,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { // no predicate can be evaluated by matching hash keys case logical.Join(left, right, LeftSemi, condition) => execution.LeftSemiJoinBNL( - planLater(left), planLater(right), condition)(sqlContext) :: Nil + planLater(left), planLater(right), condition) :: Nil case _ => Nil } } @@ -47,9 +47,19 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { /** * Uses the ExtractEquiJoinKeys pattern to find joins where at least some of the predicates can be * evaluated by matching hash keys. + * + * This strategy applies a simple optimization based on the estimates of the physical sizes of + * the two join sides. When planning a [[execution.BroadcastHashJoin]], if one side has an + * estimated physical size smaller than the user-settable threshold + * [[org.apache.spark.sql.SQLConf.AUTO_BROADCASTJOIN_THRESHOLD]], the planner would mark it as the + * ''build'' relation and mark the other relation as the ''stream'' side. The build table will be + * ''broadcasted'' to all of the executors involved in the join, as a + * [[org.apache.spark.broadcast.Broadcast]] object. If both estimates exceed the threshold, they + * will instead be used to decide the build side in a [[execution.ShuffledHashJoin]]. */ object HashJoin extends Strategy with PredicateHelper { - private[this] def broadcastHashJoin( + + private[this] def makeBroadcastHashJoin( leftKeys: Seq[Expression], rightKeys: Seq[Expression], left: LogicalPlan, @@ -57,102 +67,102 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { condition: Option[Expression], side: BuildSide) = { val broadcastHashJoin = execution.BroadcastHashJoin( - leftKeys, rightKeys, side, planLater(left), planLater(right))(sqlContext) + leftKeys, rightKeys, side, planLater(left), planLater(right)) condition.map(Filter(_, broadcastHashJoin)).getOrElse(broadcastHashJoin) :: Nil } - def broadcastTables: Seq[String] = sqlContext.joinBroadcastTables.split(",").toBuffer - def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - case ExtractEquiJoinKeys( - Inner, - leftKeys, - rightKeys, - condition, - left, - right @ PhysicalOperation(_, _, b: BaseRelation)) - if broadcastTables.contains(b.tableName) => - broadcastHashJoin(leftKeys, rightKeys, left, right, condition, BuildRight) + case ExtractEquiJoinKeys(Inner, leftKeys, rightKeys, condition, left, right) + if sqlContext.autoBroadcastJoinThreshold > 0 && + right.statistics.sizeInBytes <= sqlContext.autoBroadcastJoinThreshold => + makeBroadcastHashJoin(leftKeys, rightKeys, left, right, condition, BuildRight) - case ExtractEquiJoinKeys( - Inner, - leftKeys, - rightKeys, - condition, - left @ PhysicalOperation(_, _, b: BaseRelation), - right) - if broadcastTables.contains(b.tableName) => - broadcastHashJoin(leftKeys, rightKeys, left, right, condition, BuildLeft) + case ExtractEquiJoinKeys(Inner, leftKeys, rightKeys, condition, left, right) + if sqlContext.autoBroadcastJoinThreshold > 0 && + left.statistics.sizeInBytes <= sqlContext.autoBroadcastJoinThreshold => + makeBroadcastHashJoin(leftKeys, rightKeys, left, right, condition, BuildLeft) case ExtractEquiJoinKeys(Inner, leftKeys, rightKeys, condition, left, right) => + val buildSide = + if (right.statistics.sizeInBytes <= left.statistics.sizeInBytes) { + BuildRight + } else { + BuildLeft + } val hashJoin = execution.ShuffledHashJoin( - leftKeys, rightKeys, BuildRight, planLater(left), planLater(right)) + leftKeys, rightKeys, buildSide, planLater(left), planLater(right)) condition.map(Filter(_, hashJoin)).getOrElse(hashJoin) :: Nil + case ExtractEquiJoinKeys(joinType, leftKeys, rightKeys, condition, left, right) => + execution.HashOuterJoin( + leftKeys, rightKeys, joinType, condition, planLater(left), planLater(right)) :: Nil + case _ => Nil } } - object PartialAggregation extends Strategy { + object HashAggregation extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - case logical.Aggregate(groupingExpressions, aggregateExpressions, child) => - // Collect all aggregate expressions. - val allAggregates = - aggregateExpressions.flatMap(_ collect { case a: AggregateExpression => a }) - // Collect all aggregate expressions that can be computed partially. - val partialAggregates = - aggregateExpressions.flatMap(_ collect { case p: PartialAggregate => p }) + // Aggregations that can be performed in two phases, before and after the shuffle. - // Only do partial aggregation if supported by all aggregate expressions. - if (allAggregates.size == partialAggregates.size) { - // Create a map of expressions to their partial evaluations for all aggregate expressions. - val partialEvaluations: Map[Long, SplitEvaluation] = - partialAggregates.map(a => (a.id, a.asPartial)).toMap - - // We need to pass all grouping expressions though so the grouping can happen a second - // time. However some of them might be unnamed so we alias them allowing them to be - // referenced in the second aggregation. - val namedGroupingExpressions: Map[Expression, NamedExpression] = groupingExpressions.map { - case n: NamedExpression => (n, n) - case other => (other, Alias(other, "PartialGroup")()) - }.toMap - - // Replace aggregations with a new expression that computes the result from the already - // computed partial evaluations and grouping values. - val rewrittenAggregateExpressions = aggregateExpressions.map(_.transformUp { - case e: Expression if partialEvaluations.contains(e.id) => - partialEvaluations(e.id).finalEvaluation - case e: Expression if namedGroupingExpressions.contains(e) => - namedGroupingExpressions(e).toAttribute - }).asInstanceOf[Seq[NamedExpression]] - - val partialComputation = - (namedGroupingExpressions.values ++ - partialEvaluations.values.flatMap(_.partialEvaluations)).toSeq - - // Construct two phased aggregation. - execution.Aggregate( + // Cases where all aggregates can be codegened. + case PartialAggregation( + namedGroupingAttributes, + rewrittenAggregateExpressions, + groupingExpressions, + partialComputation, + child) + if canBeCodeGened( + allAggregates(partialComputation) ++ + allAggregates(rewrittenAggregateExpressions)) && + codegenEnabled => + execution.GeneratedAggregate( partial = false, - namedGroupingExpressions.values.map(_.toAttribute).toSeq, + namedGroupingAttributes, rewrittenAggregateExpressions, - execution.Aggregate( + execution.GeneratedAggregate( partial = true, groupingExpressions, partialComputation, - planLater(child))(sqlContext))(sqlContext) :: Nil - } else { - Nil - } + planLater(child))) :: Nil + + // Cases where some aggregate can not be codegened + case PartialAggregation( + namedGroupingAttributes, + rewrittenAggregateExpressions, + groupingExpressions, + partialComputation, + child) => + execution.Aggregate( + partial = false, + namedGroupingAttributes, + rewrittenAggregateExpressions, + execution.Aggregate( + partial = true, + groupingExpressions, + partialComputation, + planLater(child))) :: Nil + case _ => Nil } + + def canBeCodeGened(aggs: Seq[AggregateExpression]) = !aggs.exists { + case _: Sum | _: Count => false + case _ => true + } + + def allAggregates(exprs: Seq[Expression]) = + exprs.flatMap(_.collect { case a: AggregateExpression => a }) } object BroadcastNestedLoopJoin extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case logical.Join(left, right, joinType, condition) => + val buildSide = + if (right.statistics.sizeInBytes <= left.statistics.sizeInBytes) BuildRight else BuildLeft execution.BroadcastNestedLoopJoin( - planLater(left), planLater(right), joinType, condition)(sqlContext) :: Nil + planLater(left), planLater(right), buildSide, joinType, condition) :: Nil case _ => Nil } } @@ -171,16 +181,10 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { protected lazy val singleRowRdd = sparkContext.parallelize(Seq(new GenericRow(Array[Any]()): Row), 1) - def convertToCatalyst(a: Any): Any = a match { - case s: Seq[Any] => s.map(convertToCatalyst) - case p: Product => new GenericRow(p.productIterator.map(convertToCatalyst).toArray) - case other => other - } - object TakeOrdered extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case logical.Limit(IntegerLiteral(limit), logical.Sort(order, child)) => - execution.TakeOrdered(limit, order, planLater(child))(sqlContext) :: Nil + execution.TakeOrdered(limit, order, planLater(child)) :: Nil case _ => Nil } } @@ -190,11 +194,11 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { // TODO: need to support writing to other types of files. Unify the below code paths. case logical.WriteToFile(path, child) => val relation = - ParquetRelation.create(path, child, sparkContext.hadoopConfiguration) + ParquetRelation.create(path, child, sparkContext.hadoopConfiguration, sqlContext) // Note: overwrite=false because otherwise the metadata we just created will be deleted - InsertIntoParquetTable(relation, planLater(child), overwrite=false)(sqlContext) :: Nil + InsertIntoParquetTable(relation, planLater(child), overwrite = false) :: Nil case logical.InsertIntoTable(table: ParquetRelation, partition, child, overwrite) => - InsertIntoParquetTable(table, planLater(child), overwrite)(sqlContext) :: Nil + InsertIntoParquetTable(table, planLater(child), overwrite) :: Nil case PhysicalOperation(projectList, filters: Seq[Expression], relation: ParquetRelation) => val prunePushedDownFilters = if (sparkContext.conf.getBoolean(ParquetFilters.PARQUET_FILTER_PUSHDOWN_ENABLED, true)) { @@ -223,7 +227,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { projectList, filters, prunePushedDownFilters, - ParquetTableScan(_, relation, filters)(sqlContext)) :: Nil + ParquetTableScan(_, relation, filters)) :: Nil case _ => Nil } @@ -247,8 +251,8 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case logical.Distinct(child) => - execution.Aggregate( - partial = false, child.output, child.output, planLater(child))(sqlContext) :: Nil + execution.Distinct(partial = false, + execution.Distinct(partial = true, planLater(child))) :: Nil case logical.Sort(sortExprs, child) => // This sort is a global sort. Its requiredDistribution will be an OrderedDistribution. execution.Sort(sortExprs, global = true, planLater(child)):: Nil @@ -261,20 +265,19 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case logical.Filter(condition, child) => execution.Filter(condition, planLater(child)) :: Nil case logical.Aggregate(group, agg, child) => - execution.Aggregate(partial = false, group, agg, planLater(child))(sqlContext) :: Nil + execution.Aggregate(partial = false, group, agg, planLater(child)) :: Nil case logical.Sample(fraction, withReplacement, seed, child) => execution.Sample(fraction, withReplacement, seed, planLater(child)) :: Nil case logical.LocalRelation(output, data) => - val dataAsRdd = - sparkContext.parallelize(data.map(r => - new GenericRow(r.productIterator.map(convertToCatalyst).toArray): Row)) - execution.ExistingRdd(output, dataAsRdd) :: Nil + ExistingRdd( + output, + ExistingRdd.productToRowRdd(sparkContext.parallelize(data, numPartitions))) :: Nil case logical.Limit(IntegerLiteral(limit), child) => - execution.Limit(limit, planLater(child))(sqlContext) :: Nil + execution.Limit(limit, planLater(child)) :: Nil case Unions(unionChildren) => - execution.Union(unionChildren.map(planLater))(sqlContext) :: Nil - case logical.Except(left,right) => - execution.Except(planLater(left),planLater(right)) :: Nil + execution.Union(unionChildren.map(planLater)) :: Nil + case logical.Except(left, right) => + execution.Except(planLater(left), planLater(right)) :: Nil case logical.Intersect(left, right) => execution.Intersect(planLater(left), planLater(right)) :: Nil case logical.Generate(generator, join, outer, _, child) => @@ -283,7 +286,9 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { execution.ExistingRdd(Nil, singleRowRdd) :: Nil case logical.Repartition(expressions, child) => execution.Exchange(HashPartitioning(expressions, numPartitions), planLater(child)) :: Nil - case SparkLogicalPlan(existingPlan, _) => existingPlan :: Nil + case e @ EvaluatePython(udf, child) => + BatchPythonEvaluation(udf, e.output, planLater(child)) :: Nil + case SparkLogicalPlan(existingPlan) => existingPlan :: Nil case _ => Nil } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index 97abd636ab5fb..0027f3cf1fc79 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.errors._ import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.physical.{OrderedDistribution, UnspecifiedDistribution} +import org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, OrderedDistribution, UnspecifiedDistribution} import org.apache.spark.util.MutablePair /** @@ -37,9 +37,11 @@ import org.apache.spark.util.MutablePair case class Project(projectList: Seq[NamedExpression], child: SparkPlan) extends UnaryNode { override def output = projectList.map(_.toAttribute) - override def execute() = child.execute().mapPartitions { iter => - @transient val reusableProjection = new MutableProjection(projectList) - iter.map(reusableProjection) + @transient lazy val buildProjection = newMutableProjection(projectList, child.output) + + def execute() = child.execute().mapPartitions { iter => + val resuableProjection = buildProjection() + iter.map(resuableProjection) } } @@ -50,8 +52,10 @@ case class Project(projectList: Seq[NamedExpression], child: SparkPlan) extends case class Filter(condition: Expression, child: SparkPlan) extends UnaryNode { override def output = child.output - override def execute() = child.execute().mapPartitions { iter => - iter.filter(condition.eval(_).asInstanceOf[Boolean]) + @transient lazy val conditionEvaluator = newPredicate(condition, child.output) + + def execute() = child.execute().mapPartitions { iter => + iter.filter(conditionEvaluator) } } @@ -72,12 +76,10 @@ case class Sample(fraction: Double, withReplacement: Boolean, seed: Long, child: * :: DeveloperApi :: */ @DeveloperApi -case class Union(children: Seq[SparkPlan])(@transient sqlContext: SQLContext) extends SparkPlan { +case class Union(children: Seq[SparkPlan]) extends SparkPlan { // TODO: attributes output by union should be distinct for nullability purposes override def output = children.head.output - override def execute() = sqlContext.sparkContext.union(children.map(_.execute())) - - override def otherCopyArgs = sqlContext :: Nil + override def execute() = sparkContext.union(children.map(_.execute())) } /** @@ -89,13 +91,11 @@ case class Union(children: Seq[SparkPlan])(@transient sqlContext: SQLContext) ex * repartition all the data to a single partition to compute the global limit. */ @DeveloperApi -case class Limit(limit: Int, child: SparkPlan)(@transient sqlContext: SQLContext) +case class Limit(limit: Int, child: SparkPlan) extends UnaryNode { // TODO: Implement a partition local limit, and use a strategy to generate the proper limit plan: // partition local limit -> exchange into one partition -> partition local limit again - override def otherCopyArgs = sqlContext :: Nil - override def output = child.output /** @@ -148,7 +148,7 @@ case class Limit(limit: Int, child: SparkPlan)(@transient sqlContext: SQLContext iter.take(limit).map(row => mutablePair.update(false, row)) } val part = new HashPartitioner(1) - val shuffled = new ShuffledRDD[Boolean, Row, Row, MutablePair[Boolean, Row]](rdd, part) + val shuffled = new ShuffledRDD[Boolean, Row, Row](rdd, part) shuffled.setSerializer(new SparkSqlSerializer(new SparkConf(false))) shuffled.mapPartitions(_.take(limit).map(_._2)) } @@ -161,20 +161,18 @@ case class Limit(limit: Int, child: SparkPlan)(@transient sqlContext: SQLContext * Spark's top operator does the opposite in ordering so we name it TakeOrdered to avoid confusion. */ @DeveloperApi -case class TakeOrdered(limit: Int, sortOrder: Seq[SortOrder], child: SparkPlan) - (@transient sqlContext: SQLContext) extends UnaryNode { - override def otherCopyArgs = sqlContext :: Nil +case class TakeOrdered(limit: Int, sortOrder: Seq[SortOrder], child: SparkPlan) extends UnaryNode { override def output = child.output - @transient - lazy val ordering = new RowOrdering(sortOrder) + val ordering = new RowOrdering(sortOrder, child.output) + // TODO: Is this copying for no reason? override def executeCollect() = child.execute().map(_.copy()).takeOrdered(limit)(ordering) // TODO: Terminal split should be implemented differently from non-terminal split. // TODO: Pick num splits based on |limit|. - override def execute() = sqlContext.sparkContext.makeRDD(executeCollect(), 1) + override def execute() = sparkContext.makeRDD(executeCollect(), 1) } /** @@ -189,15 +187,13 @@ case class Sort( override def requiredChildDistribution = if (global) OrderedDistribution(sortOrder) :: Nil else UnspecifiedDistribution :: Nil - @transient - lazy val ordering = new RowOrdering(sortOrder) override def execute() = attachTree(this, "sort") { - // TODO: Optimize sorting operation? child.execute() - .mapPartitions( - iterator => iterator.map(_.copy()).toArray.sorted(ordering).iterator, - preservesPartitioning = true) + .mapPartitions( { iterator => + val ordering = newOrdering(sortOrder, child.output) + iterator.map(_.copy()).toArray.sorted(ordering).iterator + }, preservesPartitioning = true) } override def output = child.output @@ -248,6 +244,37 @@ object ExistingRdd { case class ExistingRdd(output: Seq[Attribute], rdd: RDD[Row]) extends LeafNode { override def execute() = rdd } +/** + * :: DeveloperApi :: + * Computes the set of distinct input rows using a HashSet. + * @param partial when true the distinct operation is performed partially, per partition, without + * shuffling the data. + * @param child the input query plan. + */ +@DeveloperApi +case class Distinct(partial: Boolean, child: SparkPlan) extends UnaryNode { + override def output = child.output + + override def requiredChildDistribution = + if (partial) UnspecifiedDistribution :: Nil else ClusteredDistribution(child.output) :: Nil + + override def execute() = { + child.execute().mapPartitions { iter => + val hashSet = new scala.collection.mutable.HashSet[Row]() + + var currentRow: Row = null + while (iter.hasNext) { + currentRow = iter.next() + if (!hashSet.contains(currentRow)) { + hashSet.add(currentRow.copy()) + } + } + + hashSet.iterator + } + } +} + /** * :: DeveloperApi :: diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala index 98d2f89c8ae71..38f37564f1788 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala @@ -17,12 +17,13 @@ package org.apache.spark.sql.execution +import org.apache.spark.Logging import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericRow} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.{Row, SQLContext} +import org.apache.spark.sql.{Row, SQLConf, SQLContext} trait Command { /** @@ -44,28 +45,53 @@ trait Command { case class SetCommand( key: Option[String], value: Option[String], output: Seq[Attribute])( @transient context: SQLContext) - extends LeafNode with Command { + extends LeafNode with Command with Logging { - override protected[sql] lazy val sideEffectResult: Seq[(String, String)] = (key, value) match { + override protected[sql] lazy val sideEffectResult: Seq[String] = (key, value) match { // Set value for key k. case (Some(k), Some(v)) => - context.set(k, v) - Array(k -> v) + if (k == SQLConf.Deprecated.MAPRED_REDUCE_TASKS) { + logWarning(s"Property ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS} is deprecated, " + + s"automatically converted to ${SQLConf.SHUFFLE_PARTITIONS} instead.") + context.setConf(SQLConf.SHUFFLE_PARTITIONS, v) + Array(s"${SQLConf.SHUFFLE_PARTITIONS}=$v") + } else { + context.setConf(k, v) + Array(s"$k=$v") + } // Query the value bound to key k. case (Some(k), _) => - Array(k -> context.getOption(k).getOrElse("")) + // TODO (lian) This is just a workaround to make the Simba ODBC driver work. + // Should remove this once we get the ODBC driver updated. + if (k == "-v") { + val hiveJars = Seq( + "hive-exec-0.12.0.jar", + "hive-service-0.12.0.jar", + "hive-common-0.12.0.jar", + "hive-hwi-0.12.0.jar", + "hive-0.12.0.jar").mkString(":") + + Array( + "system:java.class.path=" + hiveJars, + "system:sun.java.command=shark.SharkServer2") + } + else { + Array(s"$k=${context.getConf(k, "")}") + } // Query all key-value pairs that are set in the SQLConf of the context. case (None, None) => - context.getAll + context.getAllConfs.map { case (k, v) => + s"$k=$v" + }.toSeq case _ => throw new IllegalArgumentException() } def execute(): RDD[Row] = { - val rows = sideEffectResult.map { case (k, v) => new GenericRow(Array[Any](k, v)) } + val rows = sideEffectResult.map { line => new GenericRow(Array[Any](line)) } context.sparkContext.parallelize(rows, 1) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala index c6fbd6d2f6930..f31df051824d7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala @@ -41,13 +41,13 @@ package object debug { */ @DeveloperApi implicit class DebugQuery(query: SchemaRDD) { - def debug(implicit sc: SparkContext): Unit = { + def debug(): Unit = { val plan = query.queryExecution.executedPlan val visited = new collection.mutable.HashSet[Long]() val debugPlan = plan transform { case s: SparkPlan if !visited.contains(s.id) => visited += s.id - DebugNode(sc, s) + DebugNode(s) } println(s"Results returned: ${debugPlan.execute().count()}") debugPlan.foreach { @@ -57,9 +57,7 @@ package object debug { } } - private[sql] case class DebugNode( - @transient sparkContext: SparkContext, - child: SparkPlan) extends UnaryNode { + private[sql] case class DebugNode(child: SparkPlan) extends UnaryNode { def references = Set.empty def output = child.output @@ -107,7 +105,9 @@ package object debug { var i = 0 while (i < numColumns) { val value = currentRow(i) - columnStats(i).elementTypes += HashSet(value.getClass.getName) + if (value != null) { + columnStats(i).elementTypes += HashSet(value.getClass.getName) + } i += 1 } currentRow diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala index 7d1f11caae838..c86811e838bd8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.execution +import java.util.{HashMap => JavaHashMap} + import scala.collection.mutable.{ArrayBuffer, BitSet} import scala.concurrent.ExecutionContext.Implicits.global import scala.concurrent._ @@ -38,6 +40,8 @@ case object BuildLeft extends BuildSide case object BuildRight extends BuildSide trait HashJoin { + self: SparkPlan => + val leftKeys: Seq[Expression] val rightKeys: Seq[Expression] val buildSide: BuildSide @@ -56,9 +60,9 @@ trait HashJoin { def output = left.output ++ right.output - @transient lazy val buildSideKeyGenerator = new Projection(buildKeys, buildPlan.output) + @transient lazy val buildSideKeyGenerator = newProjection(buildKeys, buildPlan.output) @transient lazy val streamSideKeyGenerator = - () => new MutableProjection(streamedKeys, streamedPlan.output) + newMutableProjection(streamedKeys, streamedPlan.output) def joinIterators(buildIter: Iterator[Row], streamIter: Iterator[Row]): Iterator[Row] = { // TODO: Use Spark's HashMap implementation. @@ -70,7 +74,7 @@ trait HashJoin { while (buildIter.hasNext) { currentRow = buildIter.next() val rowKey = buildSideKeyGenerator(currentRow) - if(!rowKey.anyNull) { + if (!rowKey.anyNull) { val existingMatchList = hashTable.get(rowKey) val matchList = if (existingMatchList == null) { val newMatchList = new ArrayBuffer[Row]() @@ -134,6 +138,201 @@ trait HashJoin { } } +/** + * :: DeveloperApi :: + * Performs a hash based outer join for two child relations by shuffling the data using + * the join keys. This operator requires loading the associated partition in both side into memory. + */ +@DeveloperApi +case class HashOuterJoin( + leftKeys: Seq[Expression], + rightKeys: Seq[Expression], + joinType: JoinType, + condition: Option[Expression], + left: SparkPlan, + right: SparkPlan) extends BinaryNode { + + override def outputPartitioning: Partitioning = joinType match { + case LeftOuter => left.outputPartitioning + case RightOuter => right.outputPartitioning + case FullOuter => UnknownPartitioning(left.outputPartitioning.numPartitions) + case x => throw new Exception(s"HashOuterJoin should not take $x as the JoinType") + } + + override def requiredChildDistribution = + ClusteredDistribution(leftKeys) :: ClusteredDistribution(rightKeys) :: Nil + + override def output = { + joinType match { + case LeftOuter => + left.output ++ right.output.map(_.withNullability(true)) + case RightOuter => + left.output.map(_.withNullability(true)) ++ right.output + case FullOuter => + left.output.map(_.withNullability(true)) ++ right.output.map(_.withNullability(true)) + case x => + throw new Exception(s"HashOuterJoin should not take $x as the JoinType") + } + } + + @transient private[this] lazy val DUMMY_LIST = Seq[Row](null) + @transient private[this] lazy val EMPTY_LIST = Seq.empty[Row] + + // TODO we need to rewrite all of the iterators with our own implementation instead of the Scala + // iterator for performance purpose. + + private[this] def leftOuterIterator( + key: Row, leftIter: Iterable[Row], rightIter: Iterable[Row]): Iterator[Row] = { + val joinedRow = new JoinedRow() + val rightNullRow = new GenericRow(right.output.length) + val boundCondition = + condition.map(newPredicate(_, left.output ++ right.output)).getOrElse((row: Row) => true) + + leftIter.iterator.flatMap { l => + joinedRow.withLeft(l) + var matched = false + (if (!key.anyNull) rightIter.collect { case r if (boundCondition(joinedRow.withRight(r))) => + matched = true + joinedRow.copy + } else { + Nil + }) ++ DUMMY_LIST.filter(_ => !matched).map( _ => { + // DUMMY_LIST.filter(_ => !matched) is a tricky way to add additional row, + // as we don't know whether we need to append it until finish iterating all of the + // records in right side. + // If we didn't get any proper row, then append a single row with empty right + joinedRow.withRight(rightNullRow).copy + }) + } + } + + private[this] def rightOuterIterator( + key: Row, leftIter: Iterable[Row], rightIter: Iterable[Row]): Iterator[Row] = { + val joinedRow = new JoinedRow() + val leftNullRow = new GenericRow(left.output.length) + val boundCondition = + condition.map(newPredicate(_, left.output ++ right.output)).getOrElse((row: Row) => true) + + rightIter.iterator.flatMap { r => + joinedRow.withRight(r) + var matched = false + (if (!key.anyNull) leftIter.collect { case l if (boundCondition(joinedRow.withLeft(l))) => + matched = true + joinedRow.copy + } else { + Nil + }) ++ DUMMY_LIST.filter(_ => !matched).map( _ => { + // DUMMY_LIST.filter(_ => !matched) is a tricky way to add additional row, + // as we don't know whether we need to append it until finish iterating all of the + // records in left side. + // If we didn't get any proper row, then append a single row with empty left. + joinedRow.withLeft(leftNullRow).copy + }) + } + } + + private[this] def fullOuterIterator( + key: Row, leftIter: Iterable[Row], rightIter: Iterable[Row]): Iterator[Row] = { + val joinedRow = new JoinedRow() + val leftNullRow = new GenericRow(left.output.length) + val rightNullRow = new GenericRow(right.output.length) + val boundCondition = + condition.map(newPredicate(_, left.output ++ right.output)).getOrElse((row: Row) => true) + + if (!key.anyNull) { + // Store the positions of records in right, if one of its associated row satisfy + // the join condition. + val rightMatchedSet = scala.collection.mutable.Set[Int]() + leftIter.iterator.flatMap[Row] { l => + joinedRow.withLeft(l) + var matched = false + rightIter.zipWithIndex.collect { + // 1. For those matched (satisfy the join condition) records with both sides filled, + // append them directly + + case (r, idx) if (boundCondition(joinedRow.withRight(r)))=> { + matched = true + // if the row satisfy the join condition, add its index into the matched set + rightMatchedSet.add(idx) + joinedRow.copy + } + } ++ DUMMY_LIST.filter(_ => !matched).map( _ => { + // 2. For those unmatched records in left, append additional records with empty right. + + // DUMMY_LIST.filter(_ => !matched) is a tricky way to add additional row, + // as we don't know whether we need to append it until finish iterating all + // of the records in right side. + // If we didn't get any proper row, then append a single row with empty right. + joinedRow.withRight(rightNullRow).copy + }) + } ++ rightIter.zipWithIndex.collect { + // 3. For those unmatched records in right, append additional records with empty left. + + // Re-visiting the records in right, and append additional row with empty left, if its not + // in the matched set. + case (r, idx) if (!rightMatchedSet.contains(idx)) => { + joinedRow(leftNullRow, r).copy + } + } + } else { + leftIter.iterator.map[Row] { l => + joinedRow(l, rightNullRow).copy + } ++ rightIter.iterator.map[Row] { r => + joinedRow(leftNullRow, r).copy + } + } + } + + private[this] def buildHashTable( + iter: Iterator[Row], keyGenerator: Projection): JavaHashMap[Row, ArrayBuffer[Row]] = { + val hashTable = new JavaHashMap[Row, ArrayBuffer[Row]]() + while (iter.hasNext) { + val currentRow = iter.next() + val rowKey = keyGenerator(currentRow) + + var existingMatchList = hashTable.get(rowKey) + if (existingMatchList == null) { + existingMatchList = new ArrayBuffer[Row]() + hashTable.put(rowKey, existingMatchList) + } + + existingMatchList += currentRow.copy() + } + + hashTable + } + + def execute() = { + left.execute().zipPartitions(right.execute()) { (leftIter, rightIter) => + // TODO this probably can be replaced by external sort (sort merged join?) + // Build HashMap for current partition in left relation + val leftHashTable = buildHashTable(leftIter, newProjection(leftKeys, left.output)) + // Build HashMap for current partition in right relation + val rightHashTable = buildHashTable(rightIter, newProjection(rightKeys, right.output)) + + import scala.collection.JavaConversions._ + val boundCondition = + condition.map(newPredicate(_, left.output ++ right.output)).getOrElse((row: Row) => true) + joinType match { + case LeftOuter => leftHashTable.keysIterator.flatMap { key => + leftOuterIterator(key, leftHashTable.getOrElse(key, EMPTY_LIST), + rightHashTable.getOrElse(key, EMPTY_LIST)) + } + case RightOuter => rightHashTable.keysIterator.flatMap { key => + rightOuterIterator(key, leftHashTable.getOrElse(key, EMPTY_LIST), + rightHashTable.getOrElse(key, EMPTY_LIST)) + } + case FullOuter => (leftHashTable.keySet ++ rightHashTable.keySet).iterator.flatMap { key => + fullOuterIterator(key, + leftHashTable.getOrElse(key, EMPTY_LIST), + rightHashTable.getOrElse(key, EMPTY_LIST)) + } + case x => throw new Exception(s"HashOuterJoin should not take $x as the JoinType") + } + } + } +} + /** * :: DeveloperApi :: * Performs an inner hash join of two child relations by first shuffling the data using the join @@ -187,7 +386,7 @@ case class LeftSemiJoinHash( while (buildIter.hasNext) { currentRow = buildIter.next() val rowKey = buildSideKeyGenerator(currentRow) - if(!rowKey.anyNull) { + if (!rowKey.anyNull) { val keyExists = hashSet.contains(rowKey) if (!keyExists) { hashSet.add(rowKey) @@ -217,18 +416,16 @@ case class BroadcastHashJoin( rightKeys: Seq[Expression], buildSide: BuildSide, left: SparkPlan, - right: SparkPlan)(@transient sqlContext: SQLContext) extends BinaryNode with HashJoin { - - override def otherCopyArgs = sqlContext :: Nil + right: SparkPlan) extends BinaryNode with HashJoin { - override def outputPartitioning: Partitioning = left.outputPartitioning + override def outputPartitioning: Partitioning = streamedPlan.outputPartitioning override def requiredChildDistribution = UnspecifiedDistribution :: UnspecifiedDistribution :: Nil @transient lazy val broadcastFuture = future { - sqlContext.sparkContext.broadcast(buildPlan.executeCollect()) + sparkContext.broadcast(buildPlan.executeCollect()) } def execute() = { @@ -248,14 +445,11 @@ case class BroadcastHashJoin( @DeveloperApi case class LeftSemiJoinBNL( streamed: SparkPlan, broadcast: SparkPlan, condition: Option[Expression]) - (@transient sqlContext: SQLContext) extends BinaryNode { // TODO: Override requiredChildDistribution. override def outputPartitioning: Partitioning = streamed.outputPartitioning - override def otherCopyArgs = sqlContext :: Nil - def output = left.output /** The Streamed Relation */ @@ -271,7 +465,7 @@ case class LeftSemiJoinBNL( def execute() = { val broadcastedRelation = - sqlContext.sparkContext.broadcast(broadcast.execute().map(_.copy()).collect().toIndexedSeq) + sparkContext.broadcast(broadcast.execute().map(_.copy()).collect().toIndexedSeq) streamed.execute().mapPartitions { streamedIter => val joinedRow = new JoinedRow @@ -300,8 +494,14 @@ case class LeftSemiJoinBNL( case class CartesianProduct(left: SparkPlan, right: SparkPlan) extends BinaryNode { def output = left.output ++ right.output - def execute() = left.execute().map(_.copy()).cartesian(right.execute().map(_.copy())).map { - case (l: Row, r: Row) => buildRow(l ++ r) + def execute() = { + val leftResults = left.execute().map(_.copy()) + val rightResults = right.execute().map(_.copy()) + + leftResults.cartesian(rightResults).mapPartitions { iter => + val joinedRow = new JoinedRow + iter.map(r => joinedRow(r._1, r._2)) + } } } @@ -310,14 +510,20 @@ case class CartesianProduct(left: SparkPlan, right: SparkPlan) extends BinaryNod */ @DeveloperApi case class BroadcastNestedLoopJoin( - streamed: SparkPlan, broadcast: SparkPlan, joinType: JoinType, condition: Option[Expression]) - (@transient sqlContext: SQLContext) - extends BinaryNode { + left: SparkPlan, + right: SparkPlan, + buildSide: BuildSide, + joinType: JoinType, + condition: Option[Expression]) extends BinaryNode { // TODO: Override requiredChildDistribution. - override def outputPartitioning: Partitioning = streamed.outputPartitioning + /** BuildRight means the right relation <=> the broadcast relation. */ + val (streamed, broadcast) = buildSide match { + case BuildRight => (left, right) + case BuildLeft => (right, left) + } - override def otherCopyArgs = sqlContext :: Nil + override def outputPartitioning: Partitioning = streamed.outputPartitioning override def output = { joinType match { @@ -332,11 +538,6 @@ case class BroadcastNestedLoopJoin( } } - /** The Streamed Relation */ - def left = streamed - /** The Broadcast relation */ - def right = broadcast - @transient lazy val boundCondition = InterpretedPredicate( condition @@ -345,58 +546,80 @@ case class BroadcastNestedLoopJoin( def execute() = { val broadcastedRelation = - sqlContext.sparkContext.broadcast(broadcast.execute().map(_.copy()).collect().toIndexedSeq) + sparkContext.broadcast(broadcast.execute().map(_.copy()).collect().toIndexedSeq) - val streamedPlusMatches = streamed.execute().mapPartitions { streamedIter => + /** All rows that either match both-way, or rows from streamed joined with nulls. */ + val matchesOrStreamedRowsWithNulls = streamed.execute().mapPartitions { streamedIter => val matchedRows = new ArrayBuffer[Row] // TODO: Use Spark's BitSet. - val includedBroadcastTuples = new BitSet(broadcastedRelation.value.size) + val includedBroadcastTuples = + new scala.collection.mutable.BitSet(broadcastedRelation.value.size) val joinedRow = new JoinedRow + val leftNulls = new GenericMutableRow(left.output.size) + val rightNulls = new GenericMutableRow(right.output.size) streamedIter.foreach { streamedRow => var i = 0 - var matched = false + var streamRowMatched = false while (i < broadcastedRelation.value.size) { // TODO: One bitset per partition instead of per row. val broadcastedRow = broadcastedRelation.value(i) - if (boundCondition(joinedRow(streamedRow, broadcastedRow))) { - matchedRows += buildRow(streamedRow ++ broadcastedRow) - matched = true - includedBroadcastTuples += i + buildSide match { + case BuildRight if boundCondition(joinedRow(streamedRow, broadcastedRow)) => + matchedRows += joinedRow(streamedRow, broadcastedRow).copy() + streamRowMatched = true + includedBroadcastTuples += i + case BuildLeft if boundCondition(joinedRow(broadcastedRow, streamedRow)) => + matchedRows += joinedRow(broadcastedRow, streamedRow).copy() + streamRowMatched = true + includedBroadcastTuples += i + case _ => } i += 1 } - if (!matched && (joinType == LeftOuter || joinType == FullOuter)) { - matchedRows += buildRow(streamedRow ++ Array.fill(right.output.size)(null)) + (streamRowMatched, joinType, buildSide) match { + case (false, LeftOuter | FullOuter, BuildRight) => + matchedRows += joinedRow(streamedRow, rightNulls).copy() + case (false, RightOuter | FullOuter, BuildLeft) => + matchedRows += joinedRow(leftNulls, streamedRow).copy() + case _ => } } Iterator((matchedRows, includedBroadcastTuples)) } - val includedBroadcastTuples = streamedPlusMatches.map(_._2) + val includedBroadcastTuples = matchesOrStreamedRowsWithNulls.map(_._2) val allIncludedBroadcastTuples = if (includedBroadcastTuples.count == 0) { new scala.collection.mutable.BitSet(broadcastedRelation.value.size) } else { - streamedPlusMatches.map(_._2).reduce(_ ++ _) + includedBroadcastTuples.reduce(_ ++ _) } - val rightOuterMatches: Seq[Row] = - if (joinType == RightOuter || joinType == FullOuter) { - broadcastedRelation.value.zipWithIndex.filter { - case (row, i) => !allIncludedBroadcastTuples.contains(i) - }.map { - // TODO: Use projection. - case (row, _) => buildRow(Vector.fill(left.output.size)(null) ++ row) + val leftNulls = new GenericMutableRow(left.output.size) + val rightNulls = new GenericMutableRow(right.output.size) + /** Rows from broadcasted joined with nulls. */ + val broadcastRowsWithNulls: Seq[Row] = { + val arrBuf: collection.mutable.ArrayBuffer[Row] = collection.mutable.ArrayBuffer() + var i = 0 + val rel = broadcastedRelation.value + while (i < rel.length) { + if (!allIncludedBroadcastTuples.contains(i)) { + (joinType, buildSide) match { + case (RightOuter | FullOuter, BuildRight) => arrBuf += new JoinedRow(leftNulls, rel(i)) + case (LeftOuter | FullOuter, BuildLeft) => arrBuf += new JoinedRow(rel(i), rightNulls) + case _ => + } } - } else { - Vector() + i += 1 } + arrBuf.toSeq + } // TODO: Breaks lineage. - sqlContext.sparkContext.union( - streamedPlusMatches.flatMap(_._1), sqlContext.sparkContext.makeRDD(rightOuterMatches)) + sparkContext.union( + matchesOrStreamedRowsWithNulls.flatMap(_._1), sparkContext.makeRDD(broadcastRowsWithNulls)) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala new file mode 100644 index 0000000000000..b92091b560b1c --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala @@ -0,0 +1,177 @@ +/* +* Licensed to the Apache Software Foundation (ASF) under one or more +* contributor license agreements. See the NOTICE file distributed with +* this work for additional information regarding copyright ownership. +* The ASF licenses this file to You under the Apache License, Version 2.0 +* (the "License"); you may not use this file except in compliance with +* the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ + +package org.apache.spark.sql.execution + +import java.util.{List => JList, Map => JMap} + +import net.razorvine.pickle.{Pickler, Unpickler} +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.api.python.PythonRDD +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.logical +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.{Accumulator, Logging => SparkLogging} + +import scala.collection.JavaConversions._ + +/** + * A serialized version of a Python lambda function. Suitable for use in a [[PythonRDD]]. + */ +private[spark] case class PythonUDF( + name: String, + command: Array[Byte], + envVars: JMap[String, String], + pythonIncludes: JList[String], + pythonExec: String, + accumulator: Accumulator[JList[Array[Byte]]], + dataType: DataType, + children: Seq[Expression]) extends Expression with SparkLogging { + + override def toString = s"PythonUDF#$name(${children.mkString(",")})" + + def nullable: Boolean = true + def references: Set[Attribute] = children.flatMap(_.references).toSet + + override def eval(input: Row) = sys.error("PythonUDFs can not be directly evaluated.") +} + +/** + * Extracts PythonUDFs from operators, rewriting the query plan so that the UDF can be evaluated + * alone in a batch. + * + * This has the limitation that the input to the Python UDF is not allowed include attributes from + * multiple child operators. + */ +private[spark] object ExtractPythonUdfs extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan) = plan transform { + // Skip EvaluatePython nodes. + case p: EvaluatePython => p + + case l: LogicalPlan => + // Extract any PythonUDFs from the current operator. + val udfs = l.expressions.flatMap(_.collect { case udf: PythonUDF => udf}) + if (udfs.isEmpty) { + // If there aren't any, we are done. + l + } else { + // Pick the UDF we are going to evaluate (TODO: Support evaluating multiple UDFs at a time) + // If there is more than one, we will add another evaluation operator in a subsequent pass. + val udf = udfs.head + + var evaluation: EvaluatePython = null + + // Rewrite the child that has the input required for the UDF + val newChildren = l.children.map { child => + // Check to make sure that the UDF can be evaluated with only the input of this child. + // Other cases are disallowed as they are ambiguous or would require a cartisian product. + if (udf.references.subsetOf(child.outputSet)) { + evaluation = EvaluatePython(udf, child) + evaluation + } else if (udf.references.intersect(child.outputSet).nonEmpty) { + sys.error(s"Invalid PythonUDF $udf, requires attributes from more than one child.") + } else { + child + } + } + + assert(evaluation != null, "Unable to evaluate PythonUDF. Missing input attributes.") + + // Trim away the new UDF value if it was only used for filtering or something. + logical.Project( + l.output, + l.transformExpressions { + case p: PythonUDF if p.id == udf.id => evaluation.resultAttribute + }.withNewChildren(newChildren)) + } + } +} + +/** + * :: DeveloperApi :: + * Evaluates a [[PythonUDF]], appending the result to the end of the input tuple. + */ +@DeveloperApi +case class EvaluatePython(udf: PythonUDF, child: LogicalPlan) extends logical.UnaryNode { + val resultAttribute = AttributeReference("pythonUDF", udf.dataType, nullable=true)() + + def references = Set.empty + def output = child.output :+ resultAttribute +} + +/** + * :: DeveloperApi :: + * Uses PythonRDD to evaluate a [[PythonUDF]], one partition of tuples at a time. The input + * data is cached and zipped with the result of the udf evaluation. + */ +@DeveloperApi +case class BatchPythonEvaluation(udf: PythonUDF, output: Seq[Attribute], child: SparkPlan) + extends SparkPlan { + def children = child :: Nil + + def execute() = { + // TODO: Clean up after ourselves? + val childResults = child.execute().map(_.copy()).cache() + + val parent = childResults.mapPartitions { iter => + val pickle = new Pickler + val currentRow = newMutableProjection(udf.children, child.output)() + iter.grouped(1000).map { inputRows => + val toBePickled = inputRows.map(currentRow(_).toArray).toArray + pickle.dumps(toBePickled) + } + } + + val pyRDD = new PythonRDD( + parent, + udf.command, + udf.envVars, + udf.pythonIncludes, + false, + udf.pythonExec, + Seq[Broadcast[Array[Byte]]](), + udf.accumulator + ).mapPartitions { iter => + val pickle = new Unpickler + iter.flatMap { pickedResult => + val unpickledBatch = pickle.loads(pickedResult) + unpickledBatch.asInstanceOf[java.util.ArrayList[Any]] + } + }.mapPartitions { iter => + val row = new GenericMutableRow(1) + iter.map { result => + row(0) = udf.dataType match { + case StringType => result.toString + case other => result + } + row: Row + } + } + + childResults.zip(pyRDD).mapPartitions { iter => + val joinedRow = new JoinedRow() + iter.map { + case (row, udfResult) => + joinedRow(row, udfResult) + } + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala index f6cbca96483e2..1c0b03c684f10 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala @@ -17,7 +17,8 @@ package org.apache.spark.sql.json -import scala.collection.JavaConversions._ +import scala.collection.Map +import scala.collection.convert.Wrappers.{JMapWrapper, JListWrapper} import scala.math.BigDecimal import com.fasterxml.jackson.databind.ObjectMapper @@ -25,30 +26,25 @@ import com.fasterxml.jackson.databind.ObjectMapper import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.analysis.HiveTypeCoercion import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.types._ -import org.apache.spark.sql.execution.{ExistingRdd, SparkLogicalPlan} -import org.apache.spark.sql.Logging +import org.apache.spark.sql.catalyst.ScalaReflection +import org.apache.spark.Logging private[sql] object JsonRDD extends Logging { + private[sql] def jsonStringToRow( + json: RDD[String], + schema: StructType): RDD[Row] = { + parseJson(json).map(parsed => asRow(parsed, schema)) + } + private[sql] def inferSchema( json: RDD[String], - samplingRatio: Double = 1.0): LogicalPlan = { + samplingRatio: Double = 1.0): StructType = { require(samplingRatio > 0, s"samplingRatio ($samplingRatio) should be greater than 0") val schemaData = if (samplingRatio > 0.99) json else json.sample(false, samplingRatio, 1) val allKeys = parseJson(schemaData).map(allKeysWithValueTypes).reduce(_ ++ _) - val baseSchema = createSchema(allKeys) - - createLogicalPlan(json, baseSchema) - } - - private def createLogicalPlan( - json: RDD[String], - baseSchema: StructType): LogicalPlan = { - val schema = nullTypeToStringType(baseSchema) - - SparkLogicalPlan(ExistingRdd(asAttributes(schema), parseJson(json).map(asRow(_, schema)))) + createSchema(allKeys) } private def createSchema(allKeys: Set[(String, DataType)]): StructType = { @@ -72,8 +68,8 @@ private[sql] object JsonRDD extends Logging { val (topLevel, structLike) = values.partition(_.size == 1) val topLevelFields = topLevel.filter { name => resolved.get(prefix ++ name).get match { - case ArrayType(StructType(Nil)) => false - case ArrayType(_) => true + case ArrayType(StructType(Nil), _) => false + case ArrayType(_, _) => true case struct: StructType => false case _ => true } @@ -87,7 +83,8 @@ private[sql] object JsonRDD extends Logging { val structType = makeStruct(nestedFields, prefix :+ name) val dataType = resolved.get(prefix :+ name).get dataType match { - case array: ArrayType => Some(StructField(name, ArrayType(structType), nullable = true)) + case array: ArrayType => + Some(StructField(name, ArrayType(structType, array.containsNull), nullable = true)) case struct: StructType => Some(StructField(name, structType, nullable = true)) // dataType is StringType means that we have resolved type conflicts involving // primitive types and complex types. So, the type of name has been relaxed to @@ -106,6 +103,24 @@ private[sql] object JsonRDD extends Logging { makeStruct(resolved.keySet.toSeq, Nil) } + private[sql] def nullTypeToStringType(struct: StructType): StructType = { + val fields = struct.fields.map { + case StructField(fieldName, dataType, nullable) => { + val newType = dataType match { + case NullType => StringType + case ArrayType(NullType, containsNull) => ArrayType(StringType, containsNull) + case ArrayType(struct: StructType, containsNull) => + ArrayType(nullTypeToStringType(struct), containsNull) + case struct: StructType =>nullTypeToStringType(struct) + case other: DataType => other + } + StructField(fieldName, newType, nullable) + } + } + + StructType(fields) + } + /** * Returns the most general data type for two given data types. */ @@ -136,8 +151,8 @@ private[sql] object JsonRDD extends Logging { case StructField(name, _, _) => name }) } - case (ArrayType(elementType1), ArrayType(elementType2)) => - ArrayType(compatibleType(elementType1, elementType2)) + case (ArrayType(elementType1, containsNull1), ArrayType(elementType2, containsNull2)) => + ArrayType(compatibleType(elementType1, elementType2), containsNull1 || containsNull2) // TODO: We should use JsonObjectStringType to mark that values of field will be // strings and every string is a Json object. case (_, _) => StringType @@ -145,18 +160,13 @@ private[sql] object JsonRDD extends Logging { } } - private def typeOfPrimitiveValue(value: Any): DataType = { - value match { - case value: java.lang.String => StringType - case value: java.lang.Integer => IntegerType - case value: java.lang.Long => LongType + private def typeOfPrimitiveValue: PartialFunction[Any, DataType] = { + ScalaReflection.typeOfObject orElse { // Since we do not have a data type backed by BigInteger, // when we see a Java BigInteger, we use DecimalType. case value: java.math.BigInteger => DecimalType - case value: java.lang.Double => DoubleType + // DecimalType's JVMType is scala BigDecimal. case value: java.math.BigDecimal => DecimalType - case value: java.lang.Boolean => BooleanType - case null => NullType // Unexpected data type. case _ => StringType } @@ -169,12 +179,13 @@ private[sql] object JsonRDD extends Logging { * treat the element as String. */ private def typeOfArray(l: Seq[Any]): ArrayType = { + val containsNull = l.exists(v => v == null) val elements = l.flatMap(v => Option(v)) if (elements.isEmpty) { // If this JSON array is empty, we use NullType as a placeholder. // If this array is not empty in other JSON objects, we can resolve // the type after we have passed through all JSON objects. - ArrayType(NullType) + ArrayType(NullType, containsNull) } else { val elementType = elements.map { e => e match { @@ -186,7 +197,7 @@ private[sql] object JsonRDD extends Logging { } }.reduce((type1: DataType, type2: DataType) => compatibleType(type1, type2)) - ArrayType(elementType) + ArrayType(elementType, containsNull) } } @@ -204,24 +215,25 @@ private[sql] object JsonRDD extends Logging { case (key, value) => (s"`$key`", value) }.toSet keyValuePairs.flatMap { - case (key: String, struct: Map[String, Any]) => { - // The value associted with the key is an JSON object. - allKeysWithValueTypes(struct).map { + case (key: String, struct: Map[_, _]) => { + // The value associated with the key is an JSON object. + allKeysWithValueTypes(struct.asInstanceOf[Map[String, Any]]).map { case (k, dataType) => (s"$key.$k", dataType) } ++ Set((key, StructType(Nil))) } - case (key: String, array: List[Any]) => { - // The value associted with the key is an array. + case (key: String, array: Seq[_]) => { + // The value associated with the key is an array. typeOfArray(array) match { - case ArrayType(StructType(Nil)) => { + case ArrayType(StructType(Nil), containsNull) => { // The elements of this arrays are structs. - array.asInstanceOf[List[Map[String, Any]]].flatMap { + array.asInstanceOf[Seq[Map[String, Any]]].flatMap { element => allKeysWithValueTypes(element) }.map { case (k, dataType) => (s"$key.$k", dataType) - } :+ (key, ArrayType(StructType(Nil))) + } :+ (key, ArrayType(StructType(Nil), containsNull)) } - case ArrayType(elementType) => (key, ArrayType(elementType)) :: Nil + case ArrayType(elementType, containsNull) => + (key, ArrayType(elementType, containsNull)) :: Nil } } case (key: String, value) => (key, typeOfPrimitiveValue(value)) :: Nil @@ -229,19 +241,19 @@ private[sql] object JsonRDD extends Logging { } /** - * Converts a Java Map/List to a Scala Map/List. + * Converts a Java Map/List to a Scala Map/Seq. * We do not use Jackson's scala module at here because * DefaultScalaModule in jackson-module-scala will make * the parsing very slow. */ private def scalafy(obj: Any): Any = obj match { - case map: java.util.Map[String, Object] => + case map: java.util.Map[_, _] => // .map(identity) is used as a workaround of non-serializable Map // generated by .mapValues. // This issue is documented at https://issues.scala-lang.org/browse/SI-7005 - map.toMap.mapValues(scalafy).map(identity) - case list: java.util.List[Object] => - list.toList.map(scalafy) + JMapWrapper(map).mapValues(scalafy).map(identity) + case list: java.util.List[_] => + JListWrapper(list).map(scalafy) case atom => atom } @@ -259,8 +271,11 @@ private[sql] object JsonRDD extends Logging { // the ObjectMapper will take the last value associated with this duplicate key. // For example: for {"key": 1, "key":2}, we will get "key"->2. val mapper = new ObjectMapper() - iter.map(record => mapper.readValue(record, classOf[java.util.Map[String, Any]])) - }).map(scalafy).map(_.asInstanceOf[Map[String, Any]]) + iter.map { record => + val parsed = scalafy(mapper.readValue(record, classOf[java.util.Map[String, Any]])) + parsed.asInstanceOf[Map[String, Any]] + } + }) } private def toLong(value: Any): Long = { @@ -320,8 +335,8 @@ private[sql] object JsonRDD extends Logging { private def toString(value: Any): String = { value match { - case value: Map[String, Any] => toJsonObjectString(value) - case value: Seq[Any] => toJsonArrayString(value) + case value: Map[_, _] => toJsonObjectString(value.asInstanceOf[Map[String, Any]]) + case value: Seq[_] => toJsonArrayString(value) case value => Option(value).map(_.toString).orNull } } @@ -331,7 +346,7 @@ private[sql] object JsonRDD extends Logging { null } else { desiredType match { - case ArrayType(elementType) => + case ArrayType(elementType, _) => value.asInstanceOf[Seq[Any]].map(enforceCorrectType(_, elementType)) case StringType => toString(value) case IntegerType => value.asInstanceOf[IntegerType.JvmType] @@ -345,6 +360,7 @@ private[sql] object JsonRDD extends Logging { } private def asRow(json: Map[String,Any], schema: StructType): Row = { + // TODO: Reuse the row instead of creating a new one for every record. val row = new GenericMutableRow(schema.fields.length) schema.fields.zipWithIndex.foreach { // StructType @@ -353,7 +369,7 @@ private[sql] object JsonRDD extends Logging { v => asRow(v.asInstanceOf[Map[String, Any]], fields)).orNull) // ArrayType(StructType) - case (StructField(name, ArrayType(structType: StructType), _), i) => + case (StructField(name, ArrayType(structType: StructType, _), _), i) => row.update(i, json.get(name).flatMap(v => Option(v)).map( v => v.asInstanceOf[Seq[Any]].map( @@ -367,32 +383,4 @@ private[sql] object JsonRDD extends Logging { row } - - private def nullTypeToStringType(struct: StructType): StructType = { - val fields = struct.fields.map { - case StructField(fieldName, dataType, nullable) => { - val newType = dataType match { - case NullType => StringType - case ArrayType(NullType) => ArrayType(StringType) - case struct: StructType => nullTypeToStringType(struct) - case other: DataType => other - } - StructField(fieldName, newType, nullable) - } - } - - StructType(fields) - } - - private def asAttributes(struct: StructType): Seq[AttributeReference] = { - struct.fields.map(f => AttributeReference(f.name, f.dataType, nullable = true)()) - } - - private def asStruct(attributes: Seq[AttributeReference]): StructType = { - val fields = attributes.map { - case AttributeReference(name, dataType, nullable) => StructField(name, dataType, nullable) - } - - StructType(fields) - } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/package.scala new file mode 100644 index 0000000000000..f513eae9c2d13 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/package.scala @@ -0,0 +1,407 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark + +import org.apache.spark.annotation.DeveloperApi + +/** + * Allows the execution of relational queries, including those expressed in SQL using Spark. + * + * @groupname dataType Data types + * @groupdesc Spark SQL data types. + * @groupprio dataType -3 + * @groupname field Field + * @groupprio field -2 + * @groupname row Row + * @groupprio row -1 + */ +package object sql { + + /** + * :: DeveloperApi :: + * + * Represents one row of output from a relational operator. + * @group row + */ + @DeveloperApi + type Row = catalyst.expressions.Row + + /** + * :: DeveloperApi :: + * + * A [[Row]] object can be constructed by providing field values. Example: + * {{{ + * import org.apache.spark.sql._ + * + * // Create a Row from values. + * Row(value1, value2, value3, ...) + * // Create a Row from a Seq of values. + * Row.fromSeq(Seq(value1, value2, ...)) + * }}} + * + * A value of a row can be accessed through both generic access by ordinal, + * which will incur boxing overhead for primitives, as well as native primitive access. + * An example of generic access by ordinal: + * {{{ + * import org.apache.spark.sql._ + * + * val row = Row(1, true, "a string", null) + * // row: Row = [1,true,a string,null] + * val firstValue = row(0) + * // firstValue: Any = 1 + * val fourthValue = row(3) + * // fourthValue: Any = null + * }}} + * + * For native primitive access, it is invalid to use the native primitive interface to retrieve + * a value that is null, instead a user must check `isNullAt` before attempting to retrieve a + * value that might be null. + * An example of native primitive access: + * {{{ + * // using the row from the previous example. + * val firstValue = row.getInt(0) + * // firstValue: Int = 1 + * val isNull = row.isNullAt(3) + * // isNull: Boolean = true + * }}} + * + * Interfaces related to native primitive access are: + * + * `isNullAt(i: Int): Boolean` + * + * `getInt(i: Int): Int` + * + * `getLong(i: Int): Long` + * + * `getDouble(i: Int): Double` + * + * `getFloat(i: Int): Float` + * + * `getBoolean(i: Int): Boolean` + * + * `getShort(i: Int): Short` + * + * `getByte(i: Int): Byte` + * + * `getString(i: Int): String` + * + * Fields in a [[Row]] object can be extracted in a pattern match. Example: + * {{{ + * import org.apache.spark.sql._ + * + * val pairs = sql("SELECT key, value FROM src").rdd.map { + * case Row(key: Int, value: String) => + * key -> value + * } + * }}} + * + * @group row + */ + @DeveloperApi + val Row = catalyst.expressions.Row + + /** + * :: DeveloperApi :: + * + * The base type of all Spark SQL data types. + * + * @group dataType + */ + @DeveloperApi + type DataType = catalyst.types.DataType + + /** + * :: DeveloperApi :: + * + * The data type representing `String` values + * + * @group dataType + */ + @DeveloperApi + val StringType = catalyst.types.StringType + + /** + * :: DeveloperApi :: + * + * The data type representing `Array[Byte]` values. + * + * @group dataType + */ + @DeveloperApi + val BinaryType = catalyst.types.BinaryType + + /** + * :: DeveloperApi :: + * + * The data type representing `Boolean` values. + * + *@group dataType + */ + @DeveloperApi + val BooleanType = catalyst.types.BooleanType + + /** + * :: DeveloperApi :: + * + * The data type representing `java.sql.Timestamp` values. + * + * @group dataType + */ + @DeveloperApi + val TimestampType = catalyst.types.TimestampType + + /** + * :: DeveloperApi :: + * + * The data type representing `scala.math.BigDecimal` values. + * + * @group dataType + */ + @DeveloperApi + val DecimalType = catalyst.types.DecimalType + + /** + * :: DeveloperApi :: + * + * The data type representing `Double` values. + * + * @group dataType + */ + @DeveloperApi + val DoubleType = catalyst.types.DoubleType + + /** + * :: DeveloperApi :: + * + * The data type representing `Float` values. + * + * @group dataType + */ + @DeveloperApi + val FloatType = catalyst.types.FloatType + + /** + * :: DeveloperApi :: + * + * The data type representing `Byte` values. + * + * @group dataType + */ + @DeveloperApi + val ByteType = catalyst.types.ByteType + + /** + * :: DeveloperApi :: + * + * The data type representing `Int` values. + * + * @group dataType + */ + @DeveloperApi + val IntegerType = catalyst.types.IntegerType + + /** + * :: DeveloperApi :: + * + * The data type representing `Long` values. + * + * @group dataType + */ + @DeveloperApi + val LongType = catalyst.types.LongType + + /** + * :: DeveloperApi :: + * + * The data type representing `Short` values. + * + * @group dataType + */ + @DeveloperApi + val ShortType = catalyst.types.ShortType + + /** + * :: DeveloperApi :: + * + * The data type for collections of multiple values. + * Internally these are represented as columns that contain a ``scala.collection.Seq``. + * + * An [[ArrayType]] object comprises two fields, `elementType: [[DataType]]` and + * `containsNull: Boolean`. The field of `elementType` is used to specify the type of + * array elements. The field of `containsNull` is used to specify if the array has `null` values. + * + * @group dataType + */ + @DeveloperApi + type ArrayType = catalyst.types.ArrayType + + /** + * :: DeveloperApi :: + * + * An [[ArrayType]] object can be constructed with two ways, + * {{{ + * ArrayType(elementType: DataType, containsNull: Boolean) + * }}} and + * {{{ + * ArrayType(elementType: DataType) + * }}} + * For `ArrayType(elementType)`, the field of `containsNull` is set to `false`. + * + * @group dataType + */ + @DeveloperApi + val ArrayType = catalyst.types.ArrayType + + /** + * :: DeveloperApi :: + * + * The data type representing `Map`s. A [[MapType]] object comprises three fields, + * `keyType: [[DataType]]`, `valueType: [[DataType]]` and `valueContainsNull: Boolean`. + * The field of `keyType` is used to specify the type of keys in the map. + * The field of `valueType` is used to specify the type of values in the map. + * The field of `valueContainsNull` is used to specify if values of this map has `null` values. + * For values of a MapType column, keys are not allowed to have `null` values. + * + * @group dataType + */ + @DeveloperApi + type MapType = catalyst.types.MapType + + /** + * :: DeveloperApi :: + * + * A [[MapType]] object can be constructed with two ways, + * {{{ + * MapType(keyType: DataType, valueType: DataType, valueContainsNull: Boolean) + * }}} and + * {{{ + * MapType(keyType: DataType, valueType: DataType) + * }}} + * For `MapType(keyType: DataType, valueType: DataType)`, + * the field of `valueContainsNull` is set to `true`. + * + * @group dataType + */ + @DeveloperApi + val MapType = catalyst.types.MapType + + /** + * :: DeveloperApi :: + * + * The data type representing [[Row]]s. + * A [[StructType]] object comprises a [[Seq]] of [[StructField]]s. + * + * @group dataType + */ + @DeveloperApi + type StructType = catalyst.types.StructType + + /** + * :: DeveloperApi :: + * + * A [[StructType]] object can be constructed by + * {{{ + * StructType(fields: Seq[StructField]) + * }}} + * For a [[StructType]] object, one or multiple [[StructField]]s can be extracted by names. + * If multiple [[StructField]]s are extracted, a [[StructType]] object will be returned. + * If a provided name does not have a matching field, it will be ignored. For the case + * of extracting a single StructField, a `null` will be returned. + * Example: + * {{{ + * import org.apache.spark.sql._ + * + * val struct = + * StructType( + * StructField("a", IntegerType, true) :: + * StructField("b", LongType, false) :: + * StructField("c", BooleanType, false) :: Nil) + * + * // Extract a single StructField. + * val singleField = struct("b") + * // singleField: StructField = StructField(b,LongType,false) + * + * // This struct does not have a field called "d". null will be returned. + * val nonExisting = struct("d") + * // nonExisting: StructField = null + * + * // Extract multiple StructFields. Field names are provided in a set. + * // A StructType object will be returned. + * val twoFields = struct(Set("b", "c")) + * // twoFields: StructType = + * // StructType(List(StructField(b,LongType,false), StructField(c,BooleanType,false))) + * + * // Those names do not have matching fields will be ignored. + * // For the case shown below, "d" will be ignored and + * // it is treated as struct(Set("b", "c")). + * val ignoreNonExisting = struct(Set("b", "c", "d")) + * // ignoreNonExisting: StructType = + * // StructType(List(StructField(b,LongType,false), StructField(c,BooleanType,false))) + * }}} + * + * A [[Row]] object is used as a value of the StructType. + * Example: + * {{{ + * import org.apache.spark.sql._ + * + * val innerStruct = + * StructType( + * StructField("f1", IntegerType, true) :: + * StructField("f2", LongType, false) :: + * StructField("f3", BooleanType, false) :: Nil) + * + * val struct = StructType( + * StructField("a", innerStruct, true) :: Nil) + * + * // Create a Row with the schema defined by struct + * val row = Row(Row(1, 2, true)) + * // row: Row = [[1,2,true]] + * }}} + * + * @group dataType + */ + @DeveloperApi + val StructType = catalyst.types.StructType + + /** + * :: DeveloperApi :: + * + * A [[StructField]] object represents a field in a [[StructType]] object. + * A [[StructField]] object comprises three fields, `name: [[String]]`, `dataType: [[DataType]]`, + * and `nullable: Boolean`. The field of `name` is the name of a `StructField`. The field of + * `dataType` specifies the data type of a `StructField`. + * The field of `nullable` specifies if values of a `StructField` can contain `null` values. + * + * @group field + */ + @DeveloperApi + type StructField = catalyst.types.StructField + + /** + * :: DeveloperApi :: + * + * A [[StructField]] object can be constructed by + * {{{ + * StructField(name: String, dataType: DataType, nullable: Boolean) + * }}} + * + * @group dataType + */ + @DeveloperApi + val StructField = catalyst.types.StructField +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala index 889a408e3c393..0a3b59cbc233a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala @@ -75,21 +75,21 @@ private[sql] object CatalystConverter { val fieldType: DataType = field.dataType fieldType match { // For native JVM types we use a converter with native arrays - case ArrayType(elementType: NativeType) => { + case ArrayType(elementType: NativeType, false) => { new CatalystNativeArrayConverter(elementType, fieldIndex, parent) } // This is for other types of arrays, including those with nested fields - case ArrayType(elementType: DataType) => { + case ArrayType(elementType: DataType, false) => { new CatalystArrayConverter(elementType, fieldIndex, parent) } case StructType(fields: Seq[StructField]) => { new CatalystStructConverter(fields.toArray, fieldIndex, parent) } - case MapType(keyType: DataType, valueType: DataType) => { + case MapType(keyType: DataType, valueType: DataType, valueContainsNull: Boolean) => { new CatalystMapConverter( Array( new FieldType(MAP_KEY_SCHEMA_NAME, keyType, false), - new FieldType(MAP_VALUE_SCHEMA_NAME, valueType, true)), + new FieldType(MAP_VALUE_SCHEMA_NAME, valueType, valueContainsNull)), fieldIndex, parent) } @@ -114,7 +114,7 @@ private[sql] object CatalystConverter { } } // All other primitive types use the default converter - case ctype: NativeType => { // note: need the type tag here! + case ctype: PrimitiveType => { // note: need the type tag here! new CatalystPrimitiveConverter(parent, fieldIndex) } case _ => throw new RuntimeException( @@ -229,9 +229,9 @@ private[parquet] class CatalystGroupConverter( this(attributes.map(a => new FieldType(a.name, a.dataType, a.nullable)), 0, null) protected [parquet] val converters: Array[Converter] = - schema.map(field => - CatalystConverter.createConverter(field, schema.indexOf(field), this)) - .toArray + schema.zipWithIndex.map { + case (field, idx) => CatalystConverter.createConverter(field, idx, this) + }.toArray override val size = schema.size @@ -288,9 +288,9 @@ private[parquet] class CatalystPrimitiveRowConverter( new ParquetRelation.RowType(attributes.length)) protected [parquet] val converters: Array[Converter] = - schema.map(field => - CatalystConverter.createConverter(field, schema.indexOf(field), this)) - .toArray + schema.zipWithIndex.map { + case (field, idx) => CatalystConverter.createConverter(field, idx, this) + }.toArray override val size = schema.size diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala index cc575bedd8fcb..2298a9b933df5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala @@ -201,8 +201,9 @@ object ParquetFilters { (leftFilter, rightFilter) match { case (None, Some(filter)) => Some(filter) case (Some(filter), None) => Some(filter) - case (_, _) => - Some(new AndFilter(leftFilter.get, rightFilter.get)) + case (Some(leftF), Some(rightF)) => + Some(new AndFilter(leftF, rightF)) + case _ => None } } case p @ EqualTo(left: Literal, right: NamedExpression) if !right.nullable => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala index 9c4771d1a9846..053b2a154389c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala @@ -27,6 +27,7 @@ import parquet.hadoop.ParquetOutputFormat import parquet.hadoop.metadata.CompressionCodecName import parquet.schema.MessageType +import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.analysis.{MultiInstanceRelation, UnresolvedException} import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, LeafNode} @@ -45,7 +46,9 @@ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, LeafNode} */ private[sql] case class ParquetRelation( path: String, - @transient conf: Option[Configuration] = None) extends LeafNode with MultiInstanceRelation { + @transient conf: Option[Configuration], + @transient sqlContext: SQLContext) + extends LeafNode with MultiInstanceRelation { self: Product => @@ -57,9 +60,13 @@ private[sql] case class ParquetRelation( .getSchema /** Attributes */ - override val output = ParquetTypesConverter.readSchemaFromFile(new Path(path), conf) + override val output = + ParquetTypesConverter.readSchemaFromFile( + new Path(path), + conf, + sqlContext.isParquetBinaryAsString) - override def newInstance = ParquetRelation(path).asInstanceOf[this.type] + override def newInstance = ParquetRelation(path, conf, sqlContext).asInstanceOf[this.type] // Equals must also take into account the output attributes so that we can distinguish between // different instances of the same relation, @@ -68,6 +75,9 @@ private[sql] case class ParquetRelation( p.path == path && p.output == output case _ => false } + + // TODO: Use data from the footers. + override lazy val statistics = Statistics(sizeInBytes = sqlContext.defaultSizeInBytes) } private[sql] object ParquetRelation { @@ -104,13 +114,14 @@ private[sql] object ParquetRelation { */ def create(pathString: String, child: LogicalPlan, - conf: Configuration): ParquetRelation = { + conf: Configuration, + sqlContext: SQLContext): ParquetRelation = { if (!child.resolved) { throw new UnresolvedException[LogicalPlan]( child, "Attempt to create Parquet table from unresolved child (when schema is not available)") } - createEmpty(pathString, child.output, false, conf) + createEmpty(pathString, child.output, false, conf, sqlContext) } /** @@ -125,14 +136,15 @@ private[sql] object ParquetRelation { def createEmpty(pathString: String, attributes: Seq[Attribute], allowExisting: Boolean, - conf: Configuration): ParquetRelation = { + conf: Configuration, + sqlContext: SQLContext): ParquetRelation = { val path = checkPath(pathString, allowExisting, conf) if (conf.get(ParquetOutputFormat.COMPRESSION) == null) { conf.set(ParquetOutputFormat.COMPRESSION, ParquetRelation.defaultCompression.name()) } ParquetRelation.enableLogForwarding() ParquetTypesConverter.writeMetaData(attributes, path, conf) - new ParquetRelation(path.toString, Some(conf)) { + new ParquetRelation(path.toString, Some(conf), sqlContext) { override val output = attributes } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala index ade823b51c9cd..759a2a586b926 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -17,54 +17,65 @@ package org.apache.spark.sql.parquet +import scala.collection.JavaConversions._ +import scala.collection.mutable +import scala.util.Try + import java.io.IOException +import java.lang.{Long => JLong} import java.text.SimpleDateFormat -import java.util.Date +import java.util.{Date, List => JList} import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.Path +import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.hadoop.mapreduce._ import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFormat} -import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat => NewFileOutputFormat, FileOutputCommitter} +import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat => NewFileOutputFormat} +import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter -import parquet.hadoop.{ParquetRecordReader, ParquetInputFormat, ParquetOutputFormat} -import parquet.hadoop.api.ReadSupport +import parquet.hadoop._ +import parquet.hadoop.api.{InitContext, ReadSupport} +import parquet.hadoop.metadata.GlobalMetaData import parquet.hadoop.util.ContextUtil -import parquet.io.InvalidRecordException +import parquet.io.ParquetDecodingException import parquet.schema.MessageType -import org.apache.spark.{Logging, SerializableWritable, TaskContext} import org.apache.spark.rdd.RDD import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, Row} import org.apache.spark.sql.execution.{LeafNode, SparkPlan, UnaryNode} +import org.apache.spark.{Logging, SerializableWritable, TaskContext} /** * Parquet table scan operator. Imports the file that backs the given * [[org.apache.spark.sql.parquet.ParquetRelation]] as a ``RDD[Row]``. */ case class ParquetTableScan( - // note: output cannot be transient, see - // https://issues.apache.org/jira/browse/SPARK-1367 - output: Seq[Attribute], + attributes: Seq[Attribute], relation: ParquetRelation, - columnPruningPred: Seq[Expression])( - @transient val sqlContext: SQLContext) + columnPruningPred: Seq[Expression]) extends LeafNode { + // The resolution of Parquet attributes is case sensitive, so we resolve the original attributes + // by exprId. note: output cannot be transient, see + // https://issues.apache.org/jira/browse/SPARK-1367 + val output = attributes.map { a => + relation.output + .find(o => o.exprId == a.exprId) + .getOrElse(sys.error(s"Invalid parquet attribute $a in ${relation.output.mkString(",")}")) + } + override def execute(): RDD[Row] = { val sc = sqlContext.sparkContext val job = new Job(sc.hadoopConfiguration) - ParquetInputFormat.setReadSupportClass( - job, - classOf[org.apache.spark.sql.parquet.RowReadSupport]) + ParquetInputFormat.setReadSupportClass(job, classOf[RowReadSupport]) + val conf: Configuration = ContextUtil.getConfiguration(job) - val fileList = FileSystemHelper.listFiles(relation.path, conf) - // add all paths in the directory but skip "hidden" ones such - // as "_SUCCESS" and "_metadata" - for (path <- fileList if !path.getName.startsWith("_")) { - NewFileInputFormat.addInputPath(job, path) + val qualifiedPath = { + val path = new Path(relation.path) + path.getFileSystem(conf).makeQualified(path) } + NewFileInputFormat.addInputPath(job, qualifiedPath) // Store both requested and original schema in `Configuration` conf.set( @@ -87,15 +98,13 @@ case class ParquetTableScan( sc.newAPIHadoopRDD( conf, - classOf[org.apache.spark.sql.parquet.FilteringParquetRowInputFormat], + classOf[FilteringParquetRowInputFormat], classOf[Void], classOf[Row]) .map(_._2) .filter(_ != null) // Parquet's record filters may produce null values } - override def otherCopyArgs = sqlContext :: Nil - /** * Applies a (candidate) projection. * @@ -105,10 +114,9 @@ case class ParquetTableScan( def pruneColumns(prunedAttributes: Seq[Attribute]): ParquetTableScan = { val success = validateProjection(prunedAttributes) if (success) { - ParquetTableScan(prunedAttributes, relation, columnPruningPred)(sqlContext) + ParquetTableScan(prunedAttributes, relation, columnPruningPred) } else { sys.error("Warning: Could not validate Parquet schema projection in pruneColumns") - this } } @@ -122,14 +130,7 @@ case class ParquetTableScan( private def validateProjection(projection: Seq[Attribute]): Boolean = { val original: MessageType = relation.parquetSchema val candidate: MessageType = ParquetTypesConverter.convertFromAttributes(projection) - try { - original.checkContains(candidate) - true - } catch { - case e: InvalidRecordException => { - false - } - } + Try(original.checkContains(candidate)).isSuccess } } @@ -152,8 +153,7 @@ case class ParquetTableScan( case class InsertIntoParquetTable( relation: ParquetRelation, child: SparkPlan, - overwrite: Boolean = false)( - @transient val sqlContext: SQLContext) + overwrite: Boolean = false) extends UnaryNode with SparkHadoopMapReduceUtil { /** @@ -173,7 +173,7 @@ case class InsertIntoParquetTable( val writeSupport = if (child.output.map(_.dataType).forall(_.isPrimitive)) { - logger.debug("Initializing MutableRowWriteSupport") + log.debug("Initializing MutableRowWriteSupport") classOf[org.apache.spark.sql.parquet.MutableRowWriteSupport] } else { classOf[org.apache.spark.sql.parquet.RowWriteSupport] @@ -205,8 +205,6 @@ case class InsertIntoParquetTable( override def output = child.output - override def otherCopyArgs = sqlContext :: Nil - /** * Stores the given Row RDD as a Hadoop file. * @@ -302,6 +300,11 @@ private[parquet] class AppendingParquetOutputFormat(offset: Int) */ private[parquet] class FilteringParquetRowInputFormat extends parquet.hadoop.ParquetInputFormat[Row] with Logging { + + private var footers: JList[Footer] = _ + + private var fileStatuses= Map.empty[Path, FileStatus] + override def createRecordReader( inputSplit: InputSplit, taskAttemptContext: TaskAttemptContext): RecordReader[Void, Row] = { @@ -318,6 +321,70 @@ private[parquet] class FilteringParquetRowInputFormat new ParquetRecordReader[Row](readSupport) } } + + override def getFooters(jobContext: JobContext): JList[Footer] = { + if (footers eq null) { + val statuses = listStatus(jobContext) + fileStatuses = statuses.map(file => file.getPath -> file).toMap + footers = getFooters(ContextUtil.getConfiguration(jobContext), statuses) + } + + footers + } + + // TODO Remove this method and related code once PARQUET-16 is fixed + // This method together with the `getFooters` method and the `fileStatuses` field are just used + // to mimic this PR: https://github.com/apache/incubator-parquet-mr/pull/17 + override def getSplits( + configuration: Configuration, + footers: JList[Footer]): JList[ParquetInputSplit] = { + + val maxSplitSize: JLong = configuration.getLong("mapred.max.split.size", Long.MaxValue) + val minSplitSize: JLong = + Math.max(getFormatMinSplitSize(), configuration.getLong("mapred.min.split.size", 0L)) + if (maxSplitSize < 0 || minSplitSize < 0) { + throw new ParquetDecodingException( + s"maxSplitSize or minSplitSie should not be negative: maxSplitSize = $maxSplitSize;" + + s" minSplitSize = $minSplitSize") + } + + val getGlobalMetaData = + classOf[ParquetFileWriter].getDeclaredMethod("getGlobalMetaData", classOf[JList[Footer]]) + getGlobalMetaData.setAccessible(true) + val globalMetaData = getGlobalMetaData.invoke(null, footers).asInstanceOf[GlobalMetaData] + + val readContext = getReadSupport(configuration).init( + new InitContext(configuration, + globalMetaData.getKeyValueMetaData(), + globalMetaData.getSchema())) + + val generateSplits = + classOf[ParquetInputFormat[_]].getDeclaredMethods.find(_.getName == "generateSplits").get + generateSplits.setAccessible(true) + + val splits = mutable.ArrayBuffer.empty[ParquetInputSplit] + for (footer <- footers) { + val fs = footer.getFile.getFileSystem(configuration) + val file = footer.getFile + val fileStatus = fileStatuses.getOrElse(file, fs.getFileStatus(file)) + val parquetMetaData = footer.getParquetMetadata + val blocks = parquetMetaData.getBlocks + val fileBlockLocations = fs.getFileBlockLocations(fileStatus, 0, fileStatus.getLen) + splits.addAll( + generateSplits.invoke( + null, + blocks, + fileBlockLocations, + fileStatus, + parquetMetaData.getFileMetaData, + readContext.getRequestedSchema.toString, + readContext.getReadSupportMetadata, + minSplitSize, + maxSplitSize).asInstanceOf[JList[ParquetInputSplit]]) + } + + splits + } } private[parquet] object FileSystemHelper { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala index 9cd5dc5bbd393..6a657c20fe46c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala @@ -17,20 +17,19 @@ package org.apache.spark.sql.parquet -import org.apache.hadoop.conf.Configuration +import java.util.{HashMap => JHashMap} +import org.apache.hadoop.conf.Configuration import parquet.column.ParquetProperties import parquet.hadoop.ParquetOutputFormat import parquet.hadoop.api.ReadSupport.ReadContext import parquet.hadoop.api.{ReadSupport, WriteSupport} import parquet.io.api._ -import parquet.schema.{MessageType, MessageTypeParser} +import parquet.schema.MessageType import org.apache.spark.Logging import org.apache.spark.sql.catalyst.expressions.{Attribute, Row} import org.apache.spark.sql.catalyst.types._ -import org.apache.spark.sql.execution.SparkSqlSerializer -import com.google.common.io.BaseEncoding /** * A `parquet.io.api.RecordMaterializer` for Rows. @@ -81,9 +80,10 @@ private[parquet] class RowReadSupport extends ReadSupport[Row] with Logging { } } // if both unavailable, fall back to deducing the schema from the given Parquet schema + // TODO: Why it can be null? if (schema == null) { log.debug("falling back to Parquet read schema") - schema = ParquetTypesConverter.convertToAttributes(parquetSchema) + schema = ParquetTypesConverter.convertToAttributes(parquetSchema, false) } log.debug(s"list of attributes that will be read: $schema") new RowRecordMaterializer(parquetSchema, schema) @@ -93,8 +93,8 @@ private[parquet] class RowReadSupport extends ReadSupport[Row] with Logging { configuration: Configuration, keyValueMetaData: java.util.Map[String, String], fileSchema: MessageType): ReadContext = { - var parquetSchema: MessageType = fileSchema - var metadata: java.util.Map[String, String] = new java.util.HashMap[String, String]() + var parquetSchema = fileSchema + val metadata = new JHashMap[String, String]() val requestedAttributes = RowReadSupport.getRequestedSchema(configuration) if (requestedAttributes != null) { @@ -109,7 +109,7 @@ private[parquet] class RowReadSupport extends ReadSupport[Row] with Logging { metadata.put(RowReadSupport.SPARK_METADATA_KEY, origAttributesStr) } - return new ReadSupport.ReadContext(parquetSchema, metadata) + new ReadSupport.ReadContext(parquetSchema, metadata) } } @@ -132,13 +132,17 @@ private[parquet] class RowWriteSupport extends WriteSupport[Row] with Logging { private[parquet] var attributes: Seq[Attribute] = null override def init(configuration: Configuration): WriteSupport.WriteContext = { - attributes = if (attributes == null) RowWriteSupport.getSchema(configuration) else attributes - + val origAttributesStr: String = configuration.get(RowWriteSupport.SPARK_ROW_SCHEMA) + val metadata = new JHashMap[String, String]() + metadata.put(RowReadSupport.SPARK_METADATA_KEY, origAttributesStr) + + if (attributes == null) { + attributes = ParquetTypesConverter.convertFromString(origAttributesStr) + } + log.debug(s"write support initialized for requested schema $attributes") ParquetRelation.enableLogForwarding() - new WriteSupport.WriteContext( - ParquetTypesConverter.convertFromAttributes(attributes), - new java.util.HashMap[java.lang.String, java.lang.String]()) + new WriteSupport.WriteContext(ParquetTypesConverter.convertFromAttributes(attributes), metadata) } override def prepareForWrite(recordConsumer: RecordConsumer): Unit = { @@ -156,7 +160,7 @@ private[parquet] class RowWriteSupport extends WriteSupport[Row] with Logging { writer.startMessage() while(index < attributes.size) { // null values indicate optional fields but we do not check currently - if (record(index) != null && record(index) != Nil) { + if (record(index) != null) { writer.startField(attributes(index).name, index) writeValue(attributes(index).dataType, record(index)) writer.endField(attributes(index).name, index) @@ -167,12 +171,12 @@ private[parquet] class RowWriteSupport extends WriteSupport[Row] with Logging { } private[parquet] def writeValue(schema: DataType, value: Any): Unit = { - if (value != null && value != Nil) { + if (value != null) { schema match { - case t @ ArrayType(_) => writeArray( + case t @ ArrayType(_, false) => writeArray( t, value.asInstanceOf[CatalystConverter.ArrayScalaType[_]]) - case t @ MapType(_, _) => writeMap( + case t @ MapType(_, _, _) => writeMap( t, value.asInstanceOf[CatalystConverter.MapScalaType[_, _]]) case t @ StructType(_) => writeStruct( @@ -184,13 +188,15 @@ private[parquet] class RowWriteSupport extends WriteSupport[Row] with Logging { } private[parquet] def writePrimitive(schema: PrimitiveType, value: Any): Unit = { - if (value != null && value != Nil) { + if (value != null) { schema match { case StringType => writer.addBinary( Binary.fromByteArray( value.asInstanceOf[String].getBytes("utf-8") ) ) + case BinaryType => writer.addBinary( + Binary.fromByteArray(value.asInstanceOf[Array[Byte]])) case IntegerType => writer.addInteger(value.asInstanceOf[Int]) case ShortType => writer.addInteger(value.asInstanceOf[Short]) case LongType => writer.addLong(value.asInstanceOf[Long]) @@ -206,12 +212,12 @@ private[parquet] class RowWriteSupport extends WriteSupport[Row] with Logging { private[parquet] def writeStruct( schema: StructType, struct: CatalystConverter.StructScalaType[_]): Unit = { - if (struct != null && struct != Nil) { + if (struct != null) { val fields = schema.fields.toArray writer.startGroup() var i = 0 while(i < fields.size) { - if (struct(i) != null && struct(i) != Nil) { + if (struct(i) != null) { writer.startField(fields(i).name, i) writeValue(fields(i).dataType, struct(i)) writer.endField(fields(i).name, i) @@ -299,6 +305,8 @@ private[parquet] class MutableRowWriteSupport extends RowWriteSupport { record(index).asInstanceOf[String].getBytes("utf-8") ) ) + case BinaryType => writer.addBinary( + Binary.fromByteArray(record(index).asInstanceOf[Array[Byte]])) case IntegerType => writer.addInteger(record.getInt(index)) case ShortType => writer.addInteger(record.getShort(index)) case LongType => writer.addLong(record.getLong(index)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala index 1dc58633a2a68..837ea7695dbb3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala @@ -22,6 +22,7 @@ import java.io.File import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} import org.apache.hadoop.mapreduce.Job +import org.apache.spark.sql.test.TestSQLContext import parquet.example.data.{GroupWriter, Group} import parquet.example.data.simple.SimpleGroup @@ -58,7 +59,7 @@ private[sql] object ParquetTestData { """message myrecord { optional boolean myboolean; optional int32 myint; - optional binary mystring; + optional binary mystring (UTF8); optional int64 mylong; optional float myfloat; optional double mydouble; @@ -87,7 +88,7 @@ private[sql] object ParquetTestData { message myrecord { required boolean myboolean; required int32 myint; - required binary mystring; + required binary mystring (UTF8); required int64 mylong; required float myfloat; required double mydouble; @@ -103,7 +104,7 @@ private[sql] object ParquetTestData { val testDir = Utils.createTempDir() val testFilterDir = Utils.createTempDir() - lazy val testData = new ParquetRelation(testDir.toURI.toString) + lazy val testData = new ParquetRelation(testDir.toURI.toString, None, TestSQLContext) val testNestedSchema1 = // based on blogpost example, source: @@ -119,14 +120,14 @@ private[sql] object ParquetTestData { // so that array types can be translated correctly. """ message AddressBook { - required binary owner; + required binary owner (UTF8); optional group ownerPhoneNumbers { - repeated binary array; + repeated binary array (UTF8); } optional group contacts { repeated group array { - required binary name; - optional binary phoneNumber; + required binary name (UTF8); + optional binary phoneNumber (UTF8); } } } @@ -181,16 +182,16 @@ private[sql] object ParquetTestData { required int32 x; optional group data1 { repeated group map { - required binary key; + required binary key (UTF8); required int32 value; } } required group data2 { repeated group map { - required binary key; + required binary key (UTF8); required group value { required int64 payload1; - optional binary payload2; + optional binary payload2 (UTF8); } } } @@ -202,8 +203,10 @@ private[sql] object ParquetTestData { val testNestedDir3 = Utils.createTempDir() val testNestedDir4 = Utils.createTempDir() - lazy val testNestedData1 = new ParquetRelation(testNestedDir1.toURI.toString) - lazy val testNestedData2 = new ParquetRelation(testNestedDir2.toURI.toString) + lazy val testNestedData1 = + new ParquetRelation(testNestedDir1.toURI.toString, None, TestSQLContext) + lazy val testNestedData2 = + new ParquetRelation(testNestedDir2.toURI.toString, None, TestSQLContext) def writeFile() = { testDir.delete() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala index f9046368e7ced..c79a9ac2dad81 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala @@ -22,6 +22,7 @@ import java.io.IOException import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.mapreduce.Job +import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter import parquet.hadoop.{ParquetFileReader, Footer, ParquetFileWriter} import parquet.hadoop.metadata.{ParquetMetadata, FileMetaData} @@ -42,20 +43,25 @@ private[parquet] object ParquetTypesConverter extends Logging { def isPrimitiveType(ctype: DataType): Boolean = classOf[PrimitiveType] isAssignableFrom ctype.getClass - def toPrimitiveDataType(parquetType : ParquetPrimitiveTypeName): DataType = parquetType match { - case ParquetPrimitiveTypeName.BINARY => StringType - case ParquetPrimitiveTypeName.BOOLEAN => BooleanType - case ParquetPrimitiveTypeName.DOUBLE => DoubleType - case ParquetPrimitiveTypeName.FIXED_LEN_BYTE_ARRAY => ArrayType(ByteType) - case ParquetPrimitiveTypeName.FLOAT => FloatType - case ParquetPrimitiveTypeName.INT32 => IntegerType - case ParquetPrimitiveTypeName.INT64 => LongType - case ParquetPrimitiveTypeName.INT96 => - // TODO: add BigInteger type? TODO(andre) use DecimalType instead???? - sys.error("Potential loss of precision: cannot convert INT96") - case _ => sys.error( - s"Unsupported parquet datatype $parquetType") - } + def toPrimitiveDataType( + parquetType: ParquetPrimitiveType, + binayAsString: Boolean): DataType = + parquetType.getPrimitiveTypeName match { + case ParquetPrimitiveTypeName.BINARY + if (parquetType.getOriginalType == ParquetOriginalType.UTF8 || + binayAsString) => StringType + case ParquetPrimitiveTypeName.BINARY => BinaryType + case ParquetPrimitiveTypeName.BOOLEAN => BooleanType + case ParquetPrimitiveTypeName.DOUBLE => DoubleType + case ParquetPrimitiveTypeName.FLOAT => FloatType + case ParquetPrimitiveTypeName.INT32 => IntegerType + case ParquetPrimitiveTypeName.INT64 => LongType + case ParquetPrimitiveTypeName.INT96 => + // TODO: add BigInteger type? TODO(andre) use DecimalType instead???? + sys.error("Potential loss of precision: cannot convert INT96") + case _ => sys.error( + s"Unsupported parquet datatype $parquetType") + } /** * Converts a given Parquet `Type` into the corresponding @@ -82,7 +88,7 @@ private[parquet] object ParquetTypesConverter extends Logging { * @param parquetType The type to convert. * @return The corresponding Catalyst type. */ - def toDataType(parquetType: ParquetType): DataType = { + def toDataType(parquetType: ParquetType, isBinaryAsString: Boolean): DataType = { def correspondsToMap(groupType: ParquetGroupType): Boolean = { if (groupType.getFieldCount != 1 || groupType.getFields.apply(0).isPrimitive) { false @@ -104,7 +110,7 @@ private[parquet] object ParquetTypesConverter extends Logging { } if (parquetType.isPrimitive) { - toPrimitiveDataType(parquetType.asPrimitiveType.getPrimitiveTypeName) + toPrimitiveDataType(parquetType.asPrimitiveType, isBinaryAsString) } else { val groupType = parquetType.asGroupType() parquetType.getOriginalType match { @@ -113,7 +119,7 @@ private[parquet] object ParquetTypesConverter extends Logging { case ParquetOriginalType.LIST => { // TODO: check enums! assert(groupType.getFieldCount == 1) val field = groupType.getFields.apply(0) - new ArrayType(toDataType(field)) + ArrayType(toDataType(field, isBinaryAsString), containsNull = false) } case ParquetOriginalType.MAP => { assert( @@ -123,32 +129,36 @@ private[parquet] object ParquetTypesConverter extends Logging { assert( keyValueGroup.getFieldCount == 2, "Parquet Map type malformatted: nested group should have 2 (key, value) fields!") - val keyType = toDataType(keyValueGroup.getFields.apply(0)) + val keyType = toDataType(keyValueGroup.getFields.apply(0), isBinaryAsString) assert(keyValueGroup.getFields.apply(0).getRepetition == Repetition.REQUIRED) - val valueType = toDataType(keyValueGroup.getFields.apply(1)) + val valueType = toDataType(keyValueGroup.getFields.apply(1), isBinaryAsString) assert(keyValueGroup.getFields.apply(1).getRepetition == Repetition.REQUIRED) - new MapType(keyType, valueType) + // TODO: set valueContainsNull explicitly instead of assuming valueContainsNull is true + // at here. + MapType(keyType, valueType) } case _ => { // Note: the order of these checks is important! if (correspondsToMap(groupType)) { // MapType val keyValueGroup = groupType.getFields.apply(0).asGroupType() - val keyType = toDataType(keyValueGroup.getFields.apply(0)) + val keyType = toDataType(keyValueGroup.getFields.apply(0), isBinaryAsString) assert(keyValueGroup.getFields.apply(0).getRepetition == Repetition.REQUIRED) - val valueType = toDataType(keyValueGroup.getFields.apply(1)) + val valueType = toDataType(keyValueGroup.getFields.apply(1), isBinaryAsString) assert(keyValueGroup.getFields.apply(1).getRepetition == Repetition.REQUIRED) - new MapType(keyType, valueType) + // TODO: set valueContainsNull explicitly instead of assuming valueContainsNull is true + // at here. + MapType(keyType, valueType) } else if (correspondsToArray(groupType)) { // ArrayType - val elementType = toDataType(groupType.getFields.apply(0)) - new ArrayType(elementType) + val elementType = toDataType(groupType.getFields.apply(0), isBinaryAsString) + ArrayType(elementType, containsNull = false) } else { // everything else: StructType val fields = groupType .getFields .map(ptype => new StructField( ptype.getName, - toDataType(ptype), + toDataType(ptype, isBinaryAsString), ptype.getRepetition != Repetition.REQUIRED)) - new StructType(fields) + StructType(fields) } } } @@ -164,18 +174,17 @@ private[parquet] object ParquetTypesConverter extends Logging { * @return The name of the corresponding Parquet primitive type */ def fromPrimitiveDataType(ctype: DataType): - Option[ParquetPrimitiveTypeName] = ctype match { - case StringType => Some(ParquetPrimitiveTypeName.BINARY) - case BooleanType => Some(ParquetPrimitiveTypeName.BOOLEAN) - case DoubleType => Some(ParquetPrimitiveTypeName.DOUBLE) - case ArrayType(ByteType) => - Some(ParquetPrimitiveTypeName.FIXED_LEN_BYTE_ARRAY) - case FloatType => Some(ParquetPrimitiveTypeName.FLOAT) - case IntegerType => Some(ParquetPrimitiveTypeName.INT32) + Option[(ParquetPrimitiveTypeName, Option[ParquetOriginalType])] = ctype match { + case StringType => Some(ParquetPrimitiveTypeName.BINARY, Some(ParquetOriginalType.UTF8)) + case BinaryType => Some(ParquetPrimitiveTypeName.BINARY, None) + case BooleanType => Some(ParquetPrimitiveTypeName.BOOLEAN, None) + case DoubleType => Some(ParquetPrimitiveTypeName.DOUBLE, None) + case FloatType => Some(ParquetPrimitiveTypeName.FLOAT, None) + case IntegerType => Some(ParquetPrimitiveTypeName.INT32, None) // There is no type for Byte or Short so we promote them to INT32. - case ShortType => Some(ParquetPrimitiveTypeName.INT32) - case ByteType => Some(ParquetPrimitiveTypeName.INT32) - case LongType => Some(ParquetPrimitiveTypeName.INT64) + case ShortType => Some(ParquetPrimitiveTypeName.INT32, None) + case ByteType => Some(ParquetPrimitiveTypeName.INT32, None) + case LongType => Some(ParquetPrimitiveTypeName.INT64, None) case _ => None } @@ -227,17 +236,18 @@ private[parquet] object ParquetTypesConverter extends Logging { if (nullable) Repetition.OPTIONAL else Repetition.REQUIRED } val primitiveType = fromPrimitiveDataType(ctype) - if (primitiveType.isDefined) { - new ParquetPrimitiveType(repetition, primitiveType.get, name) - } else { + primitiveType.map { + case (primitiveType, originalType) => + new ParquetPrimitiveType(repetition, primitiveType, name, originalType.orNull) + }.getOrElse { ctype match { - case ArrayType(elementType) => { + case ArrayType(elementType, false) => { val parquetElementType = fromDataType( elementType, CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, nullable = false, inArray = true) - ConversionPatterns.listType(repetition, name, parquetElementType) + ConversionPatterns.listType(repetition, name, parquetElementType) } case StructType(structFields) => { val fields = structFields.map { @@ -245,7 +255,7 @@ private[parquet] object ParquetTypesConverter extends Logging { } new ParquetGroupType(repetition, name, fields) } - case MapType(keyType, valueType) => { + case MapType(keyType, valueType, _) => { val parquetKeyType = fromDataType( keyType, @@ -269,7 +279,7 @@ private[parquet] object ParquetTypesConverter extends Logging { } } - def convertToAttributes(parquetSchema: ParquetType): Seq[Attribute] = { + def convertToAttributes(parquetSchema: ParquetType, isBinaryAsString: Boolean): Seq[Attribute] = { parquetSchema .asGroupType() .getFields @@ -277,7 +287,7 @@ private[parquet] object ParquetTypesConverter extends Logging { field => new AttributeReference( field.getName, - toDataType(field), + toDataType(field, isBinaryAsString), field.getRepetition != Repetition.REQUIRED)()) } @@ -365,20 +375,25 @@ private[parquet] object ParquetTypesConverter extends Logging { s"Expected $path for be a directory with Parquet files/metadata") } ParquetRelation.enableLogForwarding() - val metadataPath = new Path(path, ParquetFileWriter.PARQUET_METADATA_FILE) - // if this is a new table that was just created we will find only the metadata file - if (fs.exists(metadataPath) && fs.isFile(metadataPath)) { - ParquetFileReader.readFooter(conf, metadataPath) - } else { - // there may be one or more Parquet files in the given directory - val footers = ParquetFileReader.readFooters(conf, fs.getFileStatus(path)) - // TODO: for now we assume that all footers (if there is more than one) have identical - // metadata; we may want to add a check here at some point - if (footers.size() == 0) { - throw new IllegalArgumentException(s"Could not find Parquet metadata at path $path") - } - footers(0).getParquetMetadata + + val children = fs.listStatus(path).filterNot { status => + val name = status.getPath.getName + name(0) == '.' || name == FileOutputCommitter.SUCCEEDED_FILE_NAME } + + // NOTE (lian): Parquet "_metadata" file can be very slow if the file consists of lots of row + // groups. Since Parquet schema is replicated among all row groups, we only need to touch a + // single row group to read schema related metadata. Notice that we are making assumptions that + // all data in a single Parquet file have the same schema, which is normally true. + children + // Try any non-"_metadata" file first... + .find(_.getPath.getName != ParquetFileWriter.PARQUET_METADATA_FILE) + // ... and fallback to "_metadata" if no such file exists (which implies the Parquet file is + // empty, thus normally the "_metadata" file is expected to be fairly small). + .orElse(children.find(_.getPath.getName == ParquetFileWriter.PARQUET_METADATA_FILE)) + .map(ParquetFileReader.readFooter(conf, _)) + .getOrElse( + throw new IllegalArgumentException(s"Could not find Parquet metadata at path $path")) } /** @@ -391,7 +406,10 @@ private[parquet] object ParquetTypesConverter extends Logging { * @param conf The Hadoop configuration to use. * @return A list of attributes that make up the schema. */ - def readSchemaFromFile(origPath: Path, conf: Option[Configuration]): Seq[Attribute] = { + def readSchemaFromFile( + origPath: Path, + conf: Option[Configuration], + isBinaryAsString: Boolean): Seq[Attribute] = { val keyValueMetadata: java.util.Map[String, String] = readMetaData(origPath, conf) .getFileMetaData @@ -400,8 +418,8 @@ private[parquet] object ParquetTypesConverter extends Logging { convertFromString(keyValueMetadata.get(RowReadSupport.SPARK_METADATA_KEY)) } else { val attributes = convertToAttributes( - readMetaData(origPath, conf).getFileMetaData.getSchema) - log.warn(s"Falling back to schema conversion from Parquet types; result: $attributes") + readMetaData(origPath, conf).getFileMetaData.getSchema, isBinaryAsString) + log.info(s"Falling back to schema conversion from Parquet types; result: $attributes") attributes } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala b/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala new file mode 100644 index 0000000000000..77353f4eb0227 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala @@ -0,0 +1,110 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.types.util + +import org.apache.spark.sql._ +import org.apache.spark.sql.api.java.{DataType => JDataType, StructField => JStructField} + +import scala.collection.JavaConverters._ + +protected[sql] object DataTypeConversions { + + /** + * Returns the equivalent StructField in Scala for the given StructField in Java. + */ + def asJavaStructField(scalaStructField: StructField): JStructField = { + JDataType.createStructField( + scalaStructField.name, + asJavaDataType(scalaStructField.dataType), + scalaStructField.nullable) + } + + /** + * Returns the equivalent DataType in Java for the given DataType in Scala. + */ + def asJavaDataType(scalaDataType: DataType): JDataType = scalaDataType match { + case StringType => JDataType.StringType + case BinaryType => JDataType.BinaryType + case BooleanType => JDataType.BooleanType + case TimestampType => JDataType.TimestampType + case DecimalType => JDataType.DecimalType + case DoubleType => JDataType.DoubleType + case FloatType => JDataType.FloatType + case ByteType => JDataType.ByteType + case IntegerType => JDataType.IntegerType + case LongType => JDataType.LongType + case ShortType => JDataType.ShortType + + case arrayType: ArrayType => JDataType.createArrayType( + asJavaDataType(arrayType.elementType), arrayType.containsNull) + case mapType: MapType => JDataType.createMapType( + asJavaDataType(mapType.keyType), + asJavaDataType(mapType.valueType), + mapType.valueContainsNull) + case structType: StructType => JDataType.createStructType( + structType.fields.map(asJavaStructField).asJava) + } + + /** + * Returns the equivalent StructField in Scala for the given StructField in Java. + */ + def asScalaStructField(javaStructField: JStructField): StructField = { + StructField( + javaStructField.getName, + asScalaDataType(javaStructField.getDataType), + javaStructField.isNullable) + } + + /** + * Returns the equivalent DataType in Scala for the given DataType in Java. + */ + def asScalaDataType(javaDataType: JDataType): DataType = javaDataType match { + case stringType: org.apache.spark.sql.api.java.StringType => + StringType + case binaryType: org.apache.spark.sql.api.java.BinaryType => + BinaryType + case booleanType: org.apache.spark.sql.api.java.BooleanType => + BooleanType + case timestampType: org.apache.spark.sql.api.java.TimestampType => + TimestampType + case decimalType: org.apache.spark.sql.api.java.DecimalType => + DecimalType + case doubleType: org.apache.spark.sql.api.java.DoubleType => + DoubleType + case floatType: org.apache.spark.sql.api.java.FloatType => + FloatType + case byteType: org.apache.spark.sql.api.java.ByteType => + ByteType + case integerType: org.apache.spark.sql.api.java.IntegerType => + IntegerType + case longType: org.apache.spark.sql.api.java.LongType => + LongType + case shortType: org.apache.spark.sql.api.java.ShortType => + ShortType + + case arrayType: org.apache.spark.sql.api.java.ArrayType => + ArrayType(asScalaDataType(arrayType.getElementType), arrayType.isContainsNull) + case mapType: org.apache.spark.sql.api.java.MapType => + MapType( + asScalaDataType(mapType.getKeyType), + asScalaDataType(mapType.getValueType), + mapType.isValueContainsNull) + case structType: org.apache.spark.sql.api.java.StructType => + StructType(structType.getFields.map(asScalaStructField)) + } +} diff --git a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaAPISuite.java b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaAPISuite.java new file mode 100644 index 0000000000000..a9a11285def54 --- /dev/null +++ b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaAPISuite.java @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +import org.apache.spark.sql.api.java.UDF1; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.runners.Suite; +import org.junit.runner.RunWith; + +import org.apache.spark.api.java.JavaSparkContext; + +// The test suite itself is Serializable so that anonymous Function implementations can be +// serialized, as an alternative to converting these anonymous classes to static inner classes; +// see http://stackoverflow.com/questions/758570/. +public class JavaAPISuite implements Serializable { + private transient JavaSparkContext sc; + private transient JavaSQLContext sqlContext; + + @Before + public void setUp() { + sc = new JavaSparkContext("local", "JavaAPISuite"); + sqlContext = new JavaSQLContext(sc); + } + + @After + public void tearDown() { + sc.stop(); + sc = null; + } + + @SuppressWarnings("unchecked") + @Test + public void udf1Test() { + // With Java 8 lambdas: + // sqlContext.registerFunction( + // "stringLengthTest", (String str) -> str.length(), DataType.IntegerType); + + sqlContext.registerFunction("stringLengthTest", new UDF1() { + @Override + public Integer call(String str) throws Exception { + return str.length(); + } + }, DataType.IntegerType); + + // TODO: Why do we need this cast? + Row result = (Row) sqlContext.sql("SELECT stringLengthTest('test')").first(); + assert(result.getInt(0) == 4); + } + + @SuppressWarnings("unchecked") + @Test + public void udf2Test() { + // With Java 8 lambdas: + // sqlContext.registerFunction( + // "stringLengthTest", + // (String str1, String str2) -> str1.length() + str2.length, + // DataType.IntegerType); + + sqlContext.registerFunction("stringLengthTest", new UDF2() { + @Override + public Integer call(String str1, String str2) throws Exception { + return str1.length() + str2.length(); + } + }, DataType.IntegerType); + + // TODO: Why do we need this cast? + Row result = (Row) sqlContext.sql("SELECT stringLengthTest('test', 'test2')").first(); + assert(result.getInt(0) == 9); + } +} diff --git a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaApplySchemaSuite.java b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaApplySchemaSuite.java new file mode 100644 index 0000000000000..33e5020bc636a --- /dev/null +++ b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaApplySchemaSuite.java @@ -0,0 +1,163 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; +import java.math.BigDecimal; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.Function; + +// The test suite itself is Serializable so that anonymous Function implementations can be +// serialized, as an alternative to converting these anonymous classes to static inner classes; +// see http://stackoverflow.com/questions/758570/. +public class JavaApplySchemaSuite implements Serializable { + private transient JavaSparkContext javaCtx; + private transient JavaSQLContext javaSqlCtx; + + @Before + public void setUp() { + javaCtx = new JavaSparkContext("local", "JavaApplySchemaSuite"); + javaSqlCtx = new JavaSQLContext(javaCtx); + } + + @After + public void tearDown() { + javaCtx.stop(); + javaCtx = null; + javaSqlCtx = null; + } + + public static class Person implements Serializable { + private String name; + private int age; + + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + public int getAge() { + return age; + } + + public void setAge(int age) { + this.age = age; + } + } + + @Test + public void applySchema() { + List personList = new ArrayList(2); + Person person1 = new Person(); + person1.setName("Michael"); + person1.setAge(29); + personList.add(person1); + Person person2 = new Person(); + person2.setName("Yin"); + person2.setAge(28); + personList.add(person2); + + JavaRDD rowRDD = javaCtx.parallelize(personList).map( + new Function() { + public Row call(Person person) throws Exception { + return Row.create(person.getName(), person.getAge()); + } + }); + + List fields = new ArrayList(2); + fields.add(DataType.createStructField("name", DataType.StringType, false)); + fields.add(DataType.createStructField("age", DataType.IntegerType, false)); + StructType schema = DataType.createStructType(fields); + + JavaSchemaRDD schemaRDD = javaSqlCtx.applySchema(rowRDD, schema); + schemaRDD.registerTempTable("people"); + List actual = javaSqlCtx.sql("SELECT * FROM people").collect(); + + List expected = new ArrayList(2); + expected.add(Row.create("Michael", 29)); + expected.add(Row.create("Yin", 28)); + + Assert.assertEquals(expected, actual); + } + + @Test + public void applySchemaToJSON() { + JavaRDD jsonRDD = javaCtx.parallelize(Arrays.asList( + "{\"string\":\"this is a simple string.\", \"integer\":10, \"long\":21474836470, " + + "\"bigInteger\":92233720368547758070, \"double\":1.7976931348623157E308, " + + "\"boolean\":true, \"null\":null}", + "{\"string\":\"this is another simple string.\", \"integer\":11, \"long\":21474836469, " + + "\"bigInteger\":92233720368547758069, \"double\":1.7976931348623157E305, " + + "\"boolean\":false, \"null\":null}")); + List fields = new ArrayList(7); + fields.add(DataType.createStructField("bigInteger", DataType.DecimalType, true)); + fields.add(DataType.createStructField("boolean", DataType.BooleanType, true)); + fields.add(DataType.createStructField("double", DataType.DoubleType, true)); + fields.add(DataType.createStructField("integer", DataType.IntegerType, true)); + fields.add(DataType.createStructField("long", DataType.LongType, true)); + fields.add(DataType.createStructField("null", DataType.StringType, true)); + fields.add(DataType.createStructField("string", DataType.StringType, true)); + StructType expectedSchema = DataType.createStructType(fields); + List expectedResult = new ArrayList(2); + expectedResult.add( + Row.create( + new BigDecimal("92233720368547758070"), + true, + 1.7976931348623157E308, + 10, + 21474836470L, + null, + "this is a simple string.")); + expectedResult.add( + Row.create( + new BigDecimal("92233720368547758069"), + false, + 1.7976931348623157E305, + 11, + 21474836469L, + null, + "this is another simple string.")); + + JavaSchemaRDD schemaRDD1 = javaSqlCtx.jsonRDD(jsonRDD); + StructType actualSchema1 = schemaRDD1.schema(); + Assert.assertEquals(expectedSchema, actualSchema1); + schemaRDD1.registerTempTable("jsonTable1"); + List actual1 = javaSqlCtx.sql("select * from jsonTable1").collect(); + Assert.assertEquals(expectedResult, actual1); + + JavaSchemaRDD schemaRDD2 = javaSqlCtx.jsonRDD(jsonRDD, expectedSchema); + StructType actualSchema2 = schemaRDD2.schema(); + Assert.assertEquals(expectedSchema, actualSchema2); + schemaRDD1.registerTempTable("jsonTable2"); + List actual2 = javaSqlCtx.sql("select * from jsonTable2").collect(); + Assert.assertEquals(expectedResult, actual2); + } +} diff --git a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaRowSuite.java b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaRowSuite.java new file mode 100644 index 0000000000000..52d07b5425cc3 --- /dev/null +++ b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaRowSuite.java @@ -0,0 +1,170 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.math.BigDecimal; +import java.sql.Timestamp; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +public class JavaRowSuite { + private byte byteValue; + private short shortValue; + private int intValue; + private long longValue; + private float floatValue; + private double doubleValue; + private BigDecimal decimalValue; + private boolean booleanValue; + private String stringValue; + private byte[] binaryValue; + private Timestamp timestampValue; + + @Before + public void setUp() { + byteValue = (byte)127; + shortValue = (short)32767; + intValue = 2147483647; + longValue = 9223372036854775807L; + floatValue = (float)3.4028235E38; + doubleValue = 1.7976931348623157E308; + decimalValue = new BigDecimal("1.7976931348623157E328"); + booleanValue = true; + stringValue = "this is a string"; + binaryValue = stringValue.getBytes(); + timestampValue = Timestamp.valueOf("2014-06-30 09:20:00.0"); + } + + @Test + public void constructSimpleRow() { + Row simpleRow = Row.create( + byteValue, // ByteType + new Byte(byteValue), + shortValue, // ShortType + new Short(shortValue), + intValue, // IntegerType + new Integer(intValue), + longValue, // LongType + new Long(longValue), + floatValue, // FloatType + new Float(floatValue), + doubleValue, // DoubleType + new Double(doubleValue), + decimalValue, // DecimalType + booleanValue, // BooleanType + new Boolean(booleanValue), + stringValue, // StringType + binaryValue, // BinaryType + timestampValue, // TimestampType + null // null + ); + + Assert.assertEquals(byteValue, simpleRow.getByte(0)); + Assert.assertEquals(byteValue, simpleRow.get(0)); + Assert.assertEquals(byteValue, simpleRow.getByte(1)); + Assert.assertEquals(byteValue, simpleRow.get(1)); + Assert.assertEquals(shortValue, simpleRow.getShort(2)); + Assert.assertEquals(shortValue, simpleRow.get(2)); + Assert.assertEquals(shortValue, simpleRow.getShort(3)); + Assert.assertEquals(shortValue, simpleRow.get(3)); + Assert.assertEquals(intValue, simpleRow.getInt(4)); + Assert.assertEquals(intValue, simpleRow.get(4)); + Assert.assertEquals(intValue, simpleRow.getInt(5)); + Assert.assertEquals(intValue, simpleRow.get(5)); + Assert.assertEquals(longValue, simpleRow.getLong(6)); + Assert.assertEquals(longValue, simpleRow.get(6)); + Assert.assertEquals(longValue, simpleRow.getLong(7)); + Assert.assertEquals(longValue, simpleRow.get(7)); + // When we create the row, we do not do any conversion + // for a float/double value, so we just set the delta to 0. + Assert.assertEquals(floatValue, simpleRow.getFloat(8), 0); + Assert.assertEquals(floatValue, simpleRow.get(8)); + Assert.assertEquals(floatValue, simpleRow.getFloat(9), 0); + Assert.assertEquals(floatValue, simpleRow.get(9)); + Assert.assertEquals(doubleValue, simpleRow.getDouble(10), 0); + Assert.assertEquals(doubleValue, simpleRow.get(10)); + Assert.assertEquals(doubleValue, simpleRow.getDouble(11), 0); + Assert.assertEquals(doubleValue, simpleRow.get(11)); + Assert.assertEquals(decimalValue, simpleRow.get(12)); + Assert.assertEquals(booleanValue, simpleRow.getBoolean(13)); + Assert.assertEquals(booleanValue, simpleRow.get(13)); + Assert.assertEquals(booleanValue, simpleRow.getBoolean(14)); + Assert.assertEquals(booleanValue, simpleRow.get(14)); + Assert.assertEquals(stringValue, simpleRow.getString(15)); + Assert.assertEquals(stringValue, simpleRow.get(15)); + Assert.assertEquals(binaryValue, simpleRow.get(16)); + Assert.assertEquals(timestampValue, simpleRow.get(17)); + Assert.assertEquals(true, simpleRow.isNullAt(18)); + Assert.assertEquals(null, simpleRow.get(18)); + } + + @Test + public void constructComplexRow() { + // Simple array + List simpleStringArray = Arrays.asList( + stringValue + " (1)", stringValue + " (2)", stringValue + "(3)"); + + // Simple map + Map simpleMap = new HashMap(); + simpleMap.put(stringValue + " (1)", longValue); + simpleMap.put(stringValue + " (2)", longValue - 1); + simpleMap.put(stringValue + " (3)", longValue - 2); + + // Simple struct + Row simpleStruct = Row.create( + doubleValue, stringValue, timestampValue, null); + + // Complex array + List> arrayOfMaps = Arrays.asList(simpleMap); + List arrayOfRows = Arrays.asList(simpleStruct); + + // Complex map + Map, Row> complexMap = new HashMap, Row>(); + complexMap.put(arrayOfRows, simpleStruct); + + // Complex struct + Row complexStruct = Row.create( + simpleStringArray, + simpleMap, + simpleStruct, + arrayOfMaps, + arrayOfRows, + complexMap, + null); + Assert.assertEquals(simpleStringArray, complexStruct.get(0)); + Assert.assertEquals(simpleMap, complexStruct.get(1)); + Assert.assertEquals(simpleStruct, complexStruct.get(2)); + Assert.assertEquals(arrayOfMaps, complexStruct.get(3)); + Assert.assertEquals(arrayOfRows, complexStruct.get(4)); + Assert.assertEquals(complexMap, complexStruct.get(5)); + Assert.assertEquals(null, complexStruct.get(6)); + + // A very complex row + Row complexRow = Row.create(arrayOfMaps, arrayOfRows, complexMap, complexStruct); + Assert.assertEquals(arrayOfMaps, complexRow.get(0)); + Assert.assertEquals(arrayOfRows, complexRow.get(1)); + Assert.assertEquals(complexMap, complexRow.get(2)); + Assert.assertEquals(complexStruct, complexRow.get(3)); + } +} diff --git a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaSideDataTypeConversionSuite.java b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaSideDataTypeConversionSuite.java new file mode 100644 index 0000000000000..d099a48a1f4b6 --- /dev/null +++ b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaSideDataTypeConversionSuite.java @@ -0,0 +1,148 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.util.List; +import java.util.ArrayList; + +import org.junit.Assert; +import org.junit.Test; + +import org.apache.spark.sql.types.util.DataTypeConversions; + +public class JavaSideDataTypeConversionSuite { + public void checkDataType(DataType javaDataType) { + org.apache.spark.sql.catalyst.types.DataType scalaDataType = + DataTypeConversions.asScalaDataType(javaDataType); + DataType actual = DataTypeConversions.asJavaDataType(scalaDataType); + Assert.assertEquals(javaDataType, actual); + } + + @Test + public void createDataTypes() { + // Simple DataTypes. + checkDataType(DataType.StringType); + checkDataType(DataType.BinaryType); + checkDataType(DataType.BooleanType); + checkDataType(DataType.TimestampType); + checkDataType(DataType.DecimalType); + checkDataType(DataType.DoubleType); + checkDataType(DataType.FloatType); + checkDataType(DataType.ByteType); + checkDataType(DataType.IntegerType); + checkDataType(DataType.LongType); + checkDataType(DataType.ShortType); + + // Simple ArrayType. + DataType simpleJavaArrayType = DataType.createArrayType(DataType.StringType, true); + checkDataType(simpleJavaArrayType); + + // Simple MapType. + DataType simpleJavaMapType = DataType.createMapType(DataType.StringType, DataType.LongType); + checkDataType(simpleJavaMapType); + + // Simple StructType. + List simpleFields = new ArrayList(); + simpleFields.add(DataType.createStructField("a", DataType.DecimalType, false)); + simpleFields.add(DataType.createStructField("b", DataType.BooleanType, true)); + simpleFields.add(DataType.createStructField("c", DataType.LongType, true)); + simpleFields.add(DataType.createStructField("d", DataType.BinaryType, false)); + DataType simpleJavaStructType = DataType.createStructType(simpleFields); + checkDataType(simpleJavaStructType); + + // Complex StructType. + List complexFields = new ArrayList(); + complexFields.add(DataType.createStructField("simpleArray", simpleJavaArrayType, true)); + complexFields.add(DataType.createStructField("simpleMap", simpleJavaMapType, true)); + complexFields.add(DataType.createStructField("simpleStruct", simpleJavaStructType, true)); + complexFields.add(DataType.createStructField("boolean", DataType.BooleanType, false)); + DataType complexJavaStructType = DataType.createStructType(complexFields); + checkDataType(complexJavaStructType); + + // Complex ArrayType. + DataType complexJavaArrayType = DataType.createArrayType(complexJavaStructType, true); + checkDataType(complexJavaArrayType); + + // Complex MapType. + DataType complexJavaMapType = + DataType.createMapType(complexJavaStructType, complexJavaArrayType, false); + checkDataType(complexJavaMapType); + } + + @Test + public void illegalArgument() { + // ArrayType + try { + DataType.createArrayType(null, true); + Assert.fail(); + } catch (IllegalArgumentException expectedException) { + } + + // MapType + try { + DataType.createMapType(null, DataType.StringType); + Assert.fail(); + } catch (IllegalArgumentException expectedException) { + } + try { + DataType.createMapType(DataType.StringType, null); + Assert.fail(); + } catch (IllegalArgumentException expectedException) { + } + try { + DataType.createMapType(null, null); + Assert.fail(); + } catch (IllegalArgumentException expectedException) { + } + + // StructField + try { + DataType.createStructField(null, DataType.StringType, true); + } catch (IllegalArgumentException expectedException) { + } + try { + DataType.createStructField("name", null, true); + } catch (IllegalArgumentException expectedException) { + } + try { + DataType.createStructField(null, null, true); + } catch (IllegalArgumentException expectedException) { + } + + // StructType + try { + List simpleFields = new ArrayList(); + simpleFields.add(DataType.createStructField("a", DataType.DecimalType, false)); + simpleFields.add(DataType.createStructField("b", DataType.BooleanType, true)); + simpleFields.add(DataType.createStructField("c", DataType.LongType, true)); + simpleFields.add(null); + DataType.createStructType(simpleFields); + Assert.fail(); + } catch (IllegalArgumentException expectedException) { + } + try { + List simpleFields = new ArrayList(); + simpleFields.add(DataType.createStructField("a", DataType.DecimalType, false)); + simpleFields.add(DataType.createStructField("a", DataType.BooleanType, true)); + simpleFields.add(DataType.createStructField("c", DataType.LongType, true)); + DataType.createStructType(simpleFields); + Assert.fail(); + } catch (IllegalArgumentException expectedException) { + } + } +} diff --git a/sql/core/src/test/resources/log4j.properties b/sql/core/src/test/resources/log4j.properties index dffd15a61838b..c7e0ff1cf6494 100644 --- a/sql/core/src/test/resources/log4j.properties +++ b/sql/core/src/test/resources/log4j.properties @@ -36,6 +36,9 @@ log4j.appender.FA.layout.ConversionPattern=%d{HH:mm:ss.SSS} %p %c{1}: %m%n log4j.appender.FA.Threshold = INFO # Some packages are noisy for no good reason. +log4j.additivity.parquet.hadoop.ParquetRecordReader=false +log4j.logger.parquet.hadoop.ParquetRecordReader=OFF + log4j.additivity.org.apache.hadoop.hive.serde2.lazy.LazyStruct=false log4j.logger.org.apache.hadoop.hive.serde2.lazy.LazyStruct=OFF diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index c3c0dcb1aa00b..befef46d93973 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -22,9 +22,19 @@ import org.apache.spark.sql.columnar.{InMemoryRelation, InMemoryColumnarTableSca import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.test.TestSQLContext._ +case class BigData(s: String) + class CachedTableSuite extends QueryTest { TestData // Load test tables. + test("too big for memory") { + val data = "*" * 10000 + sparkContext.parallelize(1 to 1000000, 1).map(_ => BigData(data)).registerTempTable("bigData") + cacheTable("bigData") + assert(table("bigData").count() === 1000000L) + uncacheTable("bigData") + } + test("SPARK-1669: cacheTable should be idempotent") { assume(!table("testData").logicalPlan.isInstanceOf[InMemoryRelation]) @@ -37,7 +47,7 @@ class CachedTableSuite extends QueryTest { cacheTable("testData") table("testData").queryExecution.analyzed match { - case InMemoryRelation(_, _, _: InMemoryColumnarTableScan) => + case InMemoryRelation(_, _, _, _: InMemoryColumnarTableScan) => fail("cacheTable is not idempotent") case _ => @@ -78,7 +88,7 @@ class CachedTableSuite extends QueryTest { } test("SELECT Star Cached Table") { - TestSQLContext.sql("SELECT * FROM testData").registerAsTable("selectStar") + TestSQLContext.sql("SELECT * FROM testData").registerTempTable("selectStar") TestSQLContext.cacheTable("selectStar") TestSQLContext.sql("SELECT * FROM selectStar WHERE key = 1").collect() TestSQLContext.uncacheTable("selectStar") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataTypeSuite.scala new file mode 100644 index 0000000000000..cf7d79f42db1d --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataTypeSuite.scala @@ -0,0 +1,58 @@ +/* +* Licensed to the Apache Software Foundation (ASF) under one or more +* contributor license agreements. See the NOTICE file distributed with +* this work for additional information regarding copyright ownership. +* The ASF licenses this file to You under the Apache License, Version 2.0 +* (the "License"); you may not use this file except in compliance with +* the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ + +package org.apache.spark.sql + +import org.scalatest.FunSuite + +class DataTypeSuite extends FunSuite { + + test("construct an ArrayType") { + val array = ArrayType(StringType) + + assert(ArrayType(StringType, false) === array) + } + + test("construct an MapType") { + val map = MapType(StringType, IntegerType) + + assert(MapType(StringType, IntegerType, true) === map) + } + + test("extract fields from a StructType") { + val struct = StructType( + StructField("a", IntegerType, true) :: + StructField("b", LongType, false) :: + StructField("c", StringType, true) :: + StructField("d", FloatType, true) :: Nil) + + assert(StructField("b", LongType, false) === struct("b")) + + intercept[IllegalArgumentException] { + struct("e") + } + + val expectedStruct = StructType( + StructField("b", LongType, false) :: + StructField("d", FloatType, true) :: Nil) + + assert(expectedStruct === struct(Set("b", "d"))) + intercept[IllegalArgumentException] { + struct(Set("b", "d", "e", "f")) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala index 04ac008682f5f..1a6a6c17473a3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql import org.apache.spark.sql.catalyst.analysis._ -import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.test._ /* Implicits */ @@ -33,17 +32,23 @@ class DslQuerySuite extends QueryTest { testData.collect().toSeq) } + test("repartition") { + checkAnswer( + testData.select('key).repartition(10).select('key), + testData.select('key).collect().toSeq) + } + test("agg") { checkAnswer( - testData2.groupBy('a)('a, Sum('b)), + testData2.groupBy('a)('a, sum('b)), Seq((1,3),(2,3),(3,3)) ) checkAnswer( - testData2.groupBy('a)('a, Sum('b) as 'totB).aggregate(Sum('totB)), + testData2.groupBy('a)('a, sum('b) as 'totB).aggregate(sum('totB)), 9 ) checkAnswer( - testData2.aggregate(Sum('b)), + testData2.aggregate(sum('b)), 9 ) } @@ -98,19 +103,19 @@ class DslQuerySuite extends QueryTest { Seq((3,1), (3,2), (2,1), (2,2), (1,1), (1,2))) checkAnswer( - arrayData.orderBy(GetItem('data, 0).asc), + arrayData.orderBy('data.getItem(0).asc), arrayData.collect().sortBy(_.data(0)).toSeq) checkAnswer( - arrayData.orderBy(GetItem('data, 0).desc), + arrayData.orderBy('data.getItem(0).desc), arrayData.collect().sortBy(_.data(0)).reverse.toSeq) checkAnswer( - mapData.orderBy(GetItem('data, 1).asc), + mapData.orderBy('data.getItem(1).asc), mapData.collect().sortBy(_.data(1)).toSeq) checkAnswer( - mapData.orderBy(GetItem('data, 1).desc), + mapData.orderBy('data.getItem(1).desc), mapData.collect().sortBy(_.data(1)).reverse.toSeq) } @@ -168,4 +173,25 @@ class DslQuerySuite extends QueryTest { test("zero count") { assert(emptyTableData.count() === 0) } + + test("except") { + checkAnswer( + lowerCaseData.except(upperCaseData), + (1, "a") :: + (2, "b") :: + (3, "c") :: + (4, "d") :: Nil) + checkAnswer(lowerCaseData.except(lowerCaseData), Nil) + checkAnswer(upperCaseData.except(upperCaseData), Nil) + } + + test("intersect") { + checkAnswer( + lowerCaseData.intersect(lowerCaseData), + (1, "a") :: + (2, "b") :: + (3, "c") :: + (4, "d") :: Nil) + checkAnswer(lowerCaseData.intersect(upperCaseData), Nil) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/InsertIntoSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/InsertIntoSuite.scala index 4f0b85f26254b..c87d762751e6d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/InsertIntoSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/InsertIntoSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql -import java.io.File +import _root_.java.io.File /* Implicits */ import org.apache.spark.sql.test.TestSQLContext._ @@ -31,7 +31,7 @@ class InsertIntoSuite extends QueryTest { testFilePath.delete() testFilePath.deleteOnExit() val testFile = createParquetFile[TestData](testFilePath.getCanonicalPath) - testFile.registerAsTable("createAndInsertTest") + testFile.registerTempTable("createAndInsertTest") // Add some data. testData.insertInto("createAndInsertTest") @@ -86,7 +86,7 @@ class InsertIntoSuite extends QueryTest { testFilePath.delete() testFilePath.deleteOnExit() val testFile = createParquetFile[TestData](testFilePath.getCanonicalPath) - testFile.registerAsTable("createAndInsertSQLTest") + testFile.registerTempTable("createAndInsertSQLTest") sql("INSERT INTO createAndInsertSQLTest SELECT * FROM testData") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala index 054b14f8f7ffa..6c7697ece8c56 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala @@ -17,13 +17,17 @@ package org.apache.spark.sql +import org.scalatest.BeforeAndAfterEach + +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.TestData._ -import org.apache.spark.sql.catalyst.plans.{LeftOuter, RightOuter, FullOuter, Inner} +import org.apache.spark.sql.catalyst.plans.JoinType +import org.apache.spark.sql.catalyst.plans.{LeftOuter, RightOuter, FullOuter, Inner, LeftSemi} import org.apache.spark.sql.execution._ import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.test.TestSQLContext._ -class JoinSuite extends QueryTest { +class JoinSuite extends QueryTest with BeforeAndAfterEach { // Ensures tables are loaded. TestData @@ -36,34 +40,54 @@ class JoinSuite extends QueryTest { assert(planned.size === 1) } - test("plans broadcast hash join, given hints") { - - def mkTest(buildSide: BuildSide, leftTable: String, rightTable: String) = { - TestSQLContext.settings.synchronized { - TestSQLContext.set("spark.sql.join.broadcastTables", - s"${if (buildSide == BuildRight) rightTable else leftTable}") - val rdd = sql( s"""SELECT * FROM $leftTable JOIN $rightTable ON key = a""") - // Using `sparkPlan` because for relevant patterns in HashJoin to be - // matched, other strategies need to be applied. - val physical = rdd.queryExecution.sparkPlan - val bhj = physical.collect { case j: BroadcastHashJoin if j.buildSide == buildSide => j} - - assert(bhj.size === 1, "planner does not pick up hint to generate broadcast hash join") - checkAnswer( - rdd, - Seq( - (1, "1", 1, 1), - (1, "1", 1, 2), - (2, "2", 2, 1), - (2, "2", 2, 2), - (3, "3", 3, 1), - (3, "3", 3, 2) - )) + test("join operator selection") { + def assertJoin(sqlString: String, c: Class[_]): Any = { + val rdd = sql(sqlString) + val physical = rdd.queryExecution.sparkPlan + val operators = physical.collect { + case j: ShuffledHashJoin => j + case j: HashOuterJoin => j + case j: LeftSemiJoinHash => j + case j: BroadcastHashJoin => j + case j: LeftSemiJoinBNL => j + case j: CartesianProduct => j + case j: BroadcastNestedLoopJoin => j + } + + assert(operators.size === 1) + if (operators(0).getClass() != c) { + fail(s"$sqlString expected operator: $c, but got ${operators(0)}\n physical: \n$physical") } } - mkTest(BuildRight, "testData", "testData2") - mkTest(BuildLeft, "testData", "testData2") + val cases1 = Seq( + ("SELECT * FROM testData left semi join testData2 ON key = a", classOf[LeftSemiJoinHash]), + ("SELECT * FROM testData left semi join testData2", classOf[LeftSemiJoinBNL]), + ("SELECT * FROM testData join testData2", classOf[CartesianProduct]), + ("SELECT * FROM testData join testData2 where key=2", classOf[CartesianProduct]), + ("SELECT * FROM testData left join testData2", classOf[CartesianProduct]), + ("SELECT * FROM testData right join testData2", classOf[CartesianProduct]), + ("SELECT * FROM testData full outer join testData2", classOf[CartesianProduct]), + ("SELECT * FROM testData left join testData2 where key=2", classOf[CartesianProduct]), + ("SELECT * FROM testData right join testData2 where key=2", classOf[CartesianProduct]), + ("SELECT * FROM testData full outer join testData2 where key=2", classOf[CartesianProduct]), + ("SELECT * FROM testData join testData2 where key>a", classOf[CartesianProduct]), + ("SELECT * FROM testData full outer join testData2 where key>a", classOf[CartesianProduct]), + ("SELECT * FROM testData join testData2 ON key = a", classOf[ShuffledHashJoin]), + ("SELECT * FROM testData join testData2 ON key = a and key=2", classOf[ShuffledHashJoin]), + ("SELECT * FROM testData join testData2 ON key = a where key=2", classOf[ShuffledHashJoin]), + ("SELECT * FROM testData left join testData2 ON key = a", classOf[HashOuterJoin]), + ("SELECT * FROM testData right join testData2 ON key = a where key=2", + classOf[HashOuterJoin]), + ("SELECT * FROM testData right join testData2 ON key = a and key=2", + classOf[HashOuterJoin]), + ("SELECT * FROM testData full outer join testData2 ON key = a", classOf[HashOuterJoin]), + ("SELECT * FROM testData join testData2 ON key = a", classOf[ShuffledHashJoin]), + ("SELECT * FROM testData join testData2 ON key = a and key=2", classOf[ShuffledHashJoin]), + ("SELECT * FROM testData join testData2 ON key = a where key=2", classOf[ShuffledHashJoin]) + // TODO add BroadcastNestedLoopJoin + ) + cases1.foreach { c => assertJoin(c._1, c._2) } } test("multiple-key equi-join is hash-join") { @@ -146,6 +170,58 @@ class JoinSuite extends QueryTest { (4, "D", 4, "d") :: (5, "E", null, null) :: (6, "F", null, null) :: Nil) + + checkAnswer( + upperCaseData.join(lowerCaseData, LeftOuter, Some('n === 'N && 'n > 1)), + (1, "A", null, null) :: + (2, "B", 2, "b") :: + (3, "C", 3, "c") :: + (4, "D", 4, "d") :: + (5, "E", null, null) :: + (6, "F", null, null) :: Nil) + + checkAnswer( + upperCaseData.join(lowerCaseData, LeftOuter, Some('n === 'N && 'N > 1)), + (1, "A", null, null) :: + (2, "B", 2, "b") :: + (3, "C", 3, "c") :: + (4, "D", 4, "d") :: + (5, "E", null, null) :: + (6, "F", null, null) :: Nil) + + checkAnswer( + upperCaseData.join(lowerCaseData, LeftOuter, Some('n === 'N && 'l > 'L)), + (1, "A", 1, "a") :: + (2, "B", 2, "b") :: + (3, "C", 3, "c") :: + (4, "D", 4, "d") :: + (5, "E", null, null) :: + (6, "F", null, null) :: Nil) + + // Make sure we are choosing left.outputPartitioning as the + // outputPartitioning for the outer join operator. + checkAnswer( + sql( + """ + |SELECT l.N, count(*) + |FROM upperCaseData l LEFT OUTER JOIN allNulls r ON (l.N = r.a) + |GROUP BY l.N + """.stripMargin), + (1, 1) :: + (2, 1) :: + (3, 1) :: + (4, 1) :: + (5, 1) :: + (6, 1) :: Nil) + + checkAnswer( + sql( + """ + |SELECT r.a, count(*) + |FROM upperCaseData l LEFT OUTER JOIN allNulls r ON (l.N = r.a) + |GROUP BY r.a + """.stripMargin), + (null, 6) :: Nil) } test("right outer join") { @@ -157,11 +233,63 @@ class JoinSuite extends QueryTest { (4, "d", 4, "D") :: (null, null, 5, "E") :: (null, null, 6, "F") :: Nil) + checkAnswer( + lowerCaseData.join(upperCaseData, RightOuter, Some('n === 'N && 'n > 1)), + (null, null, 1, "A") :: + (2, "b", 2, "B") :: + (3, "c", 3, "C") :: + (4, "d", 4, "D") :: + (null, null, 5, "E") :: + (null, null, 6, "F") :: Nil) + checkAnswer( + lowerCaseData.join(upperCaseData, RightOuter, Some('n === 'N && 'N > 1)), + (null, null, 1, "A") :: + (2, "b", 2, "B") :: + (3, "c", 3, "C") :: + (4, "d", 4, "D") :: + (null, null, 5, "E") :: + (null, null, 6, "F") :: Nil) + checkAnswer( + lowerCaseData.join(upperCaseData, RightOuter, Some('n === 'N && 'l > 'L)), + (1, "a", 1, "A") :: + (2, "b", 2, "B") :: + (3, "c", 3, "C") :: + (4, "d", 4, "D") :: + (null, null, 5, "E") :: + (null, null, 6, "F") :: Nil) + + // Make sure we are choosing right.outputPartitioning as the + // outputPartitioning for the outer join operator. + checkAnswer( + sql( + """ + |SELECT l.a, count(*) + |FROM allNulls l RIGHT OUTER JOIN upperCaseData r ON (l.a = r.N) + |GROUP BY l.a + """.stripMargin), + (null, 6) :: Nil) + + checkAnswer( + sql( + """ + |SELECT r.N, count(*) + |FROM allNulls l RIGHT OUTER JOIN upperCaseData r ON (l.a = r.N) + |GROUP BY r.N + """.stripMargin), + (1, 1) :: + (2, 1) :: + (3, 1) :: + (4, 1) :: + (5, 1) :: + (6, 1) :: Nil) } test("full outer join") { - val left = upperCaseData.where('N <= 4).as('left) - val right = upperCaseData.where('N >= 3).as('right) + upperCaseData.where('N <= 4).registerTempTable("left") + upperCaseData.where('N >= 3).registerTempTable("right") + + val left = UnresolvedRelation(None, "left", None) + val right = UnresolvedRelation(None, "right", None) checkAnswer( left.join(right, FullOuter, Some("left.N".attr === "right.N".attr)), @@ -171,5 +299,74 @@ class JoinSuite extends QueryTest { (4, "D", 4, "D") :: (null, null, 5, "E") :: (null, null, 6, "F") :: Nil) + + checkAnswer( + left.join(right, FullOuter, Some(("left.N".attr === "right.N".attr) && ("left.N".attr !== 3))), + (1, "A", null, null) :: + (2, "B", null, null) :: + (3, "C", null, null) :: + (null, null, 3, "C") :: + (4, "D", 4, "D") :: + (null, null, 5, "E") :: + (null, null, 6, "F") :: Nil) + + checkAnswer( + left.join(right, FullOuter, Some(("left.N".attr === "right.N".attr) && ("right.N".attr !== 3))), + (1, "A", null, null) :: + (2, "B", null, null) :: + (3, "C", null, null) :: + (null, null, 3, "C") :: + (4, "D", 4, "D") :: + (null, null, 5, "E") :: + (null, null, 6, "F") :: Nil) + + // Make sure we are UnknownPartitioning as the outputPartitioning for the outer join operator. + checkAnswer( + sql( + """ + |SELECT l.a, count(*) + |FROM allNulls l FULL OUTER JOIN upperCaseData r ON (l.a = r.N) + |GROUP BY l.a + """.stripMargin), + (null, 10) :: Nil) + + checkAnswer( + sql( + """ + |SELECT r.N, count(*) + |FROM allNulls l FULL OUTER JOIN upperCaseData r ON (l.a = r.N) + |GROUP BY r.N + """.stripMargin), + (1, 1) :: + (2, 1) :: + (3, 1) :: + (4, 1) :: + (5, 1) :: + (6, 1) :: + (null, 4) :: Nil) + + checkAnswer( + sql( + """ + |SELECT l.N, count(*) + |FROM upperCaseData l FULL OUTER JOIN allNulls r ON (l.N = r.a) + |GROUP BY l.N + """.stripMargin), + (1, 1) :: + (2, 1) :: + (3, 1) :: + (4, 1) :: + (5, 1) :: + (6, 1) :: + (null, 4) :: Nil) + + checkAnswer( + sql( + """ + |SELECT r.a, count(*) + |FROM upperCaseData l FULL OUTER JOIN allNulls r ON (l.N = r.a) + |GROUP BY r.a + """.stripMargin), + (null, 10) :: Nil) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala index 8e1e1971d968b..1fd8d27b34c59 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala @@ -45,6 +45,7 @@ class QueryTest extends PlanTest { |${rdd.queryExecution} |== Exception == |$e + |${org.apache.spark.sql.catalyst.util.stackTraceToString(e)} """.stripMargin) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala new file mode 100644 index 0000000000000..651cb735ab7d9 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala @@ -0,0 +1,46 @@ +/* +* Licensed to the Apache Software Foundation (ASF) under one or more +* contributor license agreements. See the NOTICE file distributed with +* this work for additional information regarding copyright ownership. +* The ASF licenses this file to You under the Apache License, Version 2.0 +* (the "License"); you may not use this file except in compliance with +* the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ + +package org.apache.spark.sql + +import org.scalatest.FunSuite + +import org.apache.spark.sql.catalyst.expressions.GenericMutableRow + +class RowSuite extends FunSuite { + + test("create row") { + val expected = new GenericMutableRow(4) + expected.update(0, 2147483647) + expected.update(1, "this is a string") + expected.update(2, false) + expected.update(3, null) + val actual1 = Row(2147483647, "this is a string", false, null) + assert(expected.size === actual1.size) + assert(expected.getInt(0) === actual1.getInt(0)) + assert(expected.getString(1) === actual1.getString(1)) + assert(expected.getBoolean(2) === actual1.getBoolean(2)) + assert(expected(3) === actual1(3)) + + val actual2 = Row.fromSeq(Seq(2147483647, "this is a string", false, null)) + assert(expected.size === actual2.size) + assert(expected.getInt(0) === actual2.getInt(0)) + assert(expected.getString(1) === actual2.getString(1)) + assert(expected.getBoolean(2) === actual2.getBoolean(2)) + assert(expected(3) === actual2(3)) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLConfSuite.scala index 93792f698cfaf..584f71b3c13d5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLConfSuite.scala @@ -28,50 +28,48 @@ class SQLConfSuite extends QueryTest { val testVal = "test.val.0" test("programmatic ways of basic setting and getting") { - TestSQLContext.settings.synchronized { - clear() - assert(getOption(testKey).isEmpty) - assert(getAll.toSet === Set()) + clear() + assert(getAllConfs.size === 0) - set(testKey, testVal) - assert(get(testKey) == testVal) - assert(get(testKey, testVal + "_") == testVal) - assert(getOption(testKey) == Some(testVal)) - assert(contains(testKey)) + setConf(testKey, testVal) + assert(getConf(testKey) == testVal) + assert(getConf(testKey, testVal + "_") == testVal) + assert(getAllConfs.contains(testKey)) - // Tests SQLConf as accessed from a SQLContext is mutable after - // the latter is initialized, unlike SparkConf inside a SparkContext. - assert(TestSQLContext.get(testKey) == testVal) - assert(TestSQLContext.get(testKey, testVal + "_") == testVal) - assert(TestSQLContext.getOption(testKey) == Some(testVal)) - assert(TestSQLContext.contains(testKey)) + // Tests SQLConf as accessed from a SQLContext is mutable after + // the latter is initialized, unlike SparkConf inside a SparkContext. + assert(TestSQLContext.getConf(testKey) == testVal) + assert(TestSQLContext.getConf(testKey, testVal + "_") == testVal) + assert(TestSQLContext.getAllConfs.contains(testKey)) - clear() - } + clear() } test("parse SQL set commands") { - TestSQLContext.settings.synchronized { - clear() - sql(s"set $testKey=$testVal") - assert(get(testKey, testVal + "_") == testVal) - assert(TestSQLContext.get(testKey, testVal + "_") == testVal) + clear() + sql(s"set $testKey=$testVal") + assert(getConf(testKey, testVal + "_") == testVal) + assert(TestSQLContext.getConf(testKey, testVal + "_") == testVal) - sql("set mapred.reduce.tasks=20") - assert(get("mapred.reduce.tasks", "0") == "20") - sql("set mapred.reduce.tasks = 40") - assert(get("mapred.reduce.tasks", "0") == "40") + sql("set some.property=20") + assert(getConf("some.property", "0") == "20") + sql("set some.property = 40") + assert(getConf("some.property", "0") == "40") - val key = "spark.sql.key" - val vs = "val0,val_1,val2.3,my_table" - sql(s"set $key=$vs") - assert(get(key, "0") == vs) + val key = "spark.sql.key" + val vs = "val0,val_1,val2.3,my_table" + sql(s"set $key=$vs") + assert(getConf(key, "0") == vs) - sql(s"set $key=") - assert(get(key, "0") == "") + sql(s"set $key=") + assert(getConf(key, "0") == "") - clear() - } + clear() } + test("deprecated property") { + clear() + sql(s"set ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS}=10") + assert(getConf(SQLConf.SHUFFLE_PARTITIONS) == "10") + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index fa1f32f8a49a9..9b2a36d33fca7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -17,9 +17,7 @@ package org.apache.spark.sql -import org.apache.spark.sql.catalyst.analysis.EliminateAnalysisOperators import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.columnar.{InMemoryColumnarTableScan, InMemoryRelation} import org.apache.spark.sql.test._ /* Implicits */ @@ -36,6 +34,21 @@ class SQLQuerySuite extends QueryTest { "test") } + test("SPARK-2407 Added Parser of SQL SUBSTR()") { + checkAnswer( + sql("SELECT substr(tableName, 1, 2) FROM tableName"), + "te") + checkAnswer( + sql("SELECT substr(tableName, 3) FROM tableName"), + "st") + checkAnswer( + sql("SELECT substring(tableName, 1, 2) FROM tableName"), + "te") + checkAnswer( + sql("SELECT substring(tableName, 3) FROM tableName"), + "st") + } + test("index into array") { checkAnswer( sql("SELECT data, data[0], data[0] + data[1], data[0 + 1] FROM arrayData"), @@ -397,40 +410,119 @@ class SQLQuerySuite extends QueryTest { } test("SET commands semantics using sql()") { - TestSQLContext.settings.synchronized { - clear() - val testKey = "test.key.0" - val testVal = "test.val.0" - val nonexistentKey = "nonexistent" - - // "set" itself returns all config variables currently specified in SQLConf. - assert(sql("SET").collect().size == 0) - - // "set key=val" - sql(s"SET $testKey=$testVal") - checkAnswer( - sql("SET"), - Seq(Seq(testKey, testVal)) - ) - - sql(s"SET ${testKey + testKey}=${testVal + testVal}") - checkAnswer( - sql("set"), - Seq( - Seq(testKey, testVal), - Seq(testKey + testKey, testVal + testVal)) - ) - - // "set key" - checkAnswer( - sql(s"SET $testKey"), - Seq(Seq(testKey, testVal)) - ) - checkAnswer( - sql(s"SET $nonexistentKey"), - Seq(Seq(nonexistentKey, "")) - ) - clear() + clear() + val testKey = "test.key.0" + val testVal = "test.val.0" + val nonexistentKey = "nonexistent" + + // "set" itself returns all config variables currently specified in SQLConf. + assert(sql("SET").collect().size == 0) + + // "set key=val" + sql(s"SET $testKey=$testVal") + checkAnswer( + sql("SET"), + Seq(Seq(s"$testKey=$testVal")) + ) + + sql(s"SET ${testKey + testKey}=${testVal + testVal}") + checkAnswer( + sql("set"), + Seq( + Seq(s"$testKey=$testVal"), + Seq(s"${testKey + testKey}=${testVal + testVal}")) + ) + + // "set key" + checkAnswer( + sql(s"SET $testKey"), + Seq(Seq(s"$testKey=$testVal")) + ) + checkAnswer( + sql(s"SET $nonexistentKey"), + Seq(Seq(s"$nonexistentKey=")) + ) + clear() + } + + test("apply schema") { + val schema1 = StructType( + StructField("f1", IntegerType, false) :: + StructField("f2", StringType, false) :: + StructField("f3", BooleanType, false) :: + StructField("f4", IntegerType, true) :: Nil) + + val rowRDD1 = unparsedStrings.map { r => + val values = r.split(",").map(_.trim) + val v4 = try values(3).toInt catch { + case _: NumberFormatException => null + } + Row(values(0).toInt, values(1), values(2).toBoolean, v4) } + + val schemaRDD1 = applySchema(rowRDD1, schema1) + schemaRDD1.registerTempTable("applySchema1") + checkAnswer( + sql("SELECT * FROM applySchema1"), + (1, "A1", true, null) :: + (2, "B2", false, null) :: + (3, "C3", true, null) :: + (4, "D4", true, 2147483644) :: Nil) + + checkAnswer( + sql("SELECT f1, f4 FROM applySchema1"), + (1, null) :: + (2, null) :: + (3, null) :: + (4, 2147483644) :: Nil) + + val schema2 = StructType( + StructField("f1", StructType( + StructField("f11", IntegerType, false) :: + StructField("f12", BooleanType, false) :: Nil), false) :: + StructField("f2", MapType(StringType, IntegerType, true), false) :: Nil) + + val rowRDD2 = unparsedStrings.map { r => + val values = r.split(",").map(_.trim) + val v4 = try values(3).toInt catch { + case _: NumberFormatException => null + } + Row(Row(values(0).toInt, values(2).toBoolean), Map(values(1) -> v4)) + } + + val schemaRDD2 = applySchema(rowRDD2, schema2) + schemaRDD2.registerTempTable("applySchema2") + checkAnswer( + sql("SELECT * FROM applySchema2"), + (Seq(1, true), Map("A1" -> null)) :: + (Seq(2, false), Map("B2" -> null)) :: + (Seq(3, true), Map("C3" -> null)) :: + (Seq(4, true), Map("D4" -> 2147483644)) :: Nil) + + checkAnswer( + sql("SELECT f1.f11, f2['D4'] FROM applySchema2"), + (1, null) :: + (2, null) :: + (3, null) :: + (4, 2147483644) :: Nil) + + // The value of a MapType column can be a mutable map. + val rowRDD3 = unparsedStrings.map { r => + val values = r.split(",").map(_.trim) + val v4 = try values(3).toInt catch { + case _: NumberFormatException => null + } + Row(Row(values(0).toInt, values(2).toBoolean), scala.collection.mutable.Map(values(1) -> v4)) + } + + val schemaRDD3 = applySchema(rowRDD3, schema2) + schemaRDD3.registerTempTable("applySchema3") + + checkAnswer( + sql("SELECT f1.f11, f2['D4'] FROM applySchema3"), + (1, null) :: + (2, null) :: + (3, null) :: + (4, 2147483644) :: Nil) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala index f2934da9a031d..5b84c658db942 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala @@ -61,7 +61,7 @@ class ScalaReflectionRelationSuite extends FunSuite { val data = ReflectData("a", 1, 1L, 1.toFloat, 1.toDouble, 1.toShort, 1.toByte, true, BigDecimal(1), new Timestamp(12345), Seq(1,2,3)) val rdd = sparkContext.parallelize(data :: Nil) - rdd.registerAsTable("reflectData") + rdd.registerTempTable("reflectData") assert(sql("SELECT * FROM reflectData").collect().head === data.productIterator.toSeq) } @@ -69,7 +69,7 @@ class ScalaReflectionRelationSuite extends FunSuite { test("query case class RDD with nulls") { val data = NullReflectData(null, null, null, null, null, null, null) val rdd = sparkContext.parallelize(data :: Nil) - rdd.registerAsTable("reflectNullData") + rdd.registerTempTable("reflectNullData") assert(sql("SELECT * FROM reflectNullData").collect().head === Seq.fill(7)(null)) } @@ -77,7 +77,7 @@ class ScalaReflectionRelationSuite extends FunSuite { test("query case class RDD with Nones") { val data = OptionalReflectData(None, None, None, None, None, None, None) val rdd = sparkContext.parallelize(data :: Nil) - rdd.registerAsTable("reflectOptionalData") + rdd.registerTempTable("reflectOptionalData") assert(sql("SELECT * FROM reflectOptionalData").collect().head === Seq.fill(7)(null)) } @@ -85,7 +85,7 @@ class ScalaReflectionRelationSuite extends FunSuite { // Equality is broken for Arrays, so we test that separately. test("query binary data") { val rdd = sparkContext.parallelize(ReflectBinary(Array[Byte](1)) :: Nil) - rdd.registerAsTable("reflectBinary") + rdd.registerTempTable("reflectBinary") val result = sql("SELECT data FROM reflectBinary").collect().head(0).asInstanceOf[Array[Byte]] assert(result.toSeq === Seq[Byte](1)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala index 330b20b315d63..c3ec82fb69778 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala @@ -17,18 +17,20 @@ package org.apache.spark.sql +import java.sql.Timestamp + import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.test._ /* Implicits */ -import TestSQLContext._ +import org.apache.spark.sql.test.TestSQLContext._ case class TestData(key: Int, value: String) object TestData { val testData: SchemaRDD = TestSQLContext.sparkContext.parallelize( (1 to 100).map(i => TestData(i, i.toString))) - testData.registerAsTable("testData") + testData.registerTempTable("testData") case class LargeAndSmallInts(a: Int, b: Int) val largeAndSmallInts: SchemaRDD = @@ -39,8 +41,8 @@ object TestData { LargeAndSmallInts(2, 2) :: LargeAndSmallInts(2147483646, 1) :: LargeAndSmallInts(3, 2) :: Nil) - largeAndSmallInts.registerAsTable("largeAndSmallInts") - + largeAndSmallInts.registerTempTable("largeAndSmallInts") + case class TestData2(a: Int, b: Int) val testData2: SchemaRDD = TestSQLContext.sparkContext.parallelize( @@ -50,7 +52,7 @@ object TestData { TestData2(2, 2) :: TestData2(3, 1) :: TestData2(3, 2) :: Nil) - testData2.registerAsTable("testData2") + testData2.registerTempTable("testData2") // TODO: There is no way to express null primitives as case classes currently... val testData3 = @@ -69,7 +71,7 @@ object TestData { UpperCaseData(4, "D") :: UpperCaseData(5, "E") :: UpperCaseData(6, "F") :: Nil) - upperCaseData.registerAsTable("upperCaseData") + upperCaseData.registerTempTable("upperCaseData") case class LowerCaseData(n: Int, l: String) val lowerCaseData = @@ -78,14 +80,14 @@ object TestData { LowerCaseData(2, "b") :: LowerCaseData(3, "c") :: LowerCaseData(4, "d") :: Nil) - lowerCaseData.registerAsTable("lowerCaseData") + lowerCaseData.registerTempTable("lowerCaseData") case class ArrayData(data: Seq[Int], nestedData: Seq[Seq[Int]]) val arrayData = TestSQLContext.sparkContext.parallelize( ArrayData(Seq(1,2,3), Seq(Seq(1,2,3))) :: ArrayData(Seq(2,3,4), Seq(Seq(2,3,4))) :: Nil) - arrayData.registerAsTable("arrayData") + arrayData.registerTempTable("arrayData") case class MapData(data: Map[Int, String]) val mapData = @@ -95,18 +97,18 @@ object TestData { MapData(Map(1 -> "a3", 2 -> "b3", 3 -> "c3")) :: MapData(Map(1 -> "a4", 2 -> "b4")) :: MapData(Map(1 -> "a5")) :: Nil) - mapData.registerAsTable("mapData") + mapData.registerTempTable("mapData") case class StringData(s: String) val repeatedData = TestSQLContext.sparkContext.parallelize(List.fill(2)(StringData("test"))) - repeatedData.registerAsTable("repeatedData") + repeatedData.registerTempTable("repeatedData") val nullableRepeatedData = TestSQLContext.sparkContext.parallelize( List.fill(2)(StringData(null)) ++ List.fill(2)(StringData("test"))) - nullableRepeatedData.registerAsTable("nullableRepeatedData") + nullableRepeatedData.registerTempTable("nullableRepeatedData") case class NullInts(a: Integer) val nullInts = @@ -116,7 +118,15 @@ object TestData { NullInts(3) :: NullInts(null) :: Nil ) - nullInts.registerAsTable("nullInts") + nullInts.registerTempTable("nullInts") + + val allNulls = + TestSQLContext.sparkContext.parallelize( + NullInts(null) :: + NullInts(null) :: + NullInts(null) :: + NullInts(null) :: Nil) + allNulls.registerTempTable("allNulls") case class NullStrings(n: Int, s: String) val nullStrings = @@ -124,8 +134,21 @@ object TestData { NullStrings(1, "abc") :: NullStrings(2, "ABC") :: NullStrings(3, null) :: Nil) - nullStrings.registerAsTable("nullStrings") + nullStrings.registerTempTable("nullStrings") case class TableName(tableName: String) - TestSQLContext.sparkContext.parallelize(TableName("test") :: Nil).registerAsTable("tableName") + TestSQLContext.sparkContext.parallelize(TableName("test") :: Nil).registerTempTable("tableName") + + val unparsedStrings = + TestSQLContext.sparkContext.parallelize( + "1, A1, true, null" :: + "2, B2, false, null" :: + "3, C3, true, null" :: + "4, D4, true, 2147483644" :: Nil) + + case class TimestampField(time: Timestamp) + val timestamps = TestSQLContext.sparkContext.parallelize((1 to 3).map { i => + TimestampField(new Timestamp(i)) + }) + timestamps.registerTempTable("timestamps") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala new file mode 100644 index 0000000000000..76aa9b0081d7e --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import org.apache.spark.sql.test._ + +/* Implicits */ +import TestSQLContext._ + +class UDFSuite extends QueryTest { + + test("Simple UDF") { + registerFunction("strLenScala", (_: String).length) + assert(sql("SELECT strLenScala('test')").first().getInt(0) === 4) + } + + test("TwoArgument UDF") { + registerFunction("strLenScala", (_: String).length + (_:Int)) + assert(sql("SELECT strLenScala('test', 1)").first().getInt(0) === 5) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/api/java/JavaSQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/api/java/JavaSQLSuite.scala index 020baf0c7ec6f..203ff847e94cc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/api/java/JavaSQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/api/java/JavaSQLSuite.scala @@ -59,7 +59,7 @@ class JavaSQLSuite extends FunSuite { val rdd = javaCtx.parallelize(person :: Nil) val schemaRDD = javaSqlCtx.applySchema(rdd, classOf[PersonBean]) - schemaRDD.registerAsTable("people") + schemaRDD.registerTempTable("people") javaSqlCtx.sql("SELECT * FROM people").collect() } @@ -76,7 +76,7 @@ class JavaSQLSuite extends FunSuite { val rdd = javaCtx.parallelize(bean :: Nil) val schemaRDD = javaSqlCtx.applySchema(rdd, classOf[AllTypesBean]) - schemaRDD.registerAsTable("allTypes") + schemaRDD.registerTempTable("allTypes") assert( javaSqlCtx.sql( @@ -101,7 +101,7 @@ class JavaSQLSuite extends FunSuite { val rdd = javaCtx.parallelize(bean :: Nil) val schemaRDD = javaSqlCtx.applySchema(rdd, classOf[AllTypesBean]) - schemaRDD.registerAsTable("allTypes") + schemaRDD.registerTempTable("allTypes") assert( javaSqlCtx.sql( @@ -127,7 +127,7 @@ class JavaSQLSuite extends FunSuite { var schemaRDD = javaSqlCtx.jsonRDD(rdd) - schemaRDD.registerAsTable("jsonTable1") + schemaRDD.registerTempTable("jsonTable1") assert( javaSqlCtx.sql("select * from jsonTable1").collect.head.row === @@ -144,7 +144,7 @@ class JavaSQLSuite extends FunSuite { rdd.saveAsTextFile(path) schemaRDD = javaSqlCtx.jsonFile(path) - schemaRDD.registerAsTable("jsonTable2") + schemaRDD.registerTempTable("jsonTable2") assert( javaSqlCtx.sql("select * from jsonTable2").collect.head.row === diff --git a/sql/core/src/test/scala/org/apache/spark/sql/api/java/ScalaSideDataTypeConversionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/api/java/ScalaSideDataTypeConversionSuite.scala new file mode 100644 index 0000000000000..ff1debff0f8c1 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/api/java/ScalaSideDataTypeConversionSuite.scala @@ -0,0 +1,86 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java + +import org.apache.spark.sql.types.util.DataTypeConversions +import org.scalatest.FunSuite + +import org.apache.spark.sql.{DataType => SDataType, StructField => SStructField} +import org.apache.spark.sql.{StructType => SStructType} +import DataTypeConversions._ + +class ScalaSideDataTypeConversionSuite extends FunSuite { + + def checkDataType(scalaDataType: SDataType) { + val javaDataType = asJavaDataType(scalaDataType) + val actual = asScalaDataType(javaDataType) + assert(scalaDataType === actual, s"Converted data type ${actual} " + + s"does not equal the expected data type ${scalaDataType}") + } + + test("convert data types") { + // Simple DataTypes. + checkDataType(org.apache.spark.sql.StringType) + checkDataType(org.apache.spark.sql.BinaryType) + checkDataType(org.apache.spark.sql.BooleanType) + checkDataType(org.apache.spark.sql.TimestampType) + checkDataType(org.apache.spark.sql.DecimalType) + checkDataType(org.apache.spark.sql.DoubleType) + checkDataType(org.apache.spark.sql.FloatType) + checkDataType(org.apache.spark.sql.ByteType) + checkDataType(org.apache.spark.sql.IntegerType) + checkDataType(org.apache.spark.sql.LongType) + checkDataType(org.apache.spark.sql.ShortType) + + // Simple ArrayType. + val simpleScalaArrayType = + org.apache.spark.sql.ArrayType(org.apache.spark.sql.StringType, true) + checkDataType(simpleScalaArrayType) + + // Simple MapType. + val simpleScalaMapType = + org.apache.spark.sql.MapType(org.apache.spark.sql.StringType, org.apache.spark.sql.LongType) + checkDataType(simpleScalaMapType) + + // Simple StructType. + val simpleScalaStructType = SStructType( + SStructField("a", org.apache.spark.sql.DecimalType, false) :: + SStructField("b", org.apache.spark.sql.BooleanType, true) :: + SStructField("c", org.apache.spark.sql.LongType, true) :: + SStructField("d", org.apache.spark.sql.BinaryType, false) :: Nil) + checkDataType(simpleScalaStructType) + + // Complex StructType. + val complexScalaStructType = SStructType( + SStructField("simpleArray", simpleScalaArrayType, true) :: + SStructField("simpleMap", simpleScalaMapType, true) :: + SStructField("simpleStruct", simpleScalaStructType, true) :: + SStructField("boolean", org.apache.spark.sql.BooleanType, false) :: Nil) + checkDataType(complexScalaStructType) + + // Complex ArrayType. + val complexScalaArrayType = + org.apache.spark.sql.ArrayType(complexScalaStructType, true) + checkDataType(complexScalaArrayType) + + // Complex MapType. + val complexScalaMapType = + org.apache.spark.sql.MapType(complexScalaStructType, complexScalaArrayType, false) + checkDataType(complexScalaMapType) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala index 6f0d46d816266..5f61fb5e16ea3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala @@ -22,14 +22,15 @@ import org.scalatest.FunSuite import org.apache.spark.sql.catalyst.types._ class ColumnStatsSuite extends FunSuite { - testColumnStats(classOf[BooleanColumnStats], BOOLEAN) - testColumnStats(classOf[ByteColumnStats], BYTE) - testColumnStats(classOf[ShortColumnStats], SHORT) - testColumnStats(classOf[IntColumnStats], INT) - testColumnStats(classOf[LongColumnStats], LONG) - testColumnStats(classOf[FloatColumnStats], FLOAT) - testColumnStats(classOf[DoubleColumnStats], DOUBLE) - testColumnStats(classOf[StringColumnStats], STRING) + testColumnStats(classOf[BooleanColumnStats], BOOLEAN) + testColumnStats(classOf[ByteColumnStats], BYTE) + testColumnStats(classOf[ShortColumnStats], SHORT) + testColumnStats(classOf[IntColumnStats], INT) + testColumnStats(classOf[LongColumnStats], LONG) + testColumnStats(classOf[FloatColumnStats], FLOAT) + testColumnStats(classOf[DoubleColumnStats], DOUBLE) + testColumnStats(classOf[StringColumnStats], STRING) + testColumnStats(classOf[TimestampColumnStats], TIMESTAMP) def testColumnStats[T <: NativeType, U <: NativeColumnStats[T]]( columnStatsClass: Class[U], diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala index 314b7d317ed75..75f653f3280bd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala @@ -18,10 +18,11 @@ package org.apache.spark.sql.columnar import java.nio.ByteBuffer +import java.sql.Timestamp import org.scalatest.FunSuite -import org.apache.spark.sql.Logging +import org.apache.spark.Logging import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.columnar.ColumnarTestUtils._ import org.apache.spark.sql.execution.SparkSqlSerializer @@ -32,7 +33,7 @@ class ColumnTypeSuite extends FunSuite with Logging { test("defaultSize") { val checks = Map( INT -> 4, SHORT -> 2, LONG -> 8, BYTE -> 1, DOUBLE -> 8, FLOAT -> 4, - BOOLEAN -> 1, STRING -> 8, BINARY -> 16, GENERIC -> 16) + BOOLEAN -> 1, STRING -> 8, TIMESTAMP -> 12, BINARY -> 16, GENERIC -> 16) checks.foreach { case (columnType, expectedSize) => assertResult(expectedSize, s"Wrong defaultSize for $columnType") { @@ -52,14 +53,15 @@ class ColumnTypeSuite extends FunSuite with Logging { } } - checkActualSize(INT, Int.MaxValue, 4) - checkActualSize(SHORT, Short.MaxValue, 2) - checkActualSize(LONG, Long.MaxValue, 8) - checkActualSize(BYTE, Byte.MaxValue, 1) - checkActualSize(DOUBLE, Double.MaxValue, 8) - checkActualSize(FLOAT, Float.MaxValue, 4) - checkActualSize(BOOLEAN, true, 1) - checkActualSize(STRING, "hello", 4 + "hello".getBytes("utf-8").length) + checkActualSize(INT, Int.MaxValue, 4) + checkActualSize(SHORT, Short.MaxValue, 2) + checkActualSize(LONG, Long.MaxValue, 8) + checkActualSize(BYTE, Byte.MaxValue, 1) + checkActualSize(DOUBLE, Double.MaxValue, 8) + checkActualSize(FLOAT, Float.MaxValue, 4) + checkActualSize(BOOLEAN, true, 1) + checkActualSize(STRING, "hello", 4 + "hello".getBytes("utf-8").length) + checkActualSize(TIMESTAMP, new Timestamp(0L), 12) val binary = Array.fill[Byte](4)(0: Byte) checkActualSize(BINARY, binary, 4 + 4) @@ -164,7 +166,7 @@ class ColumnTypeSuite extends FunSuite with Logging { buffer.rewind() seq.foreach { expected => - logger.info("buffer = " + buffer + ", expected = " + expected) + logInfo("buffer = " + buffer + ", expected = " + expected) val extracted = columnType.extract(buffer) assert( expected === extracted, @@ -188,17 +190,7 @@ class ColumnTypeSuite extends FunSuite with Logging { } private def hexDump(value: Any): String = { - if (value.isInstanceOf[String]) { - val sb = new StringBuilder() - for (ch <- value.asInstanceOf[String].toCharArray) { - sb.append(Integer.toHexString(ch & 0xffff)).append(' ') - } - if (! sb.isEmpty) sb.setLength(sb.length - 1) - sb.toString() - } else { - // for now .. - hexDump(value.toString) - } + value.toString.map(ch => Integer.toHexString(ch & 0xffff)).mkString(" ") } private def dumpBuffer(buff: ByteBuffer): Any = { @@ -207,7 +199,7 @@ class ColumnTypeSuite extends FunSuite with Logging { val b = buff.get() sb.append(Integer.toHexString(b & 0xff)).append(' ') } - if (! sb.isEmpty) sb.setLength(sb.length - 1) + if (sb.nonEmpty) sb.setLength(sb.length - 1) sb.toString() } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala index 04bdc43d95328..38b04dd959f70 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala @@ -20,6 +20,8 @@ package org.apache.spark.sql.columnar import scala.collection.immutable.HashSet import scala.util.Random +import java.sql.Timestamp + import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions.GenericMutableRow import org.apache.spark.sql.catalyst.types.{DataType, NativeType} @@ -39,15 +41,19 @@ object ColumnarTestUtils { } (columnType match { - case BYTE => (Random.nextInt(Byte.MaxValue * 2) - Byte.MaxValue).toByte - case SHORT => (Random.nextInt(Short.MaxValue * 2) - Short.MaxValue).toShort - case INT => Random.nextInt() - case LONG => Random.nextLong() - case FLOAT => Random.nextFloat() - case DOUBLE => Random.nextDouble() - case STRING => Random.nextString(Random.nextInt(32)) - case BOOLEAN => Random.nextBoolean() - case BINARY => randomBytes(Random.nextInt(32)) + case BYTE => (Random.nextInt(Byte.MaxValue * 2) - Byte.MaxValue).toByte + case SHORT => (Random.nextInt(Short.MaxValue * 2) - Short.MaxValue).toShort + case INT => Random.nextInt() + case LONG => Random.nextLong() + case FLOAT => Random.nextFloat() + case DOUBLE => Random.nextDouble() + case STRING => Random.nextString(Random.nextInt(32)) + case BOOLEAN => Random.nextBoolean() + case BINARY => randomBytes(Random.nextInt(32)) + case TIMESTAMP => + val timestamp = new Timestamp(Random.nextLong()) + timestamp.setNanos(Random.nextInt(999999999)) + timestamp case _ => // Using a random one-element map instead of an arbitrary object Map(Random.nextInt() -> Random.nextString(Random.nextInt(32))) @@ -96,5 +102,4 @@ object ColumnarTestUtils { (values, rows) } - } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala index 86727b93f3659..736c0f8571e9e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala @@ -28,14 +28,14 @@ class InMemoryColumnarQuerySuite extends QueryTest { test("simple columnar query") { val plan = TestSQLContext.executePlan(testData.logicalPlan).executedPlan - val scan = InMemoryRelation(useCompression = true, plan) + val scan = InMemoryRelation(useCompression = true, 5, plan) checkAnswer(scan, testData.collect().toSeq) } test("projection") { val plan = TestSQLContext.executePlan(testData.select('value, 'key).logicalPlan).executedPlan - val scan = InMemoryRelation(useCompression = true, plan) + val scan = InMemoryRelation(useCompression = true, 5, plan) checkAnswer(scan, testData.collect().map { case Row(key: Int, value: String) => value -> key @@ -44,7 +44,7 @@ class InMemoryColumnarQuerySuite extends QueryTest { test("SPARK-1436 regression: in-memory columns must be able to be accessed multiple times") { val plan = TestSQLContext.executePlan(testData.logicalPlan).executedPlan - val scan = InMemoryRelation(useCompression = true, plan) + val scan = InMemoryRelation(useCompression = true, 5, plan) checkAnswer(scan, testData.collect().toSeq) checkAnswer(scan, testData.collect().toSeq) @@ -73,4 +73,16 @@ class InMemoryColumnarQuerySuite extends QueryTest { sql("SELECT * FROM nullableRepeatedData"), nullableRepeatedData.collect().toSeq) } + + test("SPARK-2729 regression: timestamp data type") { + checkAnswer( + sql("SELECT time FROM timestamps"), + timestamps.collect().toSeq) + + TestSQLContext.cacheTable("timestamps") + + checkAnswer( + sql("SELECT time FROM timestamps"), + timestamps.collect().toSeq) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala index 35ab14cbc353d..3baa6f8ec0c83 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala @@ -41,7 +41,7 @@ object TestNullableColumnAccessor { class NullableColumnAccessorSuite extends FunSuite { import ColumnarTestUtils._ - Seq(INT, LONG, SHORT, BOOLEAN, BYTE, STRING, DOUBLE, FLOAT, BINARY, GENERIC).foreach { + Seq(INT, LONG, SHORT, BOOLEAN, BYTE, STRING, DOUBLE, FLOAT, BINARY, GENERIC, TIMESTAMP).foreach { testNullableColumnAccessor(_) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala index d8898527baa39..dc813fe146c47 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala @@ -37,7 +37,7 @@ object TestNullableColumnBuilder { class NullableColumnBuilderSuite extends FunSuite { import ColumnarTestUtils._ - Seq(INT, LONG, SHORT, BOOLEAN, BYTE, STRING, DOUBLE, FLOAT, BINARY, GENERIC).foreach { + Seq(INT, LONG, SHORT, BOOLEAN, BYTE, STRING, DOUBLE, FLOAT, BINARY, GENERIC, TIMESTAMP).foreach { testNullableColumnBuilder(_) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala index 6d688ea95cfc0..72c19fa31d980 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala @@ -42,4 +42,3 @@ object TestCompressibleColumnBuilder { builder } } - diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala index 215618e852eb2..76b1724471442 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala @@ -39,22 +39,22 @@ class PlannerSuite extends FunSuite { test("count is partially aggregated") { val query = testData.groupBy('value)(Count('key)).queryExecution.analyzed - val planned = PartialAggregation(query).head - val aggregations = planned.collect { case a: Aggregate => a } + val planned = HashAggregation(query).head + val aggregations = planned.collect { case n if n.nodeName contains "Aggregate" => n } assert(aggregations.size === 2) } test("count distinct is not partially aggregated") { val query = testData.groupBy('value)(CountDistinct('key :: Nil)).queryExecution.analyzed - val planned = PartialAggregation(query) + val planned = HashAggregation(query) assert(planned.isEmpty) } test("mixed aggregates are not partially aggregated") { val query = testData.groupBy('value)(Count('value), CountDistinct('key :: Nil)).queryExecution.analyzed - val planned = PartialAggregation(query) + val planned = HashAggregation(query) assert(planned.isEmpty) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/TgfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/TgfSuite.scala index e55648b8ed15a..2cab5e0c44d92 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/TgfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/TgfSuite.scala @@ -33,7 +33,7 @@ import org.apache.spark.sql.test.TestSQLContext._ * Note: this is only a rough example of how TGFs can be expressed, the final version will likely * involve a lot more sugar for cleaner use in Scala/Java/etc. */ -case class ExampleTGF(input: Seq[Attribute] = Seq('name, 'age)) extends Generator { +case class ExampleTGF(input: Seq[Expression] = Seq('name, 'age)) extends Generator { def children = input protected def makeOutput() = 'nameAndAge.string :: Nil diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala index e765cfc83a397..58b1e23891a3b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala @@ -17,16 +17,12 @@ package org.apache.spark.sql.json -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} -import org.apache.spark.sql.catalyst.plans.logical.LeafNode import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.json.JsonRDD.{enforceCorrectType, compatibleType} import org.apache.spark.sql.QueryTest import org.apache.spark.sql.test.TestSQLContext._ -protected case class Schema(output: Seq[Attribute]) extends LeafNode - class JsonSuite extends QueryTest { import TestJsonData._ TestJsonData @@ -127,6 +123,18 @@ class JsonSuite extends QueryTest { checkDataType(ArrayType(IntegerType), ArrayType(LongType), ArrayType(LongType)) checkDataType(ArrayType(IntegerType), ArrayType(StringType), ArrayType(StringType)) checkDataType(ArrayType(IntegerType), StructType(Nil), StringType) + checkDataType( + ArrayType(IntegerType, true), ArrayType(IntegerType), ArrayType(IntegerType, true)) + checkDataType( + ArrayType(IntegerType, true), ArrayType(IntegerType, false), ArrayType(IntegerType, true)) + checkDataType( + ArrayType(IntegerType, true), ArrayType(IntegerType, true), ArrayType(IntegerType, true)) + checkDataType( + ArrayType(IntegerType, false), ArrayType(IntegerType), ArrayType(IntegerType, false)) + checkDataType( + ArrayType(IntegerType, false), ArrayType(IntegerType, false), ArrayType(IntegerType, false)) + checkDataType( + ArrayType(IntegerType, false), ArrayType(IntegerType, false), ArrayType(IntegerType)) // StructType checkDataType(StructType(Nil), StructType(Nil), StructType(Nil)) @@ -164,18 +172,18 @@ class JsonSuite extends QueryTest { test("Primitive field and type inferring") { val jsonSchemaRDD = jsonRDD(primitiveFieldAndType) - val expectedSchema = - AttributeReference("bigInteger", DecimalType, true)() :: - AttributeReference("boolean", BooleanType, true)() :: - AttributeReference("double", DoubleType, true)() :: - AttributeReference("integer", IntegerType, true)() :: - AttributeReference("long", LongType, true)() :: - AttributeReference("null", StringType, true)() :: - AttributeReference("string", StringType, true)() :: Nil + val expectedSchema = StructType( + StructField("bigInteger", DecimalType, true) :: + StructField("boolean", BooleanType, true) :: + StructField("double", DoubleType, true) :: + StructField("integer", IntegerType, true) :: + StructField("long", LongType, true) :: + StructField("null", StringType, true) :: + StructField("string", StringType, true) :: Nil) - comparePlans(Schema(expectedSchema), Schema(jsonSchemaRDD.logicalPlan.output)) + assert(expectedSchema === jsonSchemaRDD.schema) - jsonSchemaRDD.registerAsTable("jsonTable") + jsonSchemaRDD.registerTempTable("jsonTable") checkAnswer( sql("select * from jsonTable"), @@ -192,29 +200,31 @@ class JsonSuite extends QueryTest { test("Complex field and type inferring") { val jsonSchemaRDD = jsonRDD(complexFieldAndType) - val expectedSchema = - AttributeReference("arrayOfArray1", ArrayType(ArrayType(StringType)), true)() :: - AttributeReference("arrayOfArray2", ArrayType(ArrayType(DoubleType)), true)() :: - AttributeReference("arrayOfBigInteger", ArrayType(DecimalType), true)() :: - AttributeReference("arrayOfBoolean", ArrayType(BooleanType), true)() :: - AttributeReference("arrayOfDouble", ArrayType(DoubleType), true)() :: - AttributeReference("arrayOfInteger", ArrayType(IntegerType), true)() :: - AttributeReference("arrayOfLong", ArrayType(LongType), true)() :: - AttributeReference("arrayOfNull", ArrayType(StringType), true)() :: - AttributeReference("arrayOfString", ArrayType(StringType), true)() :: - AttributeReference("arrayOfStruct", ArrayType( - StructType(StructField("field1", BooleanType, true) :: - StructField("field2", StringType, true) :: Nil)), true)() :: - AttributeReference("struct", StructType( - StructField("field1", BooleanType, true) :: - StructField("field2", DecimalType, true) :: Nil), true)() :: - AttributeReference("structWithArrayFields", StructType( + val expectedSchema = StructType( + StructField("arrayOfArray1", ArrayType(ArrayType(StringType)), true) :: + StructField("arrayOfArray2", ArrayType(ArrayType(DoubleType)), true) :: + StructField("arrayOfBigInteger", ArrayType(DecimalType), true) :: + StructField("arrayOfBoolean", ArrayType(BooleanType), true) :: + StructField("arrayOfDouble", ArrayType(DoubleType), true) :: + StructField("arrayOfInteger", ArrayType(IntegerType), true) :: + StructField("arrayOfLong", ArrayType(LongType), true) :: + StructField("arrayOfNull", ArrayType(StringType, true), true) :: + StructField("arrayOfString", ArrayType(StringType), true) :: + StructField("arrayOfStruct", ArrayType( + StructType( + StructField("field1", BooleanType, true) :: + StructField("field2", StringType, true) :: + StructField("field3", StringType, true) :: Nil)), true) :: + StructField("struct", StructType( + StructField("field1", BooleanType, true) :: + StructField("field2", DecimalType, true) :: Nil), true) :: + StructField("structWithArrayFields", StructType( StructField("field1", ArrayType(IntegerType), true) :: - StructField("field2", ArrayType(StringType), true) :: Nil), true)() :: Nil + StructField("field2", ArrayType(StringType), true) :: Nil), true) :: Nil) - comparePlans(Schema(expectedSchema), Schema(jsonSchemaRDD.logicalPlan.output)) + assert(expectedSchema === jsonSchemaRDD.schema) - jsonSchemaRDD.registerAsTable("jsonTable") + jsonSchemaRDD.registerTempTable("jsonTable") // Access elements of a primitive array. checkAnswer( @@ -254,8 +264,12 @@ class JsonSuite extends QueryTest { // Access elements of an array of structs. checkAnswer( - sql("select arrayOfStruct[0], arrayOfStruct[1], arrayOfStruct[2] from jsonTable"), - (true :: "str1" :: Nil, false :: null :: Nil, null) :: Nil + sql("select arrayOfStruct[0], arrayOfStruct[1], arrayOfStruct[2], arrayOfStruct[3] " + + "from jsonTable"), + (true :: "str1" :: null :: Nil, + false :: null :: null :: Nil, + null :: null :: null :: Nil, + null) :: Nil ) // Access a struct and fields inside of it. @@ -282,7 +296,7 @@ class JsonSuite extends QueryTest { ignore("Complex field and type inferring (Ignored)") { val jsonSchemaRDD = jsonRDD(complexFieldAndType) - jsonSchemaRDD.registerAsTable("jsonTable") + jsonSchemaRDD.registerTempTable("jsonTable") // Right now, "field1" and "field2" are treated as aliases. We should fix it. checkAnswer( @@ -301,17 +315,17 @@ class JsonSuite extends QueryTest { test("Type conflict in primitive field values") { val jsonSchemaRDD = jsonRDD(primitiveFieldValueTypeConflict) - val expectedSchema = - AttributeReference("num_bool", StringType, true)() :: - AttributeReference("num_num_1", LongType, true)() :: - AttributeReference("num_num_2", DecimalType, true)() :: - AttributeReference("num_num_3", DoubleType, true)() :: - AttributeReference("num_str", StringType, true)() :: - AttributeReference("str_bool", StringType, true)() :: Nil + val expectedSchema = StructType( + StructField("num_bool", StringType, true) :: + StructField("num_num_1", LongType, true) :: + StructField("num_num_2", DecimalType, true) :: + StructField("num_num_3", DoubleType, true) :: + StructField("num_str", StringType, true) :: + StructField("str_bool", StringType, true) :: Nil) - comparePlans(Schema(expectedSchema), Schema(jsonSchemaRDD.logicalPlan.output)) + assert(expectedSchema === jsonSchemaRDD.schema) - jsonSchemaRDD.registerAsTable("jsonTable") + jsonSchemaRDD.registerTempTable("jsonTable") checkAnswer( sql("select * from jsonTable"), @@ -365,7 +379,7 @@ class JsonSuite extends QueryTest { ignore("Type conflict in primitive field values (Ignored)") { val jsonSchemaRDD = jsonRDD(primitiveFieldValueTypeConflict) - jsonSchemaRDD.registerAsTable("jsonTable") + jsonSchemaRDD.registerTempTable("jsonTable") // Right now, the analyzer does not promote strings in a boolean expreesion. // Number and Boolean conflict: resolve the type as boolean in this query. @@ -426,17 +440,17 @@ class JsonSuite extends QueryTest { test("Type conflict in complex field values") { val jsonSchemaRDD = jsonRDD(complexFieldValueTypeConflict) - val expectedSchema = - AttributeReference("array", ArrayType(IntegerType), true)() :: - AttributeReference("num_struct", StringType, true)() :: - AttributeReference("str_array", StringType, true)() :: - AttributeReference("struct", StructType( - StructField("field", StringType, true) :: Nil), true)() :: - AttributeReference("struct_array", StringType, true)() :: Nil + val expectedSchema = StructType( + StructField("array", ArrayType(IntegerType), true) :: + StructField("num_struct", StringType, true) :: + StructField("str_array", StringType, true) :: + StructField("struct", StructType( + StructField("field", StringType, true) :: Nil), true) :: + StructField("struct_array", StringType, true) :: Nil) - comparePlans(Schema(expectedSchema), Schema(jsonSchemaRDD.logicalPlan.output)) + assert(expectedSchema === jsonSchemaRDD.schema) - jsonSchemaRDD.registerAsTable("jsonTable") + jsonSchemaRDD.registerTempTable("jsonTable") checkAnswer( sql("select * from jsonTable"), @@ -450,14 +464,14 @@ class JsonSuite extends QueryTest { test("Type conflict in array elements") { val jsonSchemaRDD = jsonRDD(arrayElementTypeConflict) - val expectedSchema = - AttributeReference("array1", ArrayType(StringType), true)() :: - AttributeReference("array2", ArrayType(StructType( - StructField("field", LongType, true) :: Nil)), true)() :: Nil + val expectedSchema = StructType( + StructField("array1", ArrayType(StringType, true), true) :: + StructField("array2", ArrayType(StructType( + StructField("field", LongType, true) :: Nil)), true) :: Nil) - comparePlans(Schema(expectedSchema), Schema(jsonSchemaRDD.logicalPlan.output)) + assert(expectedSchema === jsonSchemaRDD.schema) - jsonSchemaRDD.registerAsTable("jsonTable") + jsonSchemaRDD.registerTempTable("jsonTable") checkAnswer( sql("select * from jsonTable"), @@ -475,17 +489,17 @@ class JsonSuite extends QueryTest { test("Handling missing fields") { val jsonSchemaRDD = jsonRDD(missingFields) - val expectedSchema = - AttributeReference("a", BooleanType, true)() :: - AttributeReference("b", LongType, true)() :: - AttributeReference("c", ArrayType(IntegerType), true)() :: - AttributeReference("d", StructType( - StructField("field", BooleanType, true) :: Nil), true)() :: - AttributeReference("e", StringType, true)() :: Nil + val expectedSchema = StructType( + StructField("a", BooleanType, true) :: + StructField("b", LongType, true) :: + StructField("c", ArrayType(IntegerType), true) :: + StructField("d", StructType( + StructField("field", BooleanType, true) :: Nil), true) :: + StructField("e", StringType, true) :: Nil) - comparePlans(Schema(expectedSchema), Schema(jsonSchemaRDD.logicalPlan.output)) + assert(expectedSchema === jsonSchemaRDD.schema) - jsonSchemaRDD.registerAsTable("jsonTable") + jsonSchemaRDD.registerTempTable("jsonTable") } test("Loading a JSON dataset from a text file") { @@ -494,18 +508,18 @@ class JsonSuite extends QueryTest { primitiveFieldAndType.map(record => record.replaceAll("\n", " ")).saveAsTextFile(path) val jsonSchemaRDD = jsonFile(path) - val expectedSchema = - AttributeReference("bigInteger", DecimalType, true)() :: - AttributeReference("boolean", BooleanType, true)() :: - AttributeReference("double", DoubleType, true)() :: - AttributeReference("integer", IntegerType, true)() :: - AttributeReference("long", LongType, true)() :: - AttributeReference("null", StringType, true)() :: - AttributeReference("string", StringType, true)() :: Nil + val expectedSchema = StructType( + StructField("bigInteger", DecimalType, true) :: + StructField("boolean", BooleanType, true) :: + StructField("double", DoubleType, true) :: + StructField("integer", IntegerType, true) :: + StructField("long", LongType, true) :: + StructField("null", StringType, true) :: + StructField("string", StringType, true) :: Nil) - comparePlans(Schema(expectedSchema), Schema(jsonSchemaRDD.logicalPlan.output)) + assert(expectedSchema === jsonSchemaRDD.schema) - jsonSchemaRDD.registerAsTable("jsonTable") + jsonSchemaRDD.registerTempTable("jsonTable") checkAnswer( sql("select * from jsonTable"), @@ -518,4 +532,53 @@ class JsonSuite extends QueryTest { "this is a simple string.") :: Nil ) } + + test("Applying schemas") { + val file = getTempFilePath("json") + val path = file.toString + primitiveFieldAndType.map(record => record.replaceAll("\n", " ")).saveAsTextFile(path) + + val schema = StructType( + StructField("bigInteger", DecimalType, true) :: + StructField("boolean", BooleanType, true) :: + StructField("double", DoubleType, true) :: + StructField("integer", IntegerType, true) :: + StructField("long", LongType, true) :: + StructField("null", StringType, true) :: + StructField("string", StringType, true) :: Nil) + + val jsonSchemaRDD1 = jsonFile(path, schema) + + assert(schema === jsonSchemaRDD1.schema) + + jsonSchemaRDD1.registerTempTable("jsonTable1") + + checkAnswer( + sql("select * from jsonTable1"), + (BigDecimal("92233720368547758070"), + true, + 1.7976931348623157E308, + 10, + 21474836470L, + null, + "this is a simple string.") :: Nil + ) + + val jsonSchemaRDD2 = jsonRDD(primitiveFieldAndType, schema) + + assert(schema === jsonSchemaRDD2.schema) + + jsonSchemaRDD2.registerTempTable("jsonTable2") + + checkAnswer( + sql("select * from jsonTable2"), + (BigDecimal("92233720368547758070"), + true, + 1.7976931348623157E308, + 10, + 21474836470L, + null, + "this is a simple string.") :: Nil + ) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala index d0180f3754f22..a88310b5f1b46 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala @@ -43,7 +43,7 @@ object TestJsonData { "arrayOfDouble":[1.2, 1.7976931348623157E308, 4.9E-324, 2.2250738585072014E-308], "arrayOfBoolean":[true, false, true], "arrayOfNull":[null, null, null, null], - "arrayOfStruct":[{"field1": true, "field2": "str1"}, {"field1": false}], + "arrayOfStruct":[{"field1": true, "field2": "str1"}, {"field1": false}, {"field3": null}], "arrayOfArray1":[[1, 2, 3], ["str1", "str2"]], "arrayOfArray2":[[1, 2, 3], [1.1, 2.1, 3.1]] }""" :: Nil) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index dbf315947ff47..172dcd6aa0ee3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -21,10 +21,9 @@ import org.scalatest.{BeforeAndAfterAll, FunSuiteLike} import parquet.hadoop.ParquetFileWriter import parquet.hadoop.util.ContextUtil -import parquet.schema.MessageTypeParser - import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.mapreduce.Job + import org.apache.spark.SparkContext import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.{SqlLexical, SqlParser} @@ -65,7 +64,8 @@ case class AllDataTypes( doubleField: Double, shortField: Short, byteField: Byte, - booleanField: Boolean) + booleanField: Boolean, + binaryField: Array[Byte]) case class AllDataTypesWithNonPrimitiveType( stringField: String, @@ -76,9 +76,10 @@ case class AllDataTypesWithNonPrimitiveType( shortField: Short, byteField: Byte, booleanField: Boolean, + binaryField: Array[Byte], array: Seq[Int], map: Map[Int, String], - nested: Nested) + data: Data) class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterAll { TestData // Load test data tables. @@ -97,9 +98,9 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA ParquetTestData.writeNestedFile3() ParquetTestData.writeNestedFile4() testRDD = parquetFile(ParquetTestData.testDir.toString) - testRDD.registerAsTable("testsource") + testRDD.registerTempTable("testsource") parquetFile(ParquetTestData.testFilterDir.toString) - .registerAsTable("testfiltersource") + .registerTempTable("testfiltersource") } override def afterAll() { @@ -116,7 +117,8 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA val tempDir = getTempFilePath("parquetTest").getCanonicalPath val range = (0 to 255) TestSQLContext.sparkContext.parallelize(range) - .map(x => AllDataTypes(s"$x", x, x.toLong, x.toFloat, x.toDouble, x.toShort, x.toByte, x % 2 == 0)) + .map(x => AllDataTypes(s"$x", x, x.toLong, x.toFloat, x.toDouble, x.toShort, x.toByte, x % 2 == 0, + (0 to x).map(_.toByte).toArray)) .saveAsParquetFile(tempDir) val result = parquetFile(tempDir).collect() range.foreach { @@ -129,7 +131,59 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA assert(result(i).getShort(5) === i.toShort) assert(result(i).getByte(6) === i.toByte) assert(result(i).getBoolean(7) === (i % 2 == 0)) + assert(result(i)(8) === (0 to i).map(_.toByte).toArray) + } + } + + test("Treat binary as string") { + val oldIsParquetBinaryAsString = TestSQLContext.isParquetBinaryAsString + + // Create the test file. + val file = getTempFilePath("parquet") + val path = file.toString + val range = (0 to 255) + val rowRDD = TestSQLContext.sparkContext.parallelize(range) + .map(i => org.apache.spark.sql.Row(i, s"val_$i".getBytes)) + // We need to ask Parquet to store the String column as a Binary column. + val schema = StructType( + StructField("c1", IntegerType, false) :: + StructField("c2", BinaryType, false) :: Nil) + val schemaRDD1 = applySchema(rowRDD, schema) + schemaRDD1.saveAsParquetFile(path) + val resultWithBinary = parquetFile(path).collect + range.foreach { + i => + assert(resultWithBinary(i).getInt(0) === i) + assert(resultWithBinary(i)(1) === s"val_$i".getBytes) + } + + TestSQLContext.setConf(SQLConf.PARQUET_BINARY_AS_STRING, "true") + // This ParquetRelation always use Parquet types to derive output. + val parquetRelation = new ParquetRelation( + path.toString, + Some(TestSQLContext.sparkContext.hadoopConfiguration), + TestSQLContext) { + override val output = + ParquetTypesConverter.convertToAttributes( + ParquetTypesConverter.readMetaData(new Path(path), conf).getFileMetaData.getSchema, + TestSQLContext.isParquetBinaryAsString) + } + val schemaRDD = new SchemaRDD(TestSQLContext, parquetRelation) + val resultWithString = schemaRDD.collect + range.foreach { + i => + assert(resultWithString(i).getInt(0) === i) + assert(resultWithString(i)(1) === s"val_$i") } + + schemaRDD.registerTempTable("tmp") + checkAnswer( + sql("SELECT c1, c2 FROM tmp WHERE c2 = 'val_5' OR c2 = 'val_7'"), + (5, "val_5") :: + (7, "val_7") :: Nil) + + // Set it back. + TestSQLContext.setConf(SQLConf.PARQUET_BINARY_AS_STRING, oldIsParquetBinaryAsString.toString) } test("Read/Write All Types with non-primitive type") { @@ -138,7 +192,8 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA TestSQLContext.sparkContext.parallelize(range) .map(x => AllDataTypesWithNonPrimitiveType( s"$x", x, x.toLong, x.toFloat, x.toDouble, x.toShort, x.toByte, x % 2 == 0, - Seq(x), Map(x -> s"$x"), Nested(x, s"$x"))) + (0 to x).map(_.toByte).toArray, + (0 until x), (0 until x).map(i => i -> s"$i").toMap, Data((0 until x), Nested(x, s"$x")))) .saveAsParquetFile(tempDir) val result = parquetFile(tempDir).collect() range.foreach { @@ -151,9 +206,10 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA assert(result(i).getShort(5) === i.toShort) assert(result(i).getByte(6) === i.toByte) assert(result(i).getBoolean(7) === (i % 2 == 0)) - assert(result(i)(8) === Seq(i)) - assert(result(i)(9) === Map(i -> s"$i")) - assert(result(i)(10) === new GenericRow(Array[Any](i, s"$i"))) + assert(result(i)(8) === (0 to i).map(_.toByte).toArray) + assert(result(i)(9) === (0 until i)) + assert(result(i)(10) === (0 until i).map(i => i -> s"$i").toMap) + assert(result(i)(11) === new GenericRow(Array[Any]((0 until i), new GenericRow(Array[Any](i, s"$i"))))) } } @@ -201,18 +257,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA } test("Projection of simple Parquet file") { - val scanner = new ParquetTableScan( - ParquetTestData.testData.output, - ParquetTestData.testData, - Seq())(TestSQLContext) - val projected = scanner.pruneColumns(ParquetTypesConverter - .convertToAttributes(MessageTypeParser - .parseMessageType(ParquetTestData.subTestSchema))) - assert(projected.output.size === 2) - val result = projected - .execute() - .map(_.copy()) - .collect() + val result = ParquetTestData.testData.select('myboolean, 'mylong).collect() result.zipWithIndex.foreach { case (row, index) => { if (index % 3 == 0) @@ -250,7 +295,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA test("Creating case class RDD table") { TestSQLContext.sparkContext.parallelize((1 to 100)) .map(i => TestRDDEntry(i, s"val_$i")) - .registerAsTable("tmp") + .registerTempTable("tmp") val rdd = sql("SELECT * FROM tmp").collect().sortBy(_.getInt(0)) var counter = 1 rdd.foreach { @@ -269,7 +314,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA .map(i => TestRDDEntry(i, s"val_$i")) rdd.saveAsParquetFile(path) val readFile = parquetFile(path) - readFile.registerAsTable("tmpx") + readFile.registerTempTable("tmpx") val rdd_copy = sql("SELECT * FROM tmpx").collect() val rdd_orig = rdd.collect() for(i <- 0 to 99) { @@ -283,9 +328,9 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA val dirname = Utils.createTempDir() val source_rdd = TestSQLContext.sparkContext.parallelize((1 to 100)) .map(i => TestRDDEntry(i, s"val_$i")) - source_rdd.registerAsTable("source") + source_rdd.registerTempTable("source") val dest_rdd = createParquetFile[TestRDDEntry](dirname.toString) - dest_rdd.registerAsTable("dest") + dest_rdd.registerTempTable("dest") sql("INSERT OVERWRITE INTO dest SELECT * FROM source").collect() val rdd_copy1 = sql("SELECT * FROM dest").collect() assert(rdd_copy1.size === 100) @@ -384,11 +429,14 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA val predicate5 = new GreaterThan(attribute1, attribute2) val badfilter = ParquetFilters.createFilter(predicate5) assert(badfilter.isDefined === false) + + val predicate6 = And(GreaterThan(attribute1, attribute2), GreaterThan(attribute1, attribute2)) + val badfilter2 = ParquetFilters.createFilter(predicate6) + assert(badfilter2.isDefined === false) } test("test filter by predicate pushdown") { for(myval <- Seq("myint", "mylong", "mydouble", "myfloat")) { - println(s"testing field $myval") val query1 = sql(s"SELECT * FROM testfiltersource WHERE $myval < 150 AND $myval >= 100") assert( query1.queryExecution.executedPlan(0)(0).isInstanceOf[ParquetTableScan], @@ -550,7 +598,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA val data = nestedParserSqlContext .parquetFile(ParquetTestData.testNestedDir1.toString) .toSchemaRDD - data.registerAsTable("data") + data.registerTempTable("data") val query = nestedParserSqlContext.sql("SELECT owner, contacts[1].name FROM data") val tmp = query.collect() assert(tmp.size === 2) @@ -565,7 +613,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA val data = nestedParserSqlContext .parquetFile(ParquetTestData.testNestedDir2.toString) .toSchemaRDD - data.registerAsTable("data") + data.registerTempTable("data") val result1 = nestedParserSqlContext.sql("SELECT entries[0].value FROM data").collect() assert(result1.size === 1) assert(result1(0).size === 1) @@ -592,7 +640,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA val data = nestedParserSqlContext .parquetFile(ParquetTestData.testNestedDir3.toString) .toSchemaRDD - data.registerAsTable("data") + data.registerTempTable("data") val result1 = nestedParserSqlContext.sql("SELECT booleanNumberPairs[0].value[0].truth FROM data").collect() assert(result1.size === 1) assert(result1(0).size === 1) @@ -611,7 +659,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA val data = TestSQLContext .parquetFile(ParquetTestData.testNestedDir4.toString) .toSchemaRDD - data.registerAsTable("mapTable") + data.registerTempTable("mapTable") val result1 = sql("SELECT data1 FROM mapTable").collect() assert(result1.size === 1) assert(result1(0)(0) @@ -628,7 +676,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA val data = nestedParserSqlContext .parquetFile(ParquetTestData.testNestedDir4.toString) .toSchemaRDD - data.registerAsTable("mapTable") + data.registerTempTable("mapTable") val result1 = nestedParserSqlContext.sql("SELECT data2 FROM mapTable").collect() assert(result1.size === 1) val entry1 = result1(0)(0) @@ -661,7 +709,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA nestedParserSqlContext .parquetFile(tmpdir.toString) .toSchemaRDD - .registerAsTable("tmpcopy") + .registerTempTable("tmpcopy") val tmpdata = nestedParserSqlContext.sql("SELECT owner, contacts[1].name FROM tmpcopy").collect() assert(tmpdata.size === 2) assert(tmpdata(0).size === 2) @@ -682,7 +730,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA nestedParserSqlContext .parquetFile(tmpdir.toString) .toSchemaRDD - .registerAsTable("tmpmapcopy") + .registerTempTable("tmpmapcopy") val result1 = nestedParserSqlContext.sql("""SELECT data1["key2"] FROM tmpmapcopy""").collect() assert(result1.size === 1) assert(result1(0)(0) === 2) diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml new file mode 100644 index 0000000000000..c6f60c18804a4 --- /dev/null +++ b/sql/hive-thriftserver/pom.xml @@ -0,0 +1,82 @@ + + + + + 4.0.0 + + org.apache.spark + spark-parent + 1.1.0-SNAPSHOT + ../../pom.xml + + + org.apache.spark + spark-hive-thriftserver_2.10 + jar + Spark Project Hive Thrift Server + http://spark.apache.org/ + + hive-thriftserver + + + + + org.apache.spark + spark-hive_${scala.binary.version} + ${project.version} + + + org.spark-project.hive + hive-cli + ${hive.version} + + + org.spark-project.hive + hive-jdbc + ${hive.version} + + + org.spark-project.hive + hive-beeline + ${hive.version} + + + org.scalatest + scalatest_${scala.binary.version} + test + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + org.scalatest + scalatest-maven-plugin + + + org.apache.maven.plugins + maven-deploy-plugin + + true + + + + + diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala new file mode 100644 index 0000000000000..cadf7aaf42157 --- /dev/null +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala @@ -0,0 +1,96 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.thriftserver + +import scala.collection.JavaConversions._ + +import org.apache.commons.logging.LogFactory +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.ql.session.SessionState +import org.apache.hive.service.cli.thrift.ThriftBinaryCLIService +import org.apache.hive.service.server.{HiveServer2, ServerOptionsProcessor} + +import org.apache.spark.Logging +import org.apache.spark.sql.hive.HiveContext +import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ + +/** + * The main entry point for the Spark SQL port of HiveServer2. Starts up a `SparkSQLContext` and a + * `HiveThriftServer2` thrift server. + */ +private[hive] object HiveThriftServer2 extends Logging { + var LOG = LogFactory.getLog(classOf[HiveServer2]) + + def main(args: Array[String]) { + val optionsProcessor = new ServerOptionsProcessor("HiveThriftServer2") + + if (!optionsProcessor.process(args)) { + System.exit(-1) + } + + val ss = new SessionState(new HiveConf(classOf[SessionState])) + + // Set all properties specified via command line. + val hiveConf: HiveConf = ss.getConf + hiveConf.getAllProperties.toSeq.sortBy(_._1).foreach { case (k, v) => + logDebug(s"HiveConf var: $k=$v") + } + + SessionState.start(ss) + + logInfo("Starting SparkContext") + SparkSQLEnv.init() + SessionState.start(ss) + + Runtime.getRuntime.addShutdownHook( + new Thread() { + override def run() { + SparkSQLEnv.stop() + } + } + ) + + try { + val server = new HiveThriftServer2(SparkSQLEnv.hiveContext) + server.init(hiveConf) + server.start() + logInfo("HiveThriftServer2 started") + } catch { + case e: Exception => + logError("Error starting HiveThriftServer2", e) + System.exit(-1) + } + } +} + +private[hive] class HiveThriftServer2(hiveContext: HiveContext) + extends HiveServer2 + with ReflectedCompositeService { + + override def init(hiveConf: HiveConf) { + val sparkSqlCliService = new SparkSQLCLIService(hiveContext) + setSuperField(this, "cliService", sparkSqlCliService) + addService(sparkSqlCliService) + + val thriftCliService = new ThriftBinaryCLIService(sparkSqlCliService) + setSuperField(this, "thriftCLIService", thriftCliService) + addService(thriftCliService) + + initCompositeService(hiveConf) + } +} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ReflectionUtils.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ReflectionUtils.scala new file mode 100644 index 0000000000000..599294dfbb7d7 --- /dev/null +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ReflectionUtils.scala @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.thriftserver + +private[hive] object ReflectionUtils { + def setSuperField(obj : Object, fieldName: String, fieldValue: Object) { + setAncestorField(obj, 1, fieldName, fieldValue) + } + + def setAncestorField(obj: AnyRef, level: Int, fieldName: String, fieldValue: AnyRef) { + val ancestor = Iterator.iterate[Class[_]](obj.getClass)(_.getSuperclass).drop(level).next() + val field = ancestor.getDeclaredField(fieldName) + field.setAccessible(true) + field.set(obj, fieldValue) + } + + def getSuperField[T](obj: AnyRef, fieldName: String): T = { + getAncestorField[T](obj, 1, fieldName) + } + + def getAncestorField[T](clazz: Object, level: Int, fieldName: String): T = { + val ancestor = Iterator.iterate[Class[_]](clazz.getClass)(_.getSuperclass).drop(level).next() + val field = ancestor.getDeclaredField(fieldName) + field.setAccessible(true) + field.get(clazz).asInstanceOf[T] + } + + def invokeStatic(clazz: Class[_], methodName: String, args: (Class[_], AnyRef)*): AnyRef = { + invoke(clazz, null, methodName, args: _*) + } + + def invoke( + clazz: Class[_], + obj: AnyRef, + methodName: String, + args: (Class[_], AnyRef)*): AnyRef = { + + val (types, values) = args.unzip + val method = clazz.getDeclaredMethod(methodName, types: _*) + method.setAccessible(true) + method.invoke(obj, values.toSeq: _*) + } +} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala new file mode 100755 index 0000000000000..b092f42372171 --- /dev/null +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala @@ -0,0 +1,346 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.thriftserver + +import scala.collection.JavaConversions._ + +import java.io._ +import java.util.{ArrayList => JArrayList} + +import jline.{ConsoleReader, History} +import org.apache.commons.lang.StringUtils +import org.apache.commons.logging.LogFactory +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.hive.cli.{CliDriver, CliSessionState, OptionsProcessor} +import org.apache.hadoop.hive.common.LogUtils.LogInitializationException +import org.apache.hadoop.hive.common.{HiveInterruptCallback, HiveInterruptUtils, LogUtils} +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.ql.Driver +import org.apache.hadoop.hive.ql.exec.Utilities +import org.apache.hadoop.hive.ql.processors.{SetProcessor, CommandProcessor, CommandProcessorFactory} +import org.apache.hadoop.hive.ql.session.SessionState +import org.apache.hadoop.hive.shims.ShimLoader +import org.apache.thrift.transport.TSocket + +import org.apache.spark.Logging + +private[hive] object SparkSQLCLIDriver { + private var prompt = "spark-sql" + private var continuedPrompt = "".padTo(prompt.length, ' ') + private var transport:TSocket = _ + + installSignalHandler() + + /** + * Install an interrupt callback to cancel all Spark jobs. In Hive's CliDriver#processLine(), + * a signal handler will invoke this registered callback if a Ctrl+C signal is detected while + * a command is being processed by the current thread. + */ + def installSignalHandler() { + HiveInterruptUtils.add(new HiveInterruptCallback { + override def interrupt() { + // Handle remote execution mode + if (SparkSQLEnv.sparkContext != null) { + SparkSQLEnv.sparkContext.cancelAllJobs() + } else { + if (transport != null) { + // Force closing of TCP connection upon session termination + transport.getSocket.close() + } + } + } + }) + } + + def main(args: Array[String]) { + val oproc = new OptionsProcessor() + if (!oproc.process_stage1(args)) { + System.exit(1) + } + + // NOTE: It is critical to do this here so that log4j is reinitialized + // before any of the other core hive classes are loaded + var logInitFailed = false + var logInitDetailMessage: String = null + try { + logInitDetailMessage = LogUtils.initHiveLog4j() + } catch { + case e: LogInitializationException => + logInitFailed = true + logInitDetailMessage = e.getMessage + } + + val sessionState = new CliSessionState(new HiveConf(classOf[SessionState])) + + sessionState.in = System.in + try { + sessionState.out = new PrintStream(System.out, true, "UTF-8") + sessionState.info = new PrintStream(System.err, true, "UTF-8") + sessionState.err = new PrintStream(System.err, true, "UTF-8") + } catch { + case e: UnsupportedEncodingException => System.exit(3) + } + + if (!oproc.process_stage2(sessionState)) { + System.exit(2) + } + + if (!sessionState.getIsSilent) { + if (logInitFailed) System.err.println(logInitDetailMessage) + else SessionState.getConsole.printInfo(logInitDetailMessage) + } + + // Set all properties specified via command line. + val conf: HiveConf = sessionState.getConf + sessionState.cmdProperties.entrySet().foreach { item: java.util.Map.Entry[Object, Object] => + conf.set(item.getKey.asInstanceOf[String], item.getValue.asInstanceOf[String]) + sessionState.getOverriddenConfigurations.put( + item.getKey.asInstanceOf[String], item.getValue.asInstanceOf[String]) + } + + SessionState.start(sessionState) + + // Clean up after we exit + Runtime.getRuntime.addShutdownHook( + new Thread() { + override def run() { + SparkSQLEnv.stop() + } + } + ) + + // "-h" option has been passed, so connect to Hive thrift server. + if (sessionState.getHost != null) { + sessionState.connect() + if (sessionState.isRemoteMode) { + prompt = s"[${sessionState.getHost}:${sessionState.getPort}]" + prompt + continuedPrompt = "".padTo(prompt.length, ' ') + } + } + + if (!sessionState.isRemoteMode && !ShimLoader.getHadoopShims.usesJobShell()) { + // Hadoop-20 and above - we need to augment classpath using hiveconf + // components. + // See also: code in ExecDriver.java + var loader = conf.getClassLoader + val auxJars = HiveConf.getVar(conf, HiveConf.ConfVars.HIVEAUXJARS) + if (StringUtils.isNotBlank(auxJars)) { + loader = Utilities.addToClassPath(loader, StringUtils.split(auxJars, ",")) + } + conf.setClassLoader(loader) + Thread.currentThread().setContextClassLoader(loader) + } + + val cli = new SparkSQLCLIDriver + cli.setHiveVariables(oproc.getHiveVariables) + + // TODO work around for set the log output to console, because the HiveContext + // will set the output into an invalid buffer. + sessionState.in = System.in + try { + sessionState.out = new PrintStream(System.out, true, "UTF-8") + sessionState.info = new PrintStream(System.err, true, "UTF-8") + sessionState.err = new PrintStream(System.err, true, "UTF-8") + } catch { + case e: UnsupportedEncodingException => System.exit(3) + } + + // Execute -i init files (always in silent mode) + cli.processInitFiles(sessionState) + + if (sessionState.execString != null) { + System.exit(cli.processLine(sessionState.execString)) + } + + try { + if (sessionState.fileName != null) { + System.exit(cli.processFile(sessionState.fileName)) + } + } catch { + case e: FileNotFoundException => + System.err.println(s"Could not open input file for reading. (${e.getMessage})") + System.exit(3) + } + + val reader = new ConsoleReader() + reader.setBellEnabled(false) + // reader.setDebug(new PrintWriter(new FileWriter("writer.debug", true))) + CliDriver.getCommandCompletor.foreach((e) => reader.addCompletor(e)) + + val historyDirectory = System.getProperty("user.home") + + try { + if (new File(historyDirectory).exists()) { + val historyFile = historyDirectory + File.separator + ".hivehistory" + reader.setHistory(new History(new File(historyFile))) + } else { + System.err.println("WARNING: Directory for Hive history file: " + historyDirectory + + " does not exist. History will not be available during this session.") + } + } catch { + case e: Exception => + System.err.println("WARNING: Encountered an error while trying to initialize Hive's " + + "history file. History will not be available during this session.") + System.err.println(e.getMessage) + } + + val clientTransportTSocketField = classOf[CliSessionState].getDeclaredField("transport") + clientTransportTSocketField.setAccessible(true) + + transport = clientTransportTSocketField.get(sessionState).asInstanceOf[TSocket] + + var ret = 0 + var prefix = "" + val currentDB = ReflectionUtils.invokeStatic(classOf[CliDriver], "getFormattedDb", + classOf[HiveConf] -> conf, classOf[CliSessionState] -> sessionState) + + def promptWithCurrentDB = s"$prompt$currentDB" + def continuedPromptWithDBSpaces = continuedPrompt + ReflectionUtils.invokeStatic( + classOf[CliDriver], "spacesForString", classOf[String] -> currentDB) + + var currentPrompt = promptWithCurrentDB + var line = reader.readLine(currentPrompt + "> ") + + while (line != null) { + if (prefix.nonEmpty) { + prefix += '\n' + } + + if (line.trim().endsWith(";") && !line.trim().endsWith("\\;")) { + line = prefix + line + ret = cli.processLine(line, true) + prefix = "" + currentPrompt = promptWithCurrentDB + } else { + prefix = prefix + line + currentPrompt = continuedPromptWithDBSpaces + } + + line = reader.readLine(currentPrompt + "> ") + } + + sessionState.close() + + System.exit(ret) + } +} + +private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { + private val sessionState = SessionState.get().asInstanceOf[CliSessionState] + + private val LOG = LogFactory.getLog("CliDriver") + + private val console = new SessionState.LogHelper(LOG) + + private val conf: Configuration = + if (sessionState != null) sessionState.getConf else new Configuration() + + // Force initializing SparkSQLEnv. This is put here but not object SparkSQLCliDriver + // because the Hive unit tests do not go through the main() code path. + if (!sessionState.isRemoteMode) { + SparkSQLEnv.init() + } + + override def processCmd(cmd: String): Int = { + val cmd_trimmed: String = cmd.trim() + val tokens: Array[String] = cmd_trimmed.split("\\s+") + val cmd_1: String = cmd_trimmed.substring(tokens(0).length()).trim() + if (cmd_trimmed.toLowerCase.equals("quit") || + cmd_trimmed.toLowerCase.equals("exit") || + tokens(0).equalsIgnoreCase("source") || + cmd_trimmed.startsWith("!") || + tokens(0).toLowerCase.equals("list") || + sessionState.isRemoteMode) { + val start = System.currentTimeMillis() + super.processCmd(cmd) + val end = System.currentTimeMillis() + val timeTaken: Double = (end - start) / 1000.0 + console.printInfo(s"Time taken: $timeTaken seconds") + 0 + } else { + var ret = 0 + val hconf = conf.asInstanceOf[HiveConf] + val proc: CommandProcessor = CommandProcessorFactory.get(tokens(0), hconf) + + if (proc != null) { + if (proc.isInstanceOf[Driver] || proc.isInstanceOf[SetProcessor]) { + val driver = new SparkSQLDriver + + driver.init() + val out = sessionState.out + val start:Long = System.currentTimeMillis() + if (sessionState.getIsVerbose) { + out.println(cmd) + } + + val rc = driver.run(cmd) + ret = rc.getResponseCode + if (ret != 0) { + console.printError(rc.getErrorMessage()) + driver.close() + return ret + } + + val res = new JArrayList[String]() + + if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_CLI_PRINT_HEADER)) { + // Print the column names. + Option(driver.getSchema.getFieldSchemas).map { fields => + out.println(fields.map(_.getName).mkString("\t")) + } + } + + try { + while (!out.checkError() && driver.getResults(res)) { + res.foreach(out.println) + res.clear() + } + } catch { + case e:IOException => + console.printError( + s"""Failed with exception ${e.getClass.getName}: ${e.getMessage} + |${org.apache.hadoop.util.StringUtils.stringifyException(e)} + """.stripMargin) + ret = 1 + } + + val cret = driver.close() + if (ret == 0) { + ret = cret + } + + val end = System.currentTimeMillis() + if (end > start) { + val timeTaken:Double = (end - start) / 1000.0 + console.printInfo(s"Time taken: $timeTaken seconds", null) + } + + // Destroy the driver to release all the locks. + driver.destroy() + } else { + if (sessionState.getIsVerbose) { + sessionState.out.println(tokens(0) + " " + cmd_1) + } + ret = proc.run(cmd_1).getResponseCode + } + } + ret + } + } +} + diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala new file mode 100644 index 0000000000000..42cbf363b274f --- /dev/null +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.thriftserver + +import scala.collection.JavaConversions._ + +import java.io.IOException +import java.util.{List => JList} +import javax.security.auth.login.LoginException + +import org.apache.commons.logging.Log +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.shims.ShimLoader +import org.apache.hive.service.Service.STATE +import org.apache.hive.service.auth.HiveAuthFactory +import org.apache.hive.service.cli.CLIService +import org.apache.hive.service.{AbstractService, Service, ServiceException} + +import org.apache.spark.sql.hive.HiveContext +import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ + +private[hive] class SparkSQLCLIService(hiveContext: HiveContext) + extends CLIService + with ReflectedCompositeService { + + override def init(hiveConf: HiveConf) { + setSuperField(this, "hiveConf", hiveConf) + + val sparkSqlSessionManager = new SparkSQLSessionManager(hiveContext) + setSuperField(this, "sessionManager", sparkSqlSessionManager) + addService(sparkSqlSessionManager) + + try { + HiveAuthFactory.loginFromKeytab(hiveConf) + val serverUserName = ShimLoader.getHadoopShims + .getShortUserName(ShimLoader.getHadoopShims.getUGIForConf(hiveConf)) + setSuperField(this, "serverUserName", serverUserName) + } catch { + case e @ (_: IOException | _: LoginException) => + throw new ServiceException("Unable to login to kerberos with given principal/keytab", e) + } + + initCompositeService(hiveConf) + } +} + +private[thriftserver] trait ReflectedCompositeService { this: AbstractService => + def initCompositeService(hiveConf: HiveConf) { + // Emulating `CompositeService.init(hiveConf)` + val serviceList = getAncestorField[JList[Service]](this, 2, "serviceList") + serviceList.foreach(_.init(hiveConf)) + + // Emulating `AbstractService.init(hiveConf)` + invoke(classOf[AbstractService], this, "ensureCurrentState", classOf[STATE] -> STATE.NOTINITED) + setAncestorField(this, 3, "hiveConf", hiveConf) + invoke(classOf[AbstractService], this, "changeState", classOf[STATE] -> STATE.INITED) + getAncestorField[Log](this, 3, "LOG").info(s"Service: $getName is inited.") + } +} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala new file mode 100644 index 0000000000000..7463df1f47d43 --- /dev/null +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala @@ -0,0 +1,92 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.thriftserver + +import scala.collection.JavaConversions._ + +import java.util.{ArrayList => JArrayList} + +import org.apache.commons.lang.exception.ExceptionUtils +import org.apache.hadoop.hive.metastore.api.{FieldSchema, Schema} +import org.apache.hadoop.hive.ql.Driver +import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse + +import org.apache.spark.Logging +import org.apache.spark.sql.hive.{HiveContext, HiveMetastoreTypes} + +private[hive] class SparkSQLDriver(val context: HiveContext = SparkSQLEnv.hiveContext) + extends Driver with Logging { + + private var tableSchema: Schema = _ + private var hiveResponse: Seq[String] = _ + + override def init(): Unit = { + } + + private def getResultSetSchema(query: context.QueryExecution): Schema = { + val analyzed = query.analyzed + logDebug(s"Result Schema: ${analyzed.output}") + if (analyzed.output.size == 0) { + new Schema(new FieldSchema("Response code", "string", "") :: Nil, null) + } else { + val fieldSchemas = analyzed.output.map { attr => + new FieldSchema(attr.name, HiveMetastoreTypes.toMetastoreType(attr.dataType), "") + } + + new Schema(fieldSchemas, null) + } + } + + override def run(command: String): CommandProcessorResponse = { + // TODO unify the error code + try { + val execution = context.executePlan(context.sql(command).logicalPlan) + hiveResponse = execution.stringResult() + tableSchema = getResultSetSchema(execution) + new CommandProcessorResponse(0) + } catch { + case cause: Throwable => + logError(s"Failed in [$command]", cause) + new CommandProcessorResponse(-3, ExceptionUtils.getFullStackTrace(cause), null) + } + } + + override def close(): Int = { + hiveResponse = null + tableSchema = null + 0 + } + + override def getSchema: Schema = tableSchema + + override def getResults(res: JArrayList[String]): Boolean = { + if (hiveResponse == null) { + false + } else { + res.addAll(hiveResponse) + hiveResponse = null + true + } + } + + override def destroy() { + super.destroy() + hiveResponse = null + tableSchema = null + } +} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala new file mode 100644 index 0000000000000..582264eb59f83 --- /dev/null +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.thriftserver + +import org.apache.hadoop.hive.ql.session.SessionState + +import org.apache.spark.scheduler.{SplitInfo, StatsReportListener} +import org.apache.spark.Logging +import org.apache.spark.sql.hive.HiveContext +import org.apache.spark.{SparkConf, SparkContext} + +/** A singleton object for the master program. The slaves should not access this. */ +private[hive] object SparkSQLEnv extends Logging { + logDebug("Initializing SparkSQLEnv") + + var hiveContext: HiveContext = _ + var sparkContext: SparkContext = _ + + def init() { + if (hiveContext == null) { + sparkContext = new SparkContext(new SparkConf() + .setAppName(s"SparkSQL::${java.net.InetAddress.getLocalHost.getHostName}")) + + sparkContext.addSparkListener(new StatsReportListener()) + + hiveContext = new HiveContext(sparkContext) { + @transient override lazy val sessionState = SessionState.get() + @transient override lazy val hiveconf = sessionState.getConf + } + } + } + + /** Cleans up and shuts down the Spark SQL environments. */ + def stop() { + logDebug("Shutting down Spark SQL Environment") + // Stop the SparkContext + if (SparkSQLEnv.sparkContext != null) { + sparkContext.stop() + sparkContext = null + hiveContext = null + } + } +} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala new file mode 100644 index 0000000000000..6b3275b4eaf04 --- /dev/null +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.thriftserver + +import java.util.concurrent.Executors + +import org.apache.commons.logging.Log +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.conf.HiveConf.ConfVars +import org.apache.hive.service.cli.session.SessionManager + +import org.apache.spark.sql.hive.HiveContext +import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ +import org.apache.spark.sql.hive.thriftserver.server.SparkSQLOperationManager + +private[hive] class SparkSQLSessionManager(hiveContext: HiveContext) + extends SessionManager + with ReflectedCompositeService { + + override def init(hiveConf: HiveConf) { + setSuperField(this, "hiveConf", hiveConf) + + val backgroundPoolSize = hiveConf.getIntVar(ConfVars.HIVE_SERVER2_ASYNC_EXEC_THREADS) + setSuperField(this, "backgroundOperationPool", Executors.newFixedThreadPool(backgroundPoolSize)) + getAncestorField[Log](this, 3, "LOG").info( + s"HiveServer2: Async execution pool size $backgroundPoolSize") + + val sparkSqlOperationManager = new SparkSQLOperationManager(hiveContext) + setSuperField(this, "operationManager", sparkSqlOperationManager) + addService(sparkSqlOperationManager) + + initCompositeService(hiveConf) + } +} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala new file mode 100644 index 0000000000000..9338e8121b0fe --- /dev/null +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala @@ -0,0 +1,196 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.thriftserver.server + +import scala.collection.JavaConversions._ +import scala.collection.mutable.ArrayBuffer +import scala.math.{random, round} + +import java.sql.Timestamp +import java.util.{Map => JMap} + +import org.apache.hadoop.hive.common.`type`.HiveDecimal +import org.apache.hadoop.hive.metastore.api.FieldSchema +import org.apache.hive.service.cli._ +import org.apache.hive.service.cli.operation.{ExecuteStatementOperation, Operation, OperationManager} +import org.apache.hive.service.cli.session.HiveSession + +import org.apache.spark.Logging +import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.hive.thriftserver.ReflectionUtils +import org.apache.spark.sql.hive.{HiveContext, HiveMetastoreTypes} +import org.apache.spark.sql.{SchemaRDD, Row => SparkRow} + +/** + * Executes queries using Spark SQL, and maintains a list of handles to active queries. + */ +class SparkSQLOperationManager(hiveContext: HiveContext) extends OperationManager with Logging { + val handleToOperation = ReflectionUtils + .getSuperField[JMap[OperationHandle, Operation]](this, "handleToOperation") + + override def newExecuteStatementOperation( + parentSession: HiveSession, + statement: String, + confOverlay: JMap[String, String], + async: Boolean): ExecuteStatementOperation = synchronized { + + val operation = new ExecuteStatementOperation(parentSession, statement, confOverlay) { + private var result: SchemaRDD = _ + private var iter: Iterator[SparkRow] = _ + private var dataTypes: Array[DataType] = _ + + def close(): Unit = { + // RDDs will be cleaned automatically upon garbage collection. + logDebug("CLOSING") + } + + def getNextRowSet(order: FetchOrientation, maxRowsL: Long): RowSet = { + if (!iter.hasNext) { + new RowSet() + } else { + val maxRows = maxRowsL.toInt // Do you really want a row batch larger than Int Max? No. + var curRow = 0 + var rowSet = new ArrayBuffer[Row](maxRows) + + while (curRow < maxRows && iter.hasNext) { + val sparkRow = iter.next() + val row = new Row() + var curCol = 0 + + while (curCol < sparkRow.length) { + if (sparkRow.isNullAt(curCol)) { + addNullColumnValue(sparkRow, row, curCol) + } else { + addNonNullColumnValue(sparkRow, row, curCol) + } + curCol += 1 + } + rowSet += row + curRow += 1 + } + new RowSet(rowSet, 0) + } + } + + def addNonNullColumnValue(from: SparkRow, to: Row, ordinal: Int) { + dataTypes(ordinal) match { + case StringType => + to.addString(from(ordinal).asInstanceOf[String]) + case IntegerType => + to.addColumnValue(ColumnValue.intValue(from.getInt(ordinal))) + case BooleanType => + to.addColumnValue(ColumnValue.booleanValue(from.getBoolean(ordinal))) + case DoubleType => + to.addColumnValue(ColumnValue.doubleValue(from.getDouble(ordinal))) + case FloatType => + to.addColumnValue(ColumnValue.floatValue(from.getFloat(ordinal))) + case DecimalType => + val hiveDecimal = from.get(ordinal).asInstanceOf[BigDecimal].bigDecimal + to.addColumnValue(ColumnValue.stringValue(new HiveDecimal(hiveDecimal))) + case LongType => + to.addColumnValue(ColumnValue.longValue(from.getLong(ordinal))) + case ByteType => + to.addColumnValue(ColumnValue.byteValue(from.getByte(ordinal))) + case ShortType => + to.addColumnValue(ColumnValue.intValue(from.getShort(ordinal))) + case TimestampType => + to.addColumnValue( + ColumnValue.timestampValue(from.get(ordinal).asInstanceOf[Timestamp])) + case BinaryType | _: ArrayType | _: StructType | _: MapType => + val hiveString = result + .queryExecution + .asInstanceOf[HiveContext#QueryExecution] + .toHiveString((from.get(ordinal), dataTypes(ordinal))) + to.addColumnValue(ColumnValue.stringValue(hiveString)) + } + } + + def addNullColumnValue(from: SparkRow, to: Row, ordinal: Int) { + dataTypes(ordinal) match { + case StringType => + to.addString(null) + case IntegerType => + to.addColumnValue(ColumnValue.intValue(null)) + case BooleanType => + to.addColumnValue(ColumnValue.booleanValue(null)) + case DoubleType => + to.addColumnValue(ColumnValue.doubleValue(null)) + case FloatType => + to.addColumnValue(ColumnValue.floatValue(null)) + case DecimalType => + to.addColumnValue(ColumnValue.stringValue(null: HiveDecimal)) + case LongType => + to.addColumnValue(ColumnValue.longValue(null)) + case ByteType => + to.addColumnValue(ColumnValue.byteValue(null)) + case ShortType => + to.addColumnValue(ColumnValue.intValue(null)) + case TimestampType => + to.addColumnValue(ColumnValue.timestampValue(null)) + case BinaryType | _: ArrayType | _: StructType | _: MapType => + to.addColumnValue(ColumnValue.stringValue(null: String)) + } + } + + def getResultSetSchema: TableSchema = { + logWarning(s"Result Schema: ${result.queryExecution.analyzed.output}") + if (result.queryExecution.analyzed.output.size == 0) { + new TableSchema(new FieldSchema("Result", "string", "") :: Nil) + } else { + val schema = result.queryExecution.analyzed.output.map { attr => + new FieldSchema(attr.name, HiveMetastoreTypes.toMetastoreType(attr.dataType), "") + } + new TableSchema(schema) + } + } + + def run(): Unit = { + logInfo(s"Running query '$statement'") + setState(OperationState.RUNNING) + try { + result = hiveContext.sql(statement) + logDebug(result.queryExecution.toString()) + val groupId = round(random * 1000000).toString + hiveContext.sparkContext.setJobGroup(groupId, statement) + iter = { + val resultRdd = result.queryExecution.toRdd + val useIncrementalCollect = + hiveContext.getConf("spark.sql.thriftServer.incrementalCollect", "false").toBoolean + if (useIncrementalCollect) { + resultRdd.toLocalIterator + } else { + resultRdd.collect().iterator + } + } + dataTypes = result.queryExecution.analyzed.output.map(_.dataType).toArray + setHasResultSet(true) + } catch { + // Actually do need to catch Throwable as some failures don't inherit from Exception and + // HiveServer will silently swallow them. + case e: Throwable => + logError("Error executing query:",e) + throw new HiveSQLException(e.toString) + } + setState(OperationState.FINISHED) + } + } + + handleToOperation.put(operation.getHandle, operation) + operation + } +} diff --git a/sql/hive-thriftserver/src/test/resources/data/files/small_kv.txt b/sql/hive-thriftserver/src/test/resources/data/files/small_kv.txt new file mode 100644 index 0000000000000..850f8014b6f05 --- /dev/null +++ b/sql/hive-thriftserver/src/test/resources/data/files/small_kv.txt @@ -0,0 +1,5 @@ +238val_238 +86val_86 +311val_311 +27val_27 +165val_165 diff --git a/sql/hive-thriftserver/src/test/resources/data/files/small_kv_with_null.txt b/sql/hive-thriftserver/src/test/resources/data/files/small_kv_with_null.txt new file mode 100644 index 0000000000000..ae08c640e6c13 --- /dev/null +++ b/sql/hive-thriftserver/src/test/resources/data/files/small_kv_with_null.txt @@ -0,0 +1,10 @@ +238val_238 + +311val_311 +val_27 +val_165 +val_409 +255val_255 +278val_278 +98val_98 +val_484 diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala new file mode 100644 index 0000000000000..2bf8cfdcacd22 --- /dev/null +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.thriftserver + +import java.io.{BufferedReader, InputStreamReader, PrintWriter} + +import org.apache.hadoop.hive.conf.HiveConf.ConfVars +import org.scalatest.{BeforeAndAfterAll, FunSuite} + +class CliSuite extends FunSuite with BeforeAndAfterAll with TestUtils { + val WAREHOUSE_PATH = TestUtils.getWarehousePath("cli") + val METASTORE_PATH = TestUtils.getMetastorePath("cli") + + override def beforeAll() { + val jdbcUrl = s"jdbc:derby:;databaseName=$METASTORE_PATH;create=true" + val commands = + s"""../../bin/spark-sql + | --master local + | --hiveconf ${ConfVars.METASTORECONNECTURLKEY}="$jdbcUrl" + | --hiveconf ${ConfVars.METASTOREWAREHOUSE}=$WAREHOUSE_PATH + """.stripMargin.split("\\s+") + + val pb = new ProcessBuilder(commands: _*) + process = pb.start() + outputWriter = new PrintWriter(process.getOutputStream, true) + inputReader = new BufferedReader(new InputStreamReader(process.getInputStream)) + errorReader = new BufferedReader(new InputStreamReader(process.getErrorStream)) + waitForOutput(inputReader, "spark-sql>") + } + + override def afterAll() { + process.destroy() + process.waitFor() + } + + test("simple commands") { + val dataFilePath = getDataFile("data/files/small_kv.txt") + executeQuery("create table hive_test1(key int, val string);") + executeQuery("load data local inpath '" + dataFilePath+ "' overwrite into table hive_test1;") + executeQuery("cache table hive_test1", "Time taken") + } +} diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala new file mode 100644 index 0000000000000..aedef6ce1f5f2 --- /dev/null +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala @@ -0,0 +1,156 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.thriftserver + +import scala.collection.JavaConversions._ +import scala.concurrent.ExecutionContext.Implicits.global +import scala.concurrent._ + +import java.io.{BufferedReader, InputStreamReader} +import java.net.ServerSocket +import java.sql.{Connection, DriverManager, Statement} + +import org.apache.hadoop.hive.conf.HiveConf.ConfVars +import org.scalatest.{BeforeAndAfterAll, FunSuite} + +import org.apache.spark.Logging +import org.apache.spark.sql.catalyst.util.getTempFilePath + +/** + * Test for the HiveThriftServer2 using JDBC. + */ +class HiveThriftServer2Suite extends FunSuite with BeforeAndAfterAll with TestUtils with Logging { + + val WAREHOUSE_PATH = getTempFilePath("warehouse") + val METASTORE_PATH = getTempFilePath("metastore") + + val DRIVER_NAME = "org.apache.hive.jdbc.HiveDriver" + val TABLE = "test" + val HOST = "localhost" + val PORT = { + // Let the system to choose a random available port to avoid collision with other parallel + // builds. + val socket = new ServerSocket(0) + val port = socket.getLocalPort + socket.close() + port + } + + // If verbose is true, the test program will print all outputs coming from the Hive Thrift server. + val VERBOSE = Option(System.getenv("SPARK_SQL_TEST_VERBOSE")).getOrElse("false").toBoolean + + Class.forName(DRIVER_NAME) + + override def beforeAll() { launchServer() } + + override def afterAll() { stopServer() } + + private def launchServer(args: Seq[String] = Seq.empty) { + // Forking a new process to start the Hive Thrift server. The reason to do this is it is + // hard to clean up Hive resources entirely, so we just start a new process and kill + // that process for cleanup. + val jdbcUrl = s"jdbc:derby:;databaseName=$METASTORE_PATH;create=true" + val command = + s"""../../sbin/start-thriftserver.sh + | --master local + | --hiveconf hive.root.logger=INFO,console + | --hiveconf ${ConfVars.METASTORECONNECTURLKEY}="$jdbcUrl" + | --hiveconf ${ConfVars.METASTOREWAREHOUSE}=$METASTORE_PATH + | --hiveconf ${ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST}=$HOST + | --hiveconf ${ConfVars.HIVE_SERVER2_THRIFT_PORT}=$PORT + """.stripMargin.split("\\s+") + + val pb = new ProcessBuilder(command ++ args: _*) + val environment = pb.environment() + environment.put("HIVE_SERVER2_THRIFT_PORT", PORT.toString) + environment.put("HIVE_SERVER2_THRIFT_BIND_HOST", HOST) + process = pb.start() + inputReader = new BufferedReader(new InputStreamReader(process.getInputStream)) + errorReader = new BufferedReader(new InputStreamReader(process.getErrorStream)) + waitForOutput(inputReader, "ThriftBinaryCLIService listening on") + + // Spawn a thread to read the output from the forked process. + // Note that this is necessary since in some configurations, log4j could be blocked + // if its output to stderr are not read, and eventually blocking the entire test suite. + future { + while (true) { + val stdout = readFrom(inputReader) + val stderr = readFrom(errorReader) + if (VERBOSE && stdout.length > 0) { + println(stdout) + } + if (VERBOSE && stderr.length > 0) { + println(stderr) + } + Thread.sleep(50) + } + } + } + + private def stopServer() { + process.destroy() + process.waitFor() + } + + test("test query execution against a Hive Thrift server") { + Thread.sleep(5 * 1000) + val dataFilePath = getDataFile("data/files/small_kv.txt") + val stmt = createStatement() + stmt.execute("DROP TABLE IF EXISTS test") + stmt.execute("DROP TABLE IF EXISTS test_cached") + stmt.execute("CREATE TABLE test(key INT, val STRING)") + stmt.execute(s"LOAD DATA LOCAL INPATH '$dataFilePath' OVERWRITE INTO TABLE test") + stmt.execute("CREATE TABLE test_cached AS SELECT * FROM test LIMIT 4") + stmt.execute("CACHE TABLE test_cached") + + var rs = stmt.executeQuery("SELECT COUNT(*) FROM test") + rs.next() + assert(rs.getInt(1) === 5) + + rs = stmt.executeQuery("SELECT COUNT(*) FROM test_cached") + rs.next() + assert(rs.getInt(1) === 4) + + stmt.close() + } + + test("SPARK-3004 regression: result set containing NULL") { + Thread.sleep(5 * 1000) + val dataFilePath = getDataFile("data/files/small_kv_with_null.txt") + val stmt = createStatement() + stmt.execute("DROP TABLE IF EXISTS test_null") + stmt.execute("CREATE TABLE test_null(key INT, val STRING)") + stmt.execute(s"LOAD DATA LOCAL INPATH '$dataFilePath' OVERWRITE INTO TABLE test_null") + + val rs = stmt.executeQuery("SELECT * FROM test_null WHERE key IS NULL") + var count = 0 + while (rs.next()) { + count += 1 + } + assert(count === 5) + + stmt.close() + } + + def getConnection: Connection = { + val connectURI = s"jdbc:hive2://localhost:$PORT/" + DriverManager.getConnection(connectURI, System.getProperty("user.name"), "") + } + + def createStatement(): Statement = getConnection.createStatement() +} diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/TestUtils.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/TestUtils.scala new file mode 100644 index 0000000000000..bb2242618fbef --- /dev/null +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/TestUtils.scala @@ -0,0 +1,108 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.thriftserver + +import java.io.{BufferedReader, PrintWriter} +import java.text.SimpleDateFormat +import java.util.Date + +import org.apache.hadoop.hive.common.LogUtils +import org.apache.hadoop.hive.common.LogUtils.LogInitializationException + +object TestUtils { + val timestamp = new SimpleDateFormat("yyyyMMdd-HHmmss") + + def getWarehousePath(prefix: String): String = { + System.getProperty("user.dir") + "/test_warehouses/" + prefix + "-warehouse-" + + timestamp.format(new Date) + } + + def getMetastorePath(prefix: String): String = { + System.getProperty("user.dir") + "/test_warehouses/" + prefix + "-metastore-" + + timestamp.format(new Date) + } + + // Dummy function for initialize the log4j properties. + def init() { } + + // initialize log4j + try { + LogUtils.initHiveLog4j() + } catch { + case e: LogInitializationException => // Ignore the error. + } +} + +trait TestUtils { + var process : Process = null + var outputWriter : PrintWriter = null + var inputReader : BufferedReader = null + var errorReader : BufferedReader = null + + def executeQuery( + cmd: String, outputMessage: String = "OK", timeout: Long = 15000): String = { + println("Executing: " + cmd + ", expecting output: " + outputMessage) + outputWriter.write(cmd + "\n") + outputWriter.flush() + waitForQuery(timeout, outputMessage) + } + + protected def waitForQuery(timeout: Long, message: String): String = { + if (waitForOutput(errorReader, message, timeout)) { + Thread.sleep(500) + readOutput() + } else { + assert(false, "Didn't find \"" + message + "\" in the output:\n" + readOutput()) + null + } + } + + // Wait for the specified str to appear in the output. + protected def waitForOutput( + reader: BufferedReader, str: String, timeout: Long = 10000): Boolean = { + val startTime = System.currentTimeMillis + var out = "" + while (!out.contains(str) && System.currentTimeMillis < (startTime + timeout)) { + out += readFrom(reader) + } + out.contains(str) + } + + // Read stdout output and filter out garbage collection messages. + protected def readOutput(): String = { + val output = readFrom(inputReader) + // Remove GC Messages + val filteredOutput = output.lines.filterNot(x => x.contains("[GC") || x.contains("[Full GC")) + .mkString("\n") + filteredOutput + } + + protected def readFrom(reader: BufferedReader): String = { + var out = "" + var c = 0 + while (reader.ready) { + c = reader.read() + out += c.asInstanceOf[Char] + } + out + } + + protected def getDataFile(name: String) = { + Thread.currentThread().getContextClassLoader.getResource(name) + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala similarity index 92% rename from sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala rename to sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index cdfc2d0c17384..210753efe7678 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.hive.execution import java.io.File +import java.util.TimeZone import org.scalatest.BeforeAndAfter @@ -31,21 +32,27 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { lazy val hiveQueryDir = TestHive.getHiveFile("ql" + File.separator + "src" + File.separator + "test" + File.separator + "queries" + File.separator + "clientpositive") + var originalTimeZone: TimeZone = _ + def testCases = hiveQueryDir.listFiles.map(f => f.getName.stripSuffix(".q") -> f) override def beforeAll() { TestHive.cacheTables = true + // Timezone is fixed to America/Los_Angeles for those timezone sensitive tests (timestamp_*) + originalTimeZone = TimeZone.getDefault + TimeZone.setDefault(TimeZone.getTimeZone("America/Los_Angeles")) } override def afterAll() { TestHive.cacheTables = false + TimeZone.setDefault(originalTimeZone) } /** A list of tests deemed out of scope currently and thus completely disregarded. */ override def blackList = Seq( // These tests use hooks that are not on the classpath and thus break all subsequent execution. "hook_order", - "hook_context", + "hook_context_cs", "mapjoin_hook", "multi_sahooks", "overridden_confs", @@ -84,13 +91,16 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "udf_java_method", "create_merge_compressed", + // DFS commands + "symlink_text_input_format", + // Weird DDL differences result in failures on jenkins. "create_like2", "create_view_translate", "partitions_json", - // Timezone specific test answers. - "udf_unix_timestamp", + // This test is totally fine except that it includes wrong queries and expects errors, but error + // message format in Hive and Spark SQL differ. Should workaround this later. "udf_to_unix_timestamp", // Cant run without local map/reduce. @@ -186,7 +196,10 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { // Hive returns the results of describe as plain text. Comments with multiple lines // introduce extra lines in the Hive results, which make the result comparison fail. - "describe_comment_indent" + "describe_comment_indent", + + // Limit clause without a ordering, which causes failure. + "orc_predicate_pushdown" ) /** @@ -276,9 +289,12 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "compute_stats_empty_table", "compute_stats_long", "compute_stats_string", - "compute_stats_table", "convert_enum_to_string", + "correlationoptimizer1", + "correlationoptimizer10", "correlationoptimizer11", + "correlationoptimizer13", + "correlationoptimizer14", "correlationoptimizer15", "correlationoptimizer2", "correlationoptimizer3", @@ -286,6 +302,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "correlationoptimizer6", "correlationoptimizer7", "correlationoptimizer8", + "correlationoptimizer9", "count", "cp_mj_rc", "create_insert_outputformat", @@ -296,6 +313,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "ct_case_insensitive", "database_location", "database_properties", + "decimal_1", "decimal_4", "decimal_join", "default_partition_name", @@ -304,6 +322,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "describe_formatted_view_partitioned", "diff_part_input_formats", "disable_file_format_check", + "disallow_incompatible_type_change_off", "drop_function", "drop_index", "drop_multi_partitions", @@ -359,8 +378,10 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "groupby_map_ppr", "groupby_multi_insert_common_distinct", "groupby_multi_single_reducer2", + "groupby_multi_single_reducer3", "groupby_mutli_insert_common_distinct", "groupby_neg_float", + "groupby_ppd", "groupby_ppr", "groupby_sort_10", "groupby_sort_2", @@ -372,6 +393,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "groupby_sort_8", "groupby_sort_9", "groupby_sort_test_1", + "having", "implicit_cast1", "innerjoin", "inoutdriver", @@ -400,6 +422,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "input4", "input40", "input41", + "input49", "input4_cb_delim", "input6", "input7", @@ -481,6 +504,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "join_hive_626", "join_map_ppr", "join_nulls", + "join_nullsafe", "join_rc", "join_reorder2", "join_reorder3", @@ -491,6 +515,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "lateral_view_ppd", "leftsemijoin", "leftsemijoin_mr", + "limit_pushdown_negative", "lineage1", "literal_double", "literal_ints", @@ -598,6 +623,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "reduce_deduplicate", "reduce_deduplicate_exclude_gby", "reduce_deduplicate_exclude_join", + "reduce_deduplicate_extended", "reducesink_dedup", "rename_column", "router_join_ppr", @@ -609,6 +635,14 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "serde_regex", "serde_reported_schema", "set_variable_sub", + "show_create_table_partitioned", + "show_create_table_delimited", + "show_create_table_alter", + "show_create_table_view", + "show_create_table_serde", + "show_create_table_db_table", + "show_create_table_does_not_exist", + "show_create_table_index", "show_describe_func_quotes", "show_functions", "show_partitions", @@ -646,7 +680,13 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "stats_publisher_error_1", "subq2", "tablename_with_select", + "timestamp_1", + "timestamp_2", + "timestamp_3", "timestamp_comparison", + "timestamp_lazy", + "timestamp_null", + "timestamp_udf", "touch", "transform_ppr1", "transform_ppr2", @@ -663,8 +703,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "udf7", "udf8", "udf9", - "udf_E", - "udf_PI", "udf_abs", "udf_acos", "udf_add", @@ -704,6 +742,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "udf_double", "udf_E", "udf_elt", + "udf_equal", "udf_exp", "udf_field", "udf_find_in_set", @@ -786,6 +825,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "udf_translate", "udf_trim", "udf_ucase", + "udf_unix_timestamp", "udf_upper", "udf_var_pop", "udf_var_samp", diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 5ede76e5c3904..93d00f7c37c9b 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -31,6 +31,9 @@ jar Spark Project Hive http://spark.apache.org/ + + hive + @@ -48,6 +51,11 @@ hive-metastore ${hive.version} + + commons-httpclient + commons-httpclient + 3.1 + org.spark-project.hive hive-exec @@ -94,6 +102,36 @@ test + + + + hive + + + + org.codehaus.mojo + build-helper-maven-plugin + + + add-scala-test-sources + generate-test-sources + + add-test-source + + + + src/test/scala + compatibility/src/test/scala + + + + + + + + + + target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 7aedfcd74189b..a8da676ffa0e0 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -18,32 +18,42 @@ package org.apache.spark.sql.hive import java.io.{BufferedReader, File, InputStreamReader, PrintStream} +import java.sql.Timestamp import java.util.{ArrayList => JArrayList} import scala.collection.JavaConversions._ import scala.language.implicitConversions -import scala.reflect.runtime.universe.TypeTag +import scala.reflect.runtime.universe.{TypeTag, typeTag} +import org.apache.hadoop.fs.FileSystem +import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.Driver +import org.apache.hadoop.hive.ql.metadata.Table import org.apache.hadoop.hive.ql.processors._ import org.apache.hadoop.hive.ql.session.SessionState +import org.apache.hadoop.hive.ql.stats.StatsSetupConst +import org.apache.hadoop.hive.serde2.io.TimestampWritable import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.ScalaReflection -import org.apache.spark.sql.catalyst.analysis.{Analyzer, OverrideCatalog} +import org.apache.spark.sql.catalyst.analysis.{Analyzer, EliminateAnalysisOperators} +import org.apache.spark.sql.catalyst.analysis.{OverrideCatalog, OverrideFunctionRegistry} import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.execution.ExtractPythonUdfs import org.apache.spark.sql.execution.QueryExecutionException import org.apache.spark.sql.execution.{Command => PhysicalCommand} import org.apache.spark.sql.hive.execution.DescribeHiveTableCommand /** - * Starts up an instance of hive where metadata is stored locally. An in-process metadata data is - * created with data stored in ./metadata. Warehouse data is stored in in ./warehouse. + * DEPRECATED: Use HiveContext instead. */ +@deprecated(""" + Use HiveContext instead. It will still create a local metastore if one is not specified. + However, note that the default directory is ./metastore_db, not ./metastore + """, "1.1") class LocalHiveContext(sc: SparkContext) extends HiveContext(sc) { lazy val metastorePath = new File("metastore").getCanonicalPath @@ -51,9 +61,9 @@ class LocalHiveContext(sc: SparkContext) extends HiveContext(sc) { /** Sets up the system initially or after a RESET command */ protected def configure() { - set("javax.jdo.option.ConnectionURL", + setConf("javax.jdo.option.ConnectionURL", s"jdbc:derby:;databaseName=$metastorePath;create=true") - set("hive.metastore.warehouse.dir", warehousePath) + setConf("hive.metastore.warehouse.dir", warehousePath) } configure() // Must be called before initializing the catalog below. @@ -66,15 +76,29 @@ class LocalHiveContext(sc: SparkContext) extends HiveContext(sc) { class HiveContext(sc: SparkContext) extends SQLContext(sc) { self => + // Change the default SQL dialect to HiveQL + override private[spark] def dialect: String = getConf(SQLConf.DIALECT, "hiveql") + override protected[sql] def executePlan(plan: LogicalPlan): this.QueryExecution = new this.QueryExecution { val logical = plan } - /** - * Executes a query expressed in HiveQL using Spark, returning the result as a SchemaRDD. - */ + override def sql(sqlText: String): SchemaRDD = { + // TODO: Create a framework for registering parsers instead of just hardcoding if statements. + if (dialect == "sql") { + super.sql(sqlText) + } else if (dialect == "hiveql") { + new SchemaRDD(this, HiveQl.parseSql(sqlText)) + } else { + sys.error(s"Unsupported SQL dialect: $dialect. Try 'sql' or 'hiveql'") + } + } + + @deprecated("hiveql() is deprecated as the sql function now parses using HiveQL by default. " + + s"The SQL dialect for parsing can be set using ${SQLConf.DIALECT}", "1.1") def hiveql(hqlQuery: String): SchemaRDD = new SchemaRDD(this, HiveQl.parseSql(hqlQuery)) - /** An alias for `hiveql`. */ + @deprecated("hql() is deprecated as the sql function now parses using HiveQL by default. " + + s"The SQL dialect for parsing can be set using ${SQLConf.DIALECT}", "1.1") def hql(hqlQuery: String): SchemaRDD = hiveql(hqlQuery) /** @@ -88,9 +112,81 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { catalog.createTable("default", tableName, ScalaReflection.attributesFor[A], allowExisting) } + /** + * Analyzes the given table in the current database to generate statistics, which will be + * used in query optimizations. + * + * Right now, it only supports Hive tables and it only updates the size of a Hive table + * in the Hive metastore. + */ + def analyze(tableName: String) { + val relation = EliminateAnalysisOperators(catalog.lookupRelation(None, tableName)) + + relation match { + case relation: MetastoreRelation => { + // This method is mainly based on + // org.apache.hadoop.hive.ql.stats.StatsUtils.getFileSizeForTable(HiveConf, Table) + // in Hive 0.13 (except that we do not use fs.getContentSummary). + // TODO: Generalize statistics collection. + // TODO: Why fs.getContentSummary returns wrong size on Jenkins? + // Can we use fs.getContentSummary in future? + // Seems fs.getContentSummary returns wrong table size on Jenkins. So we use + // countFileSize to count the table size. + def calculateTableSize(fs: FileSystem, path: Path): Long = { + val fileStatus = fs.getFileStatus(path) + val size = if (fileStatus.isDir) { + fs.listStatus(path).map(status => calculateTableSize(fs, status.getPath)).sum + } else { + fileStatus.getLen + } + + size + } + + def getFileSizeForTable(conf: HiveConf, table: Table): Long = { + val path = table.getPath() + var size: Long = 0L + try { + val fs = path.getFileSystem(conf) + size = calculateTableSize(fs, path) + } catch { + case e: Exception => + logWarning( + s"Failed to get the size of table ${table.getTableName} in the " + + s"database ${table.getDbName} because of ${e.toString}", e) + size = 0L + } + + size + } + + val tableParameters = relation.hiveQlTable.getParameters + val oldTotalSize = + Option(tableParameters.get(StatsSetupConst.TOTAL_SIZE)).map(_.toLong).getOrElse(0L) + val newTotalSize = getFileSizeForTable(hiveconf, relation.hiveQlTable) + // Update the Hive metastore if the total size of the table is different than the size + // recorded in the Hive metastore. + // This logic is based on org.apache.hadoop.hive.ql.exec.StatsTask.aggregateStats(). + if (newTotalSize > 0 && newTotalSize != oldTotalSize) { + tableParameters.put(StatsSetupConst.TOTAL_SIZE, newTotalSize.toString) + val hiveTTable = relation.hiveQlTable.getTTable + hiveTTable.setParameters(tableParameters) + val tableFullName = + relation.hiveQlTable.getDbName() + "." + relation.hiveQlTable.getTableName() + + catalog.client.alterTable(tableFullName, new Table(hiveTTable)) + } + } + case otherRelation => + throw new NotImplementedError( + s"Analyze has only implemented for Hive tables, " + + s"but ${tableName} is a ${otherRelation.nodeName}") + } + } + // Circular buffer to hold what hive prints to STDOUT and ERR. Only printed when failures occur. @transient - protected val outputBuffer = new java.io.OutputStream { + protected lazy val outputBuffer = new java.io.OutputStream { var pos: Int = 0 var buffer = new Array[Int](10240) def write(i: Int): Unit = { @@ -120,21 +216,21 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { /** * SQLConf and HiveConf contracts: when the hive session is first initialized, params in * HiveConf will get picked up by the SQLConf. Additionally, any properties set by - * set() or a SET command inside hql() or sql() will be set in the SQLConf *as well as* + * set() or a SET command inside sql() will be set in the SQLConf *as well as* * in the HiveConf. */ @transient protected[hive] lazy val hiveconf = new HiveConf(classOf[SessionState]) @transient protected[hive] lazy val sessionState = { val ss = new SessionState(hiveconf) - set(hiveconf.getAllProperties) // Have SQLConf pick up the initial set of HiveConf. + setConf(hiveconf.getAllProperties) // Have SQLConf pick up the initial set of HiveConf. ss } sessionState.err = new PrintStream(outputBuffer, true, "UTF-8") sessionState.out = new PrintStream(outputBuffer, true, "UTF-8") - override def set(key: String, value: String): Unit = { - super.set(key, value) + override def setConf(key: String, value: String): Unit = { + super.setConf(key, value) runSqlHive(s"SET $key=$value") } @@ -150,10 +246,14 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { } } + // Note that HiveUDFs will be overridden by functions registered in this context. + override protected[sql] lazy val functionRegistry = + new HiveFunctionRegistry with OverrideFunctionRegistry + /* An analyzer that uses the Hive metastore. */ @transient override protected[sql] lazy val analyzer = - new Analyzer(catalog, HiveFunctionRegistry, caseSensitive = false) + new Analyzer(catalog, functionRegistry, caseSensitive = false) /** * Runs the specified SQL query using Hive. @@ -202,7 +302,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { } } catch { case e: Exception => - logger.error( + logError( s""" |====================== |HIVE FAILURE OUTPUT @@ -229,7 +329,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { HiveTableScans, DataSinks, Scripts, - PartialAggregation, + HashAggregation, LeftSemiJoin, HashJoin, BasicOperators, @@ -245,27 +345,29 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { protected[sql] abstract class QueryExecution extends super.QueryExecution { // TODO: Create mixin for the analyzer instead of overriding things here. override lazy val optimizedPlan = - optimizer(catalog.PreInsertionCasts(catalog.CreateTables(analyzed))) + optimizer(ExtractPythonUdfs(catalog.PreInsertionCasts(catalog.CreateTables(analyzed)))) override lazy val toRdd: RDD[Row] = executedPlan.execute().map(_.copy()) protected val primitiveTypes = Seq(StringType, IntegerType, LongType, DoubleType, FloatType, BooleanType, ByteType, - ShortType, DecimalType, TimestampType) + ShortType, DecimalType, TimestampType, BinaryType) - protected def toHiveString(a: (Any, DataType)): String = a match { + protected[sql] def toHiveString(a: (Any, DataType)): String = a match { case (struct: Row, StructType(fields)) => struct.zip(fields).map { case (v, t) => s""""${t.name}":${toHiveStructString(v, t.dataType)}""" }.mkString("{", ",", "}") - case (seq: Seq[_], ArrayType(typ)) => + case (seq: Seq[_], ArrayType(typ, _)) => seq.map(v => (v, typ)).map(toHiveStructString).mkString("[", ",", "]") - case (map: Map[_,_], MapType(kType, vType)) => + case (map: Map[_,_], MapType(kType, vType, _)) => map.map { case (key, value) => toHiveStructString((key, kType)) + ":" + toHiveStructString((value, vType)) }.toSeq.sorted.mkString("{", ",", "}") case (null, _) => "NULL" + case (t: Timestamp, TimestampType) => new TimestampWritable(t).toString + case (bin: Array[Byte], BinaryType) => new String(bin, "UTF-8") case (other, tpe) if primitiveTypes contains tpe => other.toString } @@ -275,9 +377,9 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { struct.zip(fields).map { case (v, t) => s""""${t.name}":${toHiveStructString(v, t.dataType)}""" }.mkString("{", ",", "}") - case (seq: Seq[_], ArrayType(typ)) => + case (seq: Seq[_], ArrayType(typ, _)) => seq.map(v => (v, typ)).map(toHiveStructString).mkString("[", ",", "]") - case (map: Map[_,_], MapType(kType, vType)) => + case (map: Map[_,_], MapType(kType, vType, _)) => map.map { case (key, value) => toHiveStructString((key, kType)) + ":" + toHiveStructString((value, vType)) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala new file mode 100644 index 0000000000000..943bbaa8ce25e --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala @@ -0,0 +1,243 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive + +import org.apache.hadoop.hive.common.`type`.HiveDecimal +import org.apache.hadoop.hive.serde2.objectinspector._ +import org.apache.hadoop.hive.serde2.objectinspector.primitive._ +import org.apache.hadoop.hive.serde2.{io => hiveIo} +import org.apache.hadoop.{io => hadoopIo} + +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.types +import org.apache.spark.sql.catalyst.types._ + +/* Implicit conversions */ +import scala.collection.JavaConversions._ + +private[hive] trait HiveInspectors { + + def javaClassToDataType(clz: Class[_]): DataType = clz match { + // writable + case c: Class[_] if c == classOf[hadoopIo.DoubleWritable] => DoubleType + case c: Class[_] if c == classOf[hiveIo.DoubleWritable] => DoubleType + case c: Class[_] if c == classOf[hiveIo.HiveDecimalWritable] => DecimalType + case c: Class[_] if c == classOf[hiveIo.ByteWritable] => ByteType + case c: Class[_] if c == classOf[hiveIo.ShortWritable] => ShortType + case c: Class[_] if c == classOf[hiveIo.TimestampWritable] => TimestampType + case c: Class[_] if c == classOf[hadoopIo.Text] => StringType + case c: Class[_] if c == classOf[hadoopIo.IntWritable] => IntegerType + case c: Class[_] if c == classOf[hadoopIo.LongWritable] => LongType + case c: Class[_] if c == classOf[hadoopIo.FloatWritable] => FloatType + case c: Class[_] if c == classOf[hadoopIo.BooleanWritable] => BooleanType + case c: Class[_] if c == classOf[hadoopIo.BytesWritable] => BinaryType + + // java class + case c: Class[_] if c == classOf[java.lang.String] => StringType + case c: Class[_] if c == classOf[java.sql.Timestamp] => TimestampType + case c: Class[_] if c == classOf[HiveDecimal] => DecimalType + case c: Class[_] if c == classOf[java.math.BigDecimal] => DecimalType + case c: Class[_] if c == classOf[Array[Byte]] => BinaryType + case c: Class[_] if c == classOf[java.lang.Short] => ShortType + case c: Class[_] if c == classOf[java.lang.Integer] => IntegerType + case c: Class[_] if c == classOf[java.lang.Long] => LongType + case c: Class[_] if c == classOf[java.lang.Double] => DoubleType + case c: Class[_] if c == classOf[java.lang.Byte] => ByteType + case c: Class[_] if c == classOf[java.lang.Float] => FloatType + case c: Class[_] if c == classOf[java.lang.Boolean] => BooleanType + + // primitive type + case c: Class[_] if c == java.lang.Short.TYPE => ShortType + case c: Class[_] if c == java.lang.Integer.TYPE => IntegerType + case c: Class[_] if c == java.lang.Long.TYPE => LongType + case c: Class[_] if c == java.lang.Double.TYPE => DoubleType + case c: Class[_] if c == java.lang.Byte.TYPE => ByteType + case c: Class[_] if c == java.lang.Float.TYPE => FloatType + case c: Class[_] if c == java.lang.Boolean.TYPE => BooleanType + + case c: Class[_] if c.isArray => ArrayType(javaClassToDataType(c.getComponentType)) + + // Hive seems to return this for struct types? + case c: Class[_] if c == classOf[java.lang.Object] => NullType + } + + /** Converts hive types to native catalyst types. */ + def unwrap(a: Any): Any = a match { + case null => null + case i: hadoopIo.IntWritable => i.get + case t: hadoopIo.Text => t.toString + case l: hadoopIo.LongWritable => l.get + case d: hadoopIo.DoubleWritable => d.get + case d: hiveIo.DoubleWritable => d.get + case s: hiveIo.ShortWritable => s.get + case b: hadoopIo.BooleanWritable => b.get + case b: hiveIo.ByteWritable => b.get + case b: hadoopIo.FloatWritable => b.get + case b: hadoopIo.BytesWritable => { + val bytes = new Array[Byte](b.getLength) + System.arraycopy(b.getBytes(), 0, bytes, 0, b.getLength) + bytes + } + case t: hiveIo.TimestampWritable => t.getTimestamp + case b: hiveIo.HiveDecimalWritable => BigDecimal(b.getHiveDecimal().bigDecimalValue()) + case list: java.util.List[_] => list.map(unwrap) + case map: java.util.Map[_,_] => map.map { case (k, v) => (unwrap(k), unwrap(v)) }.toMap + case array: Array[_] => array.map(unwrap).toSeq + case p: java.lang.Short => p + case p: java.lang.Long => p + case p: java.lang.Float => p + case p: java.lang.Integer => p + case p: java.lang.Double => p + case p: java.lang.Byte => p + case p: java.lang.Boolean => p + case str: String => str + case p: java.math.BigDecimal => p + case p: Array[Byte] => p + case p: java.sql.Timestamp => p + } + + def unwrapData(data: Any, oi: ObjectInspector): Any = oi match { + case hvoi: HiveVarcharObjectInspector => + if (data == null) null else hvoi.getPrimitiveJavaObject(data).getValue + case hdoi: HiveDecimalObjectInspector => + if (data == null) null else BigDecimal(hdoi.getPrimitiveJavaObject(data).bigDecimalValue()) + case pi: PrimitiveObjectInspector => pi.getPrimitiveJavaObject(data) + case li: ListObjectInspector => + Option(li.getList(data)) + .map(_.map(unwrapData(_, li.getListElementObjectInspector)).toSeq) + .orNull + case mi: MapObjectInspector => + Option(mi.getMap(data)).map( + _.map { + case (k,v) => + (unwrapData(k, mi.getMapKeyObjectInspector), + unwrapData(v, mi.getMapValueObjectInspector)) + }.toMap).orNull + case si: StructObjectInspector => + val allRefs = si.getAllStructFieldRefs + new GenericRow( + allRefs.map(r => + unwrapData(si.getStructFieldData(data,r), r.getFieldObjectInspector)).toArray) + } + + /** Converts native catalyst types to the types expected by Hive */ + def wrap(a: Any): AnyRef = a match { + case s: String => new hadoopIo.Text(s) // TODO why should be Text? + case i: Int => i: java.lang.Integer + case b: Boolean => b: java.lang.Boolean + case f: Float => f: java.lang.Float + case d: Double => d: java.lang.Double + case l: Long => l: java.lang.Long + case l: Short => l: java.lang.Short + case l: Byte => l: java.lang.Byte + case b: BigDecimal => b.bigDecimal + case b: Array[Byte] => b + case t: java.sql.Timestamp => t + case s: Seq[_] => seqAsJavaList(s.map(wrap)) + case m: Map[_,_] => + // Some UDFs seem to assume we pass in a HashMap. + val hashMap = new java.util.HashMap[AnyRef, AnyRef]() + hashMap.putAll(m.map { case (k, v) => wrap(k) -> wrap(v) }) + hashMap + case null => null + } + + def toInspector(dataType: DataType): ObjectInspector = dataType match { + case ArrayType(tpe, _) => + ObjectInspectorFactory.getStandardListObjectInspector(toInspector(tpe)) + case MapType(keyType, valueType, _) => + ObjectInspectorFactory.getStandardMapObjectInspector( + toInspector(keyType), toInspector(valueType)) + case StringType => PrimitiveObjectInspectorFactory.javaStringObjectInspector + case IntegerType => PrimitiveObjectInspectorFactory.javaIntObjectInspector + case DoubleType => PrimitiveObjectInspectorFactory.javaDoubleObjectInspector + case BooleanType => PrimitiveObjectInspectorFactory.javaBooleanObjectInspector + case LongType => PrimitiveObjectInspectorFactory.javaLongObjectInspector + case FloatType => PrimitiveObjectInspectorFactory.javaFloatObjectInspector + case ShortType => PrimitiveObjectInspectorFactory.javaShortObjectInspector + case ByteType => PrimitiveObjectInspectorFactory.javaByteObjectInspector + case NullType => PrimitiveObjectInspectorFactory.javaVoidObjectInspector + case BinaryType => PrimitiveObjectInspectorFactory.javaByteArrayObjectInspector + case TimestampType => PrimitiveObjectInspectorFactory.javaTimestampObjectInspector + case DecimalType => PrimitiveObjectInspectorFactory.javaHiveDecimalObjectInspector + case StructType(fields) => + ObjectInspectorFactory.getStandardStructObjectInspector( + fields.map(f => f.name), fields.map(f => toInspector(f.dataType))) + } + + def inspectorToDataType(inspector: ObjectInspector): DataType = inspector match { + case s: StructObjectInspector => + StructType(s.getAllStructFieldRefs.map(f => { + types.StructField( + f.getFieldName, inspectorToDataType(f.getFieldObjectInspector), nullable = true) + })) + case l: ListObjectInspector => ArrayType(inspectorToDataType(l.getListElementObjectInspector)) + case m: MapObjectInspector => + MapType( + inspectorToDataType(m.getMapKeyObjectInspector), + inspectorToDataType(m.getMapValueObjectInspector)) + case _: WritableStringObjectInspector => StringType + case _: JavaStringObjectInspector => StringType + case _: WritableIntObjectInspector => IntegerType + case _: JavaIntObjectInspector => IntegerType + case _: WritableDoubleObjectInspector => DoubleType + case _: JavaDoubleObjectInspector => DoubleType + case _: WritableBooleanObjectInspector => BooleanType + case _: JavaBooleanObjectInspector => BooleanType + case _: WritableLongObjectInspector => LongType + case _: JavaLongObjectInspector => LongType + case _: WritableShortObjectInspector => ShortType + case _: JavaShortObjectInspector => ShortType + case _: WritableByteObjectInspector => ByteType + case _: JavaByteObjectInspector => ByteType + case _: WritableFloatObjectInspector => FloatType + case _: JavaFloatObjectInspector => FloatType + case _: WritableBinaryObjectInspector => BinaryType + case _: JavaBinaryObjectInspector => BinaryType + case _: WritableHiveDecimalObjectInspector => DecimalType + case _: JavaHiveDecimalObjectInspector => DecimalType + case _: WritableTimestampObjectInspector => TimestampType + case _: JavaTimestampObjectInspector => TimestampType + } + + implicit class typeInfoConversions(dt: DataType) { + import org.apache.hadoop.hive.serde2.typeinfo._ + import TypeInfoFactory._ + + def toTypeInfo: TypeInfo = dt match { + case ArrayType(elemType, _) => + getListTypeInfo(elemType.toTypeInfo) + case StructType(fields) => + getStructTypeInfo(fields.map(_.name), fields.map(_.dataType.toTypeInfo)) + case MapType(keyType, valueType, _) => + getMapTypeInfo(keyType.toTypeInfo, valueType.toTypeInfo) + case BinaryType => binaryTypeInfo + case BooleanType => booleanTypeInfo + case ByteType => byteTypeInfo + case DoubleType => doubleTypeInfo + case FloatType => floatTypeInfo + case IntegerType => intTypeInfo + case LongType => longTypeInfo + case ShortType => shortTypeInfo + case StringType => stringTypeInfo + case DecimalType => decimalTypeInfo + case TimestampType => timestampTypeInfo + case NullType => voidTypeInfo + } + } +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index f83068860701f..3b371211e14cd 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -23,11 +23,12 @@ import org.apache.hadoop.hive.metastore.api.{FieldSchema, StorageDescriptor, Ser import org.apache.hadoop.hive.metastore.api.{Table => TTable, Partition => TPartition} import org.apache.hadoop.hive.ql.metadata.{Hive, Partition, Table} import org.apache.hadoop.hive.ql.plan.TableDesc -import org.apache.hadoop.hive.ql.session.SessionState +import org.apache.hadoop.hive.ql.stats.StatsSetupConst import org.apache.hadoop.hive.serde2.Deserializer import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.sql.Logging +import org.apache.spark.Logging +import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.analysis.{EliminateAnalysisOperators, Catalog} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical @@ -36,6 +37,7 @@ import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.columnar.InMemoryRelation import org.apache.spark.sql.hive.execution.HiveTableScan +import org.apache.spark.util.Utils /* Implicit conversions */ import scala.collection.JavaConversions._ @@ -43,14 +45,15 @@ import scala.collection.JavaConversions._ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with Logging { import HiveMetastoreTypes._ - val client = Hive.get(hive.hiveconf) + /** Connection to hive metastore. Usages should lock on `this`. */ + protected[hive] val client = Hive.get(hive.hiveconf) val caseSensitive: Boolean = false def lookupRelation( db: Option[String], tableName: String, - alias: Option[String]): LogicalPlan = { + alias: Option[String]): LogicalPlan = synchronized { val (dbName, tblName) = processDatabaseAndTableName(db, tableName) val databaseName = dbName.getOrElse(hive.sessionState.getCurrentDatabase) val table = client.getTable(databaseName, tblName) @@ -63,9 +66,8 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with // Since HiveQL is case insensitive for table names we make them all lowercase. MetastoreRelation( - databaseName, - tblName, - alias)(table.getTTable, partitions.map(part => part.getTPartition)) + databaseName, tblName, alias)( + table.getTTable, partitions.map(part => part.getTPartition))(hive) } def createTable( @@ -135,7 +137,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with castChildOutput(p, table, child) case p @ logical.InsertIntoTable( - InMemoryRelation(_, _, + InMemoryRelation(_, _, _, HiveTableScan(_, table, _)), _, child, _) => castChildOutput(p, table, child) } @@ -199,7 +201,9 @@ object HiveMetastoreTypes extends RegexParsers { "varchar\\((\\d+)\\)".r ^^^ StringType protected lazy val arrayType: Parser[DataType] = - "array" ~> "<" ~> dataType <~ ">" ^^ ArrayType + "array" ~> "<" ~> dataType <~ ">" ^^ { + case tpe => ArrayType(tpe) + } protected lazy val mapType: Parser[DataType] = "map" ~> "<" ~> dataType ~ "," ~ dataType <~ ">" ^^ { @@ -228,10 +232,10 @@ object HiveMetastoreTypes extends RegexParsers { } def toMetastoreType(dt: DataType): String = dt match { - case ArrayType(elementType) => s"array<${toMetastoreType(elementType)}>" + case ArrayType(elementType, _) => s"array<${toMetastoreType(elementType)}>" case StructType(fields) => s"struct<${fields.map(f => s"${f.name}:${toMetastoreType(f.dataType)}").mkString(",")}>" - case MapType(keyType, valueType) => + case MapType(keyType, valueType, _) => s"map<${toMetastoreType(keyType)},${toMetastoreType(valueType)}>" case StringType => "string" case FloatType => "float" @@ -250,21 +254,43 @@ object HiveMetastoreTypes extends RegexParsers { private[hive] case class MetastoreRelation (databaseName: String, tableName: String, alias: Option[String]) (val table: TTable, val partitions: Seq[TPartition]) - extends BaseRelation { + (@transient sqlContext: SQLContext) + extends LeafNode { + + self: Product => + // TODO: Can we use org.apache.hadoop.hive.ql.metadata.Table as the type of table and // use org.apache.hadoop.hive.ql.metadata.Partition as the type of elements of partitions. // Right now, using org.apache.hadoop.hive.ql.metadata.Table and // org.apache.hadoop.hive.ql.metadata.Partition will cause a NotSerializableException // which indicates the SerDe we used is not Serializable. - def hiveQlTable = new Table(table) + @transient lazy val hiveQlTable = new Table(table) def hiveQlPartitions = partitions.map { p => new Partition(hiveQlTable, p) } + @transient override lazy val statistics = Statistics( + sizeInBytes = { + // TODO: check if this estimate is valid for tables after partition pruning. + // NOTE: getting `totalSize` directly from params is kind of hacky, but this should be + // relatively cheap if parameters for the table are populated into the metastore. An + // alternative would be going through Hadoop's FileSystem API, which can be expensive if a lot + // of RPCs are involved. Besides `totalSize`, there are also `numFiles`, `numRows`, + // `rawDataSize` keys (see StatsSetupConst in Hive) that we can look at in the future. + BigInt( + Option(hiveQlTable.getParameters.get(StatsSetupConst.TOTAL_SIZE)) + .map(_.toLong) + .getOrElse(sqlContext.defaultSizeInBytes)) + } + ) + val tableDesc = new TableDesc( - Class.forName(hiveQlTable.getSerializationLib).asInstanceOf[Class[Deserializer]], + Class.forName( + hiveQlTable.getSerializationLib, + true, + Utils.getContextOrSparkClassLoader).asInstanceOf[Class[Deserializer]], hiveQlTable.getInputFormatClass, // The class of table should be org.apache.hadoop.hive.ql.metadata.Table because // getOutputFormatClass will use HiveFileFormatUtils.getOutputFormatSubstitute to @@ -274,14 +300,14 @@ private[hive] case class MetastoreRelation hiveQlTable.getMetadata ) - implicit class SchemaAttribute(f: FieldSchema) { - def toAttribute = AttributeReference( - f.getName, - HiveMetastoreTypes.toDataType(f.getType), - // Since data can be dumped in randomly with no validation, everything is nullable. - nullable = true - )(qualifiers = tableName +: alias.toSeq) - } + implicit class SchemaAttribute(f: FieldSchema) { + def toAttribute = AttributeReference( + f.getName, + HiveMetastoreTypes.toDataType(f.getType), + // Since data can be dumped in randomly with no validation, everything is nullable. + nullable = true + )(qualifiers = tableName +: alias.toSeq) + } // Must be a stable value since new attributes are born here. val partitionKeys = hiveQlTable.getPartitionKeys.map(_.toAttribute) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index b70104dd5be5a..1d9ba1b24a7a4 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -42,15 +42,18 @@ private[hive] case class ShellCommand(cmd: String) extends Command private[hive] case class SourceCommand(filePath: String) extends Command -private[hive] case class AddJar(jarPath: String) extends Command - private[hive] case class AddFile(filePath: String) extends Command +private[hive] case class DropTable(tableName: String, ifExists: Boolean) extends Command + +private[hive] case class AnalyzeTable(tableName: String) extends Command + /** Provides a mapping from HiveQL statements to catalyst logical plans and expression trees. */ private[hive] object HiveQl { protected val nativeCommands = Seq( "TOK_DESCFUNCTION", "TOK_DESCDATABASE", + "TOK_SHOW_CREATETABLE", "TOK_SHOW_TABLESTATUS", "TOK_SHOWDATABASES", "TOK_SHOWFUNCTIONS", @@ -74,7 +77,6 @@ private[hive] object HiveQl { "TOK_CREATEFUNCTION", "TOK_DROPFUNCTION", - "TOK_ANALYZE", "TOK_ALTERDATABASE_PROPERTIES", "TOK_ALTERINDEX_PROPERTIES", "TOK_ALTERINDEX_REBUILD", @@ -92,13 +94,11 @@ private[hive] object HiveQl { "TOK_ALTERTABLE_SKEWED", "TOK_ALTERTABLE_TOUCH", "TOK_ALTERTABLE_UNARCHIVE", - "TOK_ANALYZE", "TOK_CREATEDATABASE", "TOK_CREATEFUNCTION", "TOK_CREATEINDEX", "TOK_DROPDATABASE", "TOK_DROPINDEX", - "TOK_DROPTABLE", "TOK_MSCK", // TODO(marmbrus): Figure out how view are expanded by hive, as we might need to handle this. @@ -229,7 +229,7 @@ private[hive] object HiveQl { } else if (sql.trim.toLowerCase.startsWith("uncache table")) { CacheCommand(sql.trim.drop(14).trim, false) } else if (sql.trim.toLowerCase.startsWith("add jar")) { - AddJar(sql.trim.drop(8)) + NativeCommand(sql) } else if (sql.trim.toLowerCase.startsWith("add file")) { AddFile(sql.trim.drop(9)) } else if (sql.trim.toLowerCase.startsWith("dfs")) { @@ -240,7 +240,6 @@ private[hive] object HiveQl { ShellCommand(sql.drop(1)) } else { val tree = getAst(sql) - if (nativeCommands contains tree.getText) { NativeCommand(sql) } else { @@ -298,8 +297,11 @@ private[hive] object HiveQl { matches.headOption } - assert(remainingNodes.isEmpty, - s"Unhandled clauses: ${remainingNodes.map(dumpTree(_)).mkString("\n")}") + if (remainingNodes.nonEmpty) { + sys.error( + s"""Unhandled clauses: ${remainingNodes.map(dumpTree(_)).mkString("\n")}. + |You are likely trying to use an unsupported Hive feature."""".stripMargin) + } clauses } @@ -379,6 +381,28 @@ private[hive] object HiveQl { } protected def nodeToPlan(node: Node): LogicalPlan = node match { + // Special drop table that also uncaches. + case Token("TOK_DROPTABLE", + Token("TOK_TABNAME", tableNameParts) :: + ifExists) => + val tableName = tableNameParts.map { case Token(p, Nil) => p }.mkString(".") + DropTable(tableName, ifExists.nonEmpty) + // Support "ANALYZE TABLE tableNmae COMPUTE STATISTICS noscan" + case Token("TOK_ANALYZE", + Token("TOK_TAB", Token("TOK_TABNAME", tableNameParts) :: partitionSpec) :: + isNoscan) => + // Reference: + // https://cwiki.apache.org/confluence/display/Hive/StatsDev#StatsDev-ExistingTables + if (partitionSpec.nonEmpty) { + // Analyze partitions will be treated as a Hive native command. + NativePlaceholder + } else if (isNoscan.isEmpty) { + // If users do not specify "noscan", it will be treated as a Hive native command. + NativePlaceholder + } else { + val tableName = tableNameParts.map { case Token(p, Nil) => p }.mkString(".") + AnalyzeTable(tableName) + } // Just fake explain for any of the native commands. case Token("TOK_EXPLAIN", explainArgs) if noExplainCommands.contains(explainArgs.head.getText) => @@ -612,7 +636,7 @@ private[hive] object HiveQl { // TOK_DESTINATION means to overwrite the table. val resultDestination = (intoClause orElse destClause).getOrElse(sys.error("No destination found.")) - val overwrite = if (intoClause.isEmpty) true else false + val overwrite = intoClause.isEmpty nodeToDest( resultDestination, withLimit, @@ -743,7 +767,10 @@ private[hive] object HiveQl { case Token(allJoinTokens(joinToken), relation1 :: relation2 :: other) => - assert(other.size <= 1, s"Unhandled join child $other") + if (!(other.size <= 1)) { + sys.error(s"Unsupported join operation: $other") + } + val joinType = joinToken match { case "TOK_JOIN" => Inner case "TOK_RIGHTOUTERJOIN" => RightOuter @@ -751,7 +778,6 @@ private[hive] object HiveQl { case "TOK_FULLOUTERJOIN" => FullOuter case "TOK_LEFTSEMIJOIN" => LeftSemi } - assert(other.size <= 1, "Unhandled join clauses.") Join(nodeToRelation(relation1), nodeToRelation(relation2), joinType, @@ -860,6 +886,7 @@ private[hive] object HiveQl { val BETWEEN = "(?i)BETWEEN".r val WHEN = "(?i)WHEN".r val CASE = "(?i)CASE".r + val SUBSTR = "(?i)SUBSTR(?:ING)?".r protected def nodeToExpr(node: Node): Expression = node match { /* Attribute References */ @@ -870,10 +897,7 @@ private[hive] object HiveQl { nodeToExpr(qualifier) match { case UnresolvedAttribute(qualifierName) => UnresolvedAttribute(qualifierName + "." + cleanIdentifier(attr)) - // The precidence for . seems to be wrong, so [] binds tighter an we need to go inside to - // find the underlying attribute references. - case GetItem(UnresolvedAttribute(qualifierName), ordinal) => - GetItem(UnresolvedAttribute(qualifierName + "." + cleanIdentifier(attr)), ordinal) + case other => GetField(other, attr) } /* Stars (*) */ @@ -929,11 +953,14 @@ private[hive] object HiveQl { case Token("-", left :: right:: Nil) => Subtract(nodeToExpr(left), nodeToExpr(right)) case Token("*", left :: right:: Nil) => Multiply(nodeToExpr(left), nodeToExpr(right)) case Token("/", left :: right:: Nil) => Divide(nodeToExpr(left), nodeToExpr(right)) - case Token(DIV(), left :: right:: Nil) => Divide(nodeToExpr(left), nodeToExpr(right)) + case Token(DIV(), left :: right:: Nil) => + Cast(Divide(nodeToExpr(left), nodeToExpr(right)), LongType) case Token("%", left :: right:: Nil) => Remainder(nodeToExpr(left), nodeToExpr(right)) /* Comparisons */ case Token("=", left :: right:: Nil) => EqualTo(nodeToExpr(left), nodeToExpr(right)) + case Token("==", left :: right:: Nil) => EqualTo(nodeToExpr(left), nodeToExpr(right)) + case Token("<=>", left :: right:: Nil) => EqualNullSafe(nodeToExpr(left), nodeToExpr(right)) case Token("!=", left :: right:: Nil) => Not(EqualTo(nodeToExpr(left), nodeToExpr(right))) case Token("<>", left :: right:: Nil) => Not(EqualTo(nodeToExpr(left), nodeToExpr(right))) case Token(">", left :: right:: Nil) => GreaterThan(nodeToExpr(left), nodeToExpr(right)) @@ -987,6 +1014,10 @@ private[hive] object HiveQl { /* Other functions */ case Token("TOK_FUNCTION", Token(RAND(), Nil) :: Nil) => Rand + case Token("TOK_FUNCTION", Token(SUBSTR(), Nil) :: string :: pos :: Nil) => + Substring(nodeToExpr(string), nodeToExpr(pos), Literal(Integer.MAX_VALUE, IntegerType)) + case Token("TOK_FUNCTION", Token(SUBSTR(), Nil) :: string :: pos :: length :: Nil) => + Substring(nodeToExpr(string), nodeToExpr(pos), nodeToExpr(length)) /* UDFs - Must be last otherwise will preempt built in functions */ case Token("TOK_FUNCTION", Token(name, Nil) :: args) => diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index 4d0fab4140b21..5fcc1bd4b9adf 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -45,7 +45,7 @@ private[hive] trait HiveStrategies { case logical.InsertIntoTable(table: MetastoreRelation, partition, child, overwrite) => InsertIntoHiveTable(table, partition, planLater(child), overwrite)(hiveContext) :: Nil case logical.InsertIntoTable( - InMemoryRelation(_, _, + InMemoryRelation(_, _, _, HiveTableScan(_, table, _)), partition, child, overwrite) => InsertIntoHiveTable(table, partition, planLater(child), overwrite)(hiveContext) :: Nil case _ => Nil @@ -81,6 +81,10 @@ private[hive] trait HiveStrategies { case logical.NativeCommand(sql) => NativeCommand(sql, plan.output)(context) :: Nil + case DropTable(tableName, ifExists) => execution.DropTable(tableName, ifExists) :: Nil + + case AnalyzeTable(tableName) => execution.AnalyzeTable(tableName) :: Nil + case describe: logical.DescribeCommand => val resolvedTable = context.executePlan(describe.table).analyzed resolvedTable match { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala index 8cfde46186ca4..82c88280d7754 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala @@ -24,6 +24,8 @@ import org.apache.hadoop.hive.ql.exec.Utilities import org.apache.hadoop.hive.ql.metadata.{Partition => HivePartition, Table => HiveTable} import org.apache.hadoop.hive.ql.plan.TableDesc import org.apache.hadoop.hive.serde2.Deserializer +import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector + import org.apache.hadoop.io.Writable import org.apache.hadoop.mapred.{FileInputFormat, InputFormat, JobConf} @@ -31,13 +33,16 @@ import org.apache.spark.SerializableWritable import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.{EmptyRDD, HadoopRDD, RDD, UnionRDD} +import org.apache.spark.sql.catalyst.expressions.{Attribute, Row, GenericMutableRow, Literal, Cast} +import org.apache.spark.sql.catalyst.types.DataType + /** * A trait for subclasses that handle table scans. */ private[hive] sealed trait TableReader { - def makeRDDForTable(hiveTable: HiveTable): RDD[_] + def makeRDDForTable(hiveTable: HiveTable): RDD[Row] - def makeRDDForPartitionedTable(partitions: Seq[HivePartition]): RDD[_] + def makeRDDForPartitionedTable(partitions: Seq[HivePartition]): RDD[Row] } @@ -46,7 +51,10 @@ private[hive] sealed trait TableReader { * data warehouse directory. */ private[hive] -class HadoopTableReader(@transient _tableDesc: TableDesc, @transient sc: HiveContext) +class HadoopTableReader( + @transient attributes: Seq[Attribute], + @transient relation: MetastoreRelation, + @transient sc: HiveContext) extends TableReader { // Choose the minimum number of splits. If mapred.map.tasks is set, then use that unless @@ -63,10 +71,10 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient sc: HiveCon def hiveConf = _broadcastedHiveConf.value.value - override def makeRDDForTable(hiveTable: HiveTable): RDD[_] = + override def makeRDDForTable(hiveTable: HiveTable): RDD[Row] = makeRDDForTable( hiveTable, - _tableDesc.getDeserializerClass.asInstanceOf[Class[Deserializer]], + relation.tableDesc.getDeserializerClass.asInstanceOf[Class[Deserializer]], filterOpt = None) /** @@ -81,14 +89,14 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient sc: HiveCon def makeRDDForTable( hiveTable: HiveTable, deserializerClass: Class[_ <: Deserializer], - filterOpt: Option[PathFilter]): RDD[_] = { + filterOpt: Option[PathFilter]): RDD[Row] = { assert(!hiveTable.isPartitioned, """makeRDDForTable() cannot be called on a partitioned table, since input formats may differ across partitions. Use makeRDDForTablePartitions() instead.""") // Create local references to member variables, so that the entire `this` object won't be // serialized in the closure below. - val tableDesc = _tableDesc + val tableDesc = relation.tableDesc val broadcastedHiveConf = _broadcastedHiveConf val tablePath = hiveTable.getPath @@ -99,23 +107,20 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient sc: HiveCon .asInstanceOf[java.lang.Class[InputFormat[Writable, Writable]]] val hadoopRDD = createHadoopRdd(tableDesc, inputPathStr, ifc) + val attrsWithIndex = attributes.zipWithIndex + val mutableRow = new GenericMutableRow(attrsWithIndex.length) val deserializedHadoopRDD = hadoopRDD.mapPartitions { iter => val hconf = broadcastedHiveConf.value.value val deserializer = deserializerClass.newInstance() deserializer.initialize(hconf, tableDesc.getProperties) - // Deserialize each Writable to get the row value. - iter.map { - case v: Writable => deserializer.deserialize(v) - case value => - sys.error(s"Unable to deserialize non-Writable: $value of ${value.getClass.getName}") - } + HadoopTableReader.fillObject(iter, deserializer, attrsWithIndex, mutableRow) } deserializedHadoopRDD } - override def makeRDDForPartitionedTable(partitions: Seq[HivePartition]): RDD[_] = { + override def makeRDDForPartitionedTable(partitions: Seq[HivePartition]): RDD[Row] = { val partitionToDeserializer = partitions.map(part => (part, part.getDeserializer.getClass.asInstanceOf[Class[Deserializer]])).toMap makeRDDForPartitionedTable(partitionToDeserializer, filterOpt = None) @@ -132,9 +137,9 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient sc: HiveCon * subdirectory of each partition being read. If None, then all files are accepted. */ def makeRDDForPartitionedTable( - partitionToDeserializer: Map[HivePartition, Class[_ <: Deserializer]], - filterOpt: Option[PathFilter]): RDD[_] = { - + partitionToDeserializer: Map[HivePartition, + Class[_ <: Deserializer]], + filterOpt: Option[PathFilter]): RDD[Row] = { val hivePartitionRDDs = partitionToDeserializer.map { case (partition, partDeserializer) => val partDesc = Utilities.getPartitionDesc(partition) val partPath = partition.getPartitionPath @@ -156,29 +161,42 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient sc: HiveCon } // Create local references so that the outer object isn't serialized. - val tableDesc = _tableDesc + val tableDesc = relation.tableDesc val broadcastedHiveConf = _broadcastedHiveConf val localDeserializer = partDeserializer + val mutableRow = new GenericMutableRow(attributes.length) + + // split the attributes (output schema) into 2 categories: + // (partition keys, ordinal), (normal attributes, ordinal), the ordinal mean the + // index of the attribute in the output Row. + val (partitionKeys, attrs) = attributes.zipWithIndex.partition(attr => { + relation.partitionKeys.indexOf(attr._1) >= 0 + }) + + def fillPartitionKeys(parts: Array[String], row: GenericMutableRow) = { + partitionKeys.foreach { case (attr, ordinal) => + // get partition key ordinal for a given attribute + val partOridinal = relation.partitionKeys.indexOf(attr) + row(ordinal) = Cast(Literal(parts(partOridinal)), attr.dataType).eval(null) + } + } + // fill the partition key for the given MutableRow Object + fillPartitionKeys(partValues, mutableRow) val hivePartitionRDD = createHadoopRdd(tableDesc, inputPathStr, ifc) hivePartitionRDD.mapPartitions { iter => val hconf = broadcastedHiveConf.value.value - val rowWithPartArr = new Array[Object](2) - // Map each tuple to a row object - iter.map { value => - val deserializer = localDeserializer.newInstance() - deserializer.initialize(hconf, partProps) - val deserializedRow = deserializer.deserialize(value) - rowWithPartArr.update(0, deserializedRow) - rowWithPartArr.update(1, partValues) - rowWithPartArr.asInstanceOf[Object] - } + val deserializer = localDeserializer.newInstance() + deserializer.initialize(hconf, partProps) + + // fill the non partition key attributes + HadoopTableReader.fillObject(iter, deserializer, attrs, mutableRow) } }.toSeq // Even if we don't use any partitions, we still need an empty RDD if (hivePartitionRDDs.size == 0) { - new EmptyRDD[Object](sc.sparkContext) + new EmptyRDD[Row](sc.sparkContext) } else { new UnionRDD(hivePartitionRDDs(0).context, hivePartitionRDDs) } @@ -221,10 +239,9 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient sc: HiveCon // Only take the value (skip the key) because Hive works only with values. rdd.map(_._2) } - } -private[hive] object HadoopTableReader { +private[hive] object HadoopTableReader extends HiveInspectors { /** * Curried. After given an argument for 'path', the resulting JobConf => Unit closure is used to * instantiate a HadoopRDD. @@ -237,4 +254,40 @@ private[hive] object HadoopTableReader { val bufferSize = System.getProperty("spark.buffer.size", "65536") jobConf.set("io.file.buffer.size", bufferSize) } + + /** + * Transform the raw data(Writable object) into the Row object for an iterable input + * @param iter Iterable input which represented as Writable object + * @param deserializer Deserializer associated with the input writable object + * @param attrs Represents the row attribute names and its zero-based position in the MutableRow + * @param row reusable MutableRow object + * + * @return Iterable Row object that transformed from the given iterable input. + */ + def fillObject( + iter: Iterator[Writable], + deserializer: Deserializer, + attrs: Seq[(Attribute, Int)], + row: GenericMutableRow): Iterator[Row] = { + val soi = deserializer.getObjectInspector().asInstanceOf[StructObjectInspector] + // get the field references according to the attributes(output of the reader) required + val fieldRefs = attrs.map { case (attr, idx) => (soi.getStructFieldRef(attr.name), idx) } + + // Map each tuple to a row object + iter.map { value => + val raw = deserializer.deserialize(value) + var idx = 0; + while (idx < fieldRefs.length) { + val fieldRef = fieldRefs(idx)._1 + val fieldIdx = fieldRefs(idx)._2 + val fieldValue = soi.getStructFieldData(raw, fieldRef) + + row(fieldIdx) = unwrapData(fieldValue, fieldRef.getFieldObjectInspector()) + + idx += 1 + } + + row: Row + } + } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala index 9386008d02d51..a013f3f7a805f 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala @@ -53,15 +53,31 @@ object TestHive * hive metastore seems to lead to weird non-deterministic failures. Therefore, the execution of * test cases that rely on TestHive must be serialized. */ -class TestHiveContext(sc: SparkContext) extends LocalHiveContext(sc) { +class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { self => // By clearing the port we force Spark to pick a new one. This allows us to rerun tests // without restarting the JVM. System.clearProperty("spark.hostPort") - override lazy val warehousePath = getTempFilePath("sparkHiveWarehouse").getCanonicalPath - override lazy val metastorePath = getTempFilePath("sparkHiveMetastore").getCanonicalPath + lazy val warehousePath = getTempFilePath("sparkHiveWarehouse").getCanonicalPath + lazy val metastorePath = getTempFilePath("sparkHiveMetastore").getCanonicalPath + + /** Sets up the system initially or after a RESET command */ + protected def configure() { + setConf("javax.jdo.option.ConnectionURL", + s"jdbc:derby:;databaseName=$metastorePath;create=true") + setConf("hive.metastore.warehouse.dir", warehousePath) + } + + val testTempDir = File.createTempFile("testTempFiles", "spark.hive.tmp") + testTempDir.delete() + testTempDir.mkdir() + + // For some hive test case which contain ${system:test.tmp.dir} + System.setProperty("test.tmp.dir", testTempDir.getCanonicalPath) + + configure() // Must be called before initializing the catalog below. /** The location of the compiled hive distribution */ lazy val hiveHome = envVarToFile("HIVE_HOME") @@ -100,6 +116,7 @@ class TestHiveContext(sc: SparkContext) extends LocalHiveContext(sc) { hiveFilesTemp.mkdir() hiveFilesTemp.deleteOnExit() + val inRepoTests = if (System.getProperty("user.dir").endsWith("sql" + File.separator + "hive")) { new File("src" + File.separator + "test" + File.separator + "resources" + File.separator) } else { @@ -139,7 +156,7 @@ class TestHiveContext(sc: SparkContext) extends LocalHiveContext(sc) { describedTables ++ logical.collect { case UnresolvedRelation(databaseName, name, _) => name } val referencedTestTables = referencedTables.filter(testTables.contains) - logger.debug(s"Query references test tables: ${referencedTestTables.mkString(", ")}") + logDebug(s"Query references test tables: ${referencedTestTables.mkString(", ")}") referencedTestTables.foreach(loadTestTable) // Proceed with analysis. analyzer(logical) @@ -264,7 +281,7 @@ class TestHiveContext(sc: SparkContext) extends LocalHiveContext(sc) { if (!(loadedTables contains name)) { // Marks the table as loaded first to prevent infite mutually recursive table loading. loadedTables += name - logger.info(s"Loading test table $name") + logInfo(s"Loading test table $name") val createCmds = testTables.get(name).map(_.commands).getOrElse(sys.error(s"Unknown test table $name")) createCmds.foreach(_()) @@ -288,8 +305,8 @@ class TestHiveContext(sc: SparkContext) extends LocalHiveContext(sc) { def reset() { try { // HACK: Hive is too noisy by default. - org.apache.log4j.LogManager.getCurrentLoggers.foreach { logger => - logger.asInstanceOf[org.apache.log4j.Logger].setLevel(org.apache.log4j.Level.WARN) + org.apache.log4j.LogManager.getCurrentLoggers.foreach { log => + log.asInstanceOf[org.apache.log4j.Logger].setLevel(org.apache.log4j.Level.WARN) } // It is important that we RESET first as broken hooks that might have been set could break @@ -303,7 +320,7 @@ class TestHiveContext(sc: SparkContext) extends LocalHiveContext(sc) { loadedTables.clear() catalog.client.getAllTables("default").foreach { t => - logger.debug(s"Deleting table $t") + logDebug(s"Deleting table $t") val table = catalog.client.getTable("default", t) catalog.client.getIndexes("default", t, 255).foreach { index => @@ -316,7 +333,7 @@ class TestHiveContext(sc: SparkContext) extends LocalHiveContext(sc) { } catalog.client.getAllDatabases.filterNot(_ == "default").foreach { db => - logger.debug(s"Dropping Database: $db") + logDebug(s"Dropping Database: $db") catalog.client.dropDatabase(db, true, false, true) } @@ -338,7 +355,7 @@ class TestHiveContext(sc: SparkContext) extends LocalHiveContext(sc) { loadTestTable("srcpart") } catch { case e: Exception => - logger.error(s"FATAL ERROR: Failed to reset TestDB state. $e") + logError(s"FATAL ERROR: Failed to reset TestDB state. $e") // At this point there is really no reason to continue, but the test framework traps exits. // So instead we just pause forever so that at least the developer can see where things // started to go wrong. diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/api/java/JavaHiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/api/java/JavaHiveContext.scala index c9ee162191c96..a201d2349a2ef 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/api/java/JavaHiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/api/java/JavaHiveContext.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.hive.api.java import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.sql.api.java.{JavaSQLContext, JavaSchemaRDD} +import org.apache.spark.sql.SQLConf import org.apache.spark.sql.hive.{HiveContext, HiveQl} /** @@ -28,9 +29,21 @@ class JavaHiveContext(sparkContext: JavaSparkContext) extends JavaSQLContext(spa override val sqlContext = new HiveContext(sparkContext) + override def sql(sqlText: String): JavaSchemaRDD = { + // TODO: Create a framework for registering parsers instead of just hardcoding if statements. + if (sqlContext.dialect == "sql") { + super.sql(sqlText) + } else if (sqlContext.dialect == "hiveql") { + new JavaSchemaRDD(sqlContext, HiveQl.parseSql(sqlText)) + } else { + sys.error(s"Unsupported SQL dialect: ${sqlContext.dialect}. Try 'sql' or 'hiveql'") + } + } + /** - * Executes a query expressed in HiveQL, returning the result as a JavaSchemaRDD. + * DEPRECATED: Use sql(...) Instead */ + @Deprecated def hql(hqlQuery: String): JavaSchemaRDD = new JavaSchemaRDD(sqlContext, HiveQl.parseSql(hqlQuery)) } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala index ef8bae74530ec..577ca928b43b6 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala @@ -34,7 +34,6 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.types.{BooleanType, DataType} import org.apache.spark.sql.execution._ import org.apache.spark.sql.hive._ -import org.apache.spark.util.MutablePair /** * :: DeveloperApi :: @@ -50,8 +49,7 @@ case class HiveTableScan( relation: MetastoreRelation, partitionPruningPred: Option[Expression])( @transient val context: HiveContext) - extends LeafNode - with HiveInspectors { + extends LeafNode { require(partitionPruningPred.isEmpty || relation.hiveQlTable.isPartitioned, "Partition pruning predicates only supported for partitioned tables.") @@ -67,68 +65,19 @@ case class HiveTableScan( } @transient - private[this] val hadoopReader = new HadoopTableReader(relation.tableDesc, context) - - /** - * The hive object inspector for this table, which can be used to extract values from the - * serialized row representation. - */ - @transient - private[this] lazy val objectInspector = - relation.tableDesc.getDeserializer.getObjectInspector.asInstanceOf[StructObjectInspector] - - /** - * Functions that extract the requested attributes from the hive output. Partitioned values are - * casted from string to its declared data type. - */ - @transient - protected lazy val attributeFunctions: Seq[(Any, Array[String]) => Any] = { - attributes.map { a => - val ordinal = relation.partitionKeys.indexOf(a) - if (ordinal >= 0) { - val dataType = relation.partitionKeys(ordinal).dataType - (_: Any, partitionKeys: Array[String]) => { - castFromString(partitionKeys(ordinal), dataType) - } - } else { - val ref = objectInspector.getAllStructFieldRefs - .find(_.getFieldName == a.name) - .getOrElse(sys.error(s"Can't find attribute $a")) - val fieldObjectInspector = ref.getFieldObjectInspector - - val unwrapHiveData = fieldObjectInspector match { - case _: HiveVarcharObjectInspector => - (value: Any) => value.asInstanceOf[HiveVarchar].getValue - case _: HiveDecimalObjectInspector => - (value: Any) => BigDecimal(value.asInstanceOf[HiveDecimal].bigDecimalValue()) - case _ => - identity[Any] _ - } - - (row: Any, _: Array[String]) => { - val data = objectInspector.getStructFieldData(row, ref) - val hiveData = unwrapData(data, fieldObjectInspector) - if (hiveData != null) unwrapHiveData(hiveData) else null - } - } - } - } + private[this] val hadoopReader = new HadoopTableReader(attributes, relation, context) private[this] def castFromString(value: String, dataType: DataType) = { Cast(Literal(value), dataType).eval(null) } private def addColumnMetadataToConf(hiveConf: HiveConf) { - // Specifies IDs and internal names of columns to be scanned. - val neededColumnIDs = attributes.map(a => relation.output.indexWhere(_.name == a.name): Integer) - val columnInternalNames = neededColumnIDs.map(HiveConf.getColumnInternalName(_)).mkString(",") - - if (attributes.size == relation.output.size) { - ColumnProjectionUtils.setFullyReadColumns(hiveConf) - } else { - ColumnProjectionUtils.appendReadColumnIDs(hiveConf, neededColumnIDs) - } + // Specifies needed column IDs for those non-partitioning columns. + val neededColumnIDs = + attributes.map(a => + relation.attributes.indexWhere(_.name == a.name): Integer).filter(index => index >= 0) + ColumnProjectionUtils.appendReadColumnIDs(hiveConf, neededColumnIDs) ColumnProjectionUtils.appendReadColumnNames(hiveConf, attributes.map(_.name)) // Specifies types and object inspectors of columns to be scanned. @@ -145,17 +94,11 @@ case class HiveTableScan( .mkString(",") hiveConf.set(serdeConstants.LIST_COLUMN_TYPES, columnTypeNames) - hiveConf.set(serdeConstants.LIST_COLUMNS, columnInternalNames) + hiveConf.set(serdeConstants.LIST_COLUMNS, relation.attributes.map(_.name).mkString(",")) } addColumnMetadataToConf(context.hiveconf) - private def inputRdd = if (!relation.hiveQlTable.isPartitioned) { - hadoopReader.makeRDDForTable(relation.hiveQlTable) - } else { - hadoopReader.makeRDDForPartitionedTable(prunePartitions(relation.hiveQlPartitions)) - } - /** * Prunes partitions not involve the query plan. * @@ -179,44 +122,10 @@ case class HiveTableScan( } } - override def execute() = { - inputRdd.mapPartitions { iterator => - if (iterator.isEmpty) { - Iterator.empty - } else { - val mutableRow = new GenericMutableRow(attributes.length) - val mutablePair = new MutablePair[Any, Array[String]]() - val buffered = iterator.buffered - - // NOTE (lian): Critical path of Hive table scan, unnecessary FP style code and pattern - // matching are avoided intentionally. - val rowsAndPartitionKeys = buffered.head match { - // With partition keys - case _: Array[Any] => - buffered.map { case array: Array[Any] => - val deserializedRow = array(0) - val partitionKeys = array(1).asInstanceOf[Array[String]] - mutablePair.update(deserializedRow, partitionKeys) - } - - // Without partition keys - case _ => - val emptyPartitionKeys = Array.empty[String] - buffered.map { deserializedRow => - mutablePair.update(deserializedRow, emptyPartitionKeys) - } - } - - rowsAndPartitionKeys.map { pair => - var i = 0 - while (i < attributes.length) { - mutableRow(i) = attributeFunctions(i)(pair._1, pair._2) - i += 1 - } - mutableRow: Row - } - } - } + override def execute() = if (!relation.hiveQlTable.isPartitioned) { + hadoopReader.makeRDDForTable(relation.hiveQlTable) + } else { + hadoopReader.makeRDDForPartitionedTable(prunePartitions(relation.hiveQlPartitions)) } override def output = attributes diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index c2b0b00aa5852..39033bdeac4b0 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -131,7 +131,7 @@ case class InsertIntoHiveTable( conf, SparkHiveHadoopWriter.createPathFromString(fileSinkConf.getDirName, conf)) - logger.debug("Saving as hadoop file of type " + valueClass.getSimpleName) + log.debug("Saving as hadoop file of type " + valueClass.getSimpleName) val writer = new SparkHiveHadoopWriter(conf, fileSinkConf) writer.preSetup() diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala index 8258ee5fef0eb..0c8f676e9c5c8 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala @@ -67,7 +67,7 @@ case class ScriptTransformation( } } readerThread.start() - val outputProjection = new Projection(input) + val outputProjection = new InterpretedProjection(input, child.output) iter .map(outputProjection) // TODO: Use SerDe diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala new file mode 100644 index 0000000000000..2985169da033c --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.execution + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.expressions.Row +import org.apache.spark.sql.execution.{Command, LeafNode} +import org.apache.spark.sql.hive.HiveContext + +/** + * :: DeveloperApi :: + * Analyzes the given table in the current database to generate statistics, which will be + * used in query optimizations. + * + * Right now, it only supports Hive tables and it only updates the size of a Hive table + * in the Hive metastore. + */ +@DeveloperApi +case class AnalyzeTable(tableName: String) extends LeafNode with Command { + + def hiveContext = sqlContext.asInstanceOf[HiveContext] + + def output = Seq.empty + + override protected[sql] lazy val sideEffectResult = { + hiveContext.analyze(tableName) + Seq.empty[Any] + } + + override def execute(): RDD[Row] = { + sideEffectResult + sparkContext.emptyRDD[Row] + } +} + +/** + * :: DeveloperApi :: + * Drops a table from the metastore and removes it if it is cached. + */ +@DeveloperApi +case class DropTable(tableName: String, ifExists: Boolean) extends LeafNode with Command { + + def hiveContext = sqlContext.asInstanceOf[HiveContext] + + def output = Seq.empty + + override protected[sql] lazy val sideEffectResult: Seq[Any] = { + val ifExistsClause = if (ifExists) "IF EXISTS " else "" + hiveContext.runSqlHive(s"DROP TABLE $ifExistsClause$tableName") + hiveContext.catalog.unregisterTable(None, tableName) + Seq.empty + } + + override def execute(): RDD[Row] = { + sideEffectResult + sparkContext.emptyRDD[Row] + } +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala index 9b105308ab7cf..c6497a15efa0c 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala @@ -24,22 +24,20 @@ import org.apache.hadoop.hive.ql.exec.UDF import org.apache.hadoop.hive.ql.exec.{FunctionInfo, FunctionRegistry} import org.apache.hadoop.hive.ql.udf.{UDFType => HiveUDFType} import org.apache.hadoop.hive.ql.udf.generic._ -import org.apache.hadoop.hive.serde2.objectinspector._ -import org.apache.hadoop.hive.serde2.objectinspector.primitive._ -import org.apache.hadoop.hive.serde2.{io => hiveIo} -import org.apache.hadoop.{io => hadoopIo} -import org.apache.spark.sql.Logging +import org.apache.spark.Logging import org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.types import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.util.Utils.getContextOrSparkClassLoader /* Implicit conversions */ import scala.collection.JavaConversions._ -private[hive] object HiveFunctionRegistry - extends analysis.FunctionRegistry with HiveFunctionFactory with HiveInspectors { +private[hive] abstract class HiveFunctionRegistry + extends analysis.FunctionRegistry with HiveInspectors { + + def getFunctionInfo(name: String) = FunctionRegistry.getFunctionInfo(name) def lookupFunction(name: String, children: Seq[Expression]): Expression = { // We only look it up to see if it exists, but do not include it in the HiveUDF since it is @@ -47,111 +45,40 @@ private[hive] object HiveFunctionRegistry val functionInfo: FunctionInfo = Option(FunctionRegistry.getFunctionInfo(name)).getOrElse( sys.error(s"Couldn't find function $name")) + val functionClassName = functionInfo.getFunctionClass.getName() + if (classOf[UDF].isAssignableFrom(functionInfo.getFunctionClass)) { - val function = createFunction[UDF](name) + val function = functionInfo.getFunctionClass.newInstance().asInstanceOf[UDF] val method = function.getResolver.getEvalMethod(children.map(_.dataType.toTypeInfo)) lazy val expectedDataTypes = method.getParameterTypes.map(javaClassToDataType) HiveSimpleUdf( - name, - children.zip(expectedDataTypes).map { case (e, t) => Cast(e, t) } + functionClassName, + children.zip(expectedDataTypes).map { + case (e, NullType) => e + case (e, t) => Cast(e, t) + } ) } else if (classOf[GenericUDF].isAssignableFrom(functionInfo.getFunctionClass)) { - HiveGenericUdf(name, children) + HiveGenericUdf(functionClassName, children) } else if ( classOf[AbstractGenericUDAFResolver].isAssignableFrom(functionInfo.getFunctionClass)) { - HiveGenericUdaf(name, children) + HiveGenericUdaf(functionClassName, children) } else if (classOf[GenericUDTF].isAssignableFrom(functionInfo.getFunctionClass)) { - HiveGenericUdtf(name, Nil, children) + HiveGenericUdtf(functionClassName, Nil, children) } else { sys.error(s"No handler for udf ${functionInfo.getFunctionClass}") } } - - def javaClassToDataType(clz: Class[_]): DataType = clz match { - // writable - case c: Class[_] if c == classOf[hadoopIo.DoubleWritable] => DoubleType - case c: Class[_] if c == classOf[hiveIo.DoubleWritable] => DoubleType - case c: Class[_] if c == classOf[hiveIo.HiveDecimalWritable] => DecimalType - case c: Class[_] if c == classOf[hiveIo.ByteWritable] => ByteType - case c: Class[_] if c == classOf[hiveIo.ShortWritable] => ShortType - case c: Class[_] if c == classOf[hiveIo.TimestampWritable] => TimestampType - case c: Class[_] if c == classOf[hadoopIo.Text] => StringType - case c: Class[_] if c == classOf[hadoopIo.IntWritable] => IntegerType - case c: Class[_] if c == classOf[hadoopIo.LongWritable] => LongType - case c: Class[_] if c == classOf[hadoopIo.FloatWritable] => FloatType - case c: Class[_] if c == classOf[hadoopIo.BooleanWritable] => BooleanType - case c: Class[_] if c == classOf[hadoopIo.BytesWritable] => BinaryType - - // java class - case c: Class[_] if c == classOf[java.lang.String] => StringType - case c: Class[_] if c == classOf[java.sql.Timestamp] => TimestampType - case c: Class[_] if c == classOf[HiveDecimal] => DecimalType - case c: Class[_] if c == classOf[java.math.BigDecimal] => DecimalType - case c: Class[_] if c == classOf[Array[Byte]] => BinaryType - case c: Class[_] if c == classOf[java.lang.Short] => ShortType - case c: Class[_] if c == classOf[java.lang.Integer] => IntegerType - case c: Class[_] if c == classOf[java.lang.Long] => LongType - case c: Class[_] if c == classOf[java.lang.Double] => DoubleType - case c: Class[_] if c == classOf[java.lang.Byte] => ByteType - case c: Class[_] if c == classOf[java.lang.Float] => FloatType - case c: Class[_] if c == classOf[java.lang.Boolean] => BooleanType - - // primitive type - case c: Class[_] if c == java.lang.Short.TYPE => ShortType - case c: Class[_] if c == java.lang.Integer.TYPE => IntegerType - case c: Class[_] if c == java.lang.Long.TYPE => LongType - case c: Class[_] if c == java.lang.Double.TYPE => DoubleType - case c: Class[_] if c == java.lang.Byte.TYPE => ByteType - case c: Class[_] if c == java.lang.Float.TYPE => FloatType - case c: Class[_] if c == java.lang.Boolean.TYPE => BooleanType - - case c: Class[_] if c.isArray => ArrayType(javaClassToDataType(c.getComponentType)) - } } private[hive] trait HiveFunctionFactory { - def getFunctionInfo(name: String) = FunctionRegistry.getFunctionInfo(name) - def getFunctionClass(name: String) = getFunctionInfo(name).getFunctionClass - def createFunction[UDFType](name: String) = - getFunctionClass(name).newInstance.asInstanceOf[UDFType] - - /** Converts hive types to native catalyst types. */ - def unwrap(a: Any): Any = a match { - case null => null - case i: hadoopIo.IntWritable => i.get - case t: hadoopIo.Text => t.toString - case l: hadoopIo.LongWritable => l.get - case d: hadoopIo.DoubleWritable => d.get - case d: hiveIo.DoubleWritable => d.get - case s: hiveIo.ShortWritable => s.get - case b: hadoopIo.BooleanWritable => b.get - case b: hiveIo.ByteWritable => b.get - case b: hadoopIo.FloatWritable => b.get - case b: hadoopIo.BytesWritable => { - val bytes = new Array[Byte](b.getLength) - System.arraycopy(b.getBytes(), 0, bytes, 0, b.getLength) - bytes - } - case t: hiveIo.TimestampWritable => t.getTimestamp - case b: hiveIo.HiveDecimalWritable => BigDecimal(b.getHiveDecimal().bigDecimalValue()) - case list: java.util.List[_] => list.map(unwrap) - case map: java.util.Map[_,_] => map.map { case (k, v) => (unwrap(k), unwrap(v)) }.toMap - case array: Array[_] => array.map(unwrap).toSeq - case p: java.lang.Short => p - case p: java.lang.Long => p - case p: java.lang.Float => p - case p: java.lang.Integer => p - case p: java.lang.Double => p - case p: java.lang.Byte => p - case p: java.lang.Boolean => p - case str: String => str - case p: java.math.BigDecimal => p - case p: Array[Byte] => p - case p: java.sql.Timestamp => p - } + val functionClassName: String + + def createFunction[UDFType]() = + getContextOrSparkClassLoader.loadClass(functionClassName).newInstance.asInstanceOf[UDFType] } private[hive] abstract class HiveUdf extends Expression with Logging with HiveFunctionFactory { @@ -160,20 +87,17 @@ private[hive] abstract class HiveUdf extends Expression with Logging with HiveFu type UDFType type EvaluatedType = Any - val name: String - def nullable = true def references = children.flatMap(_.references).toSet - // FunctionInfo is not serializable so we must look it up here again. - lazy val functionInfo = getFunctionInfo(name) - lazy val function = createFunction[UDFType](name) + lazy val function = createFunction[UDFType]() - override def toString = s"$nodeName#${functionInfo.getDisplayName}(${children.mkString(",")})" + override def toString = s"$nodeName#$functionClassName(${children.mkString(",")})" } -private[hive] case class HiveSimpleUdf(name: String, children: Seq[Expression]) extends HiveUdf { - import org.apache.spark.sql.hive.HiveFunctionRegistry._ +private[hive] case class HiveSimpleUdf(functionClassName: String, children: Seq[Expression]) + extends HiveUdf with HiveInspectors { + type UDFType = UDF @transient @@ -194,22 +118,26 @@ private[hive] case class HiveSimpleUdf(name: String, children: Seq[Expression]) c.getParameterTypes.size == 1 && primitiveClasses.contains(c.getParameterTypes.head) } - val constructor = matchingConstructor.getOrElse( - sys.error(s"No matching wrapper found, options: ${argClass.getConstructors.toSeq}.")) - - (a: Any) => { - logger.debug( - s"Wrapping $a of type ${if (a == null) "null" else a.getClass.getName} using $constructor.") - // We must make sure that primitives get boxed java style. - if (a == null) { - null - } else { - constructor.newInstance(a match { - case i: Int => i: java.lang.Integer - case bd: BigDecimal => new HiveDecimal(bd.underlying()) - case other: AnyRef => other - }).asInstanceOf[AnyRef] - } + matchingConstructor match { + case Some(constructor) => + (a: Any) => { + logDebug( + s"Wrapping $a of type ${if (a == null) "null" else a.getClass.getName} $constructor.") + // We must make sure that primitives get boxed java style. + if (a == null) { + null + } else { + constructor.newInstance(a match { + case i: Int => i: java.lang.Integer + case bd: BigDecimal => new HiveDecimal(bd.underlying()) + case other: AnyRef => other + }).asInstanceOf[AnyRef] + } + } + case None => + (a: Any) => a match { + case wrapper => wrap(wrapper) + } } } @@ -226,7 +154,7 @@ private[hive] case class HiveSimpleUdf(name: String, children: Seq[Expression]) } } -private[hive] case class HiveGenericUdf(name: String, children: Seq[Expression]) +private[hive] case class HiveGenericUdf(functionClassName: String, children: Seq[Expression]) extends HiveUdf with HiveInspectors { import org.apache.hadoop.hive.ql.udf.generic.GenericUDF._ @@ -277,127 +205,8 @@ private[hive] case class HiveGenericUdf(name: String, children: Seq[Expression]) } } -private[hive] trait HiveInspectors { - - def unwrapData(data: Any, oi: ObjectInspector): Any = oi match { - case pi: PrimitiveObjectInspector => pi.getPrimitiveJavaObject(data) - case li: ListObjectInspector => - Option(li.getList(data)) - .map(_.map(unwrapData(_, li.getListElementObjectInspector)).toSeq) - .orNull - case mi: MapObjectInspector => - Option(mi.getMap(data)).map( - _.map { - case (k,v) => - (unwrapData(k, mi.getMapKeyObjectInspector), - unwrapData(v, mi.getMapValueObjectInspector)) - }.toMap).orNull - case si: StructObjectInspector => - val allRefs = si.getAllStructFieldRefs - new GenericRow( - allRefs.map(r => - unwrapData(si.getStructFieldData(data,r), r.getFieldObjectInspector)).toArray) - } - - /** Converts native catalyst types to the types expected by Hive */ - def wrap(a: Any): AnyRef = a match { - case s: String => new hadoopIo.Text(s) // TODO why should be Text? - case i: Int => i: java.lang.Integer - case b: Boolean => b: java.lang.Boolean - case f: Float => f: java.lang.Float - case d: Double => d: java.lang.Double - case l: Long => l: java.lang.Long - case l: Short => l: java.lang.Short - case l: Byte => l: java.lang.Byte - case b: BigDecimal => b.bigDecimal - case b: Array[Byte] => b - case t: java.sql.Timestamp => t - case s: Seq[_] => seqAsJavaList(s.map(wrap)) - case m: Map[_,_] => - mapAsJavaMap(m.map { case (k, v) => wrap(k) -> wrap(v) }) - case null => null - } - - def toInspector(dataType: DataType): ObjectInspector = dataType match { - case ArrayType(tpe) => ObjectInspectorFactory.getStandardListObjectInspector(toInspector(tpe)) - case MapType(keyType, valueType) => - ObjectInspectorFactory.getStandardMapObjectInspector( - toInspector(keyType), toInspector(valueType)) - case StringType => PrimitiveObjectInspectorFactory.javaStringObjectInspector - case IntegerType => PrimitiveObjectInspectorFactory.javaIntObjectInspector - case DoubleType => PrimitiveObjectInspectorFactory.javaDoubleObjectInspector - case BooleanType => PrimitiveObjectInspectorFactory.javaBooleanObjectInspector - case LongType => PrimitiveObjectInspectorFactory.javaLongObjectInspector - case FloatType => PrimitiveObjectInspectorFactory.javaFloatObjectInspector - case ShortType => PrimitiveObjectInspectorFactory.javaShortObjectInspector - case ByteType => PrimitiveObjectInspectorFactory.javaByteObjectInspector - case NullType => PrimitiveObjectInspectorFactory.javaVoidObjectInspector - case BinaryType => PrimitiveObjectInspectorFactory.javaByteArrayObjectInspector - case TimestampType => PrimitiveObjectInspectorFactory.javaTimestampObjectInspector - case DecimalType => PrimitiveObjectInspectorFactory.javaHiveDecimalObjectInspector - case StructType(fields) => - ObjectInspectorFactory.getStandardStructObjectInspector( - fields.map(f => f.name), fields.map(f => toInspector(f.dataType))) - } - - def inspectorToDataType(inspector: ObjectInspector): DataType = inspector match { - case s: StructObjectInspector => - StructType(s.getAllStructFieldRefs.map(f => { - types.StructField( - f.getFieldName, inspectorToDataType(f.getFieldObjectInspector), nullable = true) - })) - case l: ListObjectInspector => ArrayType(inspectorToDataType(l.getListElementObjectInspector)) - case m: MapObjectInspector => - MapType( - inspectorToDataType(m.getMapKeyObjectInspector), - inspectorToDataType(m.getMapValueObjectInspector)) - case _: WritableStringObjectInspector => StringType - case _: JavaStringObjectInspector => StringType - case _: WritableIntObjectInspector => IntegerType - case _: JavaIntObjectInspector => IntegerType - case _: WritableDoubleObjectInspector => DoubleType - case _: JavaDoubleObjectInspector => DoubleType - case _: WritableBooleanObjectInspector => BooleanType - case _: JavaBooleanObjectInspector => BooleanType - case _: WritableLongObjectInspector => LongType - case _: JavaLongObjectInspector => LongType - case _: WritableShortObjectInspector => ShortType - case _: JavaShortObjectInspector => ShortType - case _: WritableByteObjectInspector => ByteType - case _: JavaByteObjectInspector => ByteType - case _: WritableFloatObjectInspector => FloatType - case _: JavaFloatObjectInspector => FloatType - case _: WritableBinaryObjectInspector => BinaryType - case _: JavaBinaryObjectInspector => BinaryType - case _: WritableHiveDecimalObjectInspector => DecimalType - case _: JavaHiveDecimalObjectInspector => DecimalType - case _: WritableTimestampObjectInspector => TimestampType - case _: JavaTimestampObjectInspector => TimestampType - } - - implicit class typeInfoConversions(dt: DataType) { - import org.apache.hadoop.hive.serde2.typeinfo._ - import TypeInfoFactory._ - - def toTypeInfo: TypeInfo = dt match { - case BinaryType => binaryTypeInfo - case BooleanType => booleanTypeInfo - case ByteType => byteTypeInfo - case DoubleType => doubleTypeInfo - case FloatType => floatTypeInfo - case IntegerType => intTypeInfo - case LongType => longTypeInfo - case ShortType => shortTypeInfo - case StringType => stringTypeInfo - case DecimalType => decimalTypeInfo - case TimestampType => timestampTypeInfo - case NullType => voidTypeInfo - } - } -} - private[hive] case class HiveGenericUdaf( - name: String, + functionClassName: String, children: Seq[Expression]) extends AggregateExpression with HiveInspectors with HiveFunctionFactory { @@ -405,7 +214,7 @@ private[hive] case class HiveGenericUdaf( type UDFType = AbstractGenericUDAFResolver @transient - protected lazy val resolver: AbstractGenericUDAFResolver = createFunction(name) + protected lazy val resolver: AbstractGenericUDAFResolver = createFunction() @transient protected lazy val objectInspector = { @@ -422,9 +231,9 @@ private[hive] case class HiveGenericUdaf( def references: Set[Attribute] = children.map(_.references).flatten.toSet - override def toString = s"$nodeName#$name(${children.mkString(",")})" + override def toString = s"$nodeName#$functionClassName(${children.mkString(",")})" - def newInstance() = new HiveUdafFunction(name, children, this) + def newInstance() = new HiveUdafFunction(functionClassName, children, this) } /** @@ -439,7 +248,7 @@ private[hive] case class HiveGenericUdaf( * user defined aggregations, which have clean semantics even in a partitioned execution. */ private[hive] case class HiveGenericUdtf( - name: String, + functionClassName: String, aliasNames: Seq[String], children: Seq[Expression]) extends Generator with HiveInspectors with HiveFunctionFactory { @@ -447,10 +256,12 @@ private[hive] case class HiveGenericUdtf( override def references = children.flatMap(_.references).toSet @transient - protected lazy val function: GenericUDTF = createFunction(name) + protected lazy val function: GenericUDTF = createFunction() + @transient protected lazy val inputInspectors = children.map(_.dataType).map(toInspector) + @transient protected lazy val outputInspectors = { val structInspector = function.initialize(inputInspectors.toArray) structInspector.getAllStructFieldRefs.map(_.getFieldObjectInspector) @@ -476,7 +287,7 @@ private[hive] case class HiveGenericUdtf( override def eval(input: Row): TraversableOnce[Row] = { outputInspectors // Make sure initialized. - val inputProjection = new Projection(children) + val inputProjection = new InterpretedProjection(children) val collector = new UDTFCollector function.setCollector(collector) @@ -502,11 +313,11 @@ private[hive] case class HiveGenericUdtf( } } - override def toString = s"$nodeName#$name(${children.mkString(",")})" + override def toString = s"$nodeName#$functionClassName(${children.mkString(",")})" } private[hive] case class HiveUdafFunction( - functionName: String, + functionClassName: String, exprs: Seq[Expression], base: AggregateExpression) extends AggregateFunction @@ -515,7 +326,7 @@ private[hive] case class HiveUdafFunction( def this() = this(null, null, null) - private val resolver = createFunction[AbstractGenericUDAFResolver](functionName) + private val resolver = createFunction[AbstractGenericUDAFResolver]() private val inspectors = exprs.map(_.dataType).map(toInspector).toArray @@ -530,7 +341,7 @@ private[hive] case class HiveUdafFunction( override def eval(input: Row): Any = unwrapData(function.evaluate(buffer), returnInspector) @transient - val inputProjection = new Projection(exprs) + val inputProjection = new InterpretedProjection(exprs) def update(input: Row): Unit = { val inputs = inputProjection(input).asInstanceOf[Seq[AnyRef]].toArray diff --git a/sql/hive/src/test/resources/golden/boolean = number-0-6b6975fa1892cc48edd87dc0df48a7c0 b/sql/hive/src/test/resources/golden/boolean = number-0-6b6975fa1892cc48edd87dc0df48a7c0 new file mode 100644 index 0000000000000..4d1ebdcde2c71 --- /dev/null +++ b/sql/hive/src/test/resources/golden/boolean = number-0-6b6975fa1892cc48edd87dc0df48a7c0 @@ -0,0 +1 @@ +true true true true true true false false false false false false false false false false false false true true true true true true false false false false false false false false false false false false diff --git a/sql/hive/src/test/resources/golden/case else null-0-8ef2f741400830ef889a9dd0c817fe3d b/sql/hive/src/test/resources/golden/case else null-0-8ef2f741400830ef889a9dd0c817fe3d new file mode 100644 index 0000000000000..00750edc07d64 --- /dev/null +++ b/sql/hive/src/test/resources/golden/case else null-0-8ef2f741400830ef889a9dd0c817fe3d @@ -0,0 +1 @@ +3 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-0-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/correlationoptimizer1-0-b1e2ade89ae898650f0be4f796d8947b new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-0-b1e2ade89ae898650f0be4f796d8947b @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-1-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer1-1-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-1-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-10-5d712a42dcc1c4f7c797dabda5eb7b3a b/sql/hive/src/test/resources/golden/correlationoptimizer1-10-5d712a42dcc1c4f7c797dabda5eb7b3a new file mode 100644 index 0000000000000..58010b0040b74 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-10-5d712a42dcc1c4f7c797dabda5eb7b3a @@ -0,0 +1 @@ +3556 37 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-11-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/correlationoptimizer1-11-b1e2ade89ae898650f0be4f796d8947b new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-11-b1e2ade89ae898650f0be4f796d8947b @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-12-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer1-12-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-12-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-13-f9f839aedb3a350719c0cbc53a06ace5 b/sql/hive/src/test/resources/golden/correlationoptimizer1-13-f9f839aedb3a350719c0cbc53a06ace5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-14-dae4256e08d595317f8e09a56354a3d9 b/sql/hive/src/test/resources/golden/correlationoptimizer1-14-dae4256e08d595317f8e09a56354a3d9 new file mode 100644 index 0000000000000..235736a2807b6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-14-dae4256e08d595317f8e09a56354a3d9 @@ -0,0 +1 @@ +3556 15 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-15-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer1-15-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-15-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-16-43f356b36962f2bade5706d8cf5ae6b4 b/sql/hive/src/test/resources/golden/correlationoptimizer1-16-43f356b36962f2bade5706d8cf5ae6b4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-17-dae4256e08d595317f8e09a56354a3d9 b/sql/hive/src/test/resources/golden/correlationoptimizer1-17-dae4256e08d595317f8e09a56354a3d9 new file mode 100644 index 0000000000000..235736a2807b6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-17-dae4256e08d595317f8e09a56354a3d9 @@ -0,0 +1 @@ +3556 15 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-18-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/correlationoptimizer1-18-b1e2ade89ae898650f0be4f796d8947b new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-18-b1e2ade89ae898650f0be4f796d8947b @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-19-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer1-19-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-19-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-2-42a0eedc3751f792ad5438b2c64d3897 b/sql/hive/src/test/resources/golden/correlationoptimizer1-2-42a0eedc3751f792ad5438b2c64d3897 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-20-b4c1e27a7c1f61a3e9ae07c80e6e2973 b/sql/hive/src/test/resources/golden/correlationoptimizer1-20-b4c1e27a7c1f61a3e9ae07c80e6e2973 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-21-16c57348be42ca3cc2f80f7f92265696 b/sql/hive/src/test/resources/golden/correlationoptimizer1-21-16c57348be42ca3cc2f80f7f92265696 new file mode 100644 index 0000000000000..76c4941de407d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-21-16c57348be42ca3cc2f80f7f92265696 @@ -0,0 +1 @@ +3556 47 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-22-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer1-22-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-22-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-23-2cdc77fd60449f3547cf95d8eb09a2 b/sql/hive/src/test/resources/golden/correlationoptimizer1-23-2cdc77fd60449f3547cf95d8eb09a2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-24-16c57348be42ca3cc2f80f7f92265696 b/sql/hive/src/test/resources/golden/correlationoptimizer1-24-16c57348be42ca3cc2f80f7f92265696 new file mode 100644 index 0000000000000..76c4941de407d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-24-16c57348be42ca3cc2f80f7f92265696 @@ -0,0 +1 @@ +3556 47 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-25-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer1-25-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-25-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-26-8bcdcc5f01508f576d7bd6422c939225 b/sql/hive/src/test/resources/golden/correlationoptimizer1-26-8bcdcc5f01508f576d7bd6422c939225 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-27-d31433f229e853e8b8440b4ddc63c80e b/sql/hive/src/test/resources/golden/correlationoptimizer1-27-d31433f229e853e8b8440b4ddc63c80e new file mode 100644 index 0000000000000..76c4941de407d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-27-d31433f229e853e8b8440b4ddc63c80e @@ -0,0 +1 @@ +3556 47 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-28-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer1-28-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-28-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-29-941ecfef9448ecff56cc16bcfb233ee4 b/sql/hive/src/test/resources/golden/correlationoptimizer1-29-941ecfef9448ecff56cc16bcfb233ee4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-3-5d712a42dcc1c4f7c797dabda5eb7b3a b/sql/hive/src/test/resources/golden/correlationoptimizer1-3-5d712a42dcc1c4f7c797dabda5eb7b3a new file mode 100644 index 0000000000000..58010b0040b74 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-3-5d712a42dcc1c4f7c797dabda5eb7b3a @@ -0,0 +1 @@ +3556 37 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-30-d31433f229e853e8b8440b4ddc63c80e b/sql/hive/src/test/resources/golden/correlationoptimizer1-30-d31433f229e853e8b8440b4ddc63c80e new file mode 100644 index 0000000000000..76c4941de407d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-30-d31433f229e853e8b8440b4ddc63c80e @@ -0,0 +1 @@ +3556 47 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-31-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer1-31-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-31-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-32-ef6502d6b282c8a6d228bba395b24724 b/sql/hive/src/test/resources/golden/correlationoptimizer1-32-ef6502d6b282c8a6d228bba395b24724 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-33-ea87e76dba02a46cb958148333e397b7 b/sql/hive/src/test/resources/golden/correlationoptimizer1-33-ea87e76dba02a46cb958148333e397b7 new file mode 100644 index 0000000000000..5aa2d482094af --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-33-ea87e76dba02a46cb958148333e397b7 @@ -0,0 +1 @@ +79136 500 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-34-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer1-34-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-34-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-35-b79b220859c09354e23b533c105ccbab b/sql/hive/src/test/resources/golden/correlationoptimizer1-35-b79b220859c09354e23b533c105ccbab new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-36-ea87e76dba02a46cb958148333e397b7 b/sql/hive/src/test/resources/golden/correlationoptimizer1-36-ea87e76dba02a46cb958148333e397b7 new file mode 100644 index 0000000000000..5aa2d482094af --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-36-ea87e76dba02a46cb958148333e397b7 @@ -0,0 +1 @@ +79136 500 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-37-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer1-37-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-37-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-38-638e5300f4c892c2bf27bd91a8f81b64 b/sql/hive/src/test/resources/golden/correlationoptimizer1-38-638e5300f4c892c2bf27bd91a8f81b64 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-39-66010469a9cdb66851da9a727ef9fdad b/sql/hive/src/test/resources/golden/correlationoptimizer1-39-66010469a9cdb66851da9a727ef9fdad new file mode 100644 index 0000000000000..b4a3a9d327f47 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-39-66010469a9cdb66851da9a727ef9fdad @@ -0,0 +1 @@ +3556 500 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-4-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer1-4-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-4-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-40-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer1-40-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-40-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-41-3514c74c7f68f2d70cc6d51ac46c20 b/sql/hive/src/test/resources/golden/correlationoptimizer1-41-3514c74c7f68f2d70cc6d51ac46c20 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-42-66010469a9cdb66851da9a727ef9fdad b/sql/hive/src/test/resources/golden/correlationoptimizer1-42-66010469a9cdb66851da9a727ef9fdad new file mode 100644 index 0000000000000..b4a3a9d327f47 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-42-66010469a9cdb66851da9a727ef9fdad @@ -0,0 +1 @@ +3556 500 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-43-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer1-43-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-43-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-44-7490df6719cd7e47aa08dbcbc3266a92 b/sql/hive/src/test/resources/golden/correlationoptimizer1-44-7490df6719cd7e47aa08dbcbc3266a92 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-45-e71195e7d9f557e2abc7f03462d22dba b/sql/hive/src/test/resources/golden/correlationoptimizer1-45-e71195e7d9f557e2abc7f03462d22dba new file mode 100644 index 0000000000000..bb564e0fd06eb --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-45-e71195e7d9f557e2abc7f03462d22dba @@ -0,0 +1 @@ +3556 510 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-46-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer1-46-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-46-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-47-73da9fe2b0c2ee26c021ec3f2fa27272 b/sql/hive/src/test/resources/golden/correlationoptimizer1-47-73da9fe2b0c2ee26c021ec3f2fa27272 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-48-e71195e7d9f557e2abc7f03462d22dba b/sql/hive/src/test/resources/golden/correlationoptimizer1-48-e71195e7d9f557e2abc7f03462d22dba new file mode 100644 index 0000000000000..bb564e0fd06eb --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-48-e71195e7d9f557e2abc7f03462d22dba @@ -0,0 +1 @@ +3556 510 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-49-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/correlationoptimizer1-49-b1e2ade89ae898650f0be4f796d8947b new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-49-b1e2ade89ae898650f0be4f796d8947b @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-5-a1c80c68b9a7597096c2580c3766f7f7 b/sql/hive/src/test/resources/golden/correlationoptimizer1-5-a1c80c68b9a7597096c2580c3766f7f7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-50-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer1-50-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-50-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-51-fcf9bcb522f542637ccdea863b408448 b/sql/hive/src/test/resources/golden/correlationoptimizer1-51-fcf9bcb522f542637ccdea863b408448 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-52-3070366869308907e54797927805603 b/sql/hive/src/test/resources/golden/correlationoptimizer1-52-3070366869308907e54797927805603 new file mode 100644 index 0000000000000..edd216f16b190 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-52-3070366869308907e54797927805603 @@ -0,0 +1 @@ +3556 661329102 37 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-53-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer1-53-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-53-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-54-dad56e1f06c808b29e5dc8fb0c49efb2 b/sql/hive/src/test/resources/golden/correlationoptimizer1-54-dad56e1f06c808b29e5dc8fb0c49efb2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-55-3070366869308907e54797927805603 b/sql/hive/src/test/resources/golden/correlationoptimizer1-55-3070366869308907e54797927805603 new file mode 100644 index 0000000000000..edd216f16b190 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-55-3070366869308907e54797927805603 @@ -0,0 +1 @@ +3556 661329102 37 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-56-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer1-56-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-56-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-57-3cd3fbbbd8ee5c274fe3d6a45126cef4 b/sql/hive/src/test/resources/golden/correlationoptimizer1-57-3cd3fbbbd8ee5c274fe3d6a45126cef4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-58-a6bba6d9b422adb386b35c62cecb548 b/sql/hive/src/test/resources/golden/correlationoptimizer1-58-a6bba6d9b422adb386b35c62cecb548 new file mode 100644 index 0000000000000..9fde6099b4f08 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-58-a6bba6d9b422adb386b35c62cecb548 @@ -0,0 +1 @@ +2835 29 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-59-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer1-59-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-59-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-6-5d712a42dcc1c4f7c797dabda5eb7b3a b/sql/hive/src/test/resources/golden/correlationoptimizer1-6-5d712a42dcc1c4f7c797dabda5eb7b3a new file mode 100644 index 0000000000000..58010b0040b74 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-6-5d712a42dcc1c4f7c797dabda5eb7b3a @@ -0,0 +1 @@ +3556 37 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-60-d6bbaf0d40010159095e4cac025c50c5 b/sql/hive/src/test/resources/golden/correlationoptimizer1-60-d6bbaf0d40010159095e4cac025c50c5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-61-a6bba6d9b422adb386b35c62cecb548 b/sql/hive/src/test/resources/golden/correlationoptimizer1-61-a6bba6d9b422adb386b35c62cecb548 new file mode 100644 index 0000000000000..9fde6099b4f08 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-61-a6bba6d9b422adb386b35c62cecb548 @@ -0,0 +1 @@ +2835 29 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-7-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/correlationoptimizer1-7-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-7-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-8-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer1-8-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-8-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-9-d5bea91b4edb8be0428a336ff9c21dde b/sql/hive/src/test/resources/golden/correlationoptimizer1-9-d5bea91b4edb8be0428a336ff9c21dde new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer10-0-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/correlationoptimizer10-0-b1e2ade89ae898650f0be4f796d8947b new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer10-0-b1e2ade89ae898650f0be4f796d8947b @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer10-1-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer10-1-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer10-1-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer10-10-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer10-10-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer10-10-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer10-11-a1b7af95dfd01783c07aa23208d6160 b/sql/hive/src/test/resources/golden/correlationoptimizer10-11-a1b7af95dfd01783c07aa23208d6160 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer10-12-1322cff0bdf29aab32e638ad48c71ff9 b/sql/hive/src/test/resources/golden/correlationoptimizer10-12-1322cff0bdf29aab32e638ad48c71ff9 new file mode 100644 index 0000000000000..9ea431d9f5d18 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer10-12-1322cff0bdf29aab32e638ad48c71ff9 @@ -0,0 +1,5 @@ +66 val_66 +98 val_98 +128 +146 val_146 +150 val_150 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer10-13-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer10-13-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer10-13-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer10-14-934b668c11600dc9c013c2ddc4c0d68c b/sql/hive/src/test/resources/golden/correlationoptimizer10-14-934b668c11600dc9c013c2ddc4c0d68c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer10-15-430ff20a144fb3dbf526232d9cb2baa b/sql/hive/src/test/resources/golden/correlationoptimizer10-15-430ff20a144fb3dbf526232d9cb2baa new file mode 100644 index 0000000000000..5abecc5df25fd --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer10-15-430ff20a144fb3dbf526232d9cb2baa @@ -0,0 +1,23 @@ +181 val_181 +183 val_183 +186 val_186 +187 val_187 +187 val_187 +187 val_187 +189 val_189 +190 val_190 +191 val_191 +191 val_191 +192 val_192 +193 val_193 +193 val_193 +193 val_193 +194 val_194 +195 val_195 +195 val_195 +196 val_196 +197 val_197 +197 val_197 +199 val_199 +199 val_199 +199 val_199 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer10-16-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer10-16-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer10-16-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer10-17-9d5521ecef1353d23fd2d4f7d78e7006 b/sql/hive/src/test/resources/golden/correlationoptimizer10-17-9d5521ecef1353d23fd2d4f7d78e7006 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer10-18-430ff20a144fb3dbf526232d9cb2baa b/sql/hive/src/test/resources/golden/correlationoptimizer10-18-430ff20a144fb3dbf526232d9cb2baa new file mode 100644 index 0000000000000..5abecc5df25fd --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer10-18-430ff20a144fb3dbf526232d9cb2baa @@ -0,0 +1,23 @@ +181 val_181 +183 val_183 +186 val_186 +187 val_187 +187 val_187 +187 val_187 +189 val_189 +190 val_190 +191 val_191 +191 val_191 +192 val_192 +193 val_193 +193 val_193 +193 val_193 +194 val_194 +195 val_195 +195 val_195 +196 val_196 +197 val_197 +197 val_197 +199 val_199 +199 val_199 +199 val_199 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer10-2-f9de06a4184ab1f42793327c1497437 b/sql/hive/src/test/resources/golden/correlationoptimizer10-2-f9de06a4184ab1f42793327c1497437 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer10-3-6a01aa7ca94cda4268af894b4fd852ea b/sql/hive/src/test/resources/golden/correlationoptimizer10-3-6a01aa7ca94cda4268af894b4fd852ea new file mode 100644 index 0000000000000..d00aeb4be0340 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer10-3-6a01aa7ca94cda4268af894b4fd852ea @@ -0,0 +1,15 @@ +66 1 +98 1 +128 1 +146 1 +150 1 +213 1 +224 1 +238 1 +255 1 +273 1 +278 1 +311 1 +369 1 +401 1 +406 1 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer10-4-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer10-4-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer10-4-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer10-5-6f191930802d659058465b2e6de08dd3 b/sql/hive/src/test/resources/golden/correlationoptimizer10-5-6f191930802d659058465b2e6de08dd3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer10-6-6a01aa7ca94cda4268af894b4fd852ea b/sql/hive/src/test/resources/golden/correlationoptimizer10-6-6a01aa7ca94cda4268af894b4fd852ea new file mode 100644 index 0000000000000..d00aeb4be0340 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer10-6-6a01aa7ca94cda4268af894b4fd852ea @@ -0,0 +1,15 @@ +66 1 +98 1 +128 1 +146 1 +150 1 +213 1 +224 1 +238 1 +255 1 +273 1 +278 1 +311 1 +369 1 +401 1 +406 1 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer10-7-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer10-7-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer10-7-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer10-8-b2c8d0056b9f1b41cdaeaab4a1a5c9ec b/sql/hive/src/test/resources/golden/correlationoptimizer10-8-b2c8d0056b9f1b41cdaeaab4a1a5c9ec new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer10-9-1322cff0bdf29aab32e638ad48c71ff9 b/sql/hive/src/test/resources/golden/correlationoptimizer10-9-1322cff0bdf29aab32e638ad48c71ff9 new file mode 100644 index 0000000000000..9ea431d9f5d18 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer10-9-1322cff0bdf29aab32e638ad48c71ff9 @@ -0,0 +1,5 @@ +66 val_66 +98 val_98 +128 +146 val_146 +150 val_150 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer13-0-efd135a811fa94760736a761d220b82 b/sql/hive/src/test/resources/golden/correlationoptimizer13-0-efd135a811fa94760736a761d220b82 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer13-1-32a82500cc28465fac6f64dde0c431c6 b/sql/hive/src/test/resources/golden/correlationoptimizer13-1-32a82500cc28465fac6f64dde0c431c6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer13-2-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer13-2-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer13-2-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer13-3-bb61d9292434f37bd386e5bff683764d b/sql/hive/src/test/resources/golden/correlationoptimizer13-3-bb61d9292434f37bd386e5bff683764d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer9-0-efd135a811fa94760736a761d220b82 b/sql/hive/src/test/resources/golden/correlationoptimizer9-0-efd135a811fa94760736a761d220b82 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer9-1-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/correlationoptimizer9-1-b1e2ade89ae898650f0be4f796d8947b new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer9-1-b1e2ade89ae898650f0be4f796d8947b @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer9-10-1190d82f88f7fb1f91968f6e2e03772a b/sql/hive/src/test/resources/golden/correlationoptimizer9-10-1190d82f88f7fb1f91968f6e2e03772a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer9-11-bc2ae88b17ac2bdbd288e07194a40168 b/sql/hive/src/test/resources/golden/correlationoptimizer9-11-bc2ae88b17ac2bdbd288e07194a40168 new file mode 100644 index 0000000000000..17c838bb62b3b --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer9-11-bc2ae88b17ac2bdbd288e07194a40168 @@ -0,0 +1,9 @@ +103 val_103 103 val_103 4 4 +104 val_104 104 val_104 4 4 +105 val_105 105 val_105 1 1 +111 val_111 111 val_111 1 1 +113 val_113 113 val_113 4 4 +114 val_114 114 val_114 1 1 +116 val_116 116 val_116 1 1 +118 val_118 118 val_118 4 4 +119 val_119 119 val_119 9 9 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer9-12-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer9-12-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer9-12-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer9-13-1190d82f88f7fb1f91968f6e2e03772a b/sql/hive/src/test/resources/golden/correlationoptimizer9-13-1190d82f88f7fb1f91968f6e2e03772a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer9-14-bc2ae88b17ac2bdbd288e07194a40168 b/sql/hive/src/test/resources/golden/correlationoptimizer9-14-bc2ae88b17ac2bdbd288e07194a40168 new file mode 100644 index 0000000000000..17c838bb62b3b --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer9-14-bc2ae88b17ac2bdbd288e07194a40168 @@ -0,0 +1,9 @@ +103 val_103 103 val_103 4 4 +104 val_104 104 val_104 4 4 +105 val_105 105 val_105 1 1 +111 val_111 111 val_111 1 1 +113 val_113 113 val_113 4 4 +114 val_114 114 val_114 1 1 +116 val_116 116 val_116 1 1 +118 val_118 118 val_118 4 4 +119 val_119 119 val_119 9 9 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer9-2-32a82500cc28465fac6f64dde0c431c6 b/sql/hive/src/test/resources/golden/correlationoptimizer9-2-32a82500cc28465fac6f64dde0c431c6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer9-3-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer9-3-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer9-3-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer9-4-ec131bcf578dba99f20b16a7dc6b9b b/sql/hive/src/test/resources/golden/correlationoptimizer9-4-ec131bcf578dba99f20b16a7dc6b9b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer9-5-b4e378104bb5ab8d8ba5f905aa1ff450 b/sql/hive/src/test/resources/golden/correlationoptimizer9-5-b4e378104bb5ab8d8ba5f905aa1ff450 new file mode 100644 index 0000000000000..248a14f1f4a9f --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer9-5-b4e378104bb5ab8d8ba5f905aa1ff450 @@ -0,0 +1,9 @@ +103 103 4 4 +104 104 4 4 +105 105 1 1 +111 111 1 1 +113 113 4 4 +114 114 1 1 +116 116 1 1 +118 118 4 4 +119 119 9 9 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer9-6-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer9-6-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer9-6-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer9-7-f952899d70bd718cbdbc44a5290938c9 b/sql/hive/src/test/resources/golden/correlationoptimizer9-7-f952899d70bd718cbdbc44a5290938c9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer9-8-b4e378104bb5ab8d8ba5f905aa1ff450 b/sql/hive/src/test/resources/golden/correlationoptimizer9-8-b4e378104bb5ab8d8ba5f905aa1ff450 new file mode 100644 index 0000000000000..248a14f1f4a9f --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer9-8-b4e378104bb5ab8d8ba5f905aa1ff450 @@ -0,0 +1,9 @@ +103 103 4 4 +104 104 4 4 +105 105 1 1 +111 111 1 1 +113 113 4 4 +114 114 1 1 +116 116 1 1 +118 118 4 4 +119 119 9 9 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer9-9-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer9-9-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer9-9-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/div-0-3760f9b354ddacd7c7b01b28791d4585 b/sql/hive/src/test/resources/golden/div-0-3760f9b354ddacd7c7b01b28791d4585 new file mode 100644 index 0000000000000..17ba0bea723c6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/div-0-3760f9b354ddacd7c7b01b28791d4585 @@ -0,0 +1 @@ +0 0 0 1 2 diff --git a/sql/hive/src/test/resources/golden/division-0-63b19f8a22471c8ba0415c1d3bc276f7 b/sql/hive/src/test/resources/golden/division-0-63b19f8a22471c8ba0415c1d3bc276f7 new file mode 100644 index 0000000000000..7b7a9175114ce --- /dev/null +++ b/sql/hive/src/test/resources/golden/division-0-63b19f8a22471c8ba0415c1d3bc276f7 @@ -0,0 +1 @@ +2.0 0.5 0.3333333333333333 0.002 diff --git a/sql/hive/src/test/resources/golden/double case-0-f513687d17dcb18546fefa75000a52f2 b/sql/hive/src/test/resources/golden/double case-0-f513687d17dcb18546fefa75000a52f2 new file mode 100644 index 0000000000000..00750edc07d64 --- /dev/null +++ b/sql/hive/src/test/resources/golden/double case-0-f513687d17dcb18546fefa75000a52f2 @@ -0,0 +1 @@ +3 diff --git a/sql/hive/src/test/resources/golden/groupby_ppd-0-4b116ec2d8fb55c52b7b0d248c616ae2 b/sql/hive/src/test/resources/golden/groupby_ppd-0-4b116ec2d8fb55c52b7b0d248c616ae2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_ppd-1-db7b1db8f5e61f0fa78d2874e4d72d9d b/sql/hive/src/test/resources/golden/groupby_ppd-1-db7b1db8f5e61f0fa78d2874e4d72d9d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_ppd-2-d286410aa1d5f5c8d91b863a6d6e29c5 b/sql/hive/src/test/resources/golden/groupby_ppd-2-d286410aa1d5f5c8d91b863a6d6e29c5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/having no references-0-d2de3ba23759d25ef77cdfbab72cbb63 b/sql/hive/src/test/resources/golden/having no references-0-d2de3ba23759d25ef77cdfbab72cbb63 new file mode 100644 index 0000000000000..3f2cab688ccc2 --- /dev/null +++ b/sql/hive/src/test/resources/golden/having no references-0-d2de3ba23759d25ef77cdfbab72cbb63 @@ -0,0 +1,136 @@ +0 +5 +12 +15 +18 +24 +26 +35 +37 +42 +51 +58 +67 +70 +72 +76 +83 +84 +90 +95 +97 +98 +100 +103 +104 +113 +118 +119 +120 +125 +128 +129 +134 +137 +138 +146 +149 +152 +164 +165 +167 +169 +172 +174 +175 +176 +179 +187 +191 +193 +195 +197 +199 +200 +203 +205 +207 +208 +209 +213 +216 +217 +219 +221 +223 +224 +229 +230 +233 +237 +238 +239 +242 +255 +256 +265 +272 +273 +277 +278 +280 +281 +282 +288 +298 +307 +309 +311 +316 +317 +318 +321 +322 +325 +327 +331 +333 +342 +344 +348 +353 +367 +369 +382 +384 +395 +396 +397 +399 +401 +403 +404 +406 +409 +413 +414 +417 +424 +429 +430 +431 +438 +439 +454 +458 +459 +462 +463 +466 +468 +469 +478 +480 +489 +492 +498 diff --git a/sql/hive/src/test/resources/golden/having-0-57f3f26c0203c29c2a91a7cca557ce55 b/sql/hive/src/test/resources/golden/having-0-57f3f26c0203c29c2a91a7cca557ce55 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/having-1-ef81808faeab6d212c3cf32abfc0d873 b/sql/hive/src/test/resources/golden/having-1-ef81808faeab6d212c3cf32abfc0d873 new file mode 100644 index 0000000000000..704f1e62f14c5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/having-1-ef81808faeab6d212c3cf32abfc0d873 @@ -0,0 +1,10 @@ +4 +4 +5 +4 +5 +5 +4 +4 +5 +4 diff --git a/sql/hive/src/test/resources/golden/having-2-a2b4f52cb92f730ddb912b063636d6c1 b/sql/hive/src/test/resources/golden/having-2-a2b4f52cb92f730ddb912b063636d6c1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/having-3-3fa6387b6a4ece110ac340c7b893964e b/sql/hive/src/test/resources/golden/having-3-3fa6387b6a4ece110ac340c7b893964e new file mode 100644 index 0000000000000..b56757a60f780 --- /dev/null +++ b/sql/hive/src/test/resources/golden/having-3-3fa6387b6a4ece110ac340c7b893964e @@ -0,0 +1,308 @@ +0 val_0 +2 val_2 +4 val_4 +5 val_5 +8 val_8 +9 val_9 +10 val_10 +11 val_11 +12 val_12 +15 val_15 +17 val_17 +18 val_18 +19 val_19 +20 val_20 +24 val_24 +26 val_26 +27 val_27 +28 val_28 +30 val_30 +33 val_33 +34 val_34 +35 val_35 +37 val_37 +41 val_41 +42 val_42 +43 val_43 +44 val_44 +47 val_47 +51 val_51 +53 val_53 +54 val_54 +57 val_57 +58 val_58 +64 val_64 +65 val_65 +66 val_66 +67 val_67 +69 val_69 +70 val_70 +72 val_72 +74 val_74 +76 val_76 +77 val_77 +78 val_78 +80 val_80 +82 val_82 +83 val_83 +84 val_84 +85 val_85 +86 val_86 +87 val_87 +90 val_90 +92 val_92 +95 val_95 +96 val_96 +97 val_97 +98 val_98 +100 val_100 +103 val_103 +104 val_104 +105 val_105 +111 val_111 +113 val_113 +114 val_114 +116 val_116 +118 val_118 +119 val_119 +120 val_120 +125 val_125 +126 val_126 +128 val_128 +129 val_129 +131 val_131 +133 val_133 +134 val_134 +136 val_136 +137 val_137 +138 val_138 +143 val_143 +145 val_145 +146 val_146 +149 val_149 +150 val_150 +152 val_152 +153 val_153 +155 val_155 +156 val_156 +157 val_157 +158 val_158 +160 val_160 +162 val_162 +163 val_163 +164 val_164 +165 val_165 +166 val_166 +167 val_167 +168 val_168 +169 val_169 +170 val_170 +172 val_172 +174 val_174 +175 val_175 +176 val_176 +177 val_177 +178 val_178 +179 val_179 +180 val_180 +181 val_181 +183 val_183 +186 val_186 +187 val_187 +189 val_189 +190 val_190 +191 val_191 +192 val_192 +193 val_193 +194 val_194 +195 val_195 +196 val_196 +197 val_197 +199 val_199 +200 val_200 +201 val_201 +202 val_202 +203 val_203 +205 val_205 +207 val_207 +208 val_208 +209 val_209 +213 val_213 +214 val_214 +216 val_216 +217 val_217 +218 val_218 +219 val_219 +221 val_221 +222 val_222 +223 val_223 +224 val_224 +226 val_226 +228 val_228 +229 val_229 +230 val_230 +233 val_233 +235 val_235 +237 val_237 +238 val_238 +239 val_239 +241 val_241 +242 val_242 +244 val_244 +247 val_247 +248 val_248 +249 val_249 +252 val_252 +255 val_255 +256 val_256 +257 val_257 +258 val_258 +260 val_260 +262 val_262 +263 val_263 +265 val_265 +266 val_266 +272 val_272 +273 val_273 +274 val_274 +275 val_275 +277 val_277 +278 val_278 +280 val_280 +281 val_281 +282 val_282 +283 val_283 +284 val_284 +285 val_285 +286 val_286 +287 val_287 +288 val_288 +289 val_289 +291 val_291 +292 val_292 +296 val_296 +298 val_298 +305 val_305 +306 val_306 +307 val_307 +308 val_308 +309 val_309 +310 val_310 +311 val_311 +315 val_315 +316 val_316 +317 val_317 +318 val_318 +321 val_321 +322 val_322 +323 val_323 +325 val_325 +327 val_327 +331 val_331 +332 val_332 +333 val_333 +335 val_335 +336 val_336 +338 val_338 +339 val_339 +341 val_341 +342 val_342 +344 val_344 +345 val_345 +348 val_348 +351 val_351 +353 val_353 +356 val_356 +360 val_360 +362 val_362 +364 val_364 +365 val_365 +366 val_366 +367 val_367 +368 val_368 +369 val_369 +373 val_373 +374 val_374 +375 val_375 +377 val_377 +378 val_378 +379 val_379 +382 val_382 +384 val_384 +386 val_386 +389 val_389 +392 val_392 +393 val_393 +394 val_394 +395 val_395 +396 val_396 +397 val_397 +399 val_399 +400 val_400 +401 val_401 +402 val_402 +403 val_403 +404 val_404 +406 val_406 +407 val_407 +409 val_409 +411 val_411 +413 val_413 +414 val_414 +417 val_417 +418 val_418 +419 val_419 +421 val_421 +424 val_424 +427 val_427 +429 val_429 +430 val_430 +431 val_431 +432 val_432 +435 val_435 +436 val_436 +437 val_437 +438 val_438 +439 val_439 +443 val_443 +444 val_444 +446 val_446 +448 val_448 +449 val_449 +452 val_452 +453 val_453 +454 val_454 +455 val_455 +457 val_457 +458 val_458 +459 val_459 +460 val_460 +462 val_462 +463 val_463 +466 val_466 +467 val_467 +468 val_468 +469 val_469 +470 val_470 +472 val_472 +475 val_475 +477 val_477 +478 val_478 +479 val_479 +480 val_480 +481 val_481 +482 val_482 +483 val_483 +484 val_484 +485 val_485 +487 val_487 +489 val_489 +490 val_490 +491 val_491 +492 val_492 +493 val_493 +494 val_494 +495 val_495 +496 val_496 +497 val_497 +498 val_498 diff --git a/sql/hive/src/test/resources/golden/having-4-e9918bd385cb35db4ebcbd4e398547f4 b/sql/hive/src/test/resources/golden/having-4-e9918bd385cb35db4ebcbd4e398547f4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/having-5-4a0c4e521b8a6f6146151c13a2715ff b/sql/hive/src/test/resources/golden/having-5-4a0c4e521b8a6f6146151c13a2715ff new file mode 100644 index 0000000000000..2d7022e386303 --- /dev/null +++ b/sql/hive/src/test/resources/golden/having-5-4a0c4e521b8a6f6146151c13a2715ff @@ -0,0 +1,199 @@ +4 +5 +8 +9 +26 +27 +28 +30 +33 +34 +35 +37 +41 +42 +43 +44 +47 +51 +53 +54 +57 +58 +64 +65 +66 +67 +69 +70 +72 +74 +76 +77 +78 +80 +82 +83 +84 +85 +86 +87 +90 +92 +95 +96 +97 +98 +256 +257 +258 +260 +262 +263 +265 +266 +272 +273 +274 +275 +277 +278 +280 +281 +282 +283 +284 +285 +286 +287 +288 +289 +291 +292 +296 +298 +302 +305 +306 +307 +308 +309 +310 +311 +315 +316 +317 +318 +321 +322 +323 +325 +327 +331 +332 +333 +335 +336 +338 +339 +341 +342 +344 +345 +348 +351 +353 +356 +360 +362 +364 +365 +366 +367 +368 +369 +373 +374 +375 +377 +378 +379 +382 +384 +386 +389 +392 +393 +394 +395 +396 +397 +399 +400 +401 +402 +403 +404 +406 +407 +409 +411 +413 +414 +417 +418 +419 +421 +424 +427 +429 +430 +431 +432 +435 +436 +437 +438 +439 +443 +444 +446 +448 +449 +452 +453 +454 +455 +457 +458 +459 +460 +462 +463 +466 +467 +468 +469 +470 +472 +475 +477 +478 +479 +480 +481 +482 +483 +484 +485 +487 +489 +490 +491 +492 +493 +494 +495 +496 +497 +498 diff --git a/sql/hive/src/test/resources/golden/having-6-9f50df5b5f31c7166b0396ab434dc095 b/sql/hive/src/test/resources/golden/having-6-9f50df5b5f31c7166b0396ab434dc095 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/having-7-5ad96cb287df02080da1e2594f08d83e b/sql/hive/src/test/resources/golden/having-7-5ad96cb287df02080da1e2594f08d83e new file mode 100644 index 0000000000000..bd545ccf7430c --- /dev/null +++ b/sql/hive/src/test/resources/golden/having-7-5ad96cb287df02080da1e2594f08d83e @@ -0,0 +1,125 @@ +302 +305 +306 +307 +308 +309 +310 +311 +315 +316 +317 +318 +321 +322 +323 +325 +327 +331 +332 +333 +335 +336 +338 +339 +341 +342 +344 +345 +348 +351 +353 +356 +360 +362 +364 +365 +366 +367 +368 +369 +373 +374 +375 +377 +378 +379 +382 +384 +386 +389 +392 +393 +394 +395 +396 +397 +399 +400 +401 +402 +403 +404 +406 +407 +409 +411 +413 +414 +417 +418 +419 +421 +424 +427 +429 +430 +431 +432 +435 +436 +437 +438 +439 +443 +444 +446 +448 +449 +452 +453 +454 +455 +457 +458 +459 +460 +462 +463 +466 +467 +468 +469 +470 +472 +475 +477 +478 +479 +480 +481 +482 +483 +484 +485 +487 +489 +490 +491 +492 +493 +494 +495 +496 +497 +498 diff --git a/sql/hive/src/test/resources/golden/having-8-4aa7197e20b5a64461ca670a79488103 b/sql/hive/src/test/resources/golden/having-8-4aa7197e20b5a64461ca670a79488103 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/having-9-a79743372d86d77b0ff53a71adcb1cff b/sql/hive/src/test/resources/golden/having-9-a79743372d86d77b0ff53a71adcb1cff new file mode 100644 index 0000000000000..d77586c12b6af --- /dev/null +++ b/sql/hive/src/test/resources/golden/having-9-a79743372d86d77b0ff53a71adcb1cff @@ -0,0 +1,199 @@ +4 val_4 +5 val_5 +8 val_8 +9 val_9 +26 val_26 +27 val_27 +28 val_28 +30 val_30 +33 val_33 +34 val_34 +35 val_35 +37 val_37 +41 val_41 +42 val_42 +43 val_43 +44 val_44 +47 val_47 +51 val_51 +53 val_53 +54 val_54 +57 val_57 +58 val_58 +64 val_64 +65 val_65 +66 val_66 +67 val_67 +69 val_69 +70 val_70 +72 val_72 +74 val_74 +76 val_76 +77 val_77 +78 val_78 +80 val_80 +82 val_82 +83 val_83 +84 val_84 +85 val_85 +86 val_86 +87 val_87 +90 val_90 +92 val_92 +95 val_95 +96 val_96 +97 val_97 +98 val_98 +256 val_256 +257 val_257 +258 val_258 +260 val_260 +262 val_262 +263 val_263 +265 val_265 +266 val_266 +272 val_272 +273 val_273 +274 val_274 +275 val_275 +277 val_277 +278 val_278 +280 val_280 +281 val_281 +282 val_282 +283 val_283 +284 val_284 +285 val_285 +286 val_286 +287 val_287 +288 val_288 +289 val_289 +291 val_291 +292 val_292 +296 val_296 +298 val_298 +302 val_302 +305 val_305 +306 val_306 +307 val_307 +308 val_308 +309 val_309 +310 val_310 +311 val_311 +315 val_315 +316 val_316 +317 val_317 +318 val_318 +321 val_321 +322 val_322 +323 val_323 +325 val_325 +327 val_327 +331 val_331 +332 val_332 +333 val_333 +335 val_335 +336 val_336 +338 val_338 +339 val_339 +341 val_341 +342 val_342 +344 val_344 +345 val_345 +348 val_348 +351 val_351 +353 val_353 +356 val_356 +360 val_360 +362 val_362 +364 val_364 +365 val_365 +366 val_366 +367 val_367 +368 val_368 +369 val_369 +373 val_373 +374 val_374 +375 val_375 +377 val_377 +378 val_378 +379 val_379 +382 val_382 +384 val_384 +386 val_386 +389 val_389 +392 val_392 +393 val_393 +394 val_394 +395 val_395 +396 val_396 +397 val_397 +399 val_399 +400 val_400 +401 val_401 +402 val_402 +403 val_403 +404 val_404 +406 val_406 +407 val_407 +409 val_409 +411 val_411 +413 val_413 +414 val_414 +417 val_417 +418 val_418 +419 val_419 +421 val_421 +424 val_424 +427 val_427 +429 val_429 +430 val_430 +431 val_431 +432 val_432 +435 val_435 +436 val_436 +437 val_437 +438 val_438 +439 val_439 +443 val_443 +444 val_444 +446 val_446 +448 val_448 +449 val_449 +452 val_452 +453 val_453 +454 val_454 +455 val_455 +457 val_457 +458 val_458 +459 val_459 +460 val_460 +462 val_462 +463 val_463 +466 val_466 +467 val_467 +468 val_468 +469 val_469 +470 val_470 +472 val_472 +475 val_475 +477 val_477 +478 val_478 +479 val_479 +480 val_480 +481 val_481 +482 val_482 +483 val_483 +484 val_484 +485 val_485 +487 val_487 +489 val_489 +490 val_490 +491 val_491 +492 val_492 +493 val_493 +494 val_494 +495 val_495 +496 val_496 +497 val_497 +498 val_498 diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-0-869726b703f160eabdb7763700b53e60 b/sql/hive/src/test/resources/golden/join_nullsafe-0-869726b703f160eabdb7763700b53e60 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nullsafe-0-869726b703f160eabdb7763700b53e60 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-1-5644ab44e5ba9f2941216b8d5dc33a99 b/sql/hive/src/test/resources/golden/join_nullsafe-1-5644ab44e5ba9f2941216b8d5dc33a99 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-10-b6de4e85dcc1d1949c7431d39fa1b919 b/sql/hive/src/test/resources/golden/join_nullsafe-10-b6de4e85dcc1d1949c7431d39fa1b919 new file mode 100644 index 0000000000000..31c409082cc2f --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nullsafe-10-b6de4e85dcc1d1949c7431d39fa1b919 @@ -0,0 +1,2 @@ +NULL 10 10 NULL NULL 10 +100 100 100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-11-3aa243002a5363b84556736ef71613b1 b/sql/hive/src/test/resources/golden/join_nullsafe-11-3aa243002a5363b84556736ef71613b1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-12-3cc55b14e8256d2c51361b61986c291e b/sql/hive/src/test/resources/golden/join_nullsafe-12-3cc55b14e8256d2c51361b61986c291e new file mode 100644 index 0000000000000..9b77d13cbaab2 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nullsafe-12-3cc55b14e8256d2c51361b61986c291e @@ -0,0 +1,4 @@ +NULL NULL NULL NULL NULL NULL +NULL 10 10 NULL NULL 10 +10 NULL NULL 10 10 NULL +100 100 100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-13-69d94e229191e7b9b1a3e7eae46eb993 b/sql/hive/src/test/resources/golden/join_nullsafe-13-69d94e229191e7b9b1a3e7eae46eb993 new file mode 100644 index 0000000000000..47c0709d39851 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nullsafe-13-69d94e229191e7b9b1a3e7eae46eb993 @@ -0,0 +1,12 @@ +NULL NULL NULL NULL +NULL NULL 10 NULL +NULL NULL 48 NULL +NULL 10 NULL NULL +NULL 10 10 NULL +NULL 10 48 NULL +NULL 35 NULL NULL +NULL 35 10 NULL +NULL 35 48 NULL +10 NULL NULL 10 +48 NULL NULL NULL +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-14-cf9ff6ee72a701a8e2f3e7fb0667903c b/sql/hive/src/test/resources/golden/join_nullsafe-14-cf9ff6ee72a701a8e2f3e7fb0667903c new file mode 100644 index 0000000000000..36ba48516b658 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nullsafe-14-cf9ff6ee72a701a8e2f3e7fb0667903c @@ -0,0 +1,12 @@ +NULL NULL NULL NULL +NULL NULL NULL 35 +NULL NULL 10 NULL +NULL NULL 48 NULL +NULL 10 NULL NULL +NULL 10 10 NULL +NULL 10 48 NULL +NULL 35 NULL NULL +NULL 35 10 NULL +NULL 35 48 NULL +10 NULL NULL 10 +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-15-507d0fa6d7ce39e2d9921555cea6f8da b/sql/hive/src/test/resources/golden/join_nullsafe-15-507d0fa6d7ce39e2d9921555cea6f8da new file mode 100644 index 0000000000000..fc1fd198cf8be --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nullsafe-15-507d0fa6d7ce39e2d9921555cea6f8da @@ -0,0 +1,13 @@ +NULL NULL NULL NULL +NULL NULL NULL 35 +NULL NULL 10 NULL +NULL NULL 48 NULL +NULL 10 NULL NULL +NULL 10 10 NULL +NULL 10 48 NULL +NULL 35 NULL NULL +NULL 35 10 NULL +NULL 35 48 NULL +10 NULL NULL 10 +48 NULL NULL NULL +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-16-1c714fc339304de4db630530e5d1ce97 b/sql/hive/src/test/resources/golden/join_nullsafe-16-1c714fc339304de4db630530e5d1ce97 new file mode 100644 index 0000000000000..1cc70524f9d6d --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nullsafe-16-1c714fc339304de4db630530e5d1ce97 @@ -0,0 +1,11 @@ +NULL NULL NULL NULL +NULL NULL 10 NULL +NULL NULL 48 NULL +NULL 10 NULL NULL +NULL 10 10 NULL +NULL 10 48 NULL +NULL 35 NULL NULL +NULL 35 10 NULL +NULL 35 48 NULL +10 NULL NULL 10 +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-17-8a4b0dc781a28ad11a0db9805fe03aa8 b/sql/hive/src/test/resources/golden/join_nullsafe-17-8a4b0dc781a28ad11a0db9805fe03aa8 new file mode 100644 index 0000000000000..1cc70524f9d6d --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nullsafe-17-8a4b0dc781a28ad11a0db9805fe03aa8 @@ -0,0 +1,11 @@ +NULL NULL NULL NULL +NULL NULL 10 NULL +NULL NULL 48 NULL +NULL 10 NULL NULL +NULL 10 10 NULL +NULL 10 48 NULL +NULL 35 NULL NULL +NULL 35 10 NULL +NULL 35 48 NULL +10 NULL NULL 10 +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-18-10b2051e65cac50ee1ea1c138ec192c8 b/sql/hive/src/test/resources/golden/join_nullsafe-18-10b2051e65cac50ee1ea1c138ec192c8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-19-23ab7ac8229a53d391195be7ca092429 b/sql/hive/src/test/resources/golden/join_nullsafe-19-23ab7ac8229a53d391195be7ca092429 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-2-793e288c9e0971f0bf3f37493f76dc7 b/sql/hive/src/test/resources/golden/join_nullsafe-2-793e288c9e0971f0bf3f37493f76dc7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-20-d6fc260320c577eec9a5db0d4135d224 b/sql/hive/src/test/resources/golden/join_nullsafe-20-d6fc260320c577eec9a5db0d4135d224 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-21-a60dae725ffc543f805242611d99de4e b/sql/hive/src/test/resources/golden/join_nullsafe-21-a60dae725ffc543f805242611d99de4e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-22-24c80d0f9e3d72c48d947770fa184985 b/sql/hive/src/test/resources/golden/join_nullsafe-22-24c80d0f9e3d72c48d947770fa184985 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-23-3fe6ae20cab3417759dcc654a3a26746 b/sql/hive/src/test/resources/golden/join_nullsafe-23-3fe6ae20cab3417759dcc654a3a26746 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-24-2db30531137611e06fdba478ca7a8412 b/sql/hive/src/test/resources/golden/join_nullsafe-24-2db30531137611e06fdba478ca7a8412 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nullsafe-24-2db30531137611e06fdba478ca7a8412 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-25-e58b2754e8d9c56a473557a549d0d2b9 b/sql/hive/src/test/resources/golden/join_nullsafe-25-e58b2754e8d9c56a473557a549d0d2b9 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nullsafe-25-e58b2754e8d9c56a473557a549d0d2b9 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-26-64cabe5164130a94f387288f37b62d71 b/sql/hive/src/test/resources/golden/join_nullsafe-26-64cabe5164130a94f387288f37b62d71 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nullsafe-26-64cabe5164130a94f387288f37b62d71 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-27-e8ed4a1b574a6ca70cbfb3f7b9980aa6 b/sql/hive/src/test/resources/golden/join_nullsafe-27-e8ed4a1b574a6ca70cbfb3f7b9980aa6 new file mode 100644 index 0000000000000..66482299904bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nullsafe-27-e8ed4a1b574a6ca70cbfb3f7b9980aa6 @@ -0,0 +1,42 @@ +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL NULL 10 +NULL NULL NULL 10 +NULL NULL NULL 35 +NULL NULL NULL 35 +NULL NULL NULL 110 +NULL NULL NULL 110 +NULL NULL NULL 135 +NULL NULL NULL 135 +NULL 10 NULL NULL +NULL 10 NULL NULL +NULL 10 NULL 10 +NULL 10 NULL 35 +NULL 10 NULL 110 +NULL 10 NULL 135 +NULL 35 NULL NULL +NULL 35 NULL NULL +NULL 35 NULL 10 +NULL 35 NULL 35 +NULL 35 NULL 110 +NULL 35 NULL 135 +NULL 110 NULL NULL +NULL 110 NULL NULL +NULL 110 NULL 10 +NULL 110 NULL 35 +NULL 110 NULL 110 +NULL 110 NULL 135 +NULL 135 NULL NULL +NULL 135 NULL NULL +NULL 135 NULL 10 +NULL 135 NULL 35 +NULL 135 NULL 110 +NULL 135 NULL 135 +10 NULL 10 NULL +48 NULL 48 NULL +100 100 100 100 +110 NULL 110 NULL +148 NULL 148 NULL +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-28-5a0c946cd7033857ca99e5fb800f8525 b/sql/hive/src/test/resources/golden/join_nullsafe-28-5a0c946cd7033857ca99e5fb800f8525 new file mode 100644 index 0000000000000..2efbef0484452 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nullsafe-28-5a0c946cd7033857ca99e5fb800f8525 @@ -0,0 +1,14 @@ +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL 10 NULL 10 +NULL 35 NULL 35 +NULL 110 NULL 110 +NULL 135 NULL 135 +10 NULL 10 NULL +48 NULL 48 NULL +100 100 100 100 +110 NULL 110 NULL +148 NULL 148 NULL +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-29-514043c2ddaf6ea8f16a764adc92d1cf b/sql/hive/src/test/resources/golden/join_nullsafe-29-514043c2ddaf6ea8f16a764adc92d1cf new file mode 100644 index 0000000000000..66482299904bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nullsafe-29-514043c2ddaf6ea8f16a764adc92d1cf @@ -0,0 +1,42 @@ +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL NULL 10 +NULL NULL NULL 10 +NULL NULL NULL 35 +NULL NULL NULL 35 +NULL NULL NULL 110 +NULL NULL NULL 110 +NULL NULL NULL 135 +NULL NULL NULL 135 +NULL 10 NULL NULL +NULL 10 NULL NULL +NULL 10 NULL 10 +NULL 10 NULL 35 +NULL 10 NULL 110 +NULL 10 NULL 135 +NULL 35 NULL NULL +NULL 35 NULL NULL +NULL 35 NULL 10 +NULL 35 NULL 35 +NULL 35 NULL 110 +NULL 35 NULL 135 +NULL 110 NULL NULL +NULL 110 NULL NULL +NULL 110 NULL 10 +NULL 110 NULL 35 +NULL 110 NULL 110 +NULL 110 NULL 135 +NULL 135 NULL NULL +NULL 135 NULL NULL +NULL 135 NULL 10 +NULL 135 NULL 35 +NULL 135 NULL 110 +NULL 135 NULL 135 +10 NULL 10 NULL +48 NULL 48 NULL +100 100 100 100 +110 NULL 110 NULL +148 NULL 148 NULL +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-3-ae378fc0f875a21884e58fa35a6d52cd b/sql/hive/src/test/resources/golden/join_nullsafe-3-ae378fc0f875a21884e58fa35a6d52cd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-30-fcbf92cb1b85ab01102fbbc6caba9a88 b/sql/hive/src/test/resources/golden/join_nullsafe-30-fcbf92cb1b85ab01102fbbc6caba9a88 new file mode 100644 index 0000000000000..66482299904bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nullsafe-30-fcbf92cb1b85ab01102fbbc6caba9a88 @@ -0,0 +1,42 @@ +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL NULL 10 +NULL NULL NULL 10 +NULL NULL NULL 35 +NULL NULL NULL 35 +NULL NULL NULL 110 +NULL NULL NULL 110 +NULL NULL NULL 135 +NULL NULL NULL 135 +NULL 10 NULL NULL +NULL 10 NULL NULL +NULL 10 NULL 10 +NULL 10 NULL 35 +NULL 10 NULL 110 +NULL 10 NULL 135 +NULL 35 NULL NULL +NULL 35 NULL NULL +NULL 35 NULL 10 +NULL 35 NULL 35 +NULL 35 NULL 110 +NULL 35 NULL 135 +NULL 110 NULL NULL +NULL 110 NULL NULL +NULL 110 NULL 10 +NULL 110 NULL 35 +NULL 110 NULL 110 +NULL 110 NULL 135 +NULL 135 NULL NULL +NULL 135 NULL NULL +NULL 135 NULL 10 +NULL 135 NULL 35 +NULL 135 NULL 110 +NULL 135 NULL 135 +10 NULL 10 NULL +48 NULL 48 NULL +100 100 100 100 +110 NULL 110 NULL +148 NULL 148 NULL +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-31-1cb03e1106f79d14f22bc89d386cedcf b/sql/hive/src/test/resources/golden/join_nullsafe-31-1cb03e1106f79d14f22bc89d386cedcf new file mode 100644 index 0000000000000..66482299904bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nullsafe-31-1cb03e1106f79d14f22bc89d386cedcf @@ -0,0 +1,42 @@ +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL NULL 10 +NULL NULL NULL 10 +NULL NULL NULL 35 +NULL NULL NULL 35 +NULL NULL NULL 110 +NULL NULL NULL 110 +NULL NULL NULL 135 +NULL NULL NULL 135 +NULL 10 NULL NULL +NULL 10 NULL NULL +NULL 10 NULL 10 +NULL 10 NULL 35 +NULL 10 NULL 110 +NULL 10 NULL 135 +NULL 35 NULL NULL +NULL 35 NULL NULL +NULL 35 NULL 10 +NULL 35 NULL 35 +NULL 35 NULL 110 +NULL 35 NULL 135 +NULL 110 NULL NULL +NULL 110 NULL NULL +NULL 110 NULL 10 +NULL 110 NULL 35 +NULL 110 NULL 110 +NULL 110 NULL 135 +NULL 135 NULL NULL +NULL 135 NULL NULL +NULL 135 NULL 10 +NULL 135 NULL 35 +NULL 135 NULL 110 +NULL 135 NULL 135 +10 NULL 10 NULL +48 NULL 48 NULL +100 100 100 100 +110 NULL 110 NULL +148 NULL 148 NULL +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-32-6a0bf6127d4b042e67ae8ee15125fb87 b/sql/hive/src/test/resources/golden/join_nullsafe-32-6a0bf6127d4b042e67ae8ee15125fb87 new file mode 100644 index 0000000000000..ea001a222f357 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nullsafe-32-6a0bf6127d4b042e67ae8ee15125fb87 @@ -0,0 +1,40 @@ +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL 10 NULL +NULL NULL 10 NULL +NULL NULL 48 NULL +NULL NULL 48 NULL +NULL NULL 110 NULL +NULL NULL 110 NULL +NULL NULL 148 NULL +NULL NULL 148 NULL +NULL 10 NULL NULL +NULL 10 NULL NULL +NULL 10 10 NULL +NULL 10 48 NULL +NULL 10 110 NULL +NULL 10 148 NULL +NULL 35 NULL NULL +NULL 35 NULL NULL +NULL 35 10 NULL +NULL 35 48 NULL +NULL 35 110 NULL +NULL 35 148 NULL +NULL 110 NULL NULL +NULL 110 NULL NULL +NULL 110 10 NULL +NULL 110 48 NULL +NULL 110 110 NULL +NULL 110 148 NULL +NULL 135 NULL NULL +NULL 135 NULL NULL +NULL 135 10 NULL +NULL 135 48 NULL +NULL 135 110 NULL +NULL 135 148 NULL +10 NULL NULL 10 +100 100 100 100 +110 NULL NULL 110 +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-33-63157d43422fcedadba408537ccecd5c b/sql/hive/src/test/resources/golden/join_nullsafe-33-63157d43422fcedadba408537ccecd5c new file mode 100644 index 0000000000000..ea001a222f357 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nullsafe-33-63157d43422fcedadba408537ccecd5c @@ -0,0 +1,40 @@ +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL 10 NULL +NULL NULL 10 NULL +NULL NULL 48 NULL +NULL NULL 48 NULL +NULL NULL 110 NULL +NULL NULL 110 NULL +NULL NULL 148 NULL +NULL NULL 148 NULL +NULL 10 NULL NULL +NULL 10 NULL NULL +NULL 10 10 NULL +NULL 10 48 NULL +NULL 10 110 NULL +NULL 10 148 NULL +NULL 35 NULL NULL +NULL 35 NULL NULL +NULL 35 10 NULL +NULL 35 48 NULL +NULL 35 110 NULL +NULL 35 148 NULL +NULL 110 NULL NULL +NULL 110 NULL NULL +NULL 110 10 NULL +NULL 110 48 NULL +NULL 110 110 NULL +NULL 110 148 NULL +NULL 135 NULL NULL +NULL 135 NULL NULL +NULL 135 10 NULL +NULL 135 48 NULL +NULL 135 110 NULL +NULL 135 148 NULL +10 NULL NULL 10 +100 100 100 100 +110 NULL NULL 110 +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-34-9265f806b71c03061f93f9fbc88aa223 b/sql/hive/src/test/resources/golden/join_nullsafe-34-9265f806b71c03061f93f9fbc88aa223 new file mode 100644 index 0000000000000..1093bd89f6e3f --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nullsafe-34-9265f806b71c03061f93f9fbc88aa223 @@ -0,0 +1,42 @@ +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL 10 NULL +NULL NULL 10 NULL +NULL NULL 48 NULL +NULL NULL 48 NULL +NULL NULL 110 NULL +NULL NULL 110 NULL +NULL NULL 148 NULL +NULL NULL 148 NULL +NULL 10 NULL NULL +NULL 10 NULL NULL +NULL 10 10 NULL +NULL 10 48 NULL +NULL 10 110 NULL +NULL 10 148 NULL +NULL 35 NULL NULL +NULL 35 NULL NULL +NULL 35 10 NULL +NULL 35 48 NULL +NULL 35 110 NULL +NULL 35 148 NULL +NULL 110 NULL NULL +NULL 110 NULL NULL +NULL 110 10 NULL +NULL 110 48 NULL +NULL 110 110 NULL +NULL 110 148 NULL +NULL 135 NULL NULL +NULL 135 NULL NULL +NULL 135 10 NULL +NULL 135 48 NULL +NULL 135 110 NULL +NULL 135 148 NULL +10 NULL NULL 10 +48 NULL NULL NULL +100 100 100 100 +110 NULL NULL 110 +148 NULL NULL NULL +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-35-95815bafb81cccb8129c20d399a446fc b/sql/hive/src/test/resources/golden/join_nullsafe-35-95815bafb81cccb8129c20d399a446fc new file mode 100644 index 0000000000000..9cf0036674d6e --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nullsafe-35-95815bafb81cccb8129c20d399a446fc @@ -0,0 +1,42 @@ +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL NULL 35 +NULL NULL NULL 135 +NULL NULL 10 NULL +NULL NULL 10 NULL +NULL NULL 48 NULL +NULL NULL 48 NULL +NULL NULL 110 NULL +NULL NULL 110 NULL +NULL NULL 148 NULL +NULL NULL 148 NULL +NULL 10 NULL NULL +NULL 10 NULL NULL +NULL 10 10 NULL +NULL 10 48 NULL +NULL 10 110 NULL +NULL 10 148 NULL +NULL 35 NULL NULL +NULL 35 NULL NULL +NULL 35 10 NULL +NULL 35 48 NULL +NULL 35 110 NULL +NULL 35 148 NULL +NULL 110 NULL NULL +NULL 110 NULL NULL +NULL 110 10 NULL +NULL 110 48 NULL +NULL 110 110 NULL +NULL 110 148 NULL +NULL 135 NULL NULL +NULL 135 NULL NULL +NULL 135 10 NULL +NULL 135 48 NULL +NULL 135 110 NULL +NULL 135 148 NULL +10 NULL NULL 10 +100 100 100 100 +110 NULL NULL 110 +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-36-c4762c60cc93236b7647ebd32a40ce57 b/sql/hive/src/test/resources/golden/join_nullsafe-36-c4762c60cc93236b7647ebd32a40ce57 new file mode 100644 index 0000000000000..77f6a8ddd7c28 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nullsafe-36-c4762c60cc93236b7647ebd32a40ce57 @@ -0,0 +1,42 @@ +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL 10 NULL +NULL NULL 10 NULL +NULL NULL 48 NULL +NULL NULL 48 NULL +NULL NULL 110 NULL +NULL NULL 110 NULL +NULL NULL 148 NULL +NULL NULL 148 NULL +NULL 10 NULL 10 +NULL 35 NULL 35 +NULL 110 NULL 110 +NULL 135 NULL 135 +10 NULL NULL NULL +10 NULL NULL NULL +10 NULL 10 NULL +10 NULL 48 NULL +10 NULL 110 NULL +10 NULL 148 NULL +48 NULL NULL NULL +48 NULL NULL NULL +48 NULL 10 NULL +48 NULL 48 NULL +48 NULL 110 NULL +48 NULL 148 NULL +100 100 100 100 +110 NULL NULL NULL +110 NULL NULL NULL +110 NULL 10 NULL +110 NULL 48 NULL +110 NULL 110 NULL +110 NULL 148 NULL +148 NULL NULL NULL +148 NULL NULL NULL +148 NULL 10 NULL +148 NULL 48 NULL +148 NULL 110 NULL +148 NULL 148 NULL +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-37-a87893adfc73c9cc63ceab200bb56245 b/sql/hive/src/test/resources/golden/join_nullsafe-37-a87893adfc73c9cc63ceab200bb56245 new file mode 100644 index 0000000000000..77f6a8ddd7c28 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nullsafe-37-a87893adfc73c9cc63ceab200bb56245 @@ -0,0 +1,42 @@ +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL 10 NULL +NULL NULL 10 NULL +NULL NULL 48 NULL +NULL NULL 48 NULL +NULL NULL 110 NULL +NULL NULL 110 NULL +NULL NULL 148 NULL +NULL NULL 148 NULL +NULL 10 NULL 10 +NULL 35 NULL 35 +NULL 110 NULL 110 +NULL 135 NULL 135 +10 NULL NULL NULL +10 NULL NULL NULL +10 NULL 10 NULL +10 NULL 48 NULL +10 NULL 110 NULL +10 NULL 148 NULL +48 NULL NULL NULL +48 NULL NULL NULL +48 NULL 10 NULL +48 NULL 48 NULL +48 NULL 110 NULL +48 NULL 148 NULL +100 100 100 100 +110 NULL NULL NULL +110 NULL NULL NULL +110 NULL 10 NULL +110 NULL 48 NULL +110 NULL 110 NULL +110 NULL 148 NULL +148 NULL NULL NULL +148 NULL NULL NULL +148 NULL 10 NULL +148 NULL 48 NULL +148 NULL 110 NULL +148 NULL 148 NULL +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-38-e3dfe0044b44c8a49414479521acf762 b/sql/hive/src/test/resources/golden/join_nullsafe-38-e3dfe0044b44c8a49414479521acf762 new file mode 100644 index 0000000000000..77f6a8ddd7c28 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nullsafe-38-e3dfe0044b44c8a49414479521acf762 @@ -0,0 +1,42 @@ +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL 10 NULL +NULL NULL 10 NULL +NULL NULL 48 NULL +NULL NULL 48 NULL +NULL NULL 110 NULL +NULL NULL 110 NULL +NULL NULL 148 NULL +NULL NULL 148 NULL +NULL 10 NULL 10 +NULL 35 NULL 35 +NULL 110 NULL 110 +NULL 135 NULL 135 +10 NULL NULL NULL +10 NULL NULL NULL +10 NULL 10 NULL +10 NULL 48 NULL +10 NULL 110 NULL +10 NULL 148 NULL +48 NULL NULL NULL +48 NULL NULL NULL +48 NULL 10 NULL +48 NULL 48 NULL +48 NULL 110 NULL +48 NULL 148 NULL +100 100 100 100 +110 NULL NULL NULL +110 NULL NULL NULL +110 NULL 10 NULL +110 NULL 48 NULL +110 NULL 110 NULL +110 NULL 148 NULL +148 NULL NULL NULL +148 NULL NULL NULL +148 NULL 10 NULL +148 NULL 48 NULL +148 NULL 110 NULL +148 NULL 148 NULL +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-39-9a7e1f373b9c02e632d6c7c550b908ec b/sql/hive/src/test/resources/golden/join_nullsafe-39-9a7e1f373b9c02e632d6c7c550b908ec new file mode 100644 index 0000000000000..77f6a8ddd7c28 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nullsafe-39-9a7e1f373b9c02e632d6c7c550b908ec @@ -0,0 +1,42 @@ +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL 10 NULL +NULL NULL 10 NULL +NULL NULL 48 NULL +NULL NULL 48 NULL +NULL NULL 110 NULL +NULL NULL 110 NULL +NULL NULL 148 NULL +NULL NULL 148 NULL +NULL 10 NULL 10 +NULL 35 NULL 35 +NULL 110 NULL 110 +NULL 135 NULL 135 +10 NULL NULL NULL +10 NULL NULL NULL +10 NULL 10 NULL +10 NULL 48 NULL +10 NULL 110 NULL +10 NULL 148 NULL +48 NULL NULL NULL +48 NULL NULL NULL +48 NULL 10 NULL +48 NULL 48 NULL +48 NULL 110 NULL +48 NULL 148 NULL +100 100 100 100 +110 NULL NULL NULL +110 NULL NULL NULL +110 NULL 10 NULL +110 NULL 48 NULL +110 NULL 110 NULL +110 NULL 148 NULL +148 NULL NULL NULL +148 NULL NULL NULL +148 NULL 10 NULL +148 NULL 48 NULL +148 NULL 110 NULL +148 NULL 148 NULL +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-4-644c616d87ae426eb2f8c71638045185 b/sql/hive/src/test/resources/golden/join_nullsafe-4-644c616d87ae426eb2f8c71638045185 new file mode 100644 index 0000000000000..1cc70524f9d6d --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nullsafe-4-644c616d87ae426eb2f8c71638045185 @@ -0,0 +1,11 @@ +NULL NULL NULL NULL +NULL NULL 10 NULL +NULL NULL 48 NULL +NULL 10 NULL NULL +NULL 10 10 NULL +NULL 10 48 NULL +NULL 35 NULL NULL +NULL 35 10 NULL +NULL 35 48 NULL +10 NULL NULL 10 +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-40-3c868718e4c120cb9a72ab7318c75be3 b/sql/hive/src/test/resources/golden/join_nullsafe-40-3c868718e4c120cb9a72ab7318c75be3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-41-1f7d8737c3e2d74d5ad865535d729811 b/sql/hive/src/test/resources/golden/join_nullsafe-41-1f7d8737c3e2d74d5ad865535d729811 new file mode 100644 index 0000000000000..421049d6e509e --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nullsafe-41-1f7d8737c3e2d74d5ad865535d729811 @@ -0,0 +1,9 @@ +NULL NULL NULL NULL +NULL NULL 10 NULL +NULL NULL 48 NULL +NULL 10 NULL NULL +NULL 10 10 NULL +NULL 10 48 NULL +NULL 35 NULL NULL +NULL 35 10 NULL +NULL 35 48 NULL diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-5-1e393de94850e92b3b00536aacc9371f b/sql/hive/src/test/resources/golden/join_nullsafe-5-1e393de94850e92b3b00536aacc9371f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-6-d66451815212e7d17744184e74c6b0a0 b/sql/hive/src/test/resources/golden/join_nullsafe-6-d66451815212e7d17744184e74c6b0a0 new file mode 100644 index 0000000000000..aec3122cae5f9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nullsafe-6-d66451815212e7d17744184e74c6b0a0 @@ -0,0 +1,2 @@ +10 NULL NULL 10 10 NULL +100 100 100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-7-a3ad3cc301d9884898d3e6ab6c792d4c b/sql/hive/src/test/resources/golden/join_nullsafe-7-a3ad3cc301d9884898d3e6ab6c792d4c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-8-cc7527bcf746ab7e2cd9f28db0ead0ac b/sql/hive/src/test/resources/golden/join_nullsafe-8-cc7527bcf746ab7e2cd9f28db0ead0ac new file mode 100644 index 0000000000000..30db79efa79b4 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nullsafe-8-cc7527bcf746ab7e2cd9f28db0ead0ac @@ -0,0 +1,29 @@ +NULL NULL NULL NULL NULL NULL +NULL NULL NULL NULL NULL 10 +NULL NULL NULL NULL NULL 35 +NULL NULL 10 NULL NULL NULL +NULL NULL 10 NULL NULL 10 +NULL NULL 10 NULL NULL 35 +NULL NULL 48 NULL NULL NULL +NULL NULL 48 NULL NULL 10 +NULL NULL 48 NULL NULL 35 +NULL 10 NULL NULL NULL NULL +NULL 10 NULL NULL NULL 10 +NULL 10 NULL NULL NULL 35 +NULL 10 10 NULL NULL NULL +NULL 10 10 NULL NULL 10 +NULL 10 10 NULL NULL 35 +NULL 10 48 NULL NULL NULL +NULL 10 48 NULL NULL 10 +NULL 10 48 NULL NULL 35 +NULL 35 NULL NULL NULL NULL +NULL 35 NULL NULL NULL 10 +NULL 35 NULL NULL NULL 35 +NULL 35 10 NULL NULL NULL +NULL 35 10 NULL NULL 10 +NULL 35 10 NULL NULL 35 +NULL 35 48 NULL NULL NULL +NULL 35 48 NULL NULL 10 +NULL 35 48 NULL NULL 35 +10 NULL NULL 10 10 NULL +100 100 100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-9-88f6f40959b0d2faabd9d4b3cd853809 b/sql/hive/src/test/resources/golden/join_nullsafe-9-88f6f40959b0d2faabd9d4b3cd853809 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/limit_pushdown_negative-0-79b294d0081c3dfd36c5b8b5e78dc7fb b/sql/hive/src/test/resources/golden/limit_pushdown_negative-0-79b294d0081c3dfd36c5b8b5e78dc7fb new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/limit_pushdown_negative-0-79b294d0081c3dfd36c5b8b5e78dc7fb @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/limit_pushdown_negative-1-f87339637a48bd1533493ebbed5432a7 b/sql/hive/src/test/resources/golden/limit_pushdown_negative-1-f87339637a48bd1533493ebbed5432a7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/limit_pushdown_negative-2-de7e5ac581b870fff10dc82c75c1c79e b/sql/hive/src/test/resources/golden/limit_pushdown_negative-2-de7e5ac581b870fff10dc82c75c1c79e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/limit_pushdown_negative-3-be440c3f959ca53b758481aa90551984 b/sql/hive/src/test/resources/golden/limit_pushdown_negative-3-be440c3f959ca53b758481aa90551984 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/limit_pushdown_negative-4-4dedc8057d76af264c198beaacd7f000 b/sql/hive/src/test/resources/golden/limit_pushdown_negative-4-4dedc8057d76af264c198beaacd7f000 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/limit_pushdown_negative-5-543a20e69bd8987bc37a22c1c7ef33f1 b/sql/hive/src/test/resources/golden/limit_pushdown_negative-5-543a20e69bd8987bc37a22c1c7ef33f1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/limit_pushdown_negative-6-3f8274466914ad200b33a2c83fa6dab5 b/sql/hive/src/test/resources/golden/limit_pushdown_negative-6-3f8274466914ad200b33a2c83fa6dab5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/limit_pushdown_negative-7-fb7bf3783d4fb43673a202c4111d9092 b/sql/hive/src/test/resources/golden/limit_pushdown_negative-7-fb7bf3783d4fb43673a202c4111d9092 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/null case-0-581cdfe70091e546414b202da2cebdcb b/sql/hive/src/test/resources/golden/null case-0-581cdfe70091e546414b202da2cebdcb new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/null case-0-581cdfe70091e546414b202da2cebdcb @@ -0,0 +1 @@ +1 diff --git a/sql/hive/src/test/resources/golden/partition_based_table_scan_with_different_serde-0-1436cccda63b78dd6e43a399da6cc474 b/sql/hive/src/test/resources/golden/partition_based_table_scan_with_different_serde-0-1436cccda63b78dd6e43a399da6cc474 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_based_table_scan_with_different_serde-1-8d9bf54373f45bc35f8cb6e82771b154 b/sql/hive/src/test/resources/golden/partition_based_table_scan_with_different_serde-1-8d9bf54373f45bc35f8cb6e82771b154 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_based_table_scan_with_different_serde-2-7816c17905012cf381abf93d230faa8d b/sql/hive/src/test/resources/golden/partition_based_table_scan_with_different_serde-2-7816c17905012cf381abf93d230faa8d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_based_table_scan_with_different_serde-3-90089a6db3c3d8ee5ff5ea6b9153b3cc b/sql/hive/src/test/resources/golden/partition_based_table_scan_with_different_serde-3-90089a6db3c3d8ee5ff5ea6b9153b3cc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_based_table_scan_with_different_serde-4-8caed2a6e80250a6d38a59388679c298 b/sql/hive/src/test/resources/golden/partition_based_table_scan_with_different_serde-4-8caed2a6e80250a6d38a59388679c298 new file mode 100644 index 0000000000000..f369f21e1833f --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_based_table_scan_with_different_serde-4-8caed2a6e80250a6d38a59388679c298 @@ -0,0 +1,2 @@ +100 100 2010-01-01 +200 200 2010-01-02 diff --git a/sql/hive/src/test/resources/golden/show_create_table_alter-0-813886d6cf0875c62e89cd1d06b8b0b4 b/sql/hive/src/test/resources/golden/show_create_table_alter-0-813886d6cf0875c62e89cd1d06b8b0b4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_create_table_alter-1-2a91d52719cf4552ebeb867204552a26 b/sql/hive/src/test/resources/golden/show_create_table_alter-1-2a91d52719cf4552ebeb867204552a26 new file mode 100644 index 0000000000000..3c1fc128bedce --- /dev/null +++ b/sql/hive/src/test/resources/golden/show_create_table_alter-1-2a91d52719cf4552ebeb867204552a26 @@ -0,0 +1,18 @@ +CREATE EXTERNAL TABLE tmp_showcrt1( + key smallint, + value float) +CLUSTERED BY ( + key) +SORTED BY ( + value DESC) +INTO 5 BUCKETS +ROW FORMAT SERDE + 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' +STORED AS INPUTFORMAT + 'org.apache.hadoop.mapred.TextInputFormat' +OUTPUTFORMAT + 'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat' +LOCATION + 'file:/tmp/sparkHiveWarehouse1280221975983654134/tmp_showcrt1' +TBLPROPERTIES ( + 'transient_lastDdlTime'='1407132100') diff --git a/sql/hive/src/test/resources/golden/show_create_table_alter-10-259d978ed9543204c8b9c25b6e25b0de b/sql/hive/src/test/resources/golden/show_create_table_alter-10-259d978ed9543204c8b9c25b6e25b0de new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_create_table_alter-2-928cc85c025440b731e5ee33e437e404 b/sql/hive/src/test/resources/golden/show_create_table_alter-2-928cc85c025440b731e5ee33e437e404 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_create_table_alter-3-2a91d52719cf4552ebeb867204552a26 b/sql/hive/src/test/resources/golden/show_create_table_alter-3-2a91d52719cf4552ebeb867204552a26 new file mode 100644 index 0000000000000..2ece813dd7d56 --- /dev/null +++ b/sql/hive/src/test/resources/golden/show_create_table_alter-3-2a91d52719cf4552ebeb867204552a26 @@ -0,0 +1,22 @@ +CREATE TABLE tmp_showcrt1( + key smallint, + value float) +COMMENT 'temporary table' +CLUSTERED BY ( + key) +SORTED BY ( + value DESC) +INTO 5 BUCKETS +ROW FORMAT SERDE + 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' +STORED AS INPUTFORMAT + 'org.apache.hadoop.mapred.TextInputFormat' +OUTPUTFORMAT + 'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat' +LOCATION + 'file:/tmp/sparkHiveWarehouse1280221975983654134/tmp_showcrt1' +TBLPROPERTIES ( + 'EXTERNAL'='FALSE', + 'last_modified_by'='tianyi', + 'last_modified_time'='1407132100', + 'transient_lastDdlTime'='1407132100') diff --git a/sql/hive/src/test/resources/golden/show_create_table_alter-4-c2cb6a7d942d4dddd1aababccb1239f9 b/sql/hive/src/test/resources/golden/show_create_table_alter-4-c2cb6a7d942d4dddd1aababccb1239f9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_create_table_alter-5-2a91d52719cf4552ebeb867204552a26 b/sql/hive/src/test/resources/golden/show_create_table_alter-5-2a91d52719cf4552ebeb867204552a26 new file mode 100644 index 0000000000000..2af657bd29506 --- /dev/null +++ b/sql/hive/src/test/resources/golden/show_create_table_alter-5-2a91d52719cf4552ebeb867204552a26 @@ -0,0 +1,21 @@ +CREATE EXTERNAL TABLE tmp_showcrt1( + key smallint, + value float) +COMMENT 'changed comment' +CLUSTERED BY ( + key) +SORTED BY ( + value DESC) +INTO 5 BUCKETS +ROW FORMAT SERDE + 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' +STORED AS INPUTFORMAT + 'org.apache.hadoop.mapred.TextInputFormat' +OUTPUTFORMAT + 'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat' +LOCATION + 'file:/tmp/sparkHiveWarehouse1280221975983654134/tmp_showcrt1' +TBLPROPERTIES ( + 'last_modified_by'='tianyi', + 'last_modified_time'='1407132100', + 'transient_lastDdlTime'='1407132100') diff --git a/sql/hive/src/test/resources/golden/show_create_table_alter-6-fdd1bd7f9acf0b2c8c9b7503d4046cb b/sql/hive/src/test/resources/golden/show_create_table_alter-6-fdd1bd7f9acf0b2c8c9b7503d4046cb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_create_table_alter-7-2a91d52719cf4552ebeb867204552a26 b/sql/hive/src/test/resources/golden/show_create_table_alter-7-2a91d52719cf4552ebeb867204552a26 new file mode 100644 index 0000000000000..f793ffb7a0bfd --- /dev/null +++ b/sql/hive/src/test/resources/golden/show_create_table_alter-7-2a91d52719cf4552ebeb867204552a26 @@ -0,0 +1,21 @@ +CREATE EXTERNAL TABLE tmp_showcrt1( + key smallint, + value float) +COMMENT 'changed comment' +CLUSTERED BY ( + key) +SORTED BY ( + value DESC) +INTO 5 BUCKETS +ROW FORMAT SERDE + 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' +STORED AS INPUTFORMAT + 'org.apache.hadoop.mapred.TextInputFormat' +OUTPUTFORMAT + 'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat' +LOCATION + 'file:/tmp/sparkHiveWarehouse1280221975983654134/tmp_showcrt1' +TBLPROPERTIES ( + 'last_modified_by'='tianyi', + 'last_modified_time'='1407132101', + 'transient_lastDdlTime'='1407132101') diff --git a/sql/hive/src/test/resources/golden/show_create_table_alter-8-22ab6ed5b15a018756f454dd2294847e b/sql/hive/src/test/resources/golden/show_create_table_alter-8-22ab6ed5b15a018756f454dd2294847e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_create_table_alter-9-2a91d52719cf4552ebeb867204552a26 b/sql/hive/src/test/resources/golden/show_create_table_alter-9-2a91d52719cf4552ebeb867204552a26 new file mode 100644 index 0000000000000..c65aff26a7fc1 --- /dev/null +++ b/sql/hive/src/test/resources/golden/show_create_table_alter-9-2a91d52719cf4552ebeb867204552a26 @@ -0,0 +1,21 @@ +CREATE EXTERNAL TABLE tmp_showcrt1( + key smallint, + value float) +COMMENT 'changed comment' +CLUSTERED BY ( + key) +SORTED BY ( + value DESC) +INTO 5 BUCKETS +ROW FORMAT SERDE + 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' +STORED BY + 'org.apache.hadoop.hive.ql.metadata.DefaultStorageHandler' +WITH SERDEPROPERTIES ( + 'serialization.format'='1') +LOCATION + 'file:/tmp/sparkHiveWarehouse1280221975983654134/tmp_showcrt1' +TBLPROPERTIES ( + 'last_modified_by'='tianyi', + 'last_modified_time'='1407132101', + 'transient_lastDdlTime'='1407132101') diff --git a/sql/hive/src/test/resources/golden/show_create_table_db_table-0-67509558a4b2d39b25787cca33f52635 b/sql/hive/src/test/resources/golden/show_create_table_db_table-0-67509558a4b2d39b25787cca33f52635 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_create_table_db_table-1-549981e00a3d95f03dd5a9ef6044aa20 b/sql/hive/src/test/resources/golden/show_create_table_db_table-1-549981e00a3d95f03dd5a9ef6044aa20 new file mode 100644 index 0000000000000..707b2ae3ed1df --- /dev/null +++ b/sql/hive/src/test/resources/golden/show_create_table_db_table-1-549981e00a3d95f03dd5a9ef6044aa20 @@ -0,0 +1,2 @@ +default +tmp_feng diff --git a/sql/hive/src/test/resources/golden/show_create_table_db_table-2-34ae7e611d0aedbc62b6e420347abee b/sql/hive/src/test/resources/golden/show_create_table_db_table-2-34ae7e611d0aedbc62b6e420347abee new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_create_table_db_table-3-7a9e67189d3d4151f23b12c22bde06b5 b/sql/hive/src/test/resources/golden/show_create_table_db_table-3-7a9e67189d3d4151f23b12c22bde06b5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_create_table_db_table-4-b585371b624cbab2616a49f553a870a0 b/sql/hive/src/test/resources/golden/show_create_table_db_table-4-b585371b624cbab2616a49f553a870a0 new file mode 100644 index 0000000000000..b5a18368ed85e --- /dev/null +++ b/sql/hive/src/test/resources/golden/show_create_table_db_table-4-b585371b624cbab2616a49f553a870a0 @@ -0,0 +1,13 @@ +CREATE TABLE tmp_feng.tmp_showcrt( + key string, + value int) +ROW FORMAT SERDE + 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' +STORED AS INPUTFORMAT + 'org.apache.hadoop.mapred.TextInputFormat' +OUTPUTFORMAT + 'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat' +LOCATION + 'file:/tmp/sparkHiveWarehouse1280221975983654134/tmp_feng.db/tmp_showcrt' +TBLPROPERTIES ( + 'transient_lastDdlTime'='1407132107') diff --git a/sql/hive/src/test/resources/golden/show_create_table_db_table-5-964757b7e7f2a69fe36132c1a5712199 b/sql/hive/src/test/resources/golden/show_create_table_db_table-5-964757b7e7f2a69fe36132c1a5712199 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_create_table_db_table-6-ac09cf81e7e734cf10406f30b9fa566e b/sql/hive/src/test/resources/golden/show_create_table_db_table-6-ac09cf81e7e734cf10406f30b9fa566e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_create_table_delimited-0-97228478b9925f06726ceebb6571bf34 b/sql/hive/src/test/resources/golden/show_create_table_delimited-0-97228478b9925f06726ceebb6571bf34 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_create_table_delimited-1-2a91d52719cf4552ebeb867204552a26 b/sql/hive/src/test/resources/golden/show_create_table_delimited-1-2a91d52719cf4552ebeb867204552a26 new file mode 100644 index 0000000000000..d36ad25dc8273 --- /dev/null +++ b/sql/hive/src/test/resources/golden/show_create_table_delimited-1-2a91d52719cf4552ebeb867204552a26 @@ -0,0 +1,17 @@ +CREATE TABLE tmp_showcrt1( + key int, + value string, + newvalue bigint) +ROW FORMAT DELIMITED + FIELDS TERMINATED BY ',' + COLLECTION ITEMS TERMINATED BY '|' + MAP KEYS TERMINATED BY '%' + LINES TERMINATED BY '\n' +STORED AS INPUTFORMAT + 'org.apache.hadoop.mapred.TextInputFormat' +OUTPUTFORMAT + 'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat' +LOCATION + 'file:/tmp/tmp_showcrt1' +TBLPROPERTIES ( + 'transient_lastDdlTime'='1407132730') diff --git a/sql/hive/src/test/resources/golden/show_create_table_delimited-2-259d978ed9543204c8b9c25b6e25b0de b/sql/hive/src/test/resources/golden/show_create_table_delimited-2-259d978ed9543204c8b9c25b6e25b0de new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_create_table_partitioned-0-4be9a3b1ff0840786a1f001cba170a0c b/sql/hive/src/test/resources/golden/show_create_table_partitioned-0-4be9a3b1ff0840786a1f001cba170a0c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_create_table_partitioned-1-2a91d52719cf4552ebeb867204552a26 b/sql/hive/src/test/resources/golden/show_create_table_partitioned-1-2a91d52719cf4552ebeb867204552a26 new file mode 100644 index 0000000000000..9e572c0d7df6a --- /dev/null +++ b/sql/hive/src/test/resources/golden/show_create_table_partitioned-1-2a91d52719cf4552ebeb867204552a26 @@ -0,0 +1,16 @@ +CREATE EXTERNAL TABLE tmp_showcrt1( + key string, + newvalue boolean COMMENT 'a new value') +COMMENT 'temporary table' +PARTITIONED BY ( + value bigint COMMENT 'some value') +ROW FORMAT SERDE + 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' +STORED AS INPUTFORMAT + 'org.apache.hadoop.mapred.TextInputFormat' +OUTPUTFORMAT + 'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat' +LOCATION + 'file:/tmp/sparkHiveWarehouse1280221975983654134/tmp_showcrt1' +TBLPROPERTIES ( + 'transient_lastDdlTime'='1407132112') diff --git a/sql/hive/src/test/resources/golden/show_create_table_partitioned-2-259d978ed9543204c8b9c25b6e25b0de b/sql/hive/src/test/resources/golden/show_create_table_partitioned-2-259d978ed9543204c8b9c25b6e25b0de new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_create_table_serde-0-33f15d91810b75ee05c7b9dea0abb01c b/sql/hive/src/test/resources/golden/show_create_table_serde-0-33f15d91810b75ee05c7b9dea0abb01c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_create_table_serde-1-2a91d52719cf4552ebeb867204552a26 b/sql/hive/src/test/resources/golden/show_create_table_serde-1-2a91d52719cf4552ebeb867204552a26 new file mode 100644 index 0000000000000..69a38e1a7b20a --- /dev/null +++ b/sql/hive/src/test/resources/golden/show_create_table_serde-1-2a91d52719cf4552ebeb867204552a26 @@ -0,0 +1,15 @@ +CREATE TABLE tmp_showcrt1( + key int, + value string, + newvalue bigint) +COMMENT 'temporary table' +ROW FORMAT SERDE + 'org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe' +STORED AS INPUTFORMAT + 'org.apache.hadoop.hive.ql.io.RCFileInputFormat' +OUTPUTFORMAT + 'org.apache.hadoop.hive.ql.io.RCFileOutputFormat' +LOCATION + 'file:/tmp/sparkHiveWarehouse1280221975983654134/tmp_showcrt1' +TBLPROPERTIES ( + 'transient_lastDdlTime'='1407132115') diff --git a/sql/hive/src/test/resources/golden/show_create_table_serde-2-259d978ed9543204c8b9c25b6e25b0de b/sql/hive/src/test/resources/golden/show_create_table_serde-2-259d978ed9543204c8b9c25b6e25b0de new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_create_table_serde-3-fd12b3e0fe30f5d71c67676791b4a33b b/sql/hive/src/test/resources/golden/show_create_table_serde-3-fd12b3e0fe30f5d71c67676791b4a33b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_create_table_serde-4-2a91d52719cf4552ebeb867204552a26 b/sql/hive/src/test/resources/golden/show_create_table_serde-4-2a91d52719cf4552ebeb867204552a26 new file mode 100644 index 0000000000000..b4e693dc622fb --- /dev/null +++ b/sql/hive/src/test/resources/golden/show_create_table_serde-4-2a91d52719cf4552ebeb867204552a26 @@ -0,0 +1,14 @@ +CREATE EXTERNAL TABLE tmp_showcrt1( + key string, + value boolean) +ROW FORMAT SERDE + 'org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe' +STORED BY + 'org.apache.hadoop.hive.ql.metadata.DefaultStorageHandler' +WITH SERDEPROPERTIES ( + 'serialization.format'='$', + 'field.delim'=',') +LOCATION + 'file:/tmp/sparkHiveWarehouse1280221975983654134/tmp_showcrt1' +TBLPROPERTIES ( + 'transient_lastDdlTime'='1407132115') diff --git a/sql/hive/src/test/resources/golden/show_create_table_serde-5-259d978ed9543204c8b9c25b6e25b0de b/sql/hive/src/test/resources/golden/show_create_table_serde-5-259d978ed9543204c8b9c25b6e25b0de new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_create_table_view-0-ecef6821e4e9212e553ca38142fd0250 b/sql/hive/src/test/resources/golden/show_create_table_view-0-ecef6821e4e9212e553ca38142fd0250 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_create_table_view-1-1e931ea3fa6065107859ffbb29bb0ed7 b/sql/hive/src/test/resources/golden/show_create_table_view-1-1e931ea3fa6065107859ffbb29bb0ed7 new file mode 100644 index 0000000000000..be3fb3ce30960 --- /dev/null +++ b/sql/hive/src/test/resources/golden/show_create_table_view-1-1e931ea3fa6065107859ffbb29bb0ed7 @@ -0,0 +1 @@ +CREATE VIEW tmp_copy_src AS SELECT `src`.`key`, `src`.`value` FROM `default`.`src` diff --git a/sql/hive/src/test/resources/golden/show_create_table_view-2-ed97e9e56d95c5b3db57485cba5ad17f b/sql/hive/src/test/resources/golden/show_create_table_view-2-ed97e9e56d95c5b3db57485cba5ad17f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/single case-0-c264e319c52f1840a32959d552b99e73 b/sql/hive/src/test/resources/golden/single case-0-c264e319c52f1840a32959d552b99e73 new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/single case-0-c264e319c52f1840a32959d552b99e73 @@ -0,0 +1 @@ +1 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-10-343c75daac6695917608c17db8bf473e b/sql/hive/src/test/resources/golden/timestamp_1-10-343c75daac6695917608c17db8bf473e new file mode 100644 index 0000000000000..b77c91ccee412 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-10-343c75daac6695917608c17db8bf473e @@ -0,0 +1 @@ +1.293872461E9 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-11-cf19f7359a6d3456c4526b2c69f92d6a b/sql/hive/src/test/resources/golden/timestamp_1-11-cf19f7359a6d3456c4526b2c69f92d6a new file mode 100644 index 0000000000000..bc2423dc08a1b --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-11-cf19f7359a6d3456c4526b2c69f92d6a @@ -0,0 +1 @@ +2011-01-01 01:01:01 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-12-6328d3b3dfd295dd5ec453ffb47ff4d0 b/sql/hive/src/test/resources/golden/timestamp_1-12-6328d3b3dfd295dd5ec453ffb47ff4d0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/timestamp_1-13-90269c1e50c7ae8e75ca9cc297982135 b/sql/hive/src/test/resources/golden/timestamp_1-13-90269c1e50c7ae8e75ca9cc297982135 new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-13-90269c1e50c7ae8e75ca9cc297982135 @@ -0,0 +1 @@ +true diff --git a/sql/hive/src/test/resources/golden/timestamp_1-14-e6bfca320c4ee3aff39cf2f179d57da6 b/sql/hive/src/test/resources/golden/timestamp_1-14-e6bfca320c4ee3aff39cf2f179d57da6 new file mode 100644 index 0000000000000..987e7ca9a76df --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-14-e6bfca320c4ee3aff39cf2f179d57da6 @@ -0,0 +1 @@ +77 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-15-d0291a9bd42054b2732cb4f54cf39ae7 b/sql/hive/src/test/resources/golden/timestamp_1-15-d0291a9bd42054b2732cb4f54cf39ae7 new file mode 100644 index 0000000000000..8f3a49478b26b --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-15-d0291a9bd42054b2732cb4f54cf39ae7 @@ -0,0 +1 @@ +-4787 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-16-e7b398d2a8107a42419c83771bda41e6 b/sql/hive/src/test/resources/golden/timestamp_1-16-e7b398d2a8107a42419c83771bda41e6 new file mode 100644 index 0000000000000..211a320b9ca72 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-16-e7b398d2a8107a42419c83771bda41e6 @@ -0,0 +1 @@ +1293872461 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-17-a3eeec08bccae78d0d94ad2cb923e1cf b/sql/hive/src/test/resources/golden/timestamp_1-17-a3eeec08bccae78d0d94ad2cb923e1cf new file mode 100644 index 0000000000000..211a320b9ca72 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-17-a3eeec08bccae78d0d94ad2cb923e1cf @@ -0,0 +1 @@ +1293872461 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-18-67f274bf16de625cf4e85af0c6185cac b/sql/hive/src/test/resources/golden/timestamp_1-18-67f274bf16de625cf4e85af0c6185cac new file mode 100644 index 0000000000000..502f94a71edbd --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-18-67f274bf16de625cf4e85af0c6185cac @@ -0,0 +1 @@ +1.29387251E9 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-19-343c75daac6695917608c17db8bf473e b/sql/hive/src/test/resources/golden/timestamp_1-19-343c75daac6695917608c17db8bf473e new file mode 100644 index 0000000000000..b77c91ccee412 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-19-343c75daac6695917608c17db8bf473e @@ -0,0 +1 @@ +1.293872461E9 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-20-cf19f7359a6d3456c4526b2c69f92d6a b/sql/hive/src/test/resources/golden/timestamp_1-20-cf19f7359a6d3456c4526b2c69f92d6a new file mode 100644 index 0000000000000..bc2423dc08a1b --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-20-cf19f7359a6d3456c4526b2c69f92d6a @@ -0,0 +1 @@ +2011-01-01 01:01:01 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-21-d8fff1a6c464e50eb955babfafb0b98e b/sql/hive/src/test/resources/golden/timestamp_1-21-d8fff1a6c464e50eb955babfafb0b98e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/timestamp_1-22-90269c1e50c7ae8e75ca9cc297982135 b/sql/hive/src/test/resources/golden/timestamp_1-22-90269c1e50c7ae8e75ca9cc297982135 new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-22-90269c1e50c7ae8e75ca9cc297982135 @@ -0,0 +1 @@ +true diff --git a/sql/hive/src/test/resources/golden/timestamp_1-23-e6bfca320c4ee3aff39cf2f179d57da6 b/sql/hive/src/test/resources/golden/timestamp_1-23-e6bfca320c4ee3aff39cf2f179d57da6 new file mode 100644 index 0000000000000..987e7ca9a76df --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-23-e6bfca320c4ee3aff39cf2f179d57da6 @@ -0,0 +1 @@ +77 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-24-d0291a9bd42054b2732cb4f54cf39ae7 b/sql/hive/src/test/resources/golden/timestamp_1-24-d0291a9bd42054b2732cb4f54cf39ae7 new file mode 100644 index 0000000000000..8f3a49478b26b --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-24-d0291a9bd42054b2732cb4f54cf39ae7 @@ -0,0 +1 @@ +-4787 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-25-e7b398d2a8107a42419c83771bda41e6 b/sql/hive/src/test/resources/golden/timestamp_1-25-e7b398d2a8107a42419c83771bda41e6 new file mode 100644 index 0000000000000..211a320b9ca72 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-25-e7b398d2a8107a42419c83771bda41e6 @@ -0,0 +1 @@ +1293872461 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-26-a3eeec08bccae78d0d94ad2cb923e1cf b/sql/hive/src/test/resources/golden/timestamp_1-26-a3eeec08bccae78d0d94ad2cb923e1cf new file mode 100644 index 0000000000000..211a320b9ca72 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-26-a3eeec08bccae78d0d94ad2cb923e1cf @@ -0,0 +1 @@ +1293872461 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-27-67f274bf16de625cf4e85af0c6185cac b/sql/hive/src/test/resources/golden/timestamp_1-27-67f274bf16de625cf4e85af0c6185cac new file mode 100644 index 0000000000000..502f94a71edbd --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-27-67f274bf16de625cf4e85af0c6185cac @@ -0,0 +1 @@ +1.29387251E9 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-28-343c75daac6695917608c17db8bf473e b/sql/hive/src/test/resources/golden/timestamp_1-28-343c75daac6695917608c17db8bf473e new file mode 100644 index 0000000000000..2158e77f78768 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-28-343c75daac6695917608c17db8bf473e @@ -0,0 +1 @@ +1.2938724611E9 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-29-cf19f7359a6d3456c4526b2c69f92d6a b/sql/hive/src/test/resources/golden/timestamp_1-29-cf19f7359a6d3456c4526b2c69f92d6a new file mode 100644 index 0000000000000..4bdd802b9cda9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-29-cf19f7359a6d3456c4526b2c69f92d6a @@ -0,0 +1 @@ +2011-01-01 01:01:01.1 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-3-819633b45e3e1779bca6bcb7b77fe5a1 b/sql/hive/src/test/resources/golden/timestamp_1-3-819633b45e3e1779bca6bcb7b77fe5a1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/timestamp_1-30-273256141c33eb88194cad22eb940d21 b/sql/hive/src/test/resources/golden/timestamp_1-30-273256141c33eb88194cad22eb940d21 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/timestamp_1-31-90269c1e50c7ae8e75ca9cc297982135 b/sql/hive/src/test/resources/golden/timestamp_1-31-90269c1e50c7ae8e75ca9cc297982135 new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-31-90269c1e50c7ae8e75ca9cc297982135 @@ -0,0 +1 @@ +true diff --git a/sql/hive/src/test/resources/golden/timestamp_1-32-e6bfca320c4ee3aff39cf2f179d57da6 b/sql/hive/src/test/resources/golden/timestamp_1-32-e6bfca320c4ee3aff39cf2f179d57da6 new file mode 100644 index 0000000000000..987e7ca9a76df --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-32-e6bfca320c4ee3aff39cf2f179d57da6 @@ -0,0 +1 @@ +77 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-33-d0291a9bd42054b2732cb4f54cf39ae7 b/sql/hive/src/test/resources/golden/timestamp_1-33-d0291a9bd42054b2732cb4f54cf39ae7 new file mode 100644 index 0000000000000..8f3a49478b26b --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-33-d0291a9bd42054b2732cb4f54cf39ae7 @@ -0,0 +1 @@ +-4787 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-34-e7b398d2a8107a42419c83771bda41e6 b/sql/hive/src/test/resources/golden/timestamp_1-34-e7b398d2a8107a42419c83771bda41e6 new file mode 100644 index 0000000000000..211a320b9ca72 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-34-e7b398d2a8107a42419c83771bda41e6 @@ -0,0 +1 @@ +1293872461 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-35-a3eeec08bccae78d0d94ad2cb923e1cf b/sql/hive/src/test/resources/golden/timestamp_1-35-a3eeec08bccae78d0d94ad2cb923e1cf new file mode 100644 index 0000000000000..211a320b9ca72 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-35-a3eeec08bccae78d0d94ad2cb923e1cf @@ -0,0 +1 @@ +1293872461 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-36-67f274bf16de625cf4e85af0c6185cac b/sql/hive/src/test/resources/golden/timestamp_1-36-67f274bf16de625cf4e85af0c6185cac new file mode 100644 index 0000000000000..502f94a71edbd --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-36-67f274bf16de625cf4e85af0c6185cac @@ -0,0 +1 @@ +1.29387251E9 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-37-343c75daac6695917608c17db8bf473e b/sql/hive/src/test/resources/golden/timestamp_1-37-343c75daac6695917608c17db8bf473e new file mode 100644 index 0000000000000..b71ff60863360 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-37-343c75daac6695917608c17db8bf473e @@ -0,0 +1 @@ +1.2938724610001E9 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-38-cf19f7359a6d3456c4526b2c69f92d6a b/sql/hive/src/test/resources/golden/timestamp_1-38-cf19f7359a6d3456c4526b2c69f92d6a new file mode 100644 index 0000000000000..8b014c4cd8d6e --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-38-cf19f7359a6d3456c4526b2c69f92d6a @@ -0,0 +1 @@ +2011-01-01 01:01:01.0001 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-39-b2fe5cc7c8ee62d3bb0c120c9a6c305d b/sql/hive/src/test/resources/golden/timestamp_1-39-b2fe5cc7c8ee62d3bb0c120c9a6c305d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/timestamp_1-4-90269c1e50c7ae8e75ca9cc297982135 b/sql/hive/src/test/resources/golden/timestamp_1-4-90269c1e50c7ae8e75ca9cc297982135 new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-4-90269c1e50c7ae8e75ca9cc297982135 @@ -0,0 +1 @@ +true diff --git a/sql/hive/src/test/resources/golden/timestamp_1-40-90269c1e50c7ae8e75ca9cc297982135 b/sql/hive/src/test/resources/golden/timestamp_1-40-90269c1e50c7ae8e75ca9cc297982135 new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-40-90269c1e50c7ae8e75ca9cc297982135 @@ -0,0 +1 @@ +true diff --git a/sql/hive/src/test/resources/golden/timestamp_1-41-e6bfca320c4ee3aff39cf2f179d57da6 b/sql/hive/src/test/resources/golden/timestamp_1-41-e6bfca320c4ee3aff39cf2f179d57da6 new file mode 100644 index 0000000000000..987e7ca9a76df --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-41-e6bfca320c4ee3aff39cf2f179d57da6 @@ -0,0 +1 @@ +77 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-42-d0291a9bd42054b2732cb4f54cf39ae7 b/sql/hive/src/test/resources/golden/timestamp_1-42-d0291a9bd42054b2732cb4f54cf39ae7 new file mode 100644 index 0000000000000..8f3a49478b26b --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-42-d0291a9bd42054b2732cb4f54cf39ae7 @@ -0,0 +1 @@ +-4787 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-43-e7b398d2a8107a42419c83771bda41e6 b/sql/hive/src/test/resources/golden/timestamp_1-43-e7b398d2a8107a42419c83771bda41e6 new file mode 100644 index 0000000000000..211a320b9ca72 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-43-e7b398d2a8107a42419c83771bda41e6 @@ -0,0 +1 @@ +1293872461 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-44-a3eeec08bccae78d0d94ad2cb923e1cf b/sql/hive/src/test/resources/golden/timestamp_1-44-a3eeec08bccae78d0d94ad2cb923e1cf new file mode 100644 index 0000000000000..211a320b9ca72 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-44-a3eeec08bccae78d0d94ad2cb923e1cf @@ -0,0 +1 @@ +1293872461 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-45-67f274bf16de625cf4e85af0c6185cac b/sql/hive/src/test/resources/golden/timestamp_1-45-67f274bf16de625cf4e85af0c6185cac new file mode 100644 index 0000000000000..502f94a71edbd --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-45-67f274bf16de625cf4e85af0c6185cac @@ -0,0 +1 @@ +1.29387251E9 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-46-343c75daac6695917608c17db8bf473e b/sql/hive/src/test/resources/golden/timestamp_1-46-343c75daac6695917608c17db8bf473e new file mode 100644 index 0000000000000..b71ff60863360 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-46-343c75daac6695917608c17db8bf473e @@ -0,0 +1 @@ +1.2938724610001E9 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-47-cf19f7359a6d3456c4526b2c69f92d6a b/sql/hive/src/test/resources/golden/timestamp_1-47-cf19f7359a6d3456c4526b2c69f92d6a new file mode 100644 index 0000000000000..8b014c4cd8d6e --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-47-cf19f7359a6d3456c4526b2c69f92d6a @@ -0,0 +1 @@ +2011-01-01 01:01:01.0001 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-48-7029255241de8e8b9710801319990044 b/sql/hive/src/test/resources/golden/timestamp_1-48-7029255241de8e8b9710801319990044 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/timestamp_1-49-90269c1e50c7ae8e75ca9cc297982135 b/sql/hive/src/test/resources/golden/timestamp_1-49-90269c1e50c7ae8e75ca9cc297982135 new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-49-90269c1e50c7ae8e75ca9cc297982135 @@ -0,0 +1 @@ +true diff --git a/sql/hive/src/test/resources/golden/timestamp_1-5-e6bfca320c4ee3aff39cf2f179d57da6 b/sql/hive/src/test/resources/golden/timestamp_1-5-e6bfca320c4ee3aff39cf2f179d57da6 new file mode 100644 index 0000000000000..987e7ca9a76df --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-5-e6bfca320c4ee3aff39cf2f179d57da6 @@ -0,0 +1 @@ +77 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-50-e6bfca320c4ee3aff39cf2f179d57da6 b/sql/hive/src/test/resources/golden/timestamp_1-50-e6bfca320c4ee3aff39cf2f179d57da6 new file mode 100644 index 0000000000000..987e7ca9a76df --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-50-e6bfca320c4ee3aff39cf2f179d57da6 @@ -0,0 +1 @@ +77 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-51-d0291a9bd42054b2732cb4f54cf39ae7 b/sql/hive/src/test/resources/golden/timestamp_1-51-d0291a9bd42054b2732cb4f54cf39ae7 new file mode 100644 index 0000000000000..8f3a49478b26b --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-51-d0291a9bd42054b2732cb4f54cf39ae7 @@ -0,0 +1 @@ +-4787 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-52-e7b398d2a8107a42419c83771bda41e6 b/sql/hive/src/test/resources/golden/timestamp_1-52-e7b398d2a8107a42419c83771bda41e6 new file mode 100644 index 0000000000000..211a320b9ca72 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-52-e7b398d2a8107a42419c83771bda41e6 @@ -0,0 +1 @@ +1293872461 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-53-a3eeec08bccae78d0d94ad2cb923e1cf b/sql/hive/src/test/resources/golden/timestamp_1-53-a3eeec08bccae78d0d94ad2cb923e1cf new file mode 100644 index 0000000000000..211a320b9ca72 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-53-a3eeec08bccae78d0d94ad2cb923e1cf @@ -0,0 +1 @@ +1293872461 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-54-67f274bf16de625cf4e85af0c6185cac b/sql/hive/src/test/resources/golden/timestamp_1-54-67f274bf16de625cf4e85af0c6185cac new file mode 100644 index 0000000000000..502f94a71edbd --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-54-67f274bf16de625cf4e85af0c6185cac @@ -0,0 +1 @@ +1.29387251E9 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-55-343c75daac6695917608c17db8bf473e b/sql/hive/src/test/resources/golden/timestamp_1-55-343c75daac6695917608c17db8bf473e new file mode 100644 index 0000000000000..3eefb349894ac --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-55-343c75daac6695917608c17db8bf473e @@ -0,0 +1 @@ +1.293872461001E9 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-56-cf19f7359a6d3456c4526b2c69f92d6a b/sql/hive/src/test/resources/golden/timestamp_1-56-cf19f7359a6d3456c4526b2c69f92d6a new file mode 100644 index 0000000000000..acce9d97b5eba --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-56-cf19f7359a6d3456c4526b2c69f92d6a @@ -0,0 +1 @@ +2011-01-01 01:01:01.001000011 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-57-d362501d0176855077e65f8faf067fa8 b/sql/hive/src/test/resources/golden/timestamp_1-57-d362501d0176855077e65f8faf067fa8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/timestamp_1-6-d0291a9bd42054b2732cb4f54cf39ae7 b/sql/hive/src/test/resources/golden/timestamp_1-6-d0291a9bd42054b2732cb4f54cf39ae7 new file mode 100644 index 0000000000000..8f3a49478b26b --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-6-d0291a9bd42054b2732cb4f54cf39ae7 @@ -0,0 +1 @@ +-4787 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-7-e7b398d2a8107a42419c83771bda41e6 b/sql/hive/src/test/resources/golden/timestamp_1-7-e7b398d2a8107a42419c83771bda41e6 new file mode 100644 index 0000000000000..211a320b9ca72 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-7-e7b398d2a8107a42419c83771bda41e6 @@ -0,0 +1 @@ +1293872461 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-8-a3eeec08bccae78d0d94ad2cb923e1cf b/sql/hive/src/test/resources/golden/timestamp_1-8-a3eeec08bccae78d0d94ad2cb923e1cf new file mode 100644 index 0000000000000..211a320b9ca72 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-8-a3eeec08bccae78d0d94ad2cb923e1cf @@ -0,0 +1 @@ +1293872461 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-9-67f274bf16de625cf4e85af0c6185cac b/sql/hive/src/test/resources/golden/timestamp_1-9-67f274bf16de625cf4e85af0c6185cac new file mode 100644 index 0000000000000..502f94a71edbd --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-9-67f274bf16de625cf4e85af0c6185cac @@ -0,0 +1 @@ +1.29387251E9 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-10-5181279a0bf8939fe46ddacae015dad8 b/sql/hive/src/test/resources/golden/timestamp_2-10-5181279a0bf8939fe46ddacae015dad8 new file mode 100644 index 0000000000000..b77c91ccee412 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-10-5181279a0bf8939fe46ddacae015dad8 @@ -0,0 +1 @@ +1.293872461E9 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-11-240fce5f58794fa051824e8732c00c03 b/sql/hive/src/test/resources/golden/timestamp_2-11-240fce5f58794fa051824e8732c00c03 new file mode 100644 index 0000000000000..bc2423dc08a1b --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-11-240fce5f58794fa051824e8732c00c03 @@ -0,0 +1 @@ +2011-01-01 01:01:01 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-12-7350308cbf49d6ebd6599d3802750acd b/sql/hive/src/test/resources/golden/timestamp_2-12-7350308cbf49d6ebd6599d3802750acd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/timestamp_2-13-25f6ec69328af6cba76899194e0dd84e b/sql/hive/src/test/resources/golden/timestamp_2-13-25f6ec69328af6cba76899194e0dd84e new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-13-25f6ec69328af6cba76899194e0dd84e @@ -0,0 +1 @@ +true diff --git a/sql/hive/src/test/resources/golden/timestamp_2-14-93c769be4cff93bea6e62bfe4e2a8742 b/sql/hive/src/test/resources/golden/timestamp_2-14-93c769be4cff93bea6e62bfe4e2a8742 new file mode 100644 index 0000000000000..987e7ca9a76df --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-14-93c769be4cff93bea6e62bfe4e2a8742 @@ -0,0 +1 @@ +77 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-15-5bdbf67419cc060b82d091d80ce59bf9 b/sql/hive/src/test/resources/golden/timestamp_2-15-5bdbf67419cc060b82d091d80ce59bf9 new file mode 100644 index 0000000000000..8f3a49478b26b --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-15-5bdbf67419cc060b82d091d80ce59bf9 @@ -0,0 +1 @@ +-4787 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-16-de3c42ab06c17ae895fd7deaf7bd9571 b/sql/hive/src/test/resources/golden/timestamp_2-16-de3c42ab06c17ae895fd7deaf7bd9571 new file mode 100644 index 0000000000000..211a320b9ca72 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-16-de3c42ab06c17ae895fd7deaf7bd9571 @@ -0,0 +1 @@ +1293872461 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-17-da3937d21b7c2cfe1e624e812ae1d3ef b/sql/hive/src/test/resources/golden/timestamp_2-17-da3937d21b7c2cfe1e624e812ae1d3ef new file mode 100644 index 0000000000000..211a320b9ca72 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-17-da3937d21b7c2cfe1e624e812ae1d3ef @@ -0,0 +1 @@ +1293872461 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-18-252aebfe7882335d31bfc53a8705b7a b/sql/hive/src/test/resources/golden/timestamp_2-18-252aebfe7882335d31bfc53a8705b7a new file mode 100644 index 0000000000000..502f94a71edbd --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-18-252aebfe7882335d31bfc53a8705b7a @@ -0,0 +1 @@ +1.29387251E9 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-19-5181279a0bf8939fe46ddacae015dad8 b/sql/hive/src/test/resources/golden/timestamp_2-19-5181279a0bf8939fe46ddacae015dad8 new file mode 100644 index 0000000000000..b77c91ccee412 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-19-5181279a0bf8939fe46ddacae015dad8 @@ -0,0 +1 @@ +1.293872461E9 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-20-240fce5f58794fa051824e8732c00c03 b/sql/hive/src/test/resources/golden/timestamp_2-20-240fce5f58794fa051824e8732c00c03 new file mode 100644 index 0000000000000..bc2423dc08a1b --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-20-240fce5f58794fa051824e8732c00c03 @@ -0,0 +1 @@ +2011-01-01 01:01:01 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-21-5eb58e5d3c5b9f766f0b497bf59c47b b/sql/hive/src/test/resources/golden/timestamp_2-21-5eb58e5d3c5b9f766f0b497bf59c47b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/timestamp_2-22-25f6ec69328af6cba76899194e0dd84e b/sql/hive/src/test/resources/golden/timestamp_2-22-25f6ec69328af6cba76899194e0dd84e new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-22-25f6ec69328af6cba76899194e0dd84e @@ -0,0 +1 @@ +true diff --git a/sql/hive/src/test/resources/golden/timestamp_2-23-93c769be4cff93bea6e62bfe4e2a8742 b/sql/hive/src/test/resources/golden/timestamp_2-23-93c769be4cff93bea6e62bfe4e2a8742 new file mode 100644 index 0000000000000..987e7ca9a76df --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-23-93c769be4cff93bea6e62bfe4e2a8742 @@ -0,0 +1 @@ +77 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-24-5bdbf67419cc060b82d091d80ce59bf9 b/sql/hive/src/test/resources/golden/timestamp_2-24-5bdbf67419cc060b82d091d80ce59bf9 new file mode 100644 index 0000000000000..8f3a49478b26b --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-24-5bdbf67419cc060b82d091d80ce59bf9 @@ -0,0 +1 @@ +-4787 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-25-de3c42ab06c17ae895fd7deaf7bd9571 b/sql/hive/src/test/resources/golden/timestamp_2-25-de3c42ab06c17ae895fd7deaf7bd9571 new file mode 100644 index 0000000000000..211a320b9ca72 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-25-de3c42ab06c17ae895fd7deaf7bd9571 @@ -0,0 +1 @@ +1293872461 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-26-da3937d21b7c2cfe1e624e812ae1d3ef b/sql/hive/src/test/resources/golden/timestamp_2-26-da3937d21b7c2cfe1e624e812ae1d3ef new file mode 100644 index 0000000000000..211a320b9ca72 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-26-da3937d21b7c2cfe1e624e812ae1d3ef @@ -0,0 +1 @@ +1293872461 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-27-252aebfe7882335d31bfc53a8705b7a b/sql/hive/src/test/resources/golden/timestamp_2-27-252aebfe7882335d31bfc53a8705b7a new file mode 100644 index 0000000000000..502f94a71edbd --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-27-252aebfe7882335d31bfc53a8705b7a @@ -0,0 +1 @@ +1.29387251E9 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-28-5181279a0bf8939fe46ddacae015dad8 b/sql/hive/src/test/resources/golden/timestamp_2-28-5181279a0bf8939fe46ddacae015dad8 new file mode 100644 index 0000000000000..2158e77f78768 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-28-5181279a0bf8939fe46ddacae015dad8 @@ -0,0 +1 @@ +1.2938724611E9 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-29-240fce5f58794fa051824e8732c00c03 b/sql/hive/src/test/resources/golden/timestamp_2-29-240fce5f58794fa051824e8732c00c03 new file mode 100644 index 0000000000000..4bdd802b9cda9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-29-240fce5f58794fa051824e8732c00c03 @@ -0,0 +1 @@ +2011-01-01 01:01:01.1 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-3-a95a52c3a66e1f211ea04a0a10bd3b74 b/sql/hive/src/test/resources/golden/timestamp_2-3-a95a52c3a66e1f211ea04a0a10bd3b74 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/timestamp_2-30-ffe6b6ddaaba84152074f7781fba2243 b/sql/hive/src/test/resources/golden/timestamp_2-30-ffe6b6ddaaba84152074f7781fba2243 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/timestamp_2-31-25f6ec69328af6cba76899194e0dd84e b/sql/hive/src/test/resources/golden/timestamp_2-31-25f6ec69328af6cba76899194e0dd84e new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-31-25f6ec69328af6cba76899194e0dd84e @@ -0,0 +1 @@ +true diff --git a/sql/hive/src/test/resources/golden/timestamp_2-32-93c769be4cff93bea6e62bfe4e2a8742 b/sql/hive/src/test/resources/golden/timestamp_2-32-93c769be4cff93bea6e62bfe4e2a8742 new file mode 100644 index 0000000000000..987e7ca9a76df --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-32-93c769be4cff93bea6e62bfe4e2a8742 @@ -0,0 +1 @@ +77 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-33-5bdbf67419cc060b82d091d80ce59bf9 b/sql/hive/src/test/resources/golden/timestamp_2-33-5bdbf67419cc060b82d091d80ce59bf9 new file mode 100644 index 0000000000000..8f3a49478b26b --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-33-5bdbf67419cc060b82d091d80ce59bf9 @@ -0,0 +1 @@ +-4787 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-34-de3c42ab06c17ae895fd7deaf7bd9571 b/sql/hive/src/test/resources/golden/timestamp_2-34-de3c42ab06c17ae895fd7deaf7bd9571 new file mode 100644 index 0000000000000..211a320b9ca72 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-34-de3c42ab06c17ae895fd7deaf7bd9571 @@ -0,0 +1 @@ +1293872461 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-35-da3937d21b7c2cfe1e624e812ae1d3ef b/sql/hive/src/test/resources/golden/timestamp_2-35-da3937d21b7c2cfe1e624e812ae1d3ef new file mode 100644 index 0000000000000..211a320b9ca72 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-35-da3937d21b7c2cfe1e624e812ae1d3ef @@ -0,0 +1 @@ +1293872461 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-36-252aebfe7882335d31bfc53a8705b7a b/sql/hive/src/test/resources/golden/timestamp_2-36-252aebfe7882335d31bfc53a8705b7a new file mode 100644 index 0000000000000..502f94a71edbd --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-36-252aebfe7882335d31bfc53a8705b7a @@ -0,0 +1 @@ +1.29387251E9 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-37-5181279a0bf8939fe46ddacae015dad8 b/sql/hive/src/test/resources/golden/timestamp_2-37-5181279a0bf8939fe46ddacae015dad8 new file mode 100644 index 0000000000000..b71ff60863360 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-37-5181279a0bf8939fe46ddacae015dad8 @@ -0,0 +1 @@ +1.2938724610001E9 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-38-240fce5f58794fa051824e8732c00c03 b/sql/hive/src/test/resources/golden/timestamp_2-38-240fce5f58794fa051824e8732c00c03 new file mode 100644 index 0000000000000..8b014c4cd8d6e --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-38-240fce5f58794fa051824e8732c00c03 @@ -0,0 +1 @@ +2011-01-01 01:01:01.0001 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-39-8236608f28681eac5503195096a34181 b/sql/hive/src/test/resources/golden/timestamp_2-39-8236608f28681eac5503195096a34181 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/timestamp_2-4-25f6ec69328af6cba76899194e0dd84e b/sql/hive/src/test/resources/golden/timestamp_2-4-25f6ec69328af6cba76899194e0dd84e new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-4-25f6ec69328af6cba76899194e0dd84e @@ -0,0 +1 @@ +true diff --git a/sql/hive/src/test/resources/golden/timestamp_2-40-25f6ec69328af6cba76899194e0dd84e b/sql/hive/src/test/resources/golden/timestamp_2-40-25f6ec69328af6cba76899194e0dd84e new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-40-25f6ec69328af6cba76899194e0dd84e @@ -0,0 +1 @@ +true diff --git a/sql/hive/src/test/resources/golden/timestamp_2-41-93c769be4cff93bea6e62bfe4e2a8742 b/sql/hive/src/test/resources/golden/timestamp_2-41-93c769be4cff93bea6e62bfe4e2a8742 new file mode 100644 index 0000000000000..987e7ca9a76df --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-41-93c769be4cff93bea6e62bfe4e2a8742 @@ -0,0 +1 @@ +77 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-42-5bdbf67419cc060b82d091d80ce59bf9 b/sql/hive/src/test/resources/golden/timestamp_2-42-5bdbf67419cc060b82d091d80ce59bf9 new file mode 100644 index 0000000000000..8f3a49478b26b --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-42-5bdbf67419cc060b82d091d80ce59bf9 @@ -0,0 +1 @@ +-4787 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-43-de3c42ab06c17ae895fd7deaf7bd9571 b/sql/hive/src/test/resources/golden/timestamp_2-43-de3c42ab06c17ae895fd7deaf7bd9571 new file mode 100644 index 0000000000000..211a320b9ca72 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-43-de3c42ab06c17ae895fd7deaf7bd9571 @@ -0,0 +1 @@ +1293872461 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-44-da3937d21b7c2cfe1e624e812ae1d3ef b/sql/hive/src/test/resources/golden/timestamp_2-44-da3937d21b7c2cfe1e624e812ae1d3ef new file mode 100644 index 0000000000000..211a320b9ca72 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-44-da3937d21b7c2cfe1e624e812ae1d3ef @@ -0,0 +1 @@ +1293872461 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-45-252aebfe7882335d31bfc53a8705b7a b/sql/hive/src/test/resources/golden/timestamp_2-45-252aebfe7882335d31bfc53a8705b7a new file mode 100644 index 0000000000000..502f94a71edbd --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-45-252aebfe7882335d31bfc53a8705b7a @@ -0,0 +1 @@ +1.29387251E9 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-46-5181279a0bf8939fe46ddacae015dad8 b/sql/hive/src/test/resources/golden/timestamp_2-46-5181279a0bf8939fe46ddacae015dad8 new file mode 100644 index 0000000000000..b71ff60863360 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-46-5181279a0bf8939fe46ddacae015dad8 @@ -0,0 +1 @@ +1.2938724610001E9 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-47-240fce5f58794fa051824e8732c00c03 b/sql/hive/src/test/resources/golden/timestamp_2-47-240fce5f58794fa051824e8732c00c03 new file mode 100644 index 0000000000000..8b014c4cd8d6e --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-47-240fce5f58794fa051824e8732c00c03 @@ -0,0 +1 @@ +2011-01-01 01:01:01.0001 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-48-654e5533ec6dc911996abc7e47af8ccb b/sql/hive/src/test/resources/golden/timestamp_2-48-654e5533ec6dc911996abc7e47af8ccb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/timestamp_2-49-25f6ec69328af6cba76899194e0dd84e b/sql/hive/src/test/resources/golden/timestamp_2-49-25f6ec69328af6cba76899194e0dd84e new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-49-25f6ec69328af6cba76899194e0dd84e @@ -0,0 +1 @@ +true diff --git a/sql/hive/src/test/resources/golden/timestamp_2-5-93c769be4cff93bea6e62bfe4e2a8742 b/sql/hive/src/test/resources/golden/timestamp_2-5-93c769be4cff93bea6e62bfe4e2a8742 new file mode 100644 index 0000000000000..987e7ca9a76df --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-5-93c769be4cff93bea6e62bfe4e2a8742 @@ -0,0 +1 @@ +77 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-50-93c769be4cff93bea6e62bfe4e2a8742 b/sql/hive/src/test/resources/golden/timestamp_2-50-93c769be4cff93bea6e62bfe4e2a8742 new file mode 100644 index 0000000000000..987e7ca9a76df --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-50-93c769be4cff93bea6e62bfe4e2a8742 @@ -0,0 +1 @@ +77 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-51-5bdbf67419cc060b82d091d80ce59bf9 b/sql/hive/src/test/resources/golden/timestamp_2-51-5bdbf67419cc060b82d091d80ce59bf9 new file mode 100644 index 0000000000000..8f3a49478b26b --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-51-5bdbf67419cc060b82d091d80ce59bf9 @@ -0,0 +1 @@ +-4787 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-52-de3c42ab06c17ae895fd7deaf7bd9571 b/sql/hive/src/test/resources/golden/timestamp_2-52-de3c42ab06c17ae895fd7deaf7bd9571 new file mode 100644 index 0000000000000..211a320b9ca72 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-52-de3c42ab06c17ae895fd7deaf7bd9571 @@ -0,0 +1 @@ +1293872461 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-53-da3937d21b7c2cfe1e624e812ae1d3ef b/sql/hive/src/test/resources/golden/timestamp_2-53-da3937d21b7c2cfe1e624e812ae1d3ef new file mode 100644 index 0000000000000..211a320b9ca72 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-53-da3937d21b7c2cfe1e624e812ae1d3ef @@ -0,0 +1 @@ +1293872461 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-54-252aebfe7882335d31bfc53a8705b7a b/sql/hive/src/test/resources/golden/timestamp_2-54-252aebfe7882335d31bfc53a8705b7a new file mode 100644 index 0000000000000..502f94a71edbd --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-54-252aebfe7882335d31bfc53a8705b7a @@ -0,0 +1 @@ +1.29387251E9 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-55-5181279a0bf8939fe46ddacae015dad8 b/sql/hive/src/test/resources/golden/timestamp_2-55-5181279a0bf8939fe46ddacae015dad8 new file mode 100644 index 0000000000000..3eefb349894ac --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-55-5181279a0bf8939fe46ddacae015dad8 @@ -0,0 +1 @@ +1.293872461001E9 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-56-240fce5f58794fa051824e8732c00c03 b/sql/hive/src/test/resources/golden/timestamp_2-56-240fce5f58794fa051824e8732c00c03 new file mode 100644 index 0000000000000..acce9d97b5eba --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-56-240fce5f58794fa051824e8732c00c03 @@ -0,0 +1 @@ +2011-01-01 01:01:01.001000011 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-57-ea7192a4a5a985bcc8aab9aa79d9f028 b/sql/hive/src/test/resources/golden/timestamp_2-57-ea7192a4a5a985bcc8aab9aa79d9f028 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/timestamp_2-6-5bdbf67419cc060b82d091d80ce59bf9 b/sql/hive/src/test/resources/golden/timestamp_2-6-5bdbf67419cc060b82d091d80ce59bf9 new file mode 100644 index 0000000000000..8f3a49478b26b --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-6-5bdbf67419cc060b82d091d80ce59bf9 @@ -0,0 +1 @@ +-4787 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-7-de3c42ab06c17ae895fd7deaf7bd9571 b/sql/hive/src/test/resources/golden/timestamp_2-7-de3c42ab06c17ae895fd7deaf7bd9571 new file mode 100644 index 0000000000000..211a320b9ca72 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-7-de3c42ab06c17ae895fd7deaf7bd9571 @@ -0,0 +1 @@ +1293872461 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-8-da3937d21b7c2cfe1e624e812ae1d3ef b/sql/hive/src/test/resources/golden/timestamp_2-8-da3937d21b7c2cfe1e624e812ae1d3ef new file mode 100644 index 0000000000000..211a320b9ca72 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-8-da3937d21b7c2cfe1e624e812ae1d3ef @@ -0,0 +1 @@ +1293872461 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-9-252aebfe7882335d31bfc53a8705b7a b/sql/hive/src/test/resources/golden/timestamp_2-9-252aebfe7882335d31bfc53a8705b7a new file mode 100644 index 0000000000000..502f94a71edbd --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-9-252aebfe7882335d31bfc53a8705b7a @@ -0,0 +1 @@ +1.29387251E9 diff --git a/sql/hive/src/test/resources/golden/timestamp_3-10-7b1ec929239ee305ea9da46ebb990c67 b/sql/hive/src/test/resources/golden/timestamp_3-10-7b1ec929239ee305ea9da46ebb990c67 new file mode 100644 index 0000000000000..1b0a140b5a384 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_3-10-7b1ec929239ee305ea9da46ebb990c67 @@ -0,0 +1 @@ +1.3041352164485E9 diff --git a/sql/hive/src/test/resources/golden/timestamp_3-11-a63f40f6c4a022c16f8cf810e3b7ed2a b/sql/hive/src/test/resources/golden/timestamp_3-11-a63f40f6c4a022c16f8cf810e3b7ed2a new file mode 100644 index 0000000000000..d7ff6cd63d9f0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_3-11-a63f40f6c4a022c16f8cf810e3b7ed2a @@ -0,0 +1 @@ +2011-04-29 20:46:56.4485 diff --git a/sql/hive/src/test/resources/golden/timestamp_3-12-165256158e3db1ce19c3c9db3c8011d2 b/sql/hive/src/test/resources/golden/timestamp_3-12-165256158e3db1ce19c3c9db3c8011d2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/timestamp_3-3-6143888a940bfcac1133330764f5a31a b/sql/hive/src/test/resources/golden/timestamp_3-3-6143888a940bfcac1133330764f5a31a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/timestamp_3-4-935d0d2492beab99bbbba26ba62a1db4 b/sql/hive/src/test/resources/golden/timestamp_3-4-935d0d2492beab99bbbba26ba62a1db4 new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_3-4-935d0d2492beab99bbbba26ba62a1db4 @@ -0,0 +1 @@ +true diff --git a/sql/hive/src/test/resources/golden/timestamp_3-5-8fe348d5d9b9903a26eda32d308b8e41 b/sql/hive/src/test/resources/golden/timestamp_3-5-8fe348d5d9b9903a26eda32d308b8e41 new file mode 100644 index 0000000000000..21e72e8ac3d7e --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_3-5-8fe348d5d9b9903a26eda32d308b8e41 @@ -0,0 +1 @@ +48 diff --git a/sql/hive/src/test/resources/golden/timestamp_3-6-6be5fe01c502cd24db32a3781c97a703 b/sql/hive/src/test/resources/golden/timestamp_3-6-6be5fe01c502cd24db32a3781c97a703 new file mode 100644 index 0000000000000..ee3be2941da6e --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_3-6-6be5fe01c502cd24db32a3781c97a703 @@ -0,0 +1 @@ +-31184 diff --git a/sql/hive/src/test/resources/golden/timestamp_3-7-6066ba0451cd0fcfac4bea6376e72add b/sql/hive/src/test/resources/golden/timestamp_3-7-6066ba0451cd0fcfac4bea6376e72add new file mode 100644 index 0000000000000..1cf1952ac0372 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_3-7-6066ba0451cd0fcfac4bea6376e72add @@ -0,0 +1 @@ +1304135216 diff --git a/sql/hive/src/test/resources/golden/timestamp_3-8-22e03daa775eab145d39ec0730953f7e b/sql/hive/src/test/resources/golden/timestamp_3-8-22e03daa775eab145d39ec0730953f7e new file mode 100644 index 0000000000000..1cf1952ac0372 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_3-8-22e03daa775eab145d39ec0730953f7e @@ -0,0 +1 @@ +1304135216 diff --git a/sql/hive/src/test/resources/golden/timestamp_3-9-ffc79abb874323e165963aa39f460a9b b/sql/hive/src/test/resources/golden/timestamp_3-9-ffc79abb874323e165963aa39f460a9b new file mode 100644 index 0000000000000..d21deca762237 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_3-9-ffc79abb874323e165963aa39f460a9b @@ -0,0 +1 @@ +1.30413517E9 diff --git a/sql/hive/src/test/resources/golden/timestamp_lazy-2-cdb72e0c24fd9277a41fe0c7b1392e34 b/sql/hive/src/test/resources/golden/timestamp_lazy-2-cdb72e0c24fd9277a41fe0c7b1392e34 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/timestamp_lazy-3-79e0c72c4fb3b259dfbffd245ccaa636 b/sql/hive/src/test/resources/golden/timestamp_lazy-3-79e0c72c4fb3b259dfbffd245ccaa636 new file mode 100644 index 0000000000000..e6bfe0b1667ae --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_lazy-3-79e0c72c4fb3b259dfbffd245ccaa636 @@ -0,0 +1,5 @@ +2011-01-01 01:01:01 165 val_165 +2011-01-01 01:01:01 238 val_238 +2011-01-01 01:01:01 27 val_27 +2011-01-01 01:01:01 311 val_311 +2011-01-01 01:01:01 86 val_86 diff --git a/sql/hive/src/test/resources/golden/timestamp_lazy-4-b4c4417ce9f08baeb82ffde6ef1baa25 b/sql/hive/src/test/resources/golden/timestamp_lazy-4-b4c4417ce9f08baeb82ffde6ef1baa25 new file mode 100644 index 0000000000000..e6bfe0b1667ae --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_lazy-4-b4c4417ce9f08baeb82ffde6ef1baa25 @@ -0,0 +1,5 @@ +2011-01-01 01:01:01 165 val_165 +2011-01-01 01:01:01 238 val_238 +2011-01-01 01:01:01 27 val_27 +2011-01-01 01:01:01 311 val_311 +2011-01-01 01:01:01 86 val_86 diff --git a/sql/hive/src/test/resources/golden/timestamp_null-3-222c5ea127c747c71738b5dc5b80459c b/sql/hive/src/test/resources/golden/timestamp_null-3-222c5ea127c747c71738b5dc5b80459c new file mode 100644 index 0000000000000..7951defec192a --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_null-3-222c5ea127c747c71738b5dc5b80459c @@ -0,0 +1 @@ +NULL diff --git a/sql/hive/src/test/resources/golden/timestamp_null-4-ffc86f5c714eceabc36e92931b96beb0 b/sql/hive/src/test/resources/golden/timestamp_null-4-ffc86f5c714eceabc36e92931b96beb0 new file mode 100644 index 0000000000000..7951defec192a --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_null-4-ffc86f5c714eceabc36e92931b96beb0 @@ -0,0 +1 @@ +NULL diff --git a/sql/hive/src/test/resources/golden/udf_equal-0-36b6cdf7c5f68c91155569b1622f5876 b/sql/hive/src/test/resources/golden/udf_equal-0-36b6cdf7c5f68c91155569b1622f5876 new file mode 100644 index 0000000000000..9b9b6312a269a --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_equal-0-36b6cdf7c5f68c91155569b1622f5876 @@ -0,0 +1 @@ +a = b - Returns TRUE if a equals b and false otherwise diff --git a/sql/hive/src/test/resources/golden/udf_equal-1-2422b50b96502dde8b661acdfebd8892 b/sql/hive/src/test/resources/golden/udf_equal-1-2422b50b96502dde8b661acdfebd8892 new file mode 100644 index 0000000000000..30fdf50f62e4e --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_equal-1-2422b50b96502dde8b661acdfebd8892 @@ -0,0 +1,2 @@ +a = b - Returns TRUE if a equals b and false otherwise +Synonyms: == diff --git a/sql/hive/src/test/resources/golden/udf_equal-2-e0faab0f5e736c24bcc5503aeac55053 b/sql/hive/src/test/resources/golden/udf_equal-2-e0faab0f5e736c24bcc5503aeac55053 new file mode 100644 index 0000000000000..d6b4c860778b7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_equal-2-e0faab0f5e736c24bcc5503aeac55053 @@ -0,0 +1 @@ +a == b - Returns TRUE if a equals b and false otherwise diff --git a/sql/hive/src/test/resources/golden/udf_equal-3-39d8d6f197803de927f0af5409ec2f33 b/sql/hive/src/test/resources/golden/udf_equal-3-39d8d6f197803de927f0af5409ec2f33 new file mode 100644 index 0000000000000..71e55d6d638a6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_equal-3-39d8d6f197803de927f0af5409ec2f33 @@ -0,0 +1,2 @@ +a == b - Returns TRUE if a equals b and false otherwise +Synonyms: = diff --git a/sql/hive/src/test/resources/golden/udf_equal-4-94ac2476006425e1b3bcddf29ad07b16 b/sql/hive/src/test/resources/golden/udf_equal-4-94ac2476006425e1b3bcddf29ad07b16 new file mode 100644 index 0000000000000..015c417bc68f0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_equal-4-94ac2476006425e1b3bcddf29ad07b16 @@ -0,0 +1 @@ +false false true true NULL NULL NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/udf_equal-5-878650cf21e9360a07d204c8ffb0cde7 b/sql/hive/src/test/resources/golden/udf_equal-5-878650cf21e9360a07d204c8ffb0cde7 new file mode 100644 index 0000000000000..aa7b4b51edea7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_equal-5-878650cf21e9360a07d204c8ffb0cde7 @@ -0,0 +1 @@ +a <=> b - Returns same result with EQUAL(=) operator for non-null operands, but returns TRUE if both are NULL, FALSE if one of the them is NULL diff --git a/sql/hive/src/test/resources/golden/udf_equal-6-1635ef051fecdfc7891d9f5a9a3a545e b/sql/hive/src/test/resources/golden/udf_equal-6-1635ef051fecdfc7891d9f5a9a3a545e new file mode 100644 index 0000000000000..aa7b4b51edea7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_equal-6-1635ef051fecdfc7891d9f5a9a3a545e @@ -0,0 +1 @@ +a <=> b - Returns same result with EQUAL(=) operator for non-null operands, but returns TRUE if both are NULL, FALSE if one of the them is NULL diff --git a/sql/hive/src/test/resources/golden/udf_equal-7-78f1b96c199e307714fa1b804e5bae27 b/sql/hive/src/test/resources/golden/udf_equal-7-78f1b96c199e307714fa1b804e5bae27 new file mode 100644 index 0000000000000..05292fb23192d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_equal-7-78f1b96c199e307714fa1b804e5bae27 @@ -0,0 +1 @@ +false false true true true false false false false diff --git a/sql/hive/src/test/resources/golden/udf_unix_timestamp-0-d555c8cd733572bfa8cd3362da9480cb b/sql/hive/src/test/resources/golden/udf_unix_timestamp-0-d555c8cd733572bfa8cd3362da9480cb new file mode 100644 index 0000000000000..9913d42ffc1aa --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_unix_timestamp-0-d555c8cd733572bfa8cd3362da9480cb @@ -0,0 +1 @@ +unix_timestamp([date[, pattern]]) - Returns the UNIX timestamp diff --git a/sql/hive/src/test/resources/golden/udf_unix_timestamp-1-8a9dbadae706047715cf5f903ff4a724 b/sql/hive/src/test/resources/golden/udf_unix_timestamp-1-8a9dbadae706047715cf5f903ff4a724 new file mode 100644 index 0000000000000..ef4aa8e9595d1 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_unix_timestamp-1-8a9dbadae706047715cf5f903ff4a724 @@ -0,0 +1,2 @@ +unix_timestamp([date[, pattern]]) - Returns the UNIX timestamp +Converts the current or specified time to number of seconds since 1970-01-01. diff --git a/sql/hive/src/test/resources/golden/udf_unix_timestamp-2-28c40e51e55bed62693e626efda5d9c5 b/sql/hive/src/test/resources/golden/udf_unix_timestamp-2-28c40e51e55bed62693e626efda5d9c5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_unix_timestamp-3-732b21d386f2002b87eaf02d0b9951ed b/sql/hive/src/test/resources/golden/udf_unix_timestamp-3-732b21d386f2002b87eaf02d0b9951ed new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_unix_timestamp-4-b2e42ebb75cecf09961d36587797f6d0 b/sql/hive/src/test/resources/golden/udf_unix_timestamp-4-b2e42ebb75cecf09961d36587797f6d0 new file mode 100644 index 0000000000000..31aaa952b4cc5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_unix_timestamp-4-b2e42ebb75cecf09961d36587797f6d0 @@ -0,0 +1 @@ +2009-03-20 11:30:01 1237573801 diff --git a/sql/hive/src/test/resources/golden/udf_unix_timestamp-5-31243f5cb64356425b9f95ba011ac9d6 b/sql/hive/src/test/resources/golden/udf_unix_timestamp-5-31243f5cb64356425b9f95ba011ac9d6 new file mode 100644 index 0000000000000..6d9ee690cea4f --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_unix_timestamp-5-31243f5cb64356425b9f95ba011ac9d6 @@ -0,0 +1 @@ +2009-03-20 1237532400 diff --git a/sql/hive/src/test/resources/golden/udf_unix_timestamp-6-9b0f20bde1aaf9102b67a5498b167f31 b/sql/hive/src/test/resources/golden/udf_unix_timestamp-6-9b0f20bde1aaf9102b67a5498b167f31 new file mode 100644 index 0000000000000..e1d0cbb6c8495 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_unix_timestamp-6-9b0f20bde1aaf9102b67a5498b167f31 @@ -0,0 +1 @@ +2009 Mar 20 11:30:01 am 1237573801 diff --git a/sql/hive/src/test/resources/golden/udf_unix_timestamp-7-47f433ff6ccce4c666440cc1a228a96d b/sql/hive/src/test/resources/golden/udf_unix_timestamp-7-47f433ff6ccce4c666440cc1a228a96d new file mode 100644 index 0000000000000..6b40e687afd5d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_unix_timestamp-7-47f433ff6ccce4c666440cc1a228a96d @@ -0,0 +1 @@ +random_string NULL diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/QueryTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/QueryTest.scala index 11d8b1f0a3d96..95921c3d7ae09 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/QueryTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/QueryTest.scala @@ -51,9 +51,9 @@ class QueryTest extends FunSuite { fail( s""" |Exception thrown while executing query: - |${rdd.logicalPlan} + |${rdd.queryExecution} |== Exception == - |$e + |${stackTraceToString(e)} """.stripMargin) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala index 3132d0112c708..188579edd7bdd 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala @@ -23,6 +23,8 @@ import org.apache.spark.sql.hive.execution.HiveComparisonTest import org.apache.spark.sql.hive.test.TestHive class CachedTableSuite extends HiveComparisonTest { + import TestHive._ + TestHive.loadTestTable("src") test("cache table") { @@ -32,6 +34,20 @@ class CachedTableSuite extends HiveComparisonTest { createQueryTest("read from cached table", "SELECT * FROM src LIMIT 1", reset = false) + test("Drop cached table") { + sql("CREATE TABLE test(a INT)") + cacheTable("test") + sql("SELECT * FROM test").collect() + sql("DROP TABLE test") + intercept[org.apache.hadoop.hive.ql.metadata.InvalidTableException] { + sql("SELECT * FROM test").collect() + } + } + + test("DROP nonexistant table") { + sql("DROP TABLE IF EXISTS nonexistantTable") + } + test("check that table is cached and uncache") { TestHive.table("src").queryExecution.analyzed match { case _ : InMemoryRelation => // Found evidence of caching @@ -58,14 +74,14 @@ class CachedTableSuite extends HiveComparisonTest { } test("'CACHE TABLE' and 'UNCACHE TABLE' HiveQL statement") { - TestHive.hql("CACHE TABLE src") + TestHive.sql("CACHE TABLE src") TestHive.table("src").queryExecution.executedPlan match { case _: InMemoryColumnarTableScan => // Found evidence of caching case _ => fail(s"Table 'src' should be cached") } assert(TestHive.isCached("src"), "Table 'src' should be cached") - TestHive.hql("UNCACHE TABLE src") + TestHive.sql("UNCACHE TABLE src") TestHive.table("src").queryExecution.executedPlan match { case _: InMemoryColumnarTableScan => fail(s"Table 'src' should not be cached") case _ => // Found evidence of uncaching diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala index 833f3502154f3..7e323146f9da2 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala @@ -28,7 +28,7 @@ case class TestData(key: Int, value: String) class InsertIntoHiveTableSuite extends QueryTest { val testData = TestHive.sparkContext.parallelize( (1 to 100).map(i => TestData(i, i.toString))) - testData.registerAsTable("testData") + testData.registerTempTable("testData") test("insertInto() HiveTable") { createTable[TestData]("createAndInsertTest") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala new file mode 100644 index 0000000000000..7c82964b5ecdc --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala @@ -0,0 +1,190 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive + +import scala.reflect.ClassTag + + +import org.apache.spark.sql.{SQLConf, QueryTest} +import org.apache.spark.sql.catalyst.plans.logical.NativeCommand +import org.apache.spark.sql.execution.{BroadcastHashJoin, ShuffledHashJoin} +import org.apache.spark.sql.hive.test.TestHive +import org.apache.spark.sql.hive.test.TestHive._ + +class StatisticsSuite extends QueryTest { + + test("parse analyze commands") { + def assertAnalyzeCommand(analyzeCommand: String, c: Class[_]) { + val parsed = HiveQl.parseSql(analyzeCommand) + val operators = parsed.collect { + case a: AnalyzeTable => a + case o => o + } + + assert(operators.size === 1) + if (operators(0).getClass() != c) { + fail( + s"""$analyzeCommand expected command: $c, but got ${operators(0)} + |parsed command: + |$parsed + """.stripMargin) + } + } + + assertAnalyzeCommand( + "ANALYZE TABLE Table1 COMPUTE STATISTICS", + classOf[NativeCommand]) + assertAnalyzeCommand( + "ANALYZE TABLE Table1 PARTITION(ds='2008-04-09', hr=11) COMPUTE STATISTICS", + classOf[NativeCommand]) + assertAnalyzeCommand( + "ANALYZE TABLE Table1 PARTITION(ds='2008-04-09', hr=11) COMPUTE STATISTICS noscan", + classOf[NativeCommand]) + assertAnalyzeCommand( + "ANALYZE TABLE Table1 PARTITION(ds, hr) COMPUTE STATISTICS", + classOf[NativeCommand]) + assertAnalyzeCommand( + "ANALYZE TABLE Table1 PARTITION(ds, hr) COMPUTE STATISTICS noscan", + classOf[NativeCommand]) + + assertAnalyzeCommand( + "ANALYZE TABLE Table1 COMPUTE STATISTICS nOscAn", + classOf[AnalyzeTable]) + } + + test("analyze MetastoreRelations") { + def queryTotalSize(tableName: String): BigInt = + catalog.lookupRelation(None, tableName).statistics.sizeInBytes + + // Non-partitioned table + sql("CREATE TABLE analyzeTable (key STRING, value STRING)").collect() + sql("INSERT INTO TABLE analyzeTable SELECT * FROM src").collect() + sql("INSERT INTO TABLE analyzeTable SELECT * FROM src").collect() + + assert(queryTotalSize("analyzeTable") === defaultSizeInBytes) + + sql("ANALYZE TABLE analyzeTable COMPUTE STATISTICS noscan") + + assert(queryTotalSize("analyzeTable") === BigInt(11624)) + + sql("DROP TABLE analyzeTable").collect() + + // Partitioned table + sql( + """ + |CREATE TABLE analyzeTable_part (key STRING, value STRING) PARTITIONED BY (ds STRING) + """.stripMargin).collect() + sql( + """ + |INSERT INTO TABLE analyzeTable_part PARTITION (ds='2010-01-01') + |SELECT * FROM src + """.stripMargin).collect() + sql( + """ + |INSERT INTO TABLE analyzeTable_part PARTITION (ds='2010-01-02') + |SELECT * FROM src + """.stripMargin).collect() + sql( + """ + |INSERT INTO TABLE analyzeTable_part PARTITION (ds='2010-01-03') + |SELECT * FROM src + """.stripMargin).collect() + + assert(queryTotalSize("analyzeTable_part") === defaultSizeInBytes) + + sql("ANALYZE TABLE analyzeTable_part COMPUTE STATISTICS noscan") + + assert(queryTotalSize("analyzeTable_part") === BigInt(17436)) + + sql("DROP TABLE analyzeTable_part").collect() + + // Try to analyze a temp table + sql("""SELECT * FROM src""").registerTempTable("tempTable") + intercept[NotImplementedError] { + analyze("tempTable") + } + catalog.unregisterTable(None, "tempTable") + } + + test("estimates the size of a test MetastoreRelation") { + val rdd = sql("""SELECT * FROM src""") + val sizes = rdd.queryExecution.analyzed.collect { case mr: MetastoreRelation => + mr.statistics.sizeInBytes + } + assert(sizes.size === 1) + assert(sizes(0).equals(BigInt(5812)), + s"expected exact size 5812 for test table 'src', got: ${sizes(0)}") + } + + test("auto converts to broadcast hash join, by size estimate of a relation") { + def mkTest( + before: () => Unit, + after: () => Unit, + query: String, + expectedAnswer: Seq[Any], + ct: ClassTag[_]) = { + before() + + var rdd = sql(query) + + // Assert src has a size smaller than the threshold. + val sizes = rdd.queryExecution.analyzed.collect { + case r if ct.runtimeClass.isAssignableFrom(r.getClass) => r.statistics.sizeInBytes + } + assert(sizes.size === 2 && sizes(0) <= autoBroadcastJoinThreshold, + s"query should contain two relations, each of which has size smaller than autoConvertSize") + + // Using `sparkPlan` because for relevant patterns in HashJoin to be + // matched, other strategies need to be applied. + var bhj = rdd.queryExecution.sparkPlan.collect { case j: BroadcastHashJoin => j } + assert(bhj.size === 1, + s"actual query plans do not contain broadcast join: ${rdd.queryExecution}") + + checkAnswer(rdd, expectedAnswer) // check correctness of output + + TestHive.settings.synchronized { + val tmp = autoBroadcastJoinThreshold + + sql(s"""SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD}=-1""") + rdd = sql(query) + bhj = rdd.queryExecution.sparkPlan.collect { case j: BroadcastHashJoin => j } + assert(bhj.isEmpty, "BroadcastHashJoin still planned even though it is switched off") + + val shj = rdd.queryExecution.sparkPlan.collect { case j: ShuffledHashJoin => j } + assert(shj.size === 1, + "ShuffledHashJoin should be planned when BroadcastHashJoin is turned off") + + sql(s"""SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD}=$tmp""") + } + + after() + } + + /** Tests for MetastoreRelation */ + val metastoreQuery = """SELECT * FROM src a JOIN src b ON a.key = 238 AND a.key = b.key""" + val metastoreAnswer = Seq.fill(4)((238, "val_238", 238, "val_238")) + mkTest( + () => (), + () => (), + metastoreQuery, + metastoreAnswer, + implicitly[ClassTag[MetastoreRelation]] + ) + } + +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/api/java/JavaHiveQLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/api/java/JavaHiveQLSuite.scala index 10c8069a624e6..9644b707eb1a0 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/api/java/JavaHiveQLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/api/java/JavaHiveQLSuite.scala @@ -40,7 +40,7 @@ class JavaHiveQLSuite extends FunSuite { ignore("SELECT * FROM src") { assert( - javaHiveCtx.hql("SELECT * FROM src").collect().map(_.getInt(0)) === + javaHiveCtx.sql("SELECT * FROM src").collect().map(_.getInt(0)) === TestHive.sql("SELECT * FROM src").collect().map(_.getInt(0)).toSeq) } @@ -56,33 +56,34 @@ class JavaHiveQLSuite extends FunSuite { val tableName = "test_native_commands" assertResult(0) { - javaHiveCtx.hql(s"DROP TABLE IF EXISTS $tableName").count() + javaHiveCtx.sql(s"DROP TABLE IF EXISTS $tableName").count() } assertResult(0) { - javaHiveCtx.hql(s"CREATE TABLE $tableName(key INT, value STRING)").count() + javaHiveCtx.sql(s"CREATE TABLE $tableName(key INT, value STRING)").count() } - javaHiveCtx.hql("SHOW TABLES").registerAsTable("show_tables") + javaHiveCtx.sql("SHOW TABLES").registerTempTable("show_tables") assert( javaHiveCtx - .hql("SELECT result FROM show_tables") + .sql("SELECT result FROM show_tables") .collect() .map(_.getString(0)) .contains(tableName)) assertResult(Array(Array("key", "int", "None"), Array("value", "string", "None"))) { - javaHiveCtx.hql(s"DESCRIBE $tableName").registerAsTable("describe_table") + javaHiveCtx.sql(s"DESCRIBE $tableName").registerTempTable("describe_table") + javaHiveCtx - .hql("SELECT result FROM describe_table") + .sql("SELECT result FROM describe_table") .collect() .map(_.getString(0).split("\t").map(_.trim)) .toArray } - assert(isExplanation(javaHiveCtx.hql( + assert(isExplanation(javaHiveCtx.sql( s"EXPLAIN SELECT key, COUNT(*) FROM $tableName GROUP BY key"))) TestHive.reset() @@ -90,7 +91,7 @@ class JavaHiveQLSuite extends FunSuite { ignore("Exactly once semantics for DDL and command statements") { val tableName = "test_exactly_once" - val q0 = javaHiveCtx.hql(s"CREATE TABLE $tableName(key INT, value STRING)") + val q0 = javaHiveCtx.sql(s"CREATE TABLE $tableName(key INT, value STRING)") // If the table was not created, the following assertion would fail assert(Try(TestHive.table(tableName)).isSuccess) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala index 08ef4d9b6bb93..502ce8fb297e9 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala @@ -21,7 +21,7 @@ import java.io._ import org.scalatest.{BeforeAndAfterAll, FunSuite, GivenWhenThen} -import org.apache.spark.sql.Logging +import org.apache.spark.Logging import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.logical.{NativeCommand => LogicalNativeCommand} @@ -132,7 +132,7 @@ abstract class HiveComparisonTest answer: Seq[String]): Seq[String] = { def isSorted(plan: LogicalPlan): Boolean = plan match { - case _: Join | _: Aggregate | _: BaseRelation | _: Generate | _: Sample | _: Distinct => false + case _: Join | _: Aggregate | _: Generate | _: Sample | _: Distinct => false case PhysicalOperation(_, _, Sort(_, _)) => true case _ => plan.children.iterator.exists(isSorted) } @@ -161,6 +161,7 @@ abstract class HiveComparisonTest "transient_lastDdlTime", "grantTime", "lastUpdateTime", + "last_modified_by", "last_modified_time", "Owner:", // The following are hive specific schema parameters which we do not need to match exactly. @@ -197,7 +198,7 @@ abstract class HiveComparisonTest // If test sharding is enable, skip tests that are not in the correct shard. shardInfo.foreach { case (shardId, numShards) if testCaseName.hashCode % numShards != shardId => return - case (shardId, _) => logger.debug(s"Shard $shardId includes test '$testCaseName'") + case (shardId, _) => logDebug(s"Shard $shardId includes test '$testCaseName'") } // Skip tests found in directories specified by user. @@ -213,13 +214,13 @@ abstract class HiveComparisonTest .map(new File(_, testCaseName)) .filter(_.exists) if (runOnlyDirectories.nonEmpty && runIndicators.isEmpty) { - logger.debug( + logDebug( s"Skipping test '$testCaseName' not found in ${runOnlyDirectories.map(_.getCanonicalPath)}") return } test(testCaseName) { - logger.debug(s"=== HIVE TEST: $testCaseName ===") + logDebug(s"=== HIVE TEST: $testCaseName ===") // Clear old output for this testcase. outputDirectories.map(new File(_, testCaseName)).filter(_.exists()).foreach(_.delete()) @@ -235,19 +236,19 @@ abstract class HiveComparisonTest .filterNot(_ contains "hive.outerjoin.supports.filters") if (allQueries != queryList) - logger.warn(s"Simplifications made on unsupported operations for test $testCaseName") + logWarning(s"Simplifications made on unsupported operations for test $testCaseName") lazy val consoleTestCase = { val quotes = "\"\"\"" queryList.zipWithIndex.map { case (query, i) => - s"""val q$i = hql($quotes$query$quotes); q$i.collect()""" + s"""val q$i = sql($quotes$query$quotes); q$i.collect()""" }.mkString("\n== Console version of this test ==\n", "\n", "\n") } try { // MINOR HACK: You must run a query before calling reset the first time. - TestHive.hql("SHOW TABLES") + TestHive.sql("SHOW TABLES") if (reset) { TestHive.reset() } val hiveCacheFiles = queryList.zipWithIndex.map { @@ -257,11 +258,11 @@ abstract class HiveComparisonTest } val hiveCachedResults = hiveCacheFiles.flatMap { cachedAnswerFile => - logger.debug(s"Looking for cached answer file $cachedAnswerFile.") + logDebug(s"Looking for cached answer file $cachedAnswerFile.") if (cachedAnswerFile.exists) { Some(fileToString(cachedAnswerFile)) } else { - logger.debug(s"File $cachedAnswerFile not found") + logDebug(s"File $cachedAnswerFile not found") None } }.map { @@ -272,7 +273,7 @@ abstract class HiveComparisonTest val hiveResults: Seq[Seq[String]] = if (hiveCachedResults.size == queryList.size) { - logger.info(s"Using answer cache for test: $testCaseName") + logInfo(s"Using answer cache for test: $testCaseName") hiveCachedResults } else { @@ -287,7 +288,7 @@ abstract class HiveComparisonTest if (installHooksCommand.findAllMatchIn(queryString).nonEmpty) sys.error("hive exec hooks not supported for tests.") - logger.warn(s"Running query ${i+1}/${queryList.size} with hive.") + logWarning(s"Running query ${i+1}/${queryList.size} with hive.") // Analyze the query with catalyst to ensure test tables are loaded. val answer = hiveQuery.analyzed match { case _: ExplainCommand => Nil // No need to execute EXPLAIN queries as we don't check the output. @@ -350,14 +351,8 @@ abstract class HiveComparisonTest val resultComparison = sideBySide(hivePrintOut, catalystPrintOut).mkString("\n") - println("hive output") - hive.foreach(println) - - println("catalyst printout") - catalyst.foreach(println) - if (recomputeCache) { - logger.warn(s"Clearing cache files for failed test $testCaseName") + logWarning(s"Clearing cache files for failed test $testCaseName") hiveCacheFiles.foreach(_.delete()) } @@ -386,7 +381,7 @@ abstract class HiveComparisonTest TestHive.runSqlHive("SELECT key FROM src") } catch { case e: Exception => - logger.error(s"FATAL ERROR: Canary query threw $e This implies that the testing environment has likely been corrupted.") + logError(s"FATAL ERROR: Canary query threw $e This implies that the testing environment has likely been corrupted.") // The testing setup traps exits so wait here for a long time so the developer can see when things started // to go wrong. Thread.sleep(1000000) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala index 50ab71a9003d3..02518d516261b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala @@ -53,7 +53,7 @@ abstract class HiveQueryFileTest extends HiveComparisonTest { testCases.sorted.foreach { case (testCaseName, testCaseFile) => if (blackList.map(_.r.pattern.matcher(testCaseName).matches()).reduceLeft(_||_)) { - logger.debug(s"Blacklisted test skipped $testCaseName") + logDebug(s"Blacklisted test skipped $testCaseName") } else if (realWhiteList.map(_.r.pattern.matcher(testCaseName).matches()).reduceLeft(_||_) || runAll) { // Build a test case and submit it to scala test framework... val queriesString = fileToString(testCaseFile) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index a623d29b53973..fdb2f41f5a5b6 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -19,9 +19,11 @@ package org.apache.spark.sql.hive.execution import scala.util.Try +import org.apache.spark.sql.{SchemaRDD, Row} +import org.apache.spark.sql.hive._ import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.hive.test.TestHive._ -import org.apache.spark.sql.{SchemaRDD, Row} +import org.apache.spark.sql.{Row, SchemaRDD} case class TestData(a: Int, b: String) @@ -30,9 +32,36 @@ case class TestData(a: Int, b: String) */ class HiveQuerySuite extends HiveComparisonTest { + createQueryTest("null case", + "SELECT case when(true) then 1 else null end FROM src LIMIT 1") + + createQueryTest("single case", + """SELECT case when true then 1 else 2 end FROM src LIMIT 1""") + + createQueryTest("double case", + """SELECT case when 1 = 2 then 1 when 2 = 2 then 3 else 2 end FROM src LIMIT 1""") + + createQueryTest("case else null", + """SELECT case when 1 = 2 then 1 when 2 = 2 then 3 else null end FROM src LIMIT 1""") + + createQueryTest("having no references", + "SELECT key FROM src GROUP BY key HAVING COUNT(*) > 1") + + createQueryTest("boolean = number", + """ + |SELECT + | 1 = true, 1L = true, 1Y = true, true = 1, true = 1L, true = 1Y, + | 0 = true, 0L = true, 0Y = true, true = 0, true = 0L, true = 0Y, + | 1 = false, 1L = false, 1Y = false, false = 1, false = 1L, false = 1Y, + | 0 = false, 0L = false, 0Y = false, false = 0, false = 0L, false = 0Y, + | 2 = true, 2L = true, 2Y = true, true = 2, true = 2L, true = 2Y, + | 2 = false, 2L = false, 2Y = false, false = 2, false = 2L, false = 2Y + |FROM src LIMIT 1 + """.stripMargin) + test("CREATE TABLE AS runs once") { - hql("CREATE TABLE foo AS SELECT 1 FROM src LIMIT 1").collect() - assert(hql("SELECT COUNT(*) FROM foo").collect().head.getLong(0) === 1, + sql("CREATE TABLE foo AS SELECT 1 FROM src LIMIT 1").collect() + assert(sql("SELECT COUNT(*) FROM foo").collect().head.getLong(0) === 1, "Incorrect number of rows in created table") } @@ -40,15 +69,20 @@ class HiveQuerySuite extends HiveComparisonTest { "SELECT * FROM src WHERE key Between 1 and 2") createQueryTest("div", - "SELECT 1 DIV 2, 1 div 2, 1 dIv 2 FROM src LIMIT 1") + "SELECT 1 DIV 2, 1 div 2, 1 dIv 2, 100 DIV 51, 100 DIV 49 FROM src LIMIT 1") + + createQueryTest("division", + "SELECT 2 / 1, 1 / 2, 1 / 3, 1 / COUNT(*) FROM src LIMIT 1") test("Query expressed in SQL") { + setConf("spark.sql.dialect", "sql") assert(sql("SELECT 1").collect() === Array(Seq(1))) + setConf("spark.sql.dialect", "hiveql") + } test("Query expressed in HiveQL") { - hql("FROM src SELECT key").collect() - hiveql("FROM src SELECT key").collect() + sql("FROM src SELECT key").collect() } createQueryTest("Constant Folding Optimization for AVG_SUM_COUNT", @@ -164,12 +198,12 @@ class HiveQuerySuite extends HiveComparisonTest { "SELECT * FROM src LATERAL VIEW explode(map(key+3,key+4)) D as k, v") test("sampling") { - hql("SELECT * FROM src TABLESAMPLE(0.1 PERCENT) s") + sql("SELECT * FROM src TABLESAMPLE(0.1 PERCENT) s") } test("SchemaRDD toString") { - hql("SHOW TABLES").toString - hql("SELECT * FROM src").toString + sql("SHOW TABLES").toString + sql("SELECT * FROM src").toString } createQueryTest("case statements with key #1", @@ -197,8 +231,8 @@ class HiveQuerySuite extends HiveComparisonTest { "SELECT (CASE WHEN key > 2 THEN 3 WHEN 2 > key THEN 2 ELSE 0 END) FROM src WHERE key < 15") test("implement identity function using case statement") { - val actual = hql("SELECT (CASE key WHEN key THEN key END) FROM src").collect().toSet - val expected = hql("SELECT key FROM src").collect().toSet + val actual = sql("SELECT (CASE key WHEN key THEN key END) FROM src").collect().toSet + val expected = sql("SELECT key FROM src").collect().toSet assert(actual === expected) } @@ -206,7 +240,7 @@ class HiveQuerySuite extends HiveComparisonTest { // See https://github.com/apache/spark/pull/1055#issuecomment-45820167 for a discussion. ignore("non-boolean conditions in a CaseWhen are illegal") { intercept[Exception] { - hql("SELECT (CASE WHEN key > 2 THEN 3 WHEN 1 THEN 2 ELSE 0 END) FROM src").collect() + sql("SELECT (CASE WHEN key > 2 THEN 3 WHEN 1 THEN 2 ELSE 0 END) FROM src").collect() } } @@ -218,10 +252,10 @@ class HiveQuerySuite extends HiveComparisonTest { TestHive.sparkContext.parallelize( TestData(1, "str1") :: TestData(2, "str2") :: Nil) - testData.registerAsTable("REGisteredTABle") + testData.registerTempTable("REGisteredTABle") assertResult(Array(Array(2, "str2"))) { - hql("SELECT tablealias.A, TABLEALIAS.b FROM reGisteredTABle TableAlias " + + sql("SELECT tablealias.A, TABLEALIAS.b FROM reGisteredTABle TableAlias " + "WHERE TableAliaS.a > 1").collect() } } @@ -232,9 +266,9 @@ class HiveQuerySuite extends HiveComparisonTest { } test("SPARK-1704: Explain commands as a SchemaRDD") { - hql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") + sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") - val rdd = hql("explain select key, count(value) from src group by key") + val rdd = sql("explain select key, count(value) from src group by key") assert(isExplanation(rdd)) TestHive.reset() @@ -243,9 +277,9 @@ class HiveQuerySuite extends HiveComparisonTest { test("SPARK-2180: HAVING support in GROUP BY clauses (positive)") { val fixture = List(("foo", 2), ("bar", 1), ("foo", 4), ("bar", 3)) .zipWithIndex.map {case Pair(Pair(value, attr), key) => HavingRow(key, value, attr)} - TestHive.sparkContext.parallelize(fixture).registerAsTable("having_test") + TestHive.sparkContext.parallelize(fixture).registerTempTable("having_test") val results = - hql("SELECT value, max(attr) AS attr FROM having_test GROUP BY value HAVING attr > 3") + sql("SELECT value, max(attr) AS attr FROM having_test GROUP BY value HAVING attr > 3") .collect() .map(x => Pair(x.getString(0), x.getInt(1))) @@ -254,39 +288,39 @@ class HiveQuerySuite extends HiveComparisonTest { } test("SPARK-2180: HAVING with non-boolean clause raises no exceptions") { - hql("select key, count(*) c from src group by key having c").collect() + sql("select key, count(*) c from src group by key having c").collect() } test("SPARK-2225: turn HAVING without GROUP BY into a simple filter") { - assert(hql("select key from src having key > 490").collect().size < 100) + assert(sql("select key from src having key > 490").collect().size < 100) } test("Query Hive native command execution result") { val tableName = "test_native_commands" assertResult(0) { - hql(s"DROP TABLE IF EXISTS $tableName").count() + sql(s"DROP TABLE IF EXISTS $tableName").count() } assertResult(0) { - hql(s"CREATE TABLE $tableName(key INT, value STRING)").count() + sql(s"CREATE TABLE $tableName(key INT, value STRING)").count() } assert( - hql("SHOW TABLES") + sql("SHOW TABLES") .select('result) .collect() .map(_.getString(0)) .contains(tableName)) - assert(isExplanation(hql(s"EXPLAIN SELECT key, COUNT(*) FROM $tableName GROUP BY key"))) + assert(isExplanation(sql(s"EXPLAIN SELECT key, COUNT(*) FROM $tableName GROUP BY key"))) TestHive.reset() } test("Exactly once semantics for DDL and command statements") { val tableName = "test_exactly_once" - val q0 = hql(s"CREATE TABLE $tableName(key INT, value STRING)") + val q0 = sql(s"CREATE TABLE $tableName(key INT, value STRING)") // If the table was not created, the following assertion would fail assert(Try(table(tableName)).isSuccess) @@ -296,9 +330,9 @@ class HiveQuerySuite extends HiveComparisonTest { } test("DESCRIBE commands") { - hql(s"CREATE TABLE test_describe_commands1 (key INT, value STRING) PARTITIONED BY (dt STRING)") + sql(s"CREATE TABLE test_describe_commands1 (key INT, value STRING) PARTITIONED BY (dt STRING)") - hql( + sql( """FROM src INSERT OVERWRITE TABLE test_describe_commands1 PARTITION (dt='2008-06-08') |SELECT key, value """.stripMargin) @@ -313,7 +347,7 @@ class HiveQuerySuite extends HiveComparisonTest { Array("# col_name", "data_type", "comment"), Array("dt", "string", null)) ) { - hql("DESCRIBE test_describe_commands1") + sql("DESCRIBE test_describe_commands1") .select('col_name, 'data_type, 'comment) .collect() } @@ -328,14 +362,14 @@ class HiveQuerySuite extends HiveComparisonTest { Array("# col_name", "data_type", "comment"), Array("dt", "string", null)) ) { - hql("DESCRIBE default.test_describe_commands1") + sql("DESCRIBE default.test_describe_commands1") .select('col_name, 'data_type, 'comment) .collect() } // Describe a column is a native command assertResult(Array(Array("value", "string", "from deserializer"))) { - hql("DESCRIBE test_describe_commands1 value") + sql("DESCRIBE test_describe_commands1 value") .select('result) .collect() .map(_.getString(0).split("\t").map(_.trim)) @@ -343,7 +377,7 @@ class HiveQuerySuite extends HiveComparisonTest { // Describe a column is a native command assertResult(Array(Array("value", "string", "from deserializer"))) { - hql("DESCRIBE default.test_describe_commands1 value") + sql("DESCRIBE default.test_describe_commands1 value") .select('result) .collect() .map(_.getString(0).split("\t").map(_.trim)) @@ -361,7 +395,7 @@ class HiveQuerySuite extends HiveComparisonTest { Array("", "", ""), Array("dt", "string", "None")) ) { - hql("DESCRIBE test_describe_commands1 PARTITION (dt='2008-06-08')") + sql("DESCRIBE test_describe_commands1 PARTITION (dt='2008-06-08')") .select('result) .collect() .map(_.getString(0).split("\t").map(_.trim)) @@ -372,7 +406,7 @@ class HiveQuerySuite extends HiveComparisonTest { TestHive.sparkContext.parallelize( TestData(1, "str1") :: TestData(1, "str2") :: Nil) - testData.registerAsTable("test_describe_commands2") + testData.registerTempTable("test_describe_commands2") assertResult( Array( @@ -380,17 +414,17 @@ class HiveQuerySuite extends HiveComparisonTest { Array("a", "IntegerType", null), Array("b", "StringType", null)) ) { - hql("DESCRIBE test_describe_commands2") + sql("DESCRIBE test_describe_commands2") .select('col_name, 'data_type, 'comment) .collect() } } test("SPARK-2263: Insert Map values") { - hql("CREATE TABLE m(value MAP)") - hql("INSERT OVERWRITE TABLE m SELECT MAP(key, value) FROM src LIMIT 10") - hql("SELECT * FROM m").collect().zip(hql("SELECT * FROM src LIMIT 10").collect()).map { - case (Row(map: Map[Int, String]), Row(key: Int, value: String)) => + sql("CREATE TABLE m(value MAP)") + sql("INSERT OVERWRITE TABLE m SELECT MAP(key, value) FROM src LIMIT 10") + sql("SELECT * FROM m").collect().zip(sql("SELECT * FROM src LIMIT 10").collect()).map { + case (Row(map: Map[_, _]), Row(key: Int, value: String)) => assert(map.size === 1) assert(map.head === (key, value)) } @@ -401,19 +435,19 @@ class HiveQuerySuite extends HiveComparisonTest { val testKey = "spark.sql.key.usedfortestonly" val testVal = "val0,val_1,val2.3,my_table" - hql(s"set $testKey=$testVal") - assert(get(testKey, testVal + "_") == testVal) + sql(s"set $testKey=$testVal") + assert(getConf(testKey, testVal + "_") == testVal) - hql("set mapred.reduce.tasks=20") - assert(get("mapred.reduce.tasks", "0") == "20") - hql("set mapred.reduce.tasks = 40") - assert(get("mapred.reduce.tasks", "0") == "40") + sql("set some.property=20") + assert(getConf("some.property", "0") == "20") + sql("set some.property = 40") + assert(getConf("some.property", "0") == "40") - hql(s"set $testKey=$testVal") - assert(get(testKey, "0") == testVal) + sql(s"set $testKey=$testVal") + assert(getConf(testKey, "0") == testVal) - hql(s"set $testKey=") - assert(get(testKey, "0") == "") + sql(s"set $testKey=") + assert(getConf(testKey, "0") == "") } test("SET commands semantics for a HiveContext") { @@ -421,64 +455,62 @@ class HiveQuerySuite extends HiveComparisonTest { val testKey = "spark.sql.key.usedfortestonly" val testVal = "test.val.0" val nonexistentKey = "nonexistent" - def rowsToPairs(rows: Array[Row]) = rows.map { case Row(key: String, value: String) => - key -> value - } clear() // "set" itself returns all config variables currently specified in SQLConf. - assert(hql("SET").collect().size == 0) + // TODO: Should we be listing the default here always? probably... + assert(sql("SET").collect().size == 0) - assertResult(Array(testKey -> testVal)) { - rowsToPairs(hql(s"SET $testKey=$testVal").collect()) + assertResult(Array(s"$testKey=$testVal")) { + sql(s"SET $testKey=$testVal").collect().map(_.getString(0)) } assert(hiveconf.get(testKey, "") == testVal) - assertResult(Array(testKey -> testVal)) { - rowsToPairs(hql("SET").collect()) + assertResult(Array(s"$testKey=$testVal")) { + sql(s"SET $testKey=$testVal").collect().map(_.getString(0)) } - hql(s"SET ${testKey + testKey}=${testVal + testVal}") + sql(s"SET ${testKey + testKey}=${testVal + testVal}") assert(hiveconf.get(testKey + testKey, "") == testVal + testVal) - assertResult(Array(testKey -> testVal, (testKey + testKey) -> (testVal + testVal))) { - rowsToPairs(hql("SET").collect()) + assertResult(Array(s"$testKey=$testVal", s"${testKey + testKey}=${testVal + testVal}")) { + sql(s"SET").collect().map(_.getString(0)) } // "set key" - assertResult(Array(testKey -> testVal)) { - rowsToPairs(hql(s"SET $testKey").collect()) + assertResult(Array(s"$testKey=$testVal")) { + sql(s"SET $testKey").collect().map(_.getString(0)) } - assertResult(Array(nonexistentKey -> "")) { - rowsToPairs(hql(s"SET $nonexistentKey").collect()) + assertResult(Array(s"$nonexistentKey=")) { + sql(s"SET $nonexistentKey").collect().map(_.getString(0)) } - // Assert that sql() should have the same effects as hql() by repeating the above using sql(). + // Assert that sql() should have the same effects as sql() by repeating the above using sql(). clear() assert(sql("SET").collect().size == 0) - assertResult(Array(testKey -> testVal)) { - rowsToPairs(sql(s"SET $testKey=$testVal").collect()) + assertResult(Array(s"$testKey=$testVal")) { + sql(s"SET $testKey=$testVal").collect().map(_.getString(0)) } assert(hiveconf.get(testKey, "") == testVal) - assertResult(Array(testKey -> testVal)) { - rowsToPairs(sql("SET").collect()) + assertResult(Array(s"$testKey=$testVal")) { + sql("SET").collect().map(_.getString(0)) } sql(s"SET ${testKey + testKey}=${testVal + testVal}") assert(hiveconf.get(testKey + testKey, "") == testVal + testVal) - assertResult(Array(testKey -> testVal, (testKey + testKey) -> (testVal + testVal))) { - rowsToPairs(sql("SET").collect()) + assertResult(Array(s"$testKey=$testVal", s"${testKey + testKey}=${testVal + testVal}")) { + sql("SET").collect().map(_.getString(0)) } - assertResult(Array(testKey -> testVal)) { - rowsToPairs(sql(s"SET $testKey").collect()) + assertResult(Array(s"$testKey=$testVal")) { + sql(s"SET $testKey").collect().map(_.getString(0)) } - assertResult(Array(nonexistentKey -> "")) { - rowsToPairs(sql(s"SET $nonexistentKey").collect()) + assertResult(Array(s"$nonexistentKey=")) { + sql(s"SET $nonexistentKey").collect().map(_.getString(0)) } clear() diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala index 67594b57d3dfa..6b3ffd1c0ffe2 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.hive.execution import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.hive.test.TestHive._ -case class Data(a: Int, B: Int, n: Nested) +case class Data(a: Int, B: Int, n: Nested, nestedArray: Seq[Nested]) case class Nested(a: Int, B: Int) /** @@ -53,10 +53,16 @@ class HiveResolutionSuite extends HiveComparisonTest { test("case insensitivity with scala reflection") { // Test resolution with Scala Reflection - TestHive.sparkContext.parallelize(Data(1, 2, Nested(1,2)) :: Nil) - .registerAsTable("caseSensitivityTest") + TestHive.sparkContext.parallelize(Data(1, 2, Nested(1,2), Seq(Nested(1,2))) :: Nil) + .registerTempTable("caseSensitivityTest") - hql("SELECT a, b, A, B, n.a, n.b, n.A, n.B FROM caseSensitivityTest") + sql("SELECT a, b, A, B, n.a, n.b, n.A, n.B FROM caseSensitivityTest") + } + + test("nested repeated resolution") { + TestHive.sparkContext.parallelize(Data(1, 2, Nested(1,2), Seq(Nested(1,2))) :: Nil) + .registerTempTable("nestedRepeatedTest") + assert(sql("SELECT nestedArray[0].a FROM nestedRepeatedTest").collect().head(0) === 1) } /** diff --git a/core/src/main/java/org/apache/spark/network/netty/FileServerChannelInitializer.java b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala similarity index 50% rename from core/src/main/java/org/apache/spark/network/netty/FileServerChannelInitializer.java rename to sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala index 46efec8f8d963..c5736723b47c0 100644 --- a/core/src/main/java/org/apache/spark/network/netty/FileServerChannelInitializer.java +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala @@ -15,27 +15,27 @@ * limitations under the License. */ -package org.apache.spark.network.netty; +package org.apache.spark.sql.hive.execution -import io.netty.channel.ChannelInitializer; -import io.netty.channel.socket.SocketChannel; -import io.netty.handler.codec.DelimiterBasedFrameDecoder; -import io.netty.handler.codec.Delimiters; -import io.netty.handler.codec.string.StringDecoder; +class HiveTableScanSuite extends HiveComparisonTest { -class FileServerChannelInitializer extends ChannelInitializer { - - private final PathResolver pResolver; - - FileServerChannelInitializer(PathResolver pResolver) { - this.pResolver = pResolver; - } - - @Override - public void initChannel(SocketChannel channel) { - channel.pipeline() - .addLast("framer", new DelimiterBasedFrameDecoder(8192, Delimiters.lineDelimiter())) - .addLast("stringDecoder", new StringDecoder()) - .addLast("handler", new FileServerHandler(pResolver)); - } + createQueryTest("partition_based_table_scan_with_different_serde", + """ + |CREATE TABLE part_scan_test (key STRING, value STRING) PARTITIONED BY (ds STRING) + |ROW FORMAT SERDE + |'org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe' + |STORED AS RCFILE; + | + |FROM src + |INSERT INTO TABLE part_scan_test PARTITION (ds='2010-01-01') + |SELECT 100,100 LIMIT 1; + | + |ALTER TABLE part_scan_test SET SERDE + |'org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe'; + | + |FROM src INSERT INTO TABLE part_scan_test PARTITION (ds='2010-01-02') + |SELECT 200,200 LIMIT 1; + | + |SELECT * from part_scan_test; + """.stripMargin) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala index 7436de264a1e1..c3c18cf8ccac3 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala @@ -35,7 +35,7 @@ class HiveTypeCoercionSuite extends HiveComparisonTest { test("[SPARK-2210] boolean cast on boolean value should be removed") { val q = "select cast(cast(key=0 as boolean) as boolean) from src" - val project = TestHive.hql(q).queryExecution.executedPlan.collect { case e: Project => e }.head + val project = TestHive.sql(q).queryExecution.executedPlan.collect { case e: Project => e }.head // No cast expression introduced project.transformAllExpressions { case c: Cast => diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala index f944d010660eb..b6b8592344ef5 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala @@ -37,7 +37,7 @@ import org.apache.hadoop.hive.ql.udf.generic.GenericUDF.DeferredObject */ class HiveUdfSuite extends HiveComparisonTest { - TestHive.hql( + TestHive.sql( """ |CREATE EXTERNAL TABLE hiveUdfTestTable ( | pair STRUCT @@ -48,16 +48,16 @@ class HiveUdfSuite extends HiveComparisonTest { """.stripMargin.format(classOf[PairSerDe].getName) ) - TestHive.hql( + TestHive.sql( "ALTER TABLE hiveUdfTestTable ADD IF NOT EXISTS PARTITION(partition='testUdf') LOCATION '%s'" .format(this.getClass.getClassLoader.getResource("data/files/testUdf").getFile) ) - TestHive.hql("CREATE TEMPORARY FUNCTION testUdf AS '%s'".format(classOf[PairUdf].getName)) + TestHive.sql("CREATE TEMPORARY FUNCTION testUdf AS '%s'".format(classOf[PairUdf].getName)) - TestHive.hql("SELECT testUdf(pair) FROM hiveUdfTestTable") + TestHive.sql("SELECT testUdf(pair) FROM hiveUdfTestTable") - TestHive.hql("DROP TEMPORARY FUNCTION IF EXISTS testUdf") + TestHive.sql("DROP TEMPORARY FUNCTION IF EXISTS testUdf") } class TestPair(x: Int, y: Int) extends Writable with Serializable { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala index 34d8a061ccc83..1a6dbc0ce0c0d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala @@ -27,7 +27,7 @@ import scala.collection.JavaConversions._ */ class PruningSuite extends HiveComparisonTest { // MINOR HACK: You must run a query before calling reset the first time. - TestHive.hql("SHOW TABLES") + TestHive.sql("SHOW TABLES") // Column/partition pruning is not implemented for `InMemoryColumnarTableScan` yet, need to reset // the environment to ensure all referenced tables in this suites are not cached in-memory. diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala new file mode 100644 index 0000000000000..635a9fb0d56cb --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.execution + +import scala.reflect.ClassTag + +import org.apache.spark.sql.{SQLConf, QueryTest} +import org.apache.spark.sql.execution.{BroadcastHashJoin, ShuffledHashJoin} +import org.apache.spark.sql.hive.test.TestHive +import org.apache.spark.sql.hive.test.TestHive._ + +/** + * A collection of hive query tests where we generate the answers ourselves instead of depending on + * Hive to generate them (in contrast to HiveQuerySuite). Often this is because the query is + * valid, but Hive currently cannot execute it. + */ +class SQLQuerySuite extends QueryTest { + test("ordering not in select") { + checkAnswer( + sql("SELECT key FROM src ORDER BY value"), + sql("SELECT key FROM (SELECT key, value FROM src ORDER BY value) a").collect().toSeq) + } + + test("ordering not in agg") { + checkAnswer( + sql("SELECT key FROM src GROUP BY key, value ORDER BY value"), + sql(""" + SELECT key + FROM ( + SELECT key, value + FROM src + GROUP BY key, value + ORDER BY value) a""").collect().toSeq) + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala index 91ad59d7f82c0..6f57fe8958387 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.parquet +import java.io.File + import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite} import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Row} @@ -27,6 +29,8 @@ import org.apache.spark.util.Utils // Implicits import org.apache.spark.sql.hive.test.TestHive._ +case class Cases(lower: String, UPPER: String) + class HiveParquetSuite extends FunSuite with BeforeAndAfterAll with BeforeAndAfterEach { val dirname = Utils.createTempDir() @@ -35,9 +39,9 @@ class HiveParquetSuite extends FunSuite with BeforeAndAfterAll with BeforeAndAft override def beforeAll() { // write test data - ParquetTestData.writeFile + ParquetTestData.writeFile() testRDD = parquetFile(ParquetTestData.testDir.toString) - testRDD.registerAsTable("testsource") + testRDD.registerTempTable("testsource") } override def afterAll() { @@ -55,35 +59,49 @@ class HiveParquetSuite extends FunSuite with BeforeAndAfterAll with BeforeAndAft Utils.deleteRecursively(dirname) } + test("Case insensitive attribute names") { + val tempFile = File.createTempFile("parquet", "") + tempFile.delete() + sparkContext.parallelize(1 to 10) + .map(_.toString) + .map(i => Cases(i, i)) + .saveAsParquetFile(tempFile.getCanonicalPath) + + parquetFile(tempFile.getCanonicalPath).registerTempTable("cases") + sql("SELECT upper FROM cases").collect().map(_.getString(0)) === (1 to 10).map(_.toString) + sql("SELECT LOWER FROM cases").collect().map(_.getString(0)) === (1 to 10).map(_.toString) + } + test("SELECT on Parquet table") { - val rdd = hql("SELECT * FROM testsource").collect() + val rdd = sql("SELECT * FROM testsource").collect() assert(rdd != null) assert(rdd.forall(_.size == 6)) } test("Simple column projection + filter on Parquet table") { - val rdd = hql("SELECT myboolean, mylong FROM testsource WHERE myboolean=true").collect() + val rdd = sql("SELECT myboolean, mylong FROM testsource WHERE myboolean=true").collect() assert(rdd.size === 5, "Filter returned incorrect number of rows") assert(rdd.forall(_.getBoolean(0)), "Filter returned incorrect Boolean field value") } test("Converting Hive to Parquet Table via saveAsParquetFile") { - hql("SELECT * FROM src").saveAsParquetFile(dirname.getAbsolutePath) - parquetFile(dirname.getAbsolutePath).registerAsTable("ptable") - val rddOne = hql("SELECT * FROM src").collect().sortBy(_.getInt(0)) - val rddTwo = hql("SELECT * from ptable").collect().sortBy(_.getInt(0)) + sql("SELECT * FROM src").saveAsParquetFile(dirname.getAbsolutePath) + parquetFile(dirname.getAbsolutePath).registerTempTable("ptable") + val rddOne = sql("SELECT * FROM src").collect().sortBy(_.getInt(0)) + val rddTwo = sql("SELECT * from ptable").collect().sortBy(_.getInt(0)) + compareRDDs(rddOne, rddTwo, "src (Hive)", Seq("key:Int", "value:String")) } test("INSERT OVERWRITE TABLE Parquet table") { - hql("SELECT * FROM testsource").saveAsParquetFile(dirname.getAbsolutePath) - parquetFile(dirname.getAbsolutePath).registerAsTable("ptable") + sql("SELECT * FROM testsource").saveAsParquetFile(dirname.getAbsolutePath) + parquetFile(dirname.getAbsolutePath).registerTempTable("ptable") // let's do three overwrites for good measure - hql("INSERT OVERWRITE TABLE ptable SELECT * FROM testsource").collect() - hql("INSERT OVERWRITE TABLE ptable SELECT * FROM testsource").collect() - hql("INSERT OVERWRITE TABLE ptable SELECT * FROM testsource").collect() - val rddCopy = hql("SELECT * FROM ptable").collect() - val rddOrig = hql("SELECT * FROM testsource").collect() + sql("INSERT OVERWRITE TABLE ptable SELECT * FROM testsource").collect() + sql("INSERT OVERWRITE TABLE ptable SELECT * FROM testsource").collect() + sql("INSERT OVERWRITE TABLE ptable SELECT * FROM testsource").collect() + val rddCopy = sql("SELECT * FROM ptable").collect() + val rddOrig = sql("SELECT * FROM testsource").collect() assert(rddCopy.size === rddOrig.size, "INSERT OVERWRITE changed size of table??") compareRDDs(rddOrig, rddCopy, "testsource", ParquetTestData.testSchemaFieldNames) } diff --git a/streaming/pom.xml b/streaming/pom.xml index f506d6ce34a6f..ce35520a28609 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -27,6 +27,9 @@ org.apache.spark spark-streaming_2.10 + + streaming + jar Spark Project Streaming http://spark.apache.org/ @@ -55,6 +58,11 @@ scalacheck_${scala.binary.version} test + + junit + junit + test + com.novocode junit-interface @@ -73,11 +81,11 @@ org.apache.maven.plugins @@ -89,8 +97,8 @@ - test-jar-on-compile - compile + test-jar-on-test-compile + test-compile test-jar diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala index ac56ff709c1c4..b780282bdac37 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala @@ -35,7 +35,6 @@ class Checkpoint(@transient ssc: StreamingContext, val checkpointTime: Time) extends Logging with Serializable { val master = ssc.sc.master val framework = ssc.sc.appName - val sparkHome = ssc.sc.getSparkHome.getOrElse(null) val jars = ssc.sc.jars val graph = ssc.graph val checkpointDir = ssc.checkpointDir diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index e0677b795cb94..101cec1c7a7c2 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -98,9 +98,15 @@ class StreamingContext private[streaming] ( * @param hadoopConf Optional, configuration object if necessary for reading from * HDFS compatible filesystems */ - def this(path: String, hadoopConf: Configuration = new Configuration) = + def this(path: String, hadoopConf: Configuration) = this(null, CheckpointReader.read(path, new SparkConf(), hadoopConf).get, null) + /** + * Recreate a StreamingContext from a checkpoint file. + * @param path Path to the directory that was specified as the checkpoint directory + */ + def this(path: String) = this(path, new Configuration) + if (sc_ == null && cp_ == null) { throw new Exception("Spark Streaming cannot be initialized with " + "both SparkContext and checkpoint as null") diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingSource.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingSource.scala index 774adc3c23c21..75f0e8716dc7e 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingSource.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingSource.scala @@ -23,10 +23,10 @@ import org.apache.spark.metrics.source.Source import org.apache.spark.streaming.ui.StreamingJobProgressListener private[streaming] class StreamingSource(ssc: StreamingContext) extends Source { - val metricRegistry = new MetricRegistry - val sourceName = "%s.StreamingMetrics".format(ssc.sparkContext.appName) + override val metricRegistry = new MetricRegistry + override val sourceName = "%s.StreamingMetrics".format(ssc.sparkContext.appName) - val streamingListener = ssc.uiTab.listener + private val streamingListener = ssc.uiTab.listener private def registerGauge[T](name: String, f: StreamingJobProgressListener => T, defaultValue: T) { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala index c4bdf01fa3744..c00e11d11910f 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala @@ -737,7 +737,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( } object JavaPairDStream { - implicit def fromPairDStream[K: ClassTag, V: ClassTag](dstream: DStream[(K, V)]) = { + implicit def fromPairDStream[K: ClassTag, V: ClassTag](dstream: DStream[(K, V)]) + : JavaPairDStream[K, V] = { new JavaPairDStream[K, V](dstream) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala index 18605cac7006c..b51d5ff0be9fc 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala @@ -546,6 +546,9 @@ class JavaStreamingContext(val ssc: StreamingContext) { * JavaStreamingContext object contains a number of utility functions. */ object JavaStreamingContext { + implicit def fromStreamingContext(ssc: StreamingContext): JavaStreamingContext = new JavaStreamingContext(ssc) + + implicit def toStreamingContext(jssc: JavaStreamingContext): StreamingContext = jssc.ssc /** * Either recreate a StreamingContext from checkpoint data or create a new StreamingContext. diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonDStream.scala new file mode 100644 index 0000000000000..990feacbdc598 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonDStream.scala @@ -0,0 +1,147 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.api.python + +import java.io._ +import java.util.{List => JList, ArrayList => JArrayList, Map => JMap} + +import scala.reflect.ClassTag +import scala.collection.JavaConversions._ + + +import org.apache.spark._ +import org.apache.spark.rdd.RDD +import org.apache.spark.api.java._ +import org.apache.spark.api.python._ +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.streaming.{StreamingContext, Duration, Time} +import org.apache.spark.streaming.dstream._ +import org.apache.spark.streaming.api.java._ + + + +class PythonDStream[T: ClassTag]( + parent: DStream[T], + command: Array[Byte], + envVars: JMap[String, String], + pythonIncludes: JList[String], + preservePartitoning: Boolean, + pythonExec: String, + broadcastVars: JList[Broadcast[Array[Byte]]], + accumulator: Accumulator[JList[Array[Byte]]]) + extends DStream[Array[Byte]](parent.ssc) { + + override def dependencies = List(parent) + + override def slideDuration: Duration = parent.slideDuration + + //pythonDStream compute + override def compute(validTime: Time): Option[RDD[Array[Byte]]] = { + parent.getOrCompute(validTime) match{ + case Some(rdd) => + // create PythonRDD to compute Python functions. + val pythonRDD = new PythonRDD(rdd, command, envVars, pythonIncludes, + preservePartitoning, pythonExec, broadcastVars, accumulator) + Some(pythonRDD.asJavaRDD.rdd) + case None => None + } + } + + def foreachRDD(foreachFunc: PythonRDDFunction) { + new PythonForeachDStream(this, context.sparkContext.clean(foreachFunc, false)).register() + } + + val asJavaDStream = JavaDStream.fromDStream(this) +} + + +private class PythonPairwiseDStream(prev:DStream[Array[Byte]], partitioner: Partitioner) extends +DStream[Array[Byte]](prev.ssc){ + override def dependencies = List(prev) + + override def slideDuration: Duration = prev.slideDuration + + override def compute(validTime:Time):Option[RDD[Array[Byte]]]={ + prev.getOrCompute(validTime) match{ + case Some(rdd)=>Some(rdd) + val pairwiseRDD = new PairwiseRDD(rdd) + /* + * Since python function is executed by Scala after StreamingContext.start. + * What PythonPairwiseDStream does is equivalent to python code in pyspark. + * + * with _JavaStackTrace(self.context) as st: + * pairRDD = self.ctx._jvm.PairwiseRDD(keyed._jrdd.rdd()).asJavaPairRDD() + * partitioner = self.ctx._jvm.PythonPartitioner(numPartitions, + * id(partitionFunc)) + * jrdd = pairRDD.partitionBy(partitioner).values() + * rdd = RDD(jrdd, self.ctx, BatchedSerializer(outputSerializer)) + */ + Some(pairwiseRDD.asJavaPairRDD.partitionBy(partitioner).values().rdd) + case None => None + } + } + + val asJavaDStream = JavaDStream.fromDStream(this) +} + + +class PythonForeachDStream( + prev: DStream[Array[Byte]], + foreachFunction: PythonRDDFunction + ) extends ForEachDStream[Array[Byte]]( + prev, + (rdd: RDD[Array[Byte]], time: Time) => { + foreachFunction.call(rdd.toJavaRDD(), time.milliseconds) + } + ) { + + this.register() +} + + +/** + * This is a input stream just for the unitest. This is equivalent to a checkpointable, + * replayable, reliable message queue like Kafka. It requires a sequence as input, and + * returns the i_th element at the i_th batch under manual clock. + */ + +class PythonTestInputStream(ssc_ : JavaStreamingContext, inputRDDs: JArrayList[JavaRDD[Array[Byte]]]) + extends InputDStream[Array[Byte]](JavaStreamingContext.toStreamingContext(ssc_)) { + + def start() {} + + def stop() {} + + def compute(validTime: Time): Option[RDD[Array[Byte]]] = { + val emptyRDD = ssc.sparkContext.emptyRDD[Array[Byte]] + val index = ((validTime - zeroTime) / slideDuration - 1).toInt + val selectedRDD = { + if (inputRDDs.isEmpty) { + emptyRDD + } else if (index < inputRDDs.size()) { + inputRDDs.get(index).rdd + } else { + emptyRDD + } + } + + Some(selectedRDD) + } + + val asJavaDStream = JavaDStream.fromDStream(this) +} \ No newline at end of file diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonRDDFunction.java b/streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonRDDFunction.java new file mode 100644 index 0000000000000..eacff4b0e6006 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonRDDFunction.java @@ -0,0 +1,12 @@ +package org.apache.spark.streaming.api.python; + +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.streaming.Time; + +/* + * Interface for py4j callback function. + * This interface is related to pyspark.streaming.dstream.DStream.foreachRDD . + */ +public interface PythonRDDFunction { + JavaRDD call(JavaRDD rdd, long time); +} \ No newline at end of file diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala index 40da31318942e..1a47089e513c4 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala @@ -133,17 +133,17 @@ class ReducedWindowedDStream[K: ClassTag, V: ClassTag]( val numOldValues = oldRDDs.size val numNewValues = newRDDs.size - val mergeValues = (seqOfValues: Seq[Seq[V]]) => { - if (seqOfValues.size != 1 + numOldValues + numNewValues) { + val mergeValues = (arrayOfValues: Array[Iterable[V]]) => { + if (arrayOfValues.size != 1 + numOldValues + numNewValues) { throw new Exception("Unexpected number of sequences of reduced values") } // Getting reduced values "old time steps" that will be removed from current window - val oldValues = (1 to numOldValues).map(i => seqOfValues(i)).filter(!_.isEmpty).map(_.head) + val oldValues = (1 to numOldValues).map(i => arrayOfValues(i)).filter(!_.isEmpty).map(_.head) // Getting reduced values "new time steps" val newValues = - (1 to numNewValues).map(i => seqOfValues(numOldValues + i)).filter(!_.isEmpty).map(_.head) + (1 to numNewValues).map(i => arrayOfValues(numOldValues + i)).filter(!_.isEmpty).map(_.head) - if (seqOfValues(0).isEmpty) { + if (arrayOfValues(0).isEmpty) { // If previous window's reduce value does not exist, then at least new values should exist if (newValues.isEmpty) { throw new Exception("Neither previous window has value for key, nor new values found. " + @@ -153,7 +153,7 @@ class ReducedWindowedDStream[K: ClassTag, V: ClassTag]( newValues.reduce(reduceF) // return } else { // Get the previous window's reduced value - var tempValue = seqOfValues(0).head + var tempValue = arrayOfValues(0).head // If old values exists, then inverse reduce then from previous value if (!oldValues.isEmpty) { tempValue = invReduceF(tempValue, oldValues.reduce(reduceF)) @@ -166,7 +166,8 @@ class ReducedWindowedDStream[K: ClassTag, V: ClassTag]( } } - val mergedValuesRDD = cogroupedRDD.asInstanceOf[RDD[(K,Seq[Seq[V]])]].mapValues(mergeValues) + val mergedValuesRDD = cogroupedRDD.asInstanceOf[RDD[(K, Array[Iterable[V]])]] + .mapValues(mergeValues) if (filterFunc.isDefined) { Some(mergedValuesRDD.filter(filterFunc.get)) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ActorReceiver.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ActorReceiver.scala index 743be58950c09..1868a1ebc7b4a 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ActorReceiver.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ActorReceiver.scala @@ -68,13 +68,13 @@ object ActorSupervisorStrategy { * should be same. */ @DeveloperApi -trait ActorHelper { +trait ActorHelper extends Logging{ self: Actor => // to ensure that this can be added to Actor classes only /** Store an iterator of received data as a data block into Spark's memory. */ def store[T](iter: Iterator[T]) { - println("Storing iterator") + logDebug("Storing iterator") context.parent ! IteratorData(iter) } @@ -84,6 +84,7 @@ trait ActorHelper { * that Spark is configured to use. */ def store(bytes: ByteBuffer) { + logDebug("Storing Bytes") context.parent ! ByteBufferData(bytes) } @@ -93,7 +94,7 @@ trait ActorHelper { * being pushed into Spark's memory. */ def store[T](item: T) { - println("Storing item") + logDebug("Storing item") context.parent ! SingleItemData(item) } } @@ -157,15 +158,16 @@ private[streaming] class ActorReceiver[T: ClassTag]( def receive = { case IteratorData(iterator) => - println("received iterator") + logDebug("received iterator") store(iterator.asInstanceOf[Iterator[T]]) case SingleItemData(msg) => - println("received single") + logDebug("received single") store(msg.asInstanceOf[T]) n.incrementAndGet case ByteBufferData(bytes) => + logDebug("received bytes") store(bytes) case props: Props => diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala index 78cc2daa56e53..0316b6862f195 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala @@ -44,7 +44,7 @@ private[streaming] class BlockGenerator( listener: BlockGeneratorListener, receiverId: Int, conf: SparkConf - ) extends Logging { + ) extends RateLimiter(conf) with Logging { private case class Block(id: StreamBlockId, buffer: ArrayBuffer[Any]) @@ -81,6 +81,7 @@ private[streaming] class BlockGenerator( * will be periodically pushed into BlockManager. */ def += (data: Any): Unit = synchronized { + waitToPush() currentBuffer += data } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/RateLimiter.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/RateLimiter.scala new file mode 100644 index 0000000000000..e4f6ba626ebbf --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/RateLimiter.scala @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.receiver + +import org.apache.spark.{Logging, SparkConf} +import java.util.concurrent.TimeUnit._ + +/** Provides waitToPush() method to limit the rate at which receivers consume data. + * + * waitToPush method will block the thread if too many messages have been pushed too quickly, + * and only return when a new message has been pushed. It assumes that only one message is + * pushed at a time. + * + * The spark configuration spark.streaming.receiver.maxRate gives the maximum number of messages + * per second that each receiver will accept. + * + * @param conf spark configuration + */ +private[receiver] abstract class RateLimiter(conf: SparkConf) extends Logging { + + private var lastSyncTime = System.nanoTime + private var messagesWrittenSinceSync = 0L + private val desiredRate = conf.getInt("spark.streaming.receiver.maxRate", 0) + private val SYNC_INTERVAL = NANOSECONDS.convert(10, SECONDS) + + def waitToPush() { + if( desiredRate <= 0 ) { + return + } + val now = System.nanoTime + val elapsedNanosecs = math.max(now - lastSyncTime, 1) + val rate = messagesWrittenSinceSync.toDouble * 1000000000 / elapsedNanosecs + if (rate < desiredRate) { + // It's okay to write; just update some variables and return + messagesWrittenSinceSync += 1 + if (now > lastSyncTime + SYNC_INTERVAL) { + // Sync interval has passed; let's resync + lastSyncTime = now + messagesWrittenSinceSync = 1 + } + } else { + // Calculate how much time we should sleep to bring ourselves to the desired rate. + val targetTimeInMillis = messagesWrittenSinceSync * 1000 / desiredRate + val elapsedTimeInMillis = elapsedNanosecs / 1000000 + val sleepTimeInMillis = targetTimeInMillis - elapsedTimeInMillis + if (sleepTimeInMillis > 0) { + logTrace("Natural rate is " + rate + " per second but desired rate is " + + desiredRate + ", sleeping for " + sleepTimeInMillis + " ms to compensate.") + Thread.sleep(sleepTimeInMillis) + } + waitToPush() + } + } +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala index 09be3a50d2dfa..1f0244c251eba 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala @@ -138,7 +138,7 @@ private[streaming] abstract class ReceiverSupervisor( onReceiverStop(message, error) } catch { case t: Throwable => - stop("Error stopping receiver " + streamId, Some(t)) + logError("Error stopping receiver " + streamId + t.getStackTraceString) } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala index ce8316bb14891..d934b9cbfc3e8 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala @@ -110,8 +110,7 @@ private[streaming] class ReceiverSupervisorImpl( ) { val blockId = optionalBlockId.getOrElse(nextBlockId) val time = System.currentTimeMillis - blockManager.put(blockId, arrayBuffer.asInstanceOf[ArrayBuffer[Any]], - storageLevel, tellMaster = true) + blockManager.putArray(blockId, arrayBuffer.toArray[Any], storageLevel, tellMaster = true) logDebug("Pushed block " + blockId + " in " + (System.currentTimeMillis - time) + " ms") reportPushedBlock(blockId, arrayBuffer.size, optionalMetadata) } @@ -124,7 +123,7 @@ private[streaming] class ReceiverSupervisorImpl( ) { val blockId = optionalBlockId.getOrElse(nextBlockId) val time = System.currentTimeMillis - blockManager.put(blockId, iterator, storageLevel, tellMaster = true) + blockManager.putIterator(blockId, iterator, storageLevel, tellMaster = true) logDebug("Pushed block " + blockId + " in " + (System.currentTimeMillis - time) + " ms") reportPushedBlock(blockId, -1, optionalMetadata) } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala index cc4a65011dd72..952a74fd5f6de 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala @@ -383,7 +383,10 @@ class TestActor(port: Int) extends Actor with ActorHelper { def bytesToString(byteString: ByteString) = byteString.utf8String - override def preStart = IOManager(context.system).connect(new InetSocketAddress(port)) + override def preStart(): Unit = { + @deprecated("suppress compile time deprecation warning", "1.0.0") + val unit = IOManager(context.system).connect(new InetSocketAddress(port)) + } def receive = { case IO.Read(socket, bytes) => diff --git a/streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala index d9ac3c91f6e36..f4e11f975de94 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala @@ -145,6 +145,44 @@ class NetworkReceiverSuite extends FunSuite with Timeouts { assert(recordedData.toSet === generatedData.toSet) } + test("block generator throttling") { + val blockGeneratorListener = new FakeBlockGeneratorListener + val blockInterval = 50 + val maxRate = 200 + val conf = new SparkConf().set("spark.streaming.blockInterval", blockInterval.toString). + set("spark.streaming.receiver.maxRate", maxRate.toString) + val blockGenerator = new BlockGenerator(blockGeneratorListener, 1, conf) + val expectedBlocks = 20 + val waitTime = expectedBlocks * blockInterval + val expectedMessages = maxRate * waitTime / 1000 + val expectedMessagesPerBlock = maxRate * blockInterval / 1000 + val generatedData = new ArrayBuffer[Int] + + // Generate blocks + val startTime = System.currentTimeMillis() + blockGenerator.start() + var count = 0 + while(System.currentTimeMillis - startTime < waitTime) { + blockGenerator += count + generatedData += count + count += 1 + Thread.sleep(1) + } + blockGenerator.stop() + + val recordedData = blockGeneratorListener.arrayBuffers + assert(blockGeneratorListener.arrayBuffers.size > 0) + assert(recordedData.flatten.toSet === generatedData.toSet) + // recordedData size should be close to the expected rate + assert(recordedData.flatten.size >= expectedMessages * 0.9 && + recordedData.flatten.size <= expectedMessages * 1.1 ) + // the first and last block may be incomplete, so we slice them out + recordedData.slice(1, recordedData.size - 1).foreach { block => + assert(block.size >= expectedMessagesPerBlock * 0.8 && + block.size <= expectedMessagesPerBlock * 1.2 ) + } + } + /** * An implementation of NetworkReceiver that is used for testing a receiver's life cycle. */ diff --git a/tools/pom.xml b/tools/pom.xml index 79cd8551d0722..97abb6b2b63e0 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -26,6 +26,9 @@ org.apache.spark spark-tools_2.10 + + tools + jar Spark Project Tools http://spark.apache.org/ diff --git a/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala b/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala index 03a73f92b275e..bcf6d43ab34eb 100644 --- a/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala +++ b/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala @@ -68,12 +68,11 @@ object GenerateMIMAIgnore { for (className <- classes) { try { val classSymbol = mirror.classSymbol(Class.forName(className, false, classLoader)) - val moduleSymbol = mirror.staticModule(className) // TODO: see if it is necessary. + val moduleSymbol = mirror.staticModule(className) val directlyPrivateSpark = isPackagePrivate(classSymbol) || isPackagePrivateModule(moduleSymbol) - val developerApi = isDeveloperApi(classSymbol) - val experimental = isExperimental(classSymbol) - + val developerApi = isDeveloperApi(classSymbol) || isDeveloperApi(moduleSymbol) + val experimental = isExperimental(classSymbol) || isExperimental(moduleSymbol) /* Inner classes defined within a private[spark] class or object are effectively invisible, so we account for them as package private. */ lazy val indirectlyPrivateSpark = { @@ -87,10 +86,9 @@ object GenerateMIMAIgnore { } if (directlyPrivateSpark || indirectlyPrivateSpark || developerApi || experimental) { ignoredClasses += className - } else { - // check if this class has package-private/annotated members. - ignoredMembers ++= getAnnotatedOrPackagePrivateMembers(classSymbol) } + // check if this class has package-private/annotated members. + ignoredMembers ++= getAnnotatedOrPackagePrivateMembers(classSymbol) } catch { case _: Throwable => println("Error instrumenting class:" + className) @@ -99,9 +97,27 @@ object GenerateMIMAIgnore { (ignoredClasses.flatMap(c => Seq(c, c.replace("$", "#"))).toSet, ignoredMembers.toSet) } + /** Scala reflection does not let us see inner function even if they are upgraded + * to public for some reason. So had to resort to java reflection to get all inner + * functions with $$ in there name. + */ + def getInnerFunctions(classSymbol: unv.ClassSymbol): Seq[String] = { + try { + Class.forName(classSymbol.fullName, false, classLoader).getMethods.map(_.getName) + .filter(_.contains("$$")).map(classSymbol.fullName + "." + _) + } catch { + case t: Throwable => + println("[WARN] Unable to detect inner functions for class:" + classSymbol.fullName) + Seq.empty[String] + } + } + private def getAnnotatedOrPackagePrivateMembers(classSymbol: unv.ClassSymbol) = { - classSymbol.typeSignature.members - .filter(x => isPackagePrivate(x) || isDeveloperApi(x) || isExperimental(x)).map(_.fullName) + classSymbol.typeSignature.members.filterNot(x => + x.fullName.startsWith("java") || x.fullName.startsWith("scala") + ).filter(x => + isPackagePrivate(x) || isDeveloperApi(x) || isExperimental(x) + ).map(_.fullName) ++ getInnerFunctions(classSymbol) } def main(args: Array[String]) { diff --git a/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala b/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala index 8e8c35615a711..17bf7c2541d13 100644 --- a/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala +++ b/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala @@ -23,6 +23,7 @@ import java.util.concurrent.atomic.AtomicLong import org.apache.spark.SparkContext import org.apache.spark.serializer.KryoSerializer import org.apache.spark.util.Utils +import org.apache.spark.executor.ShuffleWriteMetrics /** * Internal utility for micro-benchmarking shuffle write performance. @@ -56,15 +57,14 @@ object StoragePerfTester { def writeOutputBytes(mapId: Int, total: AtomicLong) = { val shuffle = blockManager.shuffleBlockManager.forMapTask(1, mapId, numOutputSplits, - new KryoSerializer(sc.conf)) + new KryoSerializer(sc.conf), new ShuffleWriteMetrics()) val writers = shuffle.writers for (i <- 1 to recordsPerMap) { writers(i % numOutputSplits).write(writeData) } - writers.map {w => - w.commit() + writers.map { w => + w.commitAndClose() total.addAndGet(w.fileSegment().length) - w.close() } shuffle.releaseWriters(true) diff --git a/tox.ini b/tox.ini index 44766e529bf7f..a1fefdd0e176f 100644 --- a/tox.ini +++ b/tox.ini @@ -15,3 +15,4 @@ [pep8] max-line-length=100 +exclude=cloudpickle.py diff --git a/yarn/alpha/pom.xml b/yarn/alpha/pom.xml index b8a631dd0bb3b..51744ece0412d 100644 --- a/yarn/alpha/pom.xml +++ b/yarn/alpha/pom.xml @@ -23,6 +23,9 @@ 1.1.0-SNAPSHOT ../pom.xml + + yarn-alpha + org.apache.spark spark-yarn-alpha_2.10 diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 438737f7a6b60..62b5c3bc5f0f3 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -60,6 +60,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, private var yarnAllocator: YarnAllocationHandler = _ private var isFinished: Boolean = false private var uiAddress: String = _ + private var uiHistoryAddress: String = _ private val maxAppAttempts: Int = conf.getInt(YarnConfiguration.RM_AM_MAX_RETRIES, YarnConfiguration.DEFAULT_RM_AM_MAX_RETRIES) private var isLastAMRetry: Boolean = true @@ -184,6 +185,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, private def startUserClass(): Thread = { logInfo("Starting the user JAR in a separate Thread") + System.setProperty("spark.executor.instances", args.numExecutors.toString) val mainMethod = Class.forName( args.userClass, false /* initialize */ , @@ -236,6 +238,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, if (null != sparkContext) { uiAddress = sparkContext.ui.appUIHostPort + uiHistoryAddress = YarnSparkHadoopUtil.getUIHistoryAddress(sparkContext, sparkConf) this.yarnAllocator = YarnAllocationHandler.newAllocator( yarnConf, resourceManager, @@ -254,10 +257,6 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, sparkContext.getConf) } } - } finally { - // in case of exceptions, etc - ensure that count is atleast ALLOCATOR_LOOP_WAIT_COUNT : - // so that the loop (in ApplicationMaster.sparkContextInitialized) breaks - ApplicationMaster.incrementAllocatorLoop(ApplicationMaster.ALLOCATOR_LOOP_WAIT_COUNT) } } @@ -276,13 +275,8 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, } yarnAllocator.allocateContainers( math.max(args.numExecutors - yarnAllocator.getNumExecutorsRunning, 0)) - ApplicationMaster.incrementAllocatorLoop(1) Thread.sleep(ApplicationMaster.ALLOCATE_HEARTBEAT_INTERVAL) } - } finally { - // In case of exceptions, etc - ensure that count is at least ALLOCATOR_LOOP_WAIT_COUNT, - // so that the loop in ApplicationMaster#sparkContextInitialized() breaks. - ApplicationMaster.incrementAllocatorLoop(ApplicationMaster.ALLOCATOR_LOOP_WAIT_COUNT) } logInfo("All executors have launched.") @@ -368,7 +362,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, finishReq.setAppAttemptId(appAttemptId) finishReq.setFinishApplicationStatus(status) finishReq.setDiagnostics(diagnostics) - finishReq.setTrackingUrl(sparkConf.get("spark.yarn.historyServer.address", "")) + finishReq.setTrackingUrl(uiHistoryAddress) resourceManager.finishApplicationMaster(finishReq) } } @@ -410,24 +404,10 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, } object ApplicationMaster extends Logging { - // Number of times to wait for the allocator loop to complete. - // Each loop iteration waits for 100ms, so maximum of 3 seconds. - // This is to ensure that we have reasonable number of containers before we start // TODO: Currently, task to container is computed once (TaskSetManager) - which need not be // optimal as more containers are available. Might need to handle this better. - private val ALLOCATOR_LOOP_WAIT_COUNT = 30 private val ALLOCATE_HEARTBEAT_INTERVAL = 100 - def incrementAllocatorLoop(by: Int) { - val count = yarnAllocatorLoop.getAndAdd(by) - if (count >= ALLOCATOR_LOOP_WAIT_COUNT) { - yarnAllocatorLoop.synchronized { - // to wake threads off wait ... - yarnAllocatorLoop.notifyAll() - } - } - } - private val applicationMasters = new CopyOnWriteArrayList[ApplicationMaster]() def register(master: ApplicationMaster) { @@ -436,7 +416,6 @@ object ApplicationMaster extends Logging { val sparkContextRef: AtomicReference[SparkContext] = new AtomicReference[SparkContext](null /* initialValue */) - val yarnAllocatorLoop: AtomicInteger = new AtomicInteger(0) def sparkContextInitialized(sc: SparkContext): Boolean = { var modified = false @@ -471,21 +450,6 @@ object ApplicationMaster extends Logging { modified } - - /** - * Returns when we've either - * 1) received all the requested executors, - * 2) waited ALLOCATOR_LOOP_WAIT_COUNT * ALLOCATE_HEARTBEAT_INTERVAL ms, - * 3) hit an error that causes us to terminate trying to get containers. - */ - def waitForInitialAllocations() { - yarnAllocatorLoop.synchronized { - while (yarnAllocatorLoop.get() <= ALLOCATOR_LOOP_WAIT_COUNT) { - yarnAllocatorLoop.wait(1000L) - } - } - } - def main(argStrings: Array[String]) { SignalLogger.register(log) val args = new ApplicationMasterArguments(argStrings) diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala index bfdb6232f5113..184e2ad6c82cd 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala @@ -28,10 +28,10 @@ import org.apache.hadoop.yarn.ipc.YarnRPC import org.apache.hadoop.yarn.util.{ConverterUtils, Records} import akka.actor._ import akka.remote._ -import akka.actor.Terminated import org.apache.spark.{Logging, SecurityManager, SparkConf} import org.apache.spark.util.{Utils, AkkaUtils} import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend +import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.AddWebUIFilter import org.apache.spark.scheduler.SplitInfo import org.apache.spark.deploy.SparkHadoopUtil @@ -56,10 +56,17 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp private val yarnConf: YarnConfiguration = new YarnConfiguration(conf) private var yarnAllocator: YarnAllocationHandler = _ - private var driverClosed:Boolean = false + + private var driverClosed: Boolean = false + private var isFinished: Boolean = false + private var registered: Boolean = false + + // Default to numExecutors * 2, with minimum of 3 + private val maxNumExecutorFailures = sparkConf.getInt("spark.yarn.max.executor.failures", + sparkConf.getInt("spark.yarn.max.worker.failures", math.max(args.numExecutors * 2, 3))) val securityManager = new SecurityManager(sparkConf) - val actorSystem : ActorSystem = AkkaUtils.createActorSystem("sparkYarnAM", Utils.localHostName, 0, + val actorSystem: ActorSystem = AkkaUtils.createActorSystem("sparkYarnAM", Utils.localHostName, 0, conf = sparkConf, securityManager = securityManager)._1 var actor: ActorRef = _ @@ -81,6 +88,9 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp case x: DisassociatedEvent => logInfo(s"Driver terminated or disconnected! Shutting down. $x") driverClosed = true + case x: AddWebUIFilter => + logInfo(s"Add WebUI Filter. $x") + driver ! x } } @@ -93,25 +103,28 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp appAttemptId = getApplicationAttemptId() resourceManager = registerWithResourceManager() - val appMasterResponse: RegisterApplicationMasterResponse = registerApplicationMaster() - - // Compute number of threads for akka - val minimumMemory = appMasterResponse.getMinimumResourceCapability().getMemory() - - if (minimumMemory > 0) { - val mem = args.executorMemory + sparkConf.getInt("spark.yarn.executor.memoryOverhead", - YarnAllocationHandler.MEMORY_OVERHEAD) - val numCore = (mem / minimumMemory) + (if (0 != (mem % minimumMemory)) 1 else 0) - - if (numCore > 0) { - // do not override - hits https://issues.apache.org/jira/browse/HADOOP-8406 - // TODO: Uncomment when hadoop is on a version which has this fixed. - // args.workerCores = numCore + synchronized { + if (!isFinished) { + val appMasterResponse: RegisterApplicationMasterResponse = registerApplicationMaster() + // Compute number of threads for akka + val minimumMemory = appMasterResponse.getMinimumResourceCapability().getMemory() + + if (minimumMemory > 0) { + val mem = args.executorMemory + sparkConf.getInt("spark.yarn.executor.memoryOverhead", + YarnAllocationHandler.MEMORY_OVERHEAD) + val numCore = (mem / minimumMemory) + (if (0 != (mem % minimumMemory)) 1 else 0) + + if (numCore > 0) { + // do not override - hits https://issues.apache.org/jira/browse/HADOOP-8406 + // TODO: Uncomment when hadoop is on a version which has this fixed. + // args.workerCores = numCore + } + } + registered = true } } - waitForSparkMaster() - + addAmIpFilter() // Allocate all containers allocateExecutors() @@ -171,7 +184,8 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp } private def registerApplicationMaster(): RegisterApplicationMasterResponse = { - logInfo("Registering the ApplicationMaster") + val appUIAddress = sparkConf.get("spark.driver.appUIAddress", "") + logInfo(s"Registering the ApplicationMaster with appUIAddress: $appUIAddress") val appMasterRequest = Records.newRecord(classOf[RegisterApplicationMasterRequest]) .asInstanceOf[RegisterApplicationMasterRequest] appMasterRequest.setApplicationAttemptId(appAttemptId) @@ -180,10 +194,21 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp appMasterRequest.setHost(Utils.localHostName()) appMasterRequest.setRpcPort(0) // What do we provide here ? Might make sense to expose something sensible later ? - appMasterRequest.setTrackingUrl("") + appMasterRequest.setTrackingUrl(appUIAddress) resourceManager.registerApplicationMaster(appMasterRequest) } + // add the yarn amIpFilter that Yarn requires for properly securing the UI + private def addAmIpFilter() { + val proxy = YarnConfiguration.getProxyHostAndPort(conf) + val parts = proxy.split(":") + val proxyBase = System.getenv(ApplicationConstants.APPLICATION_WEB_PROXY_BASE_ENV) + val uriBase = "http://" + proxy + proxyBase + val amFilter = "PROXY_HOST=" + parts(0) + "," + "PROXY_URI_BASE=" + uriBase + val amFilterName = "org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter" + actor ! AddWebUIFilter(amFilterName, amFilter, proxyBase) + } + private def waitForSparkMaster() { logInfo("Waiting for spark driver to be reachable.") var driverUp = false @@ -227,11 +252,17 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp while ((yarnAllocator.getNumExecutorsRunning < args.numExecutors) && (!driverClosed)) { yarnAllocator.allocateContainers( math.max(args.numExecutors - yarnAllocator.getNumExecutorsRunning, 0)) + checkNumExecutorsFailed() Thread.sleep(100) } logInfo("All executors have launched.") - + } + private def checkNumExecutorsFailed() { + if (yarnAllocator.getNumExecutorsFailed >= maxNumExecutorFailures) { + finishApplicationMaster(FinalApplicationStatus.FAILED, + "max number of executor failures reached") + } } // TODO: We might want to extend this to allocate more containers in case they die ! @@ -241,6 +272,7 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp val t = new Thread { override def run() { while (!driverClosed) { + checkNumExecutorsFailed() val missingExecutorCount = args.numExecutors - yarnAllocator.getNumExecutorsRunning if (missingExecutorCount > 0) { logInfo("Allocating " + missingExecutorCount + @@ -266,15 +298,23 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp yarnAllocator.allocateContainers(0) } - def finishApplicationMaster(status: FinalApplicationStatus) { - - logInfo("finish ApplicationMaster with " + status) - val finishReq = Records.newRecord(classOf[FinishApplicationMasterRequest]) - .asInstanceOf[FinishApplicationMasterRequest] - finishReq.setAppAttemptId(appAttemptId) - finishReq.setFinishApplicationStatus(status) - finishReq.setTrackingUrl(sparkConf.get("spark.yarn.historyServer.address", "")) - resourceManager.finishApplicationMaster(finishReq) + def finishApplicationMaster(status: FinalApplicationStatus, appMessage: String = "") { + synchronized { + if (isFinished) { + return + } + logInfo("Unregistering ApplicationMaster with " + status) + if (registered) { + val finishReq = Records.newRecord(classOf[FinishApplicationMasterRequest]) + .asInstanceOf[FinishApplicationMasterRequest] + finishReq.setAppAttemptId(appAttemptId) + finishReq.setFinishApplicationStatus(status) + finishReq.setTrackingUrl(sparkConf.get("spark.yarn.historyServer.address", "")) + finishReq.setDiagnostics(appMessage) + resourceManager.finishApplicationMaster(finishReq) + } + isFinished = true + } } } diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala index 25cc9016b10a6..4c383ab574abe 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala @@ -26,7 +26,7 @@ class ApplicationMasterArguments(val args: Array[String]) { var userArgs: Seq[String] = Seq[String]() var executorMemory = 1024 var executorCores = 1 - var numExecutors = 2 + var numExecutors = ApplicationMasterArguments.DEFAULT_NUMBER_EXECUTORS parseArgs(args.toList) @@ -93,3 +93,7 @@ class ApplicationMasterArguments(val args: Array[String]) { System.exit(exitCode) } } + +object ApplicationMasterArguments { + val DEFAULT_NUMBER_EXECUTORS = 2 +} diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index 556f49342977a..1da0a1b675554 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -29,7 +29,7 @@ import org.apache.hadoop.fs._ import org.apache.hadoop.fs.permission.FsPermission import org.apache.hadoop.mapred.Master import org.apache.hadoop.mapreduce.MRJobConfig -import org.apache.hadoop.security.UserGroupInformation +import org.apache.hadoop.security.{Credentials, UserGroupInformation} import org.apache.hadoop.util.StringUtils import org.apache.hadoop.yarn.api._ import org.apache.hadoop.yarn.api.ApplicationConstants.Environment @@ -37,7 +37,7 @@ import org.apache.hadoop.yarn.api.protocolrecords._ import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.util.Records -import org.apache.spark.{SparkException, Logging, SparkConf, SparkContext} +import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext, SparkException} /** * The entry point (starting in Client#main() and Client#run()) for launching Spark on YARN. The @@ -109,7 +109,7 @@ trait ClientBase extends Logging { if (amMem > maxMem) { val errorMessage = "Required AM memory (%d) is above the max threshold (%d) of this cluster." - .format(args.amMemory, maxMem) + .format(amMem, maxMem) logError(errorMessage) throw new IllegalArgumentException(errorMessage) } @@ -191,23 +191,11 @@ trait ClientBase extends Logging { // Upload Spark and the application JAR to the remote file system if necessary. Add them as // local resources to the application master. val fs = FileSystem.get(conf) - - val delegTokenRenewer = Master.getMasterPrincipal(conf) - if (UserGroupInformation.isSecurityEnabled()) { - if (delegTokenRenewer == null || delegTokenRenewer.length() == 0) { - val errorMessage = "Can't get Master Kerberos principal for use as renewer" - logError(errorMessage) - throw new SparkException(errorMessage) - } - } val dst = new Path(fs.getHomeDirectory(), appStagingDir) - val replication = sparkConf.getInt("spark.yarn.submit.file.replication", 3).toShort - - if (UserGroupInformation.isSecurityEnabled()) { - val dstFs = dst.getFileSystem(conf) - dstFs.addDelegationTokens(delegTokenRenewer, credentials) - } + val nns = ClientBase.getNameNodesToAccess(sparkConf) + dst + ClientBase.obtainTokensForNamenodes(nns, conf, credentials) + val replication = sparkConf.getInt("spark.yarn.submit.file.replication", 3).toShort val localResources = HashMap[String, LocalResource]() FileSystem.mkdirs(fs, dst, new FsPermission(STAGING_DIR_PERMISSION)) @@ -232,7 +220,8 @@ trait ClientBase extends Logging { if (!ClientBase.LOCAL_SCHEME.equals(localURI.getScheme())) { val setPermissions = if (destName.equals(ClientBase.APP_JAR)) true else false val destPath = copyRemoteFile(dst, qualifyForLocal(localURI), replication, setPermissions) - distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, + val destFs = FileSystem.get(destPath.toUri(), conf) + distCacheMgr.addResource(destFs, conf, destPath, localResources, LocalResourceType.FILE, destName, statCache) } else if (confKey != null) { sparkConf.set(confKey, localPath) @@ -270,6 +259,14 @@ trait ClientBase extends Logging { localResources } + /** Get all application master environment variables set on this SparkConf */ + def getAppMasterEnv: Seq[(String, String)] = { + val prefix = "spark.yarn.appMasterEnv." + sparkConf.getAll.filter{case (k, v) => k.startsWith(prefix)} + .map{case (k, v) => (k.substring(prefix.length), v)} + } + + def setupLaunchEnv( localResources: HashMap[String, LocalResource], stagingDir: String): HashMap[String, String] = { @@ -287,6 +284,11 @@ trait ClientBase extends Logging { distCacheMgr.setDistFilesEnv(env) distCacheMgr.setDistArchivesEnv(env) + getAppMasterEnv.foreach { case (key, value) => + YarnSparkHadoopUtil.addToEnvironment(env, key, value, File.pathSeparator) + } + + // Keep this for backwards compatibility but users should move to the config sys.env.get("SPARK_YARN_USER_ENV").foreach { userEnvs => // Allow users to specify some environment variables. YarnSparkHadoopUtil.setEnvFromInputString(env, userEnvs, File.pathSeparator) @@ -416,6 +418,13 @@ trait ClientBase extends Logging { amContainer.setCommands(printableCommands) setupSecurityToken(amContainer) + + // send the acl settings into YARN to control who has access via YARN interfaces + val securityManager = new SecurityManager(sparkConf) + val acls = Map[ApplicationAccessType, String] ( + ApplicationAccessType.VIEW_APP -> securityManager.getViewAcls, + ApplicationAccessType.MODIFY_APP -> securityManager.getModifyAcls) + amContainer.setApplicationACLs(acls) amContainer } } @@ -613,4 +622,40 @@ object ClientBase extends Logging { YarnSparkHadoopUtil.addToEnvironment(env, Environment.CLASSPATH.name, path, File.pathSeparator) + /** + * Get the list of namenodes the user may access. + */ + private[yarn] def getNameNodesToAccess(sparkConf: SparkConf): Set[Path] = { + sparkConf.get("spark.yarn.access.namenodes", "").split(",").map(_.trim()).filter(!_.isEmpty) + .map(new Path(_)).toSet + } + + private[yarn] def getTokenRenewer(conf: Configuration): String = { + val delegTokenRenewer = Master.getMasterPrincipal(conf) + logDebug("delegation token renewer is: " + delegTokenRenewer) + if (delegTokenRenewer == null || delegTokenRenewer.length() == 0) { + val errorMessage = "Can't get Master Kerberos principal for use as renewer" + logError(errorMessage) + throw new SparkException(errorMessage) + } + delegTokenRenewer + } + + /** + * Obtains tokens for the namenodes passed in and adds them to the credentials. + */ + private[yarn] def obtainTokensForNamenodes(paths: Set[Path], conf: Configuration, + creds: Credentials) { + if (UserGroupInformation.isSecurityEnabled()) { + val delegTokenRenewer = getTokenRenewer(conf) + + paths.foreach { + dst => + val dstFs = dst.getFileSystem(conf) + logDebug("getting token for namenode: " + dst) + dstFs.addDelegationTokens(delegTokenRenewer, creds) + } + } + } + } diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala index 4ba7133a959ed..71a9e42846b2b 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala @@ -171,7 +171,11 @@ trait ExecutorRunnableUtil extends Logging { val extraCp = sparkConf.getOption("spark.executor.extraClassPath") ClientBase.populateClasspath(null, yarnConf, sparkConf, env, extraCp) - // Allow users to specify some environment variables + sparkConf.getExecutorEnv.foreach { case (key, value) => + YarnSparkHadoopUtil.addToEnvironment(env, key, value, File.pathSeparator) + } + + // Keep this for backwards compatibility but users should move to the config YarnSparkHadoopUtil.setEnvFromInputString(env, System.getenv("SPARK_YARN_USER_ENV"), File.pathSeparator) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index 718cb19f57261..e98308cdbd74e 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -30,6 +30,9 @@ import org.apache.hadoop.util.StringInterner import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.api.ApplicationConstants import org.apache.hadoop.conf.Configuration + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.deploy.history.HistoryServer import org.apache.spark.deploy.SparkHadoopUtil /** @@ -132,4 +135,17 @@ object YarnSparkHadoopUtil { } } + def getUIHistoryAddress(sc: SparkContext, conf: SparkConf) : String = { + val eventLogDir = sc.eventLogger match { + case Some(logger) => logger.getApplicationLogDir() + case None => "" + } + val historyServerAddress = conf.get("spark.yarn.historyServer.address", "") + if (historyServerAddress != "" && eventLogDir != "") { + historyServerAddress + HistoryServer.UI_PATH_PREFIX + s"/$eventLogDir" + } else { + "" + } + } + } diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala index 6b91e6b9eb899..3474112ded5d7 100644 --- a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala +++ b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala @@ -37,12 +37,4 @@ private[spark] class YarnClientClusterScheduler(sc: SparkContext, conf: Configur val retval = YarnAllocationHandler.lookupRack(conf, host) if (retval != null) Some(retval) else None } - - override def postStartHook() { - - // The yarn application is running, but the executor might not yet ready - // Wait for a few seconds for the slaves to bootstrap and register with master - best case attempt - Thread.sleep(2000L) - logInfo("YarnClientClusterScheduler.postStartHook done") - } } diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala index fd2694fe7278d..833e249f9f612 100644 --- a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -19,7 +19,7 @@ package org.apache.spark.scheduler.cluster import org.apache.hadoop.yarn.api.records.{ApplicationId, YarnApplicationState} import org.apache.spark.{SparkException, Logging, SparkContext} -import org.apache.spark.deploy.yarn.{Client, ClientArguments, ExecutorLauncher} +import org.apache.spark.deploy.yarn.{Client, ClientArguments, ExecutorLauncher, YarnSparkHadoopUtil} import org.apache.spark.scheduler.TaskSchedulerImpl import scala.collection.mutable.ArrayBuffer @@ -30,8 +30,15 @@ private[spark] class YarnClientSchedulerBackend( extends CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem) with Logging { + if (conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) { + minRegisteredRatio = 0.8 + } + var client: Client = null var appId: ApplicationId = null + var checkerThread: Thread = null + var stopping: Boolean = false + var totalExpectedExecutors = 0 private[spark] def addArg(optionName: String, envVar: String, sysProp: String, arrayBuf: ArrayBuffer[String]) { @@ -48,6 +55,8 @@ private[spark] class YarnClientSchedulerBackend( val driverHost = conf.get("spark.driver.host") val driverPort = conf.get("spark.driver.port") val hostport = driverHost + ":" + driverPort + conf.set("spark.driver.appUIAddress", sc.ui.appUIHostPort) + conf.set("spark.driver.appUIHistoryAddress", YarnSparkHadoopUtil.getUIHistoryAddress(sc, conf)) val argsArrayBuf = new ArrayBuffer[String]() argsArrayBuf += ( @@ -75,9 +84,11 @@ private[spark] class YarnClientSchedulerBackend( logDebug("ClientArguments called with: " + argsArrayBuf) val args = new ClientArguments(argsArrayBuf.toArray, conf) + totalExpectedExecutors = args.numExecutors client = new Client(args, conf) appId = client.runApp() waitForApp() + checkerThread = yarnApplicationStateCheckerThread() } def waitForApp() { @@ -108,10 +119,38 @@ private[spark] class YarnClientSchedulerBackend( } } + private def yarnApplicationStateCheckerThread(): Thread = { + val t = new Thread { + override def run() { + while (!stopping) { + val report = client.getApplicationReport(appId) + val state = report.getYarnApplicationState() + if (state == YarnApplicationState.FINISHED || state == YarnApplicationState.KILLED + || state == YarnApplicationState.FAILED) { + logError(s"Yarn application already ended: $state") + sc.stop() + stopping = true + } + Thread.sleep(1000L) + } + checkerThread = null + Thread.currentThread().interrupt() + } + } + t.setName("Yarn Application State Checker") + t.setDaemon(true) + t.start() + t + } + override def stop() { + stopping = true super.stop() client.stop logInfo("Stopped") } + override def sufficientResourcesRegistered(): Boolean = { + totalRegisteredExecutors.get() >= totalExpectedExecutors * minRegisteredRatio + } } diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala index 39cdd2e8a522b..9aeca4a637d38 100644 --- a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala +++ b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala @@ -47,12 +47,8 @@ private[spark] class YarnClusterScheduler(sc: SparkContext, conf: Configuration) } override def postStartHook() { - val sparkContextInitialized = ApplicationMaster.sparkContextInitialized(sc) - if (sparkContextInitialized){ - ApplicationMaster.waitForInitialAllocations() - // Wait for a few seconds for the slaves to bootstrap and register with master - best case attempt - Thread.sleep(3000L) - } + ApplicationMaster.sparkContextInitialized(sc) + super.postStartHook() logInfo("YarnClusterScheduler.postStartHook done") } } diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala new file mode 100644 index 0000000000000..55665220a6f96 --- /dev/null +++ b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.scheduler.cluster + +import org.apache.spark.SparkContext +import org.apache.spark.deploy.yarn.ApplicationMasterArguments +import org.apache.spark.scheduler.TaskSchedulerImpl +import org.apache.spark.util.IntParam + +private[spark] class YarnClusterSchedulerBackend( + scheduler: TaskSchedulerImpl, + sc: SparkContext) + extends CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem) { + + var totalExpectedExecutors = 0 + + if (conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) { + minRegisteredRatio = 0.8 + } + + override def start() { + super.start() + totalExpectedExecutors = ApplicationMasterArguments.DEFAULT_NUMBER_EXECUTORS + if (System.getenv("SPARK_EXECUTOR_INSTANCES") != null) { + totalExpectedExecutors = IntParam.unapply(System.getenv("SPARK_EXECUTOR_INSTANCES")) + .getOrElse(totalExpectedExecutors) + } + // System property can override environment variable. + totalExpectedExecutors = sc.getConf.getInt("spark.executor.instances", totalExpectedExecutors) + } + + override def sufficientResourcesRegistered(): Boolean = { + totalRegisteredExecutors.get() >= totalExpectedExecutors * minRegisteredRatio + } +} diff --git a/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala b/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala index 686714dc36488..68cc2890f3a22 100644 --- a/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala +++ b/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala @@ -31,6 +31,8 @@ import org.apache.hadoop.yarn.api.records.ContainerLaunchContext import org.apache.hadoop.yarn.conf.YarnConfiguration import org.mockito.Matchers._ import org.mockito.Mockito._ + + import org.scalatest.FunSuite import org.scalatest.Matchers @@ -38,7 +40,7 @@ import scala.collection.JavaConversions._ import scala.collection.mutable.{ HashMap => MutableHashMap } import scala.util.Try -import org.apache.spark.SparkConf +import org.apache.spark.{SparkException, SparkConf} import org.apache.spark.util.Utils class ClientBaseSuite extends FunSuite with Matchers { @@ -138,6 +140,57 @@ class ClientBaseSuite extends FunSuite with Matchers { } } + test("check access nns empty") { + val sparkConf = new SparkConf() + sparkConf.set("spark.yarn.access.namenodes", "") + val nns = ClientBase.getNameNodesToAccess(sparkConf) + nns should be(Set()) + } + + test("check access nns unset") { + val sparkConf = new SparkConf() + val nns = ClientBase.getNameNodesToAccess(sparkConf) + nns should be(Set()) + } + + test("check access nns") { + val sparkConf = new SparkConf() + sparkConf.set("spark.yarn.access.namenodes", "hdfs://nn1:8032") + val nns = ClientBase.getNameNodesToAccess(sparkConf) + nns should be(Set(new Path("hdfs://nn1:8032"))) + } + + test("check access nns space") { + val sparkConf = new SparkConf() + sparkConf.set("spark.yarn.access.namenodes", "hdfs://nn1:8032, ") + val nns = ClientBase.getNameNodesToAccess(sparkConf) + nns should be(Set(new Path("hdfs://nn1:8032"))) + } + + test("check access two nns") { + val sparkConf = new SparkConf() + sparkConf.set("spark.yarn.access.namenodes", "hdfs://nn1:8032,hdfs://nn2:8032") + val nns = ClientBase.getNameNodesToAccess(sparkConf) + nns should be(Set(new Path("hdfs://nn1:8032"), new Path("hdfs://nn2:8032"))) + } + + test("check token renewer") { + val hadoopConf = new Configuration() + hadoopConf.set("yarn.resourcemanager.address", "myrm:8033") + hadoopConf.set("yarn.resourcemanager.principal", "yarn/myrm:8032@SPARKTEST.COM") + val renewer = ClientBase.getTokenRenewer(hadoopConf) + renewer should be ("yarn/myrm:8032@SPARKTEST.COM") + } + + test("check token renewer default") { + val hadoopConf = new Configuration() + val caught = + intercept[SparkException] { + ClientBase.getTokenRenewer(hadoopConf) + } + assert(caught.getMessage === "Can't get Master Kerberos principal for use as renewer") + } + object Fixtures { val knownDefYarnAppCP: Seq[String] = diff --git a/yarn/pom.xml b/yarn/pom.xml index ef7066ef1fdfc..3faaf053634d6 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -28,6 +28,9 @@ yarn-parent_2.10 pom Spark Project YARN Parent POM + + yarn + diff --git a/yarn/stable/pom.xml b/yarn/stable/pom.xml index 0931beb505508..b6c8456d06684 100644 --- a/yarn/stable/pom.xml +++ b/yarn/stable/pom.xml @@ -23,6 +23,9 @@ 1.1.0-SNAPSHOT ../pom.xml + + yarn-stable + org.apache.spark spark-yarn_2.10 diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index ee1e9c9c23d22..035356d390c80 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -59,6 +59,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, private var yarnAllocator: YarnAllocationHandler = _ private var isFinished: Boolean = false private var uiAddress: String = _ + private var uiHistoryAddress: String = _ private val maxAppAttempts: Int = conf.getInt( YarnConfiguration.RM_AM_MAX_ATTEMPTS, YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS) private var isLastAMRetry: Boolean = true @@ -164,6 +165,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, private def startUserClass(): Thread = { logInfo("Starting the user JAR in a separate Thread") + System.setProperty("spark.executor.instances", args.numExecutors.toString) val mainMethod = Class.forName( args.userClass, false, @@ -215,6 +217,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, if (sparkContext != null) { uiAddress = sparkContext.ui.appUIHostPort + uiHistoryAddress = YarnSparkHadoopUtil.getUIHistoryAddress(sparkContext, sparkConf) this.yarnAllocator = YarnAllocationHandler.newAllocator( yarnConf, amClient, @@ -233,10 +236,6 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, sparkContext.getConf) } } - } finally { - // In case of exceptions, etc - ensure that the loop in - // ApplicationMaster#sparkContextInitialized() breaks. - ApplicationMaster.doneWithInitialAllocations() } } @@ -253,16 +252,9 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, checkNumExecutorsFailed() allocateMissingExecutor() yarnAllocator.allocateResources() - if (iters == ApplicationMaster.ALLOCATOR_LOOP_WAIT_COUNT) { - ApplicationMaster.doneWithInitialAllocations() - } Thread.sleep(ApplicationMaster.ALLOCATE_HEARTBEAT_INTERVAL) iters += 1 } - } finally { - // In case of exceptions, etc - ensure that the loop in - // ApplicationMaster#sparkContextInitialized() breaks. - ApplicationMaster.doneWithInitialAllocations() } logInfo("All executors have launched.") } @@ -322,8 +314,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, logInfo("Unregistering ApplicationMaster with " + status) if (registered) { - val trackingUrl = sparkConf.get("spark.yarn.historyServer.address", "") - amClient.unregisterApplicationMaster(status, diagnostics, trackingUrl) + amClient.unregisterApplicationMaster(status, diagnostics, uiHistoryAddress) } } } @@ -364,12 +355,8 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, } object ApplicationMaster extends Logging { - // Number of times to wait for the allocator loop to complete. - // Each loop iteration waits for 100ms, so maximum of 3 seconds. - // This is to ensure that we have reasonable number of containers before we start // TODO: Currently, task to container is computed once (TaskSetManager) - which need not be // optimal as more containers are available. Might need to handle this better. - private val ALLOCATOR_LOOP_WAIT_COUNT = 30 private val ALLOCATE_HEARTBEAT_INTERVAL = 100 private val applicationMasters = new CopyOnWriteArrayList[ApplicationMaster]() @@ -377,20 +364,6 @@ object ApplicationMaster extends Logging { val sparkContextRef: AtomicReference[SparkContext] = new AtomicReference[SparkContext](null) - // Variable used to notify the YarnClusterScheduler that it should stop waiting - // for the initial set of executors to be started and get on with its business. - val doneWithInitialAllocationsMonitor = new Object() - - @volatile var isDoneWithInitialAllocations = false - - def doneWithInitialAllocations() { - isDoneWithInitialAllocations = true - doneWithInitialAllocationsMonitor.synchronized { - // to wake threads off wait ... - doneWithInitialAllocationsMonitor.notifyAll() - } - } - def register(master: ApplicationMaster) { applicationMasters.add(master) } @@ -433,20 +406,6 @@ object ApplicationMaster extends Logging { modified } - /** - * Returns when we've either - * 1) received all the requested executors, - * 2) waited ALLOCATOR_LOOP_WAIT_COUNT * ALLOCATE_HEARTBEAT_INTERVAL ms, - * 3) hit an error that causes us to terminate trying to get containers. - */ - def waitForInitialAllocations() { - doneWithInitialAllocationsMonitor.synchronized { - while (!isDoneWithInitialAllocations) { - doneWithInitialAllocationsMonitor.wait(1000L) - } - } - } - def getApplicationAttemptId(): ApplicationAttemptId = { val containerIdString = System.getenv(ApplicationConstants.Environment.CONTAINER_ID.name()) val containerId = ConverterUtils.toContainerId(containerIdString) diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala index f71ad036ce0f2..fc7b8320d734d 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala @@ -19,22 +19,21 @@ package org.apache.spark.deploy.yarn import java.net.Socket import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.net.NetUtils -import org.apache.hadoop.yarn.api._ +import org.apache.hadoop.yarn.api.ApplicationConstants import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.api.protocolrecords._ import org.apache.hadoop.yarn.conf.YarnConfiguration -import org.apache.hadoop.yarn.util.{ConverterUtils, Records} import akka.actor._ import akka.remote._ -import akka.actor.Terminated import org.apache.spark.{Logging, SecurityManager, SparkConf} import org.apache.spark.util.{Utils, AkkaUtils} import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend +import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.AddWebUIFilter import org.apache.spark.scheduler.SplitInfo import org.apache.hadoop.yarn.client.api.AMRMClient import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.hadoop.yarn.webapp.util.WebAppUtils /** * An application master that allocates executors on behalf of a driver that is running outside @@ -55,10 +54,16 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp private val yarnConf: YarnConfiguration = new YarnConfiguration(conf) private var yarnAllocator: YarnAllocationHandler = _ - private var driverClosed:Boolean = false + private var driverClosed: Boolean = false + private var isFinished: Boolean = false + private var registered: Boolean = false private var amClient: AMRMClient[ContainerRequest] = _ + // Default to numExecutors * 2, with minimum of 3 + private val maxNumExecutorFailures = sparkConf.getInt("spark.yarn.max.executor.failures", + sparkConf.getInt("spark.yarn.max.worker.failures", math.max(args.numExecutors * 2, 3))) + val securityManager = new SecurityManager(sparkConf) val actorSystem: ActorSystem = AkkaUtils.createActorSystem("sparkYarnAM", Utils.localHostName, 0, conf = sparkConf, securityManager = securityManager)._1 @@ -82,6 +87,9 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp case x: DisassociatedEvent => logInfo(s"Driver terminated or disconnected! Shutting down. $x") driverClosed = true + case x: AddWebUIFilter => + logInfo(s"Add WebUI Filter. $x") + driver ! x } } @@ -96,9 +104,15 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp amClient.start() appAttemptId = ApplicationMaster.getApplicationAttemptId() - registerApplicationMaster() + synchronized { + if (!isFinished) { + registerApplicationMaster() + registered = true + } + } waitForSparkMaster() + addAmIpFilter() // Allocate all containers allocateExecutors() @@ -142,9 +156,20 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp } private def registerApplicationMaster(): RegisterApplicationMasterResponse = { - logInfo("Registering the ApplicationMaster") - // TODO: Find out client's Spark UI address and fill in here? - amClient.registerApplicationMaster(Utils.localHostName(), 0, "") + val appUIAddress = sparkConf.get("spark.driver.appUIAddress", "") + logInfo(s"Registering the ApplicationMaster with appUIAddress: $appUIAddress") + amClient.registerApplicationMaster(Utils.localHostName(), 0, appUIAddress) + } + + // add the yarn amIpFilter that Yarn requires for properly securing the UI + private def addAmIpFilter() { + val proxy = WebAppUtils.getProxyHostAndPort(conf) + val parts = proxy.split(":") + val proxyBase = System.getenv(ApplicationConstants.APPLICATION_WEB_PROXY_BASE_ENV) + val uriBase = "http://" + proxy + proxyBase + val amFilter = "PROXY_HOST=" + parts(0) + "," + "PROXY_URI_BASE=" + uriBase + val amFilterName = "org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter" + actor ! AddWebUIFilter(amFilterName, amFilter, proxyBase) } private def waitForSparkMaster() { @@ -193,6 +218,7 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp yarnAllocator.addResourceRequests(args.numExecutors) yarnAllocator.allocateResources() while ((yarnAllocator.getNumExecutorsRunning < args.numExecutors) && (!driverClosed)) { + checkNumExecutorsFailed() allocateMissingExecutor() yarnAllocator.allocateResources() Thread.sleep(100) @@ -211,12 +237,20 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp } } + private def checkNumExecutorsFailed() { + if (yarnAllocator.getNumExecutorsFailed >= maxNumExecutorFailures) { + finishApplicationMaster(FinalApplicationStatus.FAILED, + "max number of executor failures reached") + } + } + private def launchReporterThread(_sleepTime: Long): Thread = { val sleepTime = if (_sleepTime <= 0) 0 else _sleepTime val t = new Thread { override def run() { while (!driverClosed) { + checkNumExecutorsFailed() allocateMissingExecutor() logDebug("Sending progress") yarnAllocator.allocateResources() @@ -231,10 +265,18 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp t } - def finishApplicationMaster(status: FinalApplicationStatus) { - logInfo("Unregistering ApplicationMaster with " + status) - val trackingUrl = sparkConf.get("spark.yarn.historyServer.address", "") - amClient.unregisterApplicationMaster(status, "" /* appMessage */ , trackingUrl) + def finishApplicationMaster(status: FinalApplicationStatus, appMessage: String = "") { + synchronized { + if (isFinished) { + return + } + logInfo("Unregistering ApplicationMaster with " + status) + if (registered) { + val trackingUrl = sparkConf.get("spark.yarn.historyServer.address", "") + amClient.unregisterApplicationMaster(status, appMessage, trackingUrl) + } + isFinished = true + } } }