diff --git a/README.md b/README.md index 8906e4c1416b1..0a683a460ffac 100644 --- a/README.md +++ b/README.md @@ -69,7 +69,7 @@ Many of the example programs print usage help if no params are given. Testing first requires [building Spark](#building-spark). Once Spark is built, tests can be run using: - ./sbt/sbt test + ./dev/run-tests ## A Note About Hadoop Versions @@ -118,7 +118,10 @@ 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 using the JDBC server. +See sql-programming-guide.md for more information about using the JDBC server and CLI. +You can use those features by setting `-Phive` when building Spark as follows. + + $ sbt/sbt -Phive assembly ## Configuration @@ -136,3 +139,5 @@ submitting any copyrighted material via pull request, email, or other means you agree to license the material under the project's open source license and warrant that you have the legal authority to do so. +Please see [Contributing to Spark wiki page](https://cwiki.apache.org/SPARK/Contributing+to+Spark) +for more information. diff --git a/bin/load-spark-env.sh b/bin/load-spark-env.sh index d425f9feaac54..493d3785a081b 100644 --- a/bin/load-spark-env.sh +++ b/bin/load-spark-env.sh @@ -27,12 +27,12 @@ if [ -z "$SPARK_ENV_LOADED" ]; then # Returns the parent of the directory this script lives in. parent_dir="$(cd `dirname $0`/..; pwd)" - use_conf_dir=${SPARK_CONF_DIR:-"$parent_dir/conf"} + user_conf_dir=${SPARK_CONF_DIR:-"$parent_dir/conf"} - if [ -f "${use_conf_dir}/spark-env.sh" ]; then + if [ -f "${user_conf_dir}/spark-env.sh" ]; then # Promote all variable declarations to environment (exported) variables set -a - . "${use_conf_dir}/spark-env.sh" + . "${user_conf_dir}/spark-env.sh" set +a fi fi diff --git a/bin/spark-class b/bin/spark-class index 22acf92288b3b..c6543545a5e64 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -105,9 +105,14 @@ else exit 1 fi fi +JAVA_VERSION=$($RUNNER -version 2>&1 | sed 's/java version "\(.*\)\.\(.*\)\..*"/\1\2/; 1q') # Set JAVA_OPTS to be able to load native libraries and to set heap size -JAVA_OPTS="-XX:MaxPermSize=128m $OUR_JAVA_OPTS" +if [ "$JAVA_VERSION" -ge 18 ]; then + JAVA_OPTS="$OUR_JAVA_OPTS" +else + JAVA_OPTS="-XX:MaxPermSize=128m $OUR_JAVA_OPTS" +fi JAVA_OPTS="$JAVA_OPTS -Xms$OUR_JAVA_MEM -Xmx$OUR_JAVA_MEM" # Load extra JAVA_OPTS from conf/java-opts, if it exists diff --git a/bin/spark-class2.cmd b/bin/spark-class2.cmd index e420eb409e529..e2c5f9c385189 100755 --- a/bin/spark-class2.cmd +++ b/bin/spark-class2.cmd @@ -77,7 +77,13 @@ rem All drivers use SPARK_JAVA_OPTS + SPARK_DRIVER_MEMORY. The repl also uses SP ) rem Set JAVA_OPTS to be able to load native libraries and to set heap size -set JAVA_OPTS=-XX:MaxPermSize=128m %OUR_JAVA_OPTS% -Djava.library.path=%SPARK_LIBRARY_PATH% -Xms%OUR_JAVA_MEM% -Xmx%OUR_JAVA_MEM% +for /f "tokens=3" %%i in ('java -version 2^>^&1 ^| find "version"') do set jversion=%%i +for /f "tokens=1 delims=_" %%i in ("%jversion:~1,-1%") do set jversion=%%i +if "%jversion%" geq "1.8.0" ( + set JAVA_OPTS=%OUR_JAVA_OPTS% -Djava.library.path=%SPARK_LIBRARY_PATH% -Xms%OUR_JAVA_MEM% -Xmx%OUR_JAVA_MEM% +) else ( + set JAVA_OPTS=-XX:MaxPermSize=128m %OUR_JAVA_OPTS% -Djava.library.path=%SPARK_LIBRARY_PATH% -Xms%OUR_JAVA_MEM% -Xmx%OUR_JAVA_MEM% +) rem Attention: when changing the way the JAVA_OPTS are assembled, the change must be reflected in CommandUtils.scala! rem Test whether the user has built Spark diff --git a/bin/spark-shell b/bin/spark-shell index 8b7ccd7439551..0ab4e14f5b744 100755 --- a/bin/spark-shell +++ b/bin/spark-shell @@ -22,7 +22,7 @@ cygwin=false case "`uname`" in - CYGWIN*) cygwin=true;; + CYGWIN*) cygwin=true;; esac # Enter posix mode for bash @@ -32,9 +32,9 @@ set -o posix 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 + echo "Usage: ./bin/spark-shell [options]" + $FWDIR/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 + exit 0 } if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then @@ -46,20 +46,20 @@ 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/). - # If you're using the Mintty terminal emulator in Cygwin, may need to set the - # "Backspace sends ^H" setting in "Keys" section of the Mintty options - # (see https://github.com/sbt/sbt/issues/562). - stty -icanon min 1 -echo > /dev/null 2>&1 - export SPARK_SUBMIT_OPTS="$SPARK_SUBMIT_OPTS -Djline.terminal=unix" - $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 --class org.apache.spark.repl.Main "${SUBMISSION_OPTS[@]}" spark-shell "${APPLICATION_OPTS[@]}" - fi + if $cygwin; then + # Workaround for issue involving JLine and Cygwin + # (see http://sourceforge.net/p/jline/bugs/40/). + # If you're using the Mintty terminal emulator in Cygwin, may need to set the + # "Backspace sends ^H" setting in "Keys" section of the Mintty options + # (see https://github.com/sbt/sbt/issues/562). + stty -icanon min 1 -echo > /dev/null 2>&1 + export SPARK_SUBMIT_OPTS="$SPARK_SUBMIT_OPTS -Djline.terminal=unix" + $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 --class org.apache.spark.repl.Main "${SUBMISSION_OPTS[@]}" spark-shell "${APPLICATION_OPTS[@]}" + fi } # Copy restore-TTY-on-exit functions from Scala script so spark-shell exits properly even in diff --git a/bin/spark-sql b/bin/spark-sql index 564f1f419060f..2a3cb31f58e8d 100755 --- a/bin/spark-sql +++ b/bin/spark-sql @@ -24,6 +24,7 @@ set -o posix CLASS="org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver" +CLASS_NOT_FOUND_EXIT_STATUS=1 # Figure out where Spark is installed FWDIR="$(cd `dirname $0`/..; pwd)" @@ -43,52 +44,22 @@ function usage { $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 +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 - ;; +source $FWDIR/bin/utils.sh +SUBMIT_USAGE_FUNCTION=usage +gatherSparkSubmitOpts "$@" - -e) - ensure_arg_number $# 2 - CLI_ARGS+=("$1"); shift - CLI_ARGS+=("$1"); shift - ;; +"$FWDIR"/bin/spark-submit --class $CLASS "${SUBMISSION_OPTS[@]}" spark-internal "${APPLICATION_OPTS[@]}" +exit_status=$? - -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 +if [[ exit_status -eq CLASS_NOT_FOUND_EXIT_STATUS ]]; then + echo + echo "Failed to load Spark SQL CLI main class $CLASS." + echo "You need to build Spark with -Phive." +fi -exec "$FWDIR"/bin/spark-submit --class $CLASS "${SUBMISSION_ARGS[@]}" spark-internal "${CLI_ARGS[@]}" +exit $exit_status diff --git a/core/src/main/resources/org/apache/spark/ui/static/sorttable.js b/core/src/main/resources/org/apache/spark/ui/static/sorttable.js index 7abb9011ccf36..dbacbf19beee5 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/sorttable.js +++ b/core/src/main/resources/org/apache/spark/ui/static/sorttable.js @@ -81,15 +81,15 @@ sorttable = { if (!headrow[i].className.match(/\bsorttable_nosort\b/)) { // skip this col mtch = headrow[i].className.match(/\bsorttable_([a-z0-9]+)\b/); if (mtch) { override = mtch[1]; } - if (mtch && typeof sorttable["sort_"+override] == 'function') { - headrow[i].sorttable_sortfunction = sorttable["sort_"+override]; - } else { - headrow[i].sorttable_sortfunction = sorttable.guessType(table,i); - } - // make it clickable to sort - headrow[i].sorttable_columnindex = i; - headrow[i].sorttable_tbody = table.tBodies[0]; - dean_addEvent(headrow[i],"click", function(e) { + if (mtch && typeof sorttable["sort_"+override] == 'function') { + headrow[i].sorttable_sortfunction = sorttable["sort_"+override]; + } else { + headrow[i].sorttable_sortfunction = sorttable.guessType(table,i); + } + // make it clickable to sort + headrow[i].sorttable_columnindex = i; + headrow[i].sorttable_tbody = table.tBodies[0]; + dean_addEvent(headrow[i],"click", function(e) { if (this.className.search(/\bsorttable_sorted\b/) != -1) { // if we're already sorted by this column, just @@ -109,7 +109,7 @@ sorttable = { // re-reverse the table, which is quicker sorttable.reverse(this.sorttable_tbody); this.className = this.className.replace('sorttable_sorted_reverse', - 'sorttable_sorted'); + 'sorttable_sorted'); this.removeChild(document.getElementById('sorttable_sortrevind')); sortfwdind = document.createElement('span'); sortfwdind.id = "sorttable_sortfwdind"; @@ -117,7 +117,7 @@ sorttable = { this.appendChild(sortfwdind); return; } - + // remove sorttable_sorted classes theadrow = this.parentNode; forEach(theadrow.childNodes, function(cell) { @@ -130,36 +130,36 @@ sorttable = { if (sortfwdind) { sortfwdind.parentNode.removeChild(sortfwdind); } sortrevind = document.getElementById('sorttable_sortrevind'); if (sortrevind) { sortrevind.parentNode.removeChild(sortrevind); } - + this.className += ' sorttable_sorted'; sortfwdind = document.createElement('span'); sortfwdind.id = "sorttable_sortfwdind"; sortfwdind.innerHTML = stIsIE ? ' 6' : ' ▾'; this.appendChild(sortfwdind); - // build an array to sort. This is a Schwartzian transform thing, - // i.e., we "decorate" each row with the actual sort key, - // sort based on the sort keys, and then put the rows back in order - // which is a lot faster because you only do getInnerText once per row - row_array = []; - col = this.sorttable_columnindex; - rows = this.sorttable_tbody.rows; - for (var j=0; j 0 ) { - var q = list[i]; list[i] = list[i+1]; list[i+1] = q; - swap = true; - } - } // for - t--; + swap = false; + for(var i = b; i < t; ++i) { + if ( comp_func(list[i], list[i+1]) > 0 ) { + var q = list[i]; list[i] = list[i+1]; list[i+1] = q; + swap = true; + } + } // for + t--; - if (!swap) break; - - for(var i = t; i > b; --i) { - if ( comp_func(list[i], list[i-1]) < 0 ) { - var q = list[i]; list[i] = list[i-1]; list[i-1] = q; - swap = true; - } - } // for - b++; + if (!swap) break; + for(var i = t; i > b; --i) { + if ( comp_func(list[i], list[i-1]) < 0 ) { + var q = list[i]; list[i] = list[i-1]; list[i-1] = q; + swap = true; + } + } // for + b++; } // while(swap) } } @@ -358,11 +357,11 @@ if (document.addEventListener) { /* for Safari */ if (/WebKit/i.test(navigator.userAgent)) { // sniff - var _timer = setInterval(function() { - if (/loaded|complete/.test(document.readyState)) { - sorttable.init(); // call the onload handler - } - }, 10); + var _timer = setInterval(function() { + if (/loaded|complete/.test(document.readyState)) { + sorttable.init(); // call the onload handler + } + }, 10); } /* for other browsers */ @@ -374,66 +373,66 @@ window.onload = sorttable.init; // http://dean.edwards.name/weblog/2005/10/add-event/ function dean_addEvent(element, type, handler) { - if (element.addEventListener) { - element.addEventListener(type, handler, false); - } else { - // assign each event handler a unique ID - if (!handler.$$guid) handler.$$guid = dean_addEvent.guid++; - // create a hash table of event types for the element - if (!element.events) element.events = {}; - // create a hash table of event handlers for each element/event pair - var handlers = element.events[type]; - if (!handlers) { - handlers = element.events[type] = {}; - // store the existing event handler (if there is one) - if (element["on" + type]) { - handlers[0] = element["on" + type]; - } - } - // store the event handler in the hash table - handlers[handler.$$guid] = handler; - // assign a global event handler to do all the work - element["on" + type] = handleEvent; - } + if (element.addEventListener) { + element.addEventListener(type, handler, false); + } else { + // assign each event handler a unique ID + if (!handler.$$guid) handler.$$guid = dean_addEvent.guid++; + // create a hash table of event types for the element + if (!element.events) element.events = {}; + // create a hash table of event handlers for each element/event pair + var handlers = element.events[type]; + if (!handlers) { + handlers = element.events[type] = {}; + // store the existing event handler (if there is one) + if (element["on" + type]) { + handlers[0] = element["on" + type]; + } + } + // store the event handler in the hash table + handlers[handler.$$guid] = handler; + // assign a global event handler to do all the work + element["on" + type] = handleEvent; + } }; // a counter used to create unique IDs dean_addEvent.guid = 1; function removeEvent(element, type, handler) { - if (element.removeEventListener) { - element.removeEventListener(type, handler, false); - } else { - // delete the event handler from the hash table - if (element.events && element.events[type]) { - delete element.events[type][handler.$$guid]; - } - } + if (element.removeEventListener) { + element.removeEventListener(type, handler, false); + } else { + // delete the event handler from the hash table + if (element.events && element.events[type]) { + delete element.events[type][handler.$$guid]; + } + } }; function handleEvent(event) { - var returnValue = true; - // grab the event object (IE uses a global event object) - event = event || fixEvent(((this.ownerDocument || this.document || this).parentWindow || window).event); - // get a reference to the hash table of event handlers - var handlers = this.events[event.type]; - // execute each event handler - for (var i in handlers) { - this.$$handleEvent = handlers[i]; - if (this.$$handleEvent(event) === false) { - returnValue = false; - } - } - return returnValue; + var returnValue = true; + // grab the event object (IE uses a global event object) + event = event || fixEvent(((this.ownerDocument || this.document || this).parentWindow || window).event); + // get a reference to the hash table of event handlers + var handlers = this.events[event.type]; + // execute each event handler + for (var i in handlers) { + this.$$handleEvent = handlers[i]; + if (this.$$handleEvent(event) === false) { + returnValue = false; + } + } + return returnValue; }; function fixEvent(event) { - // add W3C standard event methods - event.preventDefault = fixEvent.preventDefault; - event.stopPropagation = fixEvent.stopPropagation; - return event; + // add W3C standard event methods + event.preventDefault = fixEvent.preventDefault; + event.stopPropagation = fixEvent.stopPropagation; + return event; }; fixEvent.preventDefault = function() { - this.returnValue = false; + this.returnValue = false; }; fixEvent.stopPropagation = function() { this.cancelBubble = true; @@ -441,55 +440,55 @@ fixEvent.stopPropagation = function() { // Dean's forEach: http://dean.edwards.name/base/forEach.js /* - forEach, version 1.0 - Copyright 2006, Dean Edwards - License: http://www.opensource.org/licenses/mit-license.php +forEach, version 1.0 +Copyright 2006, Dean Edwards +License: http://www.opensource.org/licenses/mit-license.php */ // array-like enumeration if (!Array.forEach) { // mozilla already supports this - Array.forEach = function(array, block, context) { - for (var i = 0; i < array.length; i++) { - block.call(context, array[i], i, array); - } - }; + Array.forEach = function(array, block, context) { + for (var i = 0; i < array.length; i++) { + block.call(context, array[i], i, array); + } + }; } // generic enumeration Function.prototype.forEach = function(object, block, context) { - for (var key in object) { - if (typeof this.prototype[key] == "undefined") { - block.call(context, object[key], key, object); - } - } + for (var key in object) { + if (typeof this.prototype[key] == "undefined") { + block.call(context, object[key], key, object); + } + } }; // character enumeration String.forEach = function(string, block, context) { - Array.forEach(string.split(""), function(chr, index) { - block.call(context, chr, index, string); - }); + Array.forEach(string.split(""), function(chr, index) { + block.call(context, chr, index, string); + }); }; // globally resolve forEach enumeration var forEach = function(object, block, context) { - if (object) { - var resolve = Object; // default - if (object instanceof Function) { - // functions have a "length" property - resolve = Function; - } else if (object.forEach instanceof Function) { - // the object implements a custom forEach method so use that - object.forEach(block, context); - return; - } else if (typeof object == "string") { - // the object is a string - resolve = String; - } else if (typeof object.length == "number") { - // the object is array-like - resolve = Array; - } - resolve.forEach(object, block, context); - } + if (object) { + var resolve = Object; // default + if (object instanceof Function) { + // functions have a "length" property + resolve = Function; + } else if (object.forEach instanceof Function) { + // the object implements a custom forEach method so use that + object.forEach(block, context); + return; + } else if (typeof object == "string") { + // the object is a string + resolve = String; + } else if (typeof object.length == "number") { + // the object is array-like + resolve = Array; + } + resolve.forEach(object, block, context); + } }; diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index fc36e37c53f5e..72716567ca99b 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -111,6 +111,9 @@ object SparkEnv extends Logging { private val env = new ThreadLocal[SparkEnv] @volatile private var lastSetSparkEnv : SparkEnv = _ + private[spark] val driverActorSystemName = "sparkDriver" + private[spark] val executorActorSystemName = "sparkExecutor" + def set(e: SparkEnv) { lastSetSparkEnv = e env.set(e) @@ -146,9 +149,9 @@ object SparkEnv extends Logging { } val securityManager = new SecurityManager(conf) - - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, port, conf = conf, - securityManager = securityManager) + val actorSystemName = if (isDriver) driverActorSystemName else executorActorSystemName + val (actorSystem, boundPort) = AkkaUtils.createActorSystem( + actorSystemName, hostname, port, conf, securityManager) // Figure out which port Akka actually bound to in case the original port is 0 or occupied. // This is so that we tell the executors the correct port to connect to. 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 747023812f754..ae8010300a500 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 @@ -749,6 +749,23 @@ private[spark] object PythonRDD extends Logging { } } } + + /** + * Convert an RDD of serialized Python objects to RDD of objects, that is usable by PySpark. + */ + def pythonToJava(pyRDD: JavaRDD[Array[Byte]], batched: Boolean): JavaRDD[Any] = { + pyRDD.rdd.mapPartitions { iter => + val unpickle = new Unpickler + iter.flatMap { row => + val obj = unpickle.loads(row) + if (batched) { + obj.asInstanceOf[JArrayList[_]] + } else { + Seq(obj) + } + } + }.toJavaRDD() + } } private 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 f8cdbc3c392b5..550ee72538900 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -54,6 +54,8 @@ object SparkSubmit { private val SPARK_SHELL = "spark-shell" private val PYSPARK_SHELL = "pyspark-shell" + private val CLASS_NOT_FOUND_EXIT_STATUS = 1 + // Exposed for testing private[spark] var exitFn: () => Unit = () => System.exit(-1) private[spark] var printStream: PrintStream = System.err @@ -311,8 +313,18 @@ object SparkSubmit { System.setProperty(key, value) } - val mainClass = Class.forName(childMainClass, true, loader) + var mainClass: Class[_] = null + + try { + mainClass = Class.forName(childMainClass, true, loader) + } catch { + case e: ClassNotFoundException => + e.printStackTrace(printStream) + System.exit(CLASS_NOT_FOUND_EXIT_STATUS) + } + val mainMethod = mainClass.getMethod("main", new Array[String](0).getClass) + try { mainMethod.invoke(null, childArgs.toArray) } catch { 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 687e492a0d6fc..12e98fd40d6c9 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 @@ -64,8 +64,6 @@ object CommandUtils extends Logging { Seq() } - val permGenOpt = Seq("-XX:MaxPermSize=128m") - // Figure out our classpath with the external compute-classpath script val ext = if (System.getProperty("os.name").startsWith("Windows")) ".cmd" else ".sh" val classPath = Utils.executeAndGetOutput( @@ -73,6 +71,8 @@ object CommandUtils extends Logging { extraEnvironment = command.environment) val userClassPath = command.classPathEntries ++ Seq(classPath) + val javaVersion = System.getProperty("java.version") + val permGenOpt = if (!javaVersion.startsWith("1.8")) Some("-XX:MaxPermSize=128m") else None Seq("-cp", userClassPath.filterNot(_.isEmpty).mkString(File.pathSeparator)) ++ permGenOpt ++ libraryOpts ++ workerLocalOpts ++ command.javaOpts ++ memoryOpts } 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 e5e1e72cd912b..578d806263006 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala @@ -851,8 +851,8 @@ private[spark] class ConnectionManager( messageStatuses.synchronized { messageStatuses.remove(message.id).foreach ( s => { promise.failure( - new IOException(s"sendMessageReliably failed because ack " + - "was not received within ${ackTimeout} sec")) + new IOException("sendMessageReliably failed because ack " + + s"was not received within $ackTimeout sec")) }) } } diff --git a/core/src/main/scala/org/apache/spark/partial/SumEvaluator.scala b/core/src/main/scala/org/apache/spark/partial/SumEvaluator.scala index a74f80094434c..d5336284571d2 100644 --- a/core/src/main/scala/org/apache/spark/partial/SumEvaluator.scala +++ b/core/src/main/scala/org/apache/spark/partial/SumEvaluator.scala @@ -22,7 +22,7 @@ import cern.jet.stat.Probability import org.apache.spark.util.StatCounter /** - * An ApproximateEvaluator for sums. It estimates the mean and the cont and multiplies them + * An ApproximateEvaluator for sums. It estimates the mean and the count and multiplies them * together, then uses the formula for the variance of two independent random variables to get * a variance for the result and compute a confidence interval. */ diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala index d99c76117c168..4f7133c4bc17c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala @@ -20,7 +20,7 @@ package org.apache.spark.scheduler.cluster import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{Path, FileSystem} -import org.apache.spark.{Logging, SparkContext} +import org.apache.spark.{Logging, SparkContext, SparkEnv} import org.apache.spark.scheduler.TaskSchedulerImpl private[spark] class SimrSchedulerBackend( @@ -38,8 +38,10 @@ private[spark] class SimrSchedulerBackend( override def start() { super.start() - val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( - sc.conf.get("spark.driver.host"), sc.conf.get("spark.driver.port"), + val driverUrl = "akka.tcp://%s@%s:%s/user/%s".format( + SparkEnv.driverActorSystemName, + sc.conf.get("spark.driver.host"), + sc.conf.get("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) val conf = new Configuration() 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 589dba2e40d20..32138e5246700 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, SparkConf, SparkContext} +import org.apache.spark.{Logging, SparkConf, SparkContext, SparkEnv} import org.apache.spark.deploy.{ApplicationDescription, Command} import org.apache.spark.deploy.client.{AppClient, AppClientListener} import org.apache.spark.scheduler.{ExecutorExited, ExecutorLossReason, SlaveLost, TaskSchedulerImpl} @@ -42,8 +42,10 @@ private[spark] class SparkDeploySchedulerBackend( super.start() // The endpoint for executors to talk to us - val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( - conf.get("spark.driver.host"), conf.get("spark.driver.port"), + val driverUrl = "akka.tcp://%s@%s:%s/user/%s".format( + SparkEnv.driverActorSystemName, + conf.get("spark.driver.host"), + conf.get("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) val args = Seq(driverUrl, "{{EXECUTOR_ID}}", "{{HOSTNAME}}", "{{CORES}}", "{{WORKER_URL}}") val extraJavaOpts = sc.conf.getOption("spark.executor.extraJavaOptions") diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index 9f45400bcf852..f0172504c55aa 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -28,7 +28,7 @@ import org.apache.mesos.{Scheduler => MScheduler} import org.apache.mesos._ import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, TaskState => MesosTaskState, _} -import org.apache.spark.{Logging, SparkContext, SparkException} +import org.apache.spark.{Logging, SparkContext, SparkEnv, SparkException} import org.apache.spark.scheduler.TaskSchedulerImpl import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend @@ -130,7 +130,8 @@ private[spark] class CoarseMesosSchedulerBackend( } val command = CommandInfo.newBuilder() .setEnvironment(environment) - val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( + val driverUrl = "akka.tcp://%s@%s:%s/user/%s".format( + SparkEnv.driverActorSystemName, conf.get("spark.driver.host"), conf.get("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) 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 index 22f656fa371ea..b8c9ad46ab035 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala @@ -17,12 +17,11 @@ package org.apache.spark.shuffle.sort -import java.io.{BufferedOutputStream, File, FileOutputStream, DataOutputStream} +import java.io.File 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 @@ -37,10 +36,6 @@ private[spark] class SortShuffleWriter[K, V, C]( 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 diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index d6afb73b74242..e2d32c859bbda 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -27,7 +27,7 @@ import akka.pattern.ask import com.typesafe.config.ConfigFactory import org.apache.log4j.{Level, Logger} -import org.apache.spark.{SparkException, Logging, SecurityManager, SparkConf} +import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkEnv, SparkException} /** * Various utility classes for working with Akka. @@ -192,10 +192,11 @@ private[spark] object AkkaUtils extends Logging { } def makeDriverRef(name: String, conf: SparkConf, actorSystem: ActorSystem): ActorRef = { + val driverActorSystemName = SparkEnv.driverActorSystemName 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 url = s"akka.tcp://$driverActorSystemName@$driverHost:$driverPort/user/$name" val timeout = AkkaUtils.lookupTimeout(conf) logInfo(s"Connecting to $name: $url") Await.result(actorSystem.actorSelection(url).resolveOne(timeout), timeout) diff --git a/docs/building-with-maven.md b/docs/building-with-maven.md index a7d7bd3ccb1f2..bce7412c7d4c9 100644 --- a/docs/building-with-maven.md +++ b/docs/building-with-maven.md @@ -96,10 +96,13 @@ 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 the JDBC server. +# Building With Hive and JDBC Support +To enable Hive integration for Spark SQL along with its JDBC server and CLI, +add the `-Phive` profile to your existing build options. +{% highlight bash %} +# Apache Hadoop 2.4.X with Hive support +mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=2.4.0 -Phive -DskipTests clean package +{% endhighlight %} # Spark Tests in Maven diff --git a/docs/mllib-collaborative-filtering.md b/docs/mllib-collaborative-filtering.md index ab10b2f01f87b..d5c539db791be 100644 --- a/docs/mllib-collaborative-filtering.md +++ b/docs/mllib-collaborative-filtering.md @@ -43,6 +43,17 @@ level of confidence in observed user preferences, rather than explicit ratings g model then tries to find latent factors that can be used to predict the expected preference of a user for an item. +### Scaling of the regularization parameter + +Since v1.1, we scale the regularization parameter `lambda` in solving each least squares problem by +the number of ratings the user generated in updating user factors, +or the number of ratings the product received in updating product factors. +This approach is named "ALS-WR" and discussed in the paper +"[Large-Scale Parallel Collaborative Filtering for the Netflix Prize](http://dx.doi.org/10.1007/978-3-540-68880-8_32)". +It makes `lambda` less dependent on the scale of the dataset. +So we can apply the best parameter learned from a sampled subset to the full dataset +and expect similar performance. + ## Examples
diff --git a/docs/mllib-decision-tree.md b/docs/mllib-decision-tree.md index c01a92a9a1b26..1166d9cd150c4 100644 --- a/docs/mllib-decision-tree.md +++ b/docs/mllib-decision-tree.md @@ -7,20 +7,26 @@ displayTitle: MLlib - Decision Tree * Table of contents {:toc} -Decision trees and their ensembles are popular methods for the machine learning tasks of +[Decision trees](http://en.wikipedia.org/wiki/Decision_tree_learning) +and their ensembles are popular methods for the machine learning tasks of classification and regression. Decision trees are widely used since they are easy to interpret, -handle categorical variables, extend to the multiclass classification setting, do not require +handle categorical features, extend to the multiclass classification setting, do not require feature scaling and are able to capture nonlinearities and feature interactions. Tree ensemble -algorithms such as decision forest and boosting are among the top performers for classification and +algorithms such as random forests and boosting are among the top performers for classification and regression tasks. +MLlib supports decision trees for binary and multiclass classification and for regression, +using both continuous and categorical features. The implementation partitions data by rows, +allowing distributed training with millions of instances. + ## Basic algorithm The decision tree is a greedy algorithm that performs a recursive binary partitioning of the feature -space by choosing a single element from the *best split set* where each element of the set maximizes -the information gain at a tree node. In other words, the split chosen at each tree node is chosen -from the set `$\underset{s}{\operatorname{argmax}} IG(D,s)$` where `$IG(D,s)$` is the information -gain when a split `$s$` is applied to a dataset `$D$`. +space. The tree predicts the same label for each bottommost (leaf) partition. +Each partition is chosen greedily by selecting the *best split* from a set of possible splits, +in order to maximize the information gain at a tree node. In other words, the split chosen at each +tree node is chosen from the set `$\underset{s}{\operatorname{argmax}} IG(D,s)$` where `$IG(D,s)$` +is the information gain when a split `$s$` is applied to a dataset `$D$`. ### Node impurity and information gain @@ -52,9 +58,10 @@ impurity measure for regression (variance). -The *information gain* is the difference in the parent node impurity and the weighted sum of the two -child node impurities. Assuming that a split $s$ partitions the dataset `$D$` of size `$N$` into two -datasets `$D_{left}$` and `$D_{right}$` of sizes `$N_{left}$` and `$N_{right}$`, respectively: +The *information gain* is the difference between the parent node impurity and the weighted sum of +the two child node impurities. Assuming that a split $s$ partitions the dataset `$D$` of size `$N$` +into two datasets `$D_{left}$` and `$D_{right}$` of sizes `$N_{left}$` and `$N_{right}$`, +respectively, the information gain is: `$IG(D,s) = Impurity(D) - \frac{N_{left}}{N} Impurity(D_{left}) - \frac{N_{right}}{N} Impurity(D_{right})$` @@ -62,14 +69,15 @@ datasets `$D_{left}$` and `$D_{right}$` of sizes `$N_{left}$` and `$N_{right}$`, **Continuous features** -For small datasets in single machine implementations, the split candidates for each continuous +For small datasets in single-machine implementations, the split candidates for each continuous feature are typically the unique values for the feature. Some implementations sort the feature values and then use the ordered unique values as split candidates for faster tree calculations. -Finding ordered unique feature values is computationally intensive for large distributed -datasets. One can get an approximate set of split candidates by performing a quantile calculation -over a sampled fraction of the data. The ordered splits create "bins" and the maximum number of such -bins can be specified using the `maxBins` parameters. +Sorting feature values is expensive for large distributed datasets. +This implementation computes an approximate set of split candidates by performing a quantile +calculation over a sampled fraction of the data. +The ordered splits create "bins" and the maximum number of such +bins can be specified using the `maxBins` parameter. Note that the number of bins cannot be greater than the number of instances `$N$` (a rare scenario since the default `maxBins` value is 100). The tree algorithm automatically reduces the number of @@ -77,109 +85,315 @@ bins if the condition is not satisfied. **Categorical features** -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 +For a categorical feature with `$M$` possible values (categories), one could come up with +`$2^{M-1}-1$` split candidates. For binary (0/1) classification and regression, +we can reduce the number of split candidates to `$M-1$` by ordering the +categorical feature values by the average label. (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, C, B. The two split candidates are A \| C, B -and A , C \| B 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. +details.) For example, for a binary classification problem with one categorical feature with three +categories A, B and C whose corresponding proportions of label 1 are 0.2, 0.6 and 0.4, the categorical +features are ordered as A, C, B. The two split candidates are A \| C, B +and A , C \| B where \| denotes the split. + +In multiclass classification, all `$2^{M-1}-1$` possible splits are used whenever possible. +When `$2^{M-1}-1$` is greater than the `maxBins` parameter, we use a (heuristic) method +similar to the method used for binary classification and regression. +The `$M$` categorical feature values are ordered by impurity, +and the resulting `$M-1$` split candidates are considered. ### Stopping rule The recursive tree construction is stopped at a node when one of the two conditions is met: -1. The node depth is equal to the `maxDepth` training parameter +1. The node depth is equal to the `maxDepth` training parameter. 2. No split candidate leads to an information gain at the node. +## Implementation details + ### Max memory requirements -For faster processing, the decision tree algorithm performs simultaneous histogram computations for all nodes at each level of the tree. This could lead to high memory requirements at deeper levels of the tree leading to memory overflow errors. To alleviate this problem, a 'maxMemoryInMB' training parameter is provided which specifies the maximum amount of memory at the workers (twice as much at the master) to be allocated to the histogram computation. The default value is conservatively chosen to be 128 MB to allow the decision algorithm to work in most scenarios. Once the memory requirements for a level-wise computation crosses the `maxMemoryInMB` threshold, the node training tasks at each subsequent level is split into smaller tasks. +For faster processing, the decision tree algorithm performs simultaneous histogram computations for +all nodes at each level of the tree. This could lead to high memory requirements at deeper levels +of the tree, potentially leading to memory overflow errors. To alleviate this problem, a `maxMemoryInMB` +training parameter specifies the maximum amount of memory at the workers (twice as much at the +master) to be allocated to the histogram computation. The default value is conservatively chosen to +be 128 MB to allow the decision algorithm to work in most scenarios. Once the memory requirements +for a level-wise computation cross the `maxMemoryInMB` threshold, the node training tasks at each +subsequent level are split into smaller tasks. + +Note that, if you have a large amount of memory, increasing `maxMemoryInMB` can lead to faster +training by requiring fewer passes over the data. + +### Binning feature values + +Increasing `maxBins` allows the algorithm to consider more split candidates and make fine-grained +split decisions. However, it also increases computation and communication. + +Note that the `maxBins` parameter must be at least the maximum number of categories `$M$` for +any categorical feature. + +### Scaling -### Practical limitations +Computation scales approximately linearly in the number of training instances, +in the number of features, and in the `maxBins` parameter. +Communication scales approximately linearly in the number of features and in `maxBins`. -1. The implemented algorithm reads both sparse and dense data. However, it is not optimized for sparse input. -2. Python is not supported in this release. +The implemented algorithm reads both sparse and dense data. However, it is not optimized for sparse input. ## Examples ### Classification -The example below demonstrates how to load a CSV file, parse it as an RDD of `LabeledPoint` and then -perform classification using a decision tree using Gini impurity as an impurity measure and a +The example below demonstrates how to load a +[LIBSVM data file](http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/), +parse it as an RDD of `LabeledPoint` and then +perform classification using a decision tree with Gini impurity as an impurity measure and a maximum tree depth of 5. The training error is calculated to measure the algorithm accuracy.
+
{% highlight scala %} -import org.apache.spark.SparkContext import org.apache.spark.mllib.tree.DecisionTree -import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.tree.configuration.Algo._ -import org.apache.spark.mllib.tree.impurity.Gini - -// Load and parse the data file -val data = sc.textFile("data/mllib/sample_tree_data.csv") -val parsedData = data.map { line => - val parts = line.split(',').map(_.toDouble) - LabeledPoint(parts(0), Vectors.dense(parts.tail)) -} +import org.apache.spark.mllib.util.MLUtils -// Run training algorithm to build the model +// Load and parse the data file. +// Cache the data since we will use it again to compute training error. +val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt").cache() + +// Train a DecisionTree model. +// Empty categoricalFeaturesInfo indicates all features are continuous. +val numClasses = 2 +val categoricalFeaturesInfo = Map[Int, Int]() +val impurity = "gini" val maxDepth = 5 -val model = DecisionTree.train(parsedData, Classification, Gini, maxDepth) +val maxBins = 100 + +val model = DecisionTree.trainClassifier(data, numClasses, categoricalFeaturesInfo, impurity, + maxDepth, maxBins) -// Evaluate model on training examples and compute training error -val labelAndPreds = parsedData.map { point => +// Evaluate model on training instances and compute training error +val labelAndPreds = data.map { point => val prediction = model.predict(point.features) (point.label, prediction) } -val trainErr = labelAndPreds.filter(r => r._1 != r._2).count.toDouble / parsedData.count +val trainErr = labelAndPreds.filter(r => r._1 != r._2).count.toDouble / data.count println("Training Error = " + trainErr) +println("Learned classification tree model:\n" + model) +{% endhighlight %} +
+ +
+{% highlight java %} +import java.util.HashMap; +import scala.Tuple2; +import org.apache.spark.api.java.function.Function2; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.tree.DecisionTree; +import org.apache.spark.mllib.tree.model.DecisionTreeModel; +import org.apache.spark.mllib.util.MLUtils; +import org.apache.spark.SparkConf; + +SparkConf sparkConf = new SparkConf().setAppName("JavaDecisionTree"); +JavaSparkContext sc = new JavaSparkContext(sparkConf); + +// Load and parse the data file. +// Cache the data since we will use it again to compute training error. +String datapath = "data/mllib/sample_libsvm_data.txt"; +JavaRDD data = MLUtils.loadLibSVMFile(sc.sc(), datapath).toJavaRDD().cache(); + +// Set parameters. +// Empty categoricalFeaturesInfo indicates all features are continuous. +Integer numClasses = 2; +HashMap categoricalFeaturesInfo = new HashMap(); +String impurity = "gini"; +Integer maxDepth = 5; +Integer maxBins = 100; + +// Train a DecisionTree model for classification. +final DecisionTreeModel model = DecisionTree.trainClassifier(data, numClasses, + categoricalFeaturesInfo, impurity, maxDepth, maxBins); + +// Evaluate model on training instances and compute training error +JavaPairRDD predictionAndLabel = + data.mapToPair(new PairFunction() { + @Override public Tuple2 call(LabeledPoint p) { + return new Tuple2(model.predict(p.features()), p.label()); + } + }); +Double trainErr = + 1.0 * predictionAndLabel.filter(new Function, Boolean>() { + @Override public Boolean call(Tuple2 pl) { + return !pl._1().equals(pl._2()); + } + }).count() / data.count(); +System.out.println("Training error: " + trainErr); +System.out.println("Learned classification tree model:\n" + model); +{% endhighlight %} +
+ +
+{% highlight python %} +from pyspark.mllib.regression import LabeledPoint +from pyspark.mllib.tree import DecisionTree +from pyspark.mllib.util import MLUtils + +# Load and parse the data file into an RDD of LabeledPoint. +# Cache the data since we will use it again to compute training error. +data = MLUtils.loadLibSVMFile(sc, 'data/mllib/sample_libsvm_data.txt').cache() + +# Train a DecisionTree model. +# Empty categoricalFeaturesInfo indicates all features are continuous. +model = DecisionTree.trainClassifier(data, numClasses=2, categoricalFeaturesInfo={}, + impurity='gini', maxDepth=5, maxBins=100) + +# Evaluate model on training instances and compute training error +predictions = model.predict(data.map(lambda x: x.features)) +labelsAndPredictions = data.map(lambda lp: lp.label).zip(predictions) +trainErr = labelsAndPredictions.filter(lambda (v, p): v != p).count() / float(data.count()) +print('Training Error = ' + str(trainErr)) +print('Learned classification tree model:') +print(model) {% endhighlight %} + +Note: When making predictions for a dataset, it is more efficient to do batch prediction rather +than separately calling `predict` on each data point. This is because the Python code makes calls +to an underlying `DecisionTree` model in Scala.
+
### Regression -The example below demonstrates how to load a CSV file, parse it as an RDD of `LabeledPoint` and then -perform regression using a decision tree using variance as an impurity measure and a maximum tree +The example below demonstrates how to load a +[LIBSVM data file](http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/), +parse it as an RDD of `LabeledPoint` and then +perform regression using a decision tree with variance as an impurity measure and a maximum tree depth of 5. The Mean Squared Error (MSE) is computed at the end to evaluate [goodness of fit](http://en.wikipedia.org/wiki/Goodness_of_fit).
+
{% highlight scala %} -import org.apache.spark.SparkContext import org.apache.spark.mllib.tree.DecisionTree -import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.tree.configuration.Algo._ -import org.apache.spark.mllib.tree.impurity.Variance - -// Load and parse the data file -val data = sc.textFile("data/mllib/sample_tree_data.csv") -val parsedData = data.map { line => - val parts = line.split(',').map(_.toDouble) - LabeledPoint(parts(0), Vectors.dense(parts.tail)) -} +import org.apache.spark.mllib.util.MLUtils -// Run training algorithm to build the model +// Load and parse the data file. +// Cache the data since we will use it again to compute training error. +val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt").cache() + +// Train a DecisionTree model. +// Empty categoricalFeaturesInfo indicates all features are continuous. +val categoricalFeaturesInfo = Map[Int, Int]() +val impurity = "variance" val maxDepth = 5 -val model = DecisionTree.train(parsedData, Regression, Variance, maxDepth) +val maxBins = 100 + +val model = DecisionTree.trainRegressor(data, categoricalFeaturesInfo, impurity, + maxDepth, maxBins) -// Evaluate model on training examples and compute training error -val valuesAndPreds = parsedData.map { point => +// Evaluate model on training instances and compute training error +val labelsAndPredictions = data.map { point => val prediction = model.predict(point.features) (point.label, prediction) } -val MSE = valuesAndPreds.map{ case(v, p) => math.pow((v - p), 2)}.mean() -println("training Mean Squared Error = " + MSE) +val trainMSE = labelsAndPredictions.map{ case(v, p) => math.pow((v - p), 2)}.mean() +println("Training Mean Squared Error = " + trainMSE) +println("Learned regression tree model:\n" + model) {% endhighlight %}
+ +
+{% highlight java %} +import java.util.HashMap; +import scala.Tuple2; +import org.apache.spark.api.java.function.Function2; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.tree.DecisionTree; +import org.apache.spark.mllib.tree.model.DecisionTreeModel; +import org.apache.spark.mllib.util.MLUtils; +import org.apache.spark.SparkConf; + +// Load and parse the data file. +// Cache the data since we will use it again to compute training error. +String datapath = "data/mllib/sample_libsvm_data.txt"; +JavaRDD data = MLUtils.loadLibSVMFile(sc.sc(), datapath).toJavaRDD().cache(); + +SparkConf sparkConf = new SparkConf().setAppName("JavaDecisionTree"); +JavaSparkContext sc = new JavaSparkContext(sparkConf); + +// Set parameters. +// Empty categoricalFeaturesInfo indicates all features are continuous. +HashMap categoricalFeaturesInfo = new HashMap(); +String impurity = "variance"; +Integer maxDepth = 5; +Integer maxBins = 100; + +// Train a DecisionTree model. +final DecisionTreeModel model = DecisionTree.trainRegressor(data, + categoricalFeaturesInfo, impurity, maxDepth, maxBins); + +// Evaluate model on training instances and compute training error +JavaPairRDD predictionAndLabel = + data.mapToPair(new PairFunction() { + @Override public Tuple2 call(LabeledPoint p) { + return new Tuple2(model.predict(p.features()), p.label()); + } + }); +Double trainMSE = + predictionAndLabel.map(new Function, Double>() { + @Override public Double call(Tuple2 pl) { + Double diff = pl._1() - pl._2(); + return diff * diff; + } + }).reduce(new Function2() { + @Override public Double call(Double a, Double b) { + return a + b; + } + }) / data.count(); +System.out.println("Training Mean Squared Error: " + trainMSE); +System.out.println("Learned regression tree model:\n" + model); +{% endhighlight %} +
+ +
+{% highlight python %} +from pyspark.mllib.regression import LabeledPoint +from pyspark.mllib.tree import DecisionTree +from pyspark.mllib.util import MLUtils + +# Load and parse the data file into an RDD of LabeledPoint. +# Cache the data since we will use it again to compute training error. +data = MLUtils.loadLibSVMFile(sc, 'data/mllib/sample_libsvm_data.txt').cache() + +# Train a DecisionTree model. +# Empty categoricalFeaturesInfo indicates all features are continuous. +model = DecisionTree.trainRegressor(data, categoricalFeaturesInfo={}, + impurity='variance', maxDepth=5, maxBins=100) + +# Evaluate model on training instances and compute training error +predictions = model.predict(data.map(lambda x: x.features)) +labelsAndPredictions = data.map(lambda lp: lp.label).zip(predictions) +trainMSE = labelsAndPredictions.map(lambda (v, p): (v - p) * (v - p)).sum() / float(data.count()) +print('Training Mean Squared Error = ' + str(trainMSE)) +print('Learned regression tree model:') +print(model) +{% endhighlight %} + +Note: When making predictions for a dataset, it is more efficient to do batch prediction rather +than separately calling `predict` on each data point. This is because the Python code makes calls +to an underlying `DecisionTree` model in Scala. +
+
diff --git a/docs/mllib-dimensionality-reduction.md b/docs/mllib-dimensionality-reduction.md index 065d646496131..9f2cf6d48ec75 100644 --- a/docs/mllib-dimensionality-reduction.md +++ b/docs/mllib-dimensionality-reduction.md @@ -11,7 +11,7 @@ displayTitle: MLlib - Dimensionality Reduction of reducing the number of variables under consideration. It can be used to extract latent features from raw and noisy features or compress data while maintaining the structure. -MLlib provides support for dimensionality reduction on tall-and-skinny matrices. +MLlib provides support for dimensionality reduction on the RowMatrix class. ## Singular value decomposition (SVD) @@ -39,8 +39,26 @@ If we keep the top $k$ singular values, then the dimensions of the resulting low * `$\Sigma$`: `$k \times k$`, * `$V$`: `$n \times k$`. -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. +### Performance +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^T A$. The matrix +storing the left singular vectors $U$, is computed via matrix multiplication as +$U = A (V S^{-1})$, if requested by the user via the computeU parameter. +The actual method to use is determined automatically based on the computational 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^T A) v$ in a distributive way and send it to +ARPACK to +compute $(A^T 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. + +### SVD Example + +MLlib provides SVD functionality to row-oriented matrices, provided in the +RowMatrix class.
@@ -124,9 +142,8 @@ MLlib supports PCA for tall-and-skinny matrices stored in row-oriented format.
-The following code demonstrates how to compute principal components on a tall-and-skinny `RowMatrix` +The following code demonstrates how to compute principal components on a `RowMatrix` and use them to project the vectors into a low-dimensional space. -The number of columns should be small, e.g, less than 1000. {% highlight scala %} import org.apache.spark.mllib.linalg.Matrix @@ -144,7 +161,7 @@ val projected: RowMatrix = mat.multiply(pc)
-The following code demonstrates how to compute principal components on a tall-and-skinny `RowMatrix` +The following code demonstrates how to compute principal components on a `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. diff --git a/docs/mllib-feature-extraction.md b/docs/mllib-feature-extraction.md index 4b3cb715c58c7..44f0f76220b6e 100644 --- a/docs/mllib-feature-extraction.md +++ b/docs/mllib-feature-extraction.md @@ -1,15 +1,94 @@ --- layout: global -title: Feature Extraction - MLlib -displayTitle: MLlib - Feature Extraction +title: Feature Extraction and Transformation - MLlib +displayTitle: MLlib - Feature Extraction and Transformation --- * Table of contents {:toc} + +## TF-IDF + +[Term frequency-inverse document frequency (TF-IDF)](http://en.wikipedia.org/wiki/Tf%E2%80%93idf) is a feature +vectorization method widely used in text mining to reflect the importance of a term to a document in the corpus. +Denote a term by `$t$`, a document by `$d$`, and the corpus by `$D$`. +Term frequency `$TF(t, d)$` is the number of times that term `$t$` appears in document `$d$`, +while document frequency `$DF(t, D)$` is the number of documents that contains term `$t$`. +If we only use term frequency to measure the importance, it is very easy to over-emphasize terms that +appear very often but carry little information about the document, e.g., "a", "the", and "of". +If a term appears very often across the corpus, it means it doesn't carry special information about +a particular document. +Inverse document frequency is a numerical measure of how much information a term provides: +`\[ +IDF(t, D) = \log \frac{|D| + 1}{DF(t, D) + 1}, +\]` +where `$|D|$` is the total number of documents in the corpus. +Since logarithm is used, if a term appears in all documents, its IDF value becomes 0. +Note that a smoothing term is applied to avoid dividing by zero for terms outside the corpus. +The TF-IDF measure is simply the product of TF and IDF: +`\[ +TFIDF(t, d, D) = TF(t, d) \cdot IDF(t, D). +\]` +There are several variants on the definition of term frequency and document frequency. +In MLlib, we separate TF and IDF to make them flexible. + +Our implementation of term frequency utilizes the +[hashing trick](http://en.wikipedia.org/wiki/Feature_hashing). +A raw feature is mapped into an index (term) by applying a hash function. +Then term frequencies are calculated based on the mapped indices. +This approach avoids the need to compute a global term-to-index map, +which can be expensive for a large corpus, but it suffers from potential hash collisions, +where different raw features may become the same term after hashing. +To reduce the chance of collision, we can increase the target feature dimension, i.e., +the number of buckets of the hash table. +The default feature dimension is `$2^{20} = 1,048,576$`. + +**Note:** MLlib doesn't provide tools for text segmentation. +We refer users to the [Stanford NLP Group](http://nlp.stanford.edu/) and +[scalanlp/chalk](https://github.com/scalanlp/chalk). + +
+
+ +TF and IDF are implemented in [HashingTF](api/scala/index.html#org.apache.spark.mllib.feature.HashingTF) +and [IDF](api/scala/index.html#org.apache.spark.mllib.feature.IDF). +`HashingTF` takes an `RDD[Iterable[_]]` as the input. +Each record could be an iterable of strings or other types. + +{% highlight scala %} +import org.apache.spark.rdd.RDD +import org.apache.spark.SparkContext +import org.apache.spark.mllib.feature.HashingTF +import org.apache.spark.mllib.linalg.Vector + +val sc: SparkContext = ... + +// Load documents (one per line). +val documents: RDD[Seq[String]] = sc.textFile("...").map(_.split(" ").toSeq) + +val hashingTF = new HashingTF() +val tf: RDD[Vector] = hasingTF.transform(documents) +{% endhighlight %} + +While applying `HashingTF` only needs a single pass to the data, applying `IDF` needs two passes: +first to compute the IDF vector and second to scale the term frequencies by IDF. + +{% highlight scala %} +import org.apache.spark.mllib.feature.IDF + +// ... continue from the previous example +tf.cache() +val idf = new IDF().fit(tf) +val tfidf: RDD[Vector] = idf.transform(tf) +{% endhighlight %} +
+
+ ## Word2Vec -Word2Vec computes distributed vector representation of words. The main advantage of the distributed +[Word2Vec](https://code.google.com/p/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 @@ -70,4 +149,107 @@ for((synonym, cosineSimilarity) <- synonyms) {
-## TFIDF \ No newline at end of file +## StandardScaler + +Standardizes features by scaling to unit variance and/or removing the mean using column summary +statistics on the samples in the training set. This is a very common pre-processing step. + +For example, RBF kernel of Support Vector Machines or the L1 and L2 regularized linear models +typically work better when all features have unit variance and/or zero mean. + +Standardization can improve the convergence rate during the optimization process, and also prevents +against features with very large variances exerting an overly large influence during model training. + +### Model Fitting + +[`StandardScaler`](api/scala/index.html#org.apache.spark.mllib.feature.StandardScaler) has the +following parameters in the constructor: + +* `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. +* `withStd` True by default. Scales the data to unit variance. + +We provide a [`fit`](api/scala/index.html#org.apache.spark.mllib.feature.StandardScaler) method in +`StandardScaler` which can take an input of `RDD[Vector]`, learn the summary statistics, and then +return a model which can transform the input dataset into unit variance and/or zero mean features +depending how we configure the `StandardScaler`. + +This model implements [`VectorTransformer`](api/scala/index.html#org.apache.spark.mllib.feature.VectorTransformer) +which can apply the standardization on a `Vector` to produce a transformed `Vector` or on +an `RDD[Vector]` to produce a transformed `RDD[Vector]`. + +Note that if the variance of a feature is zero, it will return default `0.0` value in the `Vector` +for that feature. + +### Example + +The example below demonstrates how to load a dataset in libsvm format, and standardize the features +so that the new features have unit variance and/or zero mean. + +
+
+{% highlight scala %} +import org.apache.spark.SparkContext._ +import org.apache.spark.mllib.feature.StandardScaler +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.util.MLUtils + +val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") + +val scaler1 = new StandardScaler().fit(data.map(x => x.features)) +val scaler2 = new StandardScaler(withMean = true, withStd = true).fit(data.map(x => x.features)) + +// data1 will be unit variance. +val data1 = data.map(x => (x.label, scaler1.transform(x.features))) + +// Without converting the features into dense vectors, transformation with zero mean will raise +// exception on sparse vector. +// data2 will be unit variance and zero mean. +val data2 = data.map(x => (x.label, scaler2.transform(Vectors.dense(x.features.toArray)))) +{% endhighlight %} +
+
+ +## Normalizer + +Normalizer scales individual samples to have unit $L^p$ norm. This is a common operation for text +classification or clustering. For example, the dot product of two $L^2$ normalized TF-IDF vectors +is the cosine similarity of the vectors. + +[`Normalizer`](api/scala/index.html#org.apache.spark.mllib.feature.Normalizer) has the following +parameter in the constructor: + +* `p` Normalization in $L^p$ space, $p = 2$ by default. + +`Normalizer` implements [`VectorTransformer`](api/scala/index.html#org.apache.spark.mllib.feature.VectorTransformer) +which can apply the normalization on a `Vector` to produce a transformed `Vector` or on +an `RDD[Vector]` to produce a transformed `RDD[Vector]`. + +Note that if the norm of the input is zero, it will return the input vector. + +### Example + +The example below demonstrates how to load a dataset in libsvm format, and normalizes the features +with $L^2$ norm, and $L^\infty$ norm. + +
+
+{% highlight scala %} +import org.apache.spark.SparkContext._ +import org.apache.spark.mllib.feature.Normalizer +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.util.MLUtils + +val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") + +val normalizer1 = new Normalizer() +val normalizer2 = new Normalizer(p = Double.PositiveInfinity) + +// Each sample in data1 will be normalized using $L^2$ norm. +val data1 = data.map(x => (x.label, normalizer1.transform(x.features))) + +// Each sample in data2 will be normalized using $L^\infty$ norm. +val data2 = data.map(x => (x.label, normalizer2.transform(x.features))) +{% endhighlight %} +
+
diff --git a/docs/mllib-guide.md b/docs/mllib-guide.md index ca0a84a8c53fd..4d4198b9e0452 100644 --- a/docs/mllib-guide.md +++ b/docs/mllib-guide.md @@ -35,18 +35,23 @@ and the migration guide below will explain all changes between releases. # Dependencies -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). +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. You need to install the -[gfortran runtime library](https://github.com/mikiobraun/jblas/wiki/Missing-Libraries) if it is not -already present on your nodes. MLlib will throw a linking error if it cannot detect these libraries -automatically. Due to license issues, we do not include `netlib-java`'s native libraries in MLlib's -dependency set. If no native library is available at runtime, you will see a warning message. To -use native libraries from `netlib-java`, please include artifact -`com.github.fommil.netlib:all:1.1.2` as a dependency of your project or build your own (see -[instructions](https://github.com/fommil/netlib-java/blob/master/README.md#machine-optimised-system-libraries)). +[gfortran runtime library](https://github.com/mikiobraun/jblas/wiki/Missing-Libraries) +if it is not already present on your nodes. +MLlib will throw a linking error if it cannot detect these libraries automatically. +Due to license issues, we do not include `netlib-java`'s native libraries in MLlib's +dependency set under default settings. +If no native library is available at runtime, you will see a warning message. +To use native libraries from `netlib-java`, please build Spark with `-Pnetlib-lgpl` or +include `com.github.fommil.netlib:all:1.1.2` as a dependency of your project. +If you want to use optimized BLAS/LAPACK libraries such as +[OpenBLAS](http://www.openblas.net/), please link its shared libraries to +`/usr/lib/libblas.so.3` and `/usr/lib/liblapack.so.3`, respectively. +BLAS/LAPACK libraries on worker nodes should be built without multithreading. To use MLlib in Python, you will need [NumPy](http://www.numpy.org) version 1.4 or newer. diff --git a/docs/mllib-stats.md b/docs/mllib-stats.md index f25dca746ba3a..511a9fbf710cc 100644 --- a/docs/mllib-stats.md +++ b/docs/mllib-stats.md @@ -25,6 +25,85 @@ displayTitle: MLlib - Statistics Functionality \newcommand{\zero}{\mathbf{0}} \]` +## Summary Statistics + +We provide column summary statistics for `RDD[Vector]` through the function `colStats` +available in `Statistics`. + +
+
+ +[`colStats()`](api/scala/index.html#org.apache.spark.mllib.stat.Statistics$) 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.Vector +import org.apache.spark.mllib.stat.{MultivariateStatisticalSummary, Statistics} + +val observations: RDD[Vector] = ... // an RDD of Vectors + +// Compute column summary statistics. +val summary: MultivariateStatisticalSummary = Statistics.colStats(observations) +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 + +{% endhighlight %} +
+ +
+ +[`colStats()`](api/java/org/apache/spark/mllib/stat/Statistics.html) 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.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.mllib.stat.MultivariateStatisticalSummary; +import org.apache.spark.mllib.stat.Statistics; + +JavaSparkContext jsc = ... + +JavaRDD mat = ... // an RDD of Vectors + +// Compute column summary statistics. +MultivariateStatisticalSummary summary = Statistics.colStats(mat.rdd()); +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 + +{% endhighlight %} +
+ +
+[`colStats()`](api/python/pyspark.mllib.stat.Statistics-class.html#colStats) returns an instance of +[`MultivariateStatisticalSummary`](api/python/pyspark.mllib.stat.MultivariateStatisticalSummary-class.html), +which contains the column-wise max, min, mean, variance, and number of nonzeros, as well as the +total count. + +{% highlight python %} +from pyspark.mllib.stat import Statistics + +sc = ... # SparkContext + +mat = ... # an RDD of Vectors + +# Compute column summary statistics. +summary = Statistics.colStats(mat) +print summary.mean() +print summary.variance() +print summary.numNonzeros() + +{% endhighlight %} +
+ +
+ ## Random data generation Random data generation is useful for randomized algorithms, prototyping, and performance testing. @@ -99,69 +178,280 @@ v = u.map(lambda x: 1.0 + 2.0 * x)
-## Stratified Sampling +## Correlations calculation -## Summary Statistics +Calculating the correlation between two series of data is a common operation in Statistics. In MLlib +we provide the flexibility to calculate pairwise correlations among many series. The supported +correlation methods are currently Pearson's and Spearman's correlation. + +
+
+[`Statistics`](api/scala/index.html#org.apache.spark.mllib.stat.Statistics$) provides methods to +calculate correlations between series. Depending on the type of input, two `RDD[Double]`s or +an `RDD[Vector]`, the output will be a `Double` or the correlation `Matrix` respectively. + +{% highlight scala %} +import org.apache.spark.SparkContext +import org.apache.spark.mllib.linalg._ +import org.apache.spark.mllib.stat.Statistics + +val sc: SparkContext = ... + +val seriesX: RDD[Double] = ... // a series +val seriesY: RDD[Double] = ... // must have the same number of partitions and cardinality as seriesX + +// compute the correlation using Pearson's method. Enter "spearman" for Spearman's method. If a +// method is not specified, Pearson's method will be used by default. +val correlation: Double = Statistics.corr(seriesX, seriesY, "pearson") + +val data: RDD[Vector] = ... // note that each Vector is a row and not a column + +// calculate the correlation matrix using Pearson's method. Use "spearman" for Spearman's method. +// If a method is not specified, Pearson's method will be used by default. +val correlMatrix: Matrix = Statistics.corr(data, "pearson") + +{% endhighlight %} +
+ +
+[`Statistics`](api/java/org/apache/spark/mllib/stat/Statistics.html) provides methods to +calculate correlations between series. Depending on the type of input, two `JavaDoubleRDD`s or +a `JavaRDD`, the output will be a `Double` or the correlation `Matrix` respectively. + +{% highlight java %} +import org.apache.spark.api.java.JavaDoubleRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.mllib.linalg.*; +import org.apache.spark.mllib.stat.Statistics; + +JavaSparkContext jsc = ... + +JavaDoubleRDD seriesX = ... // a series +JavaDoubleRDD seriesY = ... // must have the same number of partitions and cardinality as seriesX + +// compute the correlation using Pearson's method. Enter "spearman" for Spearman's method. If a +// method is not specified, Pearson's method will be used by default. +Double correlation = Statistics.corr(seriesX.srdd(), seriesY.srdd(), "pearson"); + +JavaRDD data = ... // note that each Vector is a row and not a column + +// calculate the correlation matrix using Pearson's method. Use "spearman" for Spearman's method. +// If a method is not specified, Pearson's method will be used by default. +Matrix correlMatrix = Statistics.corr(data.rdd(), "pearson"); + +{% endhighlight %} +
-### Multivariate summary statistics +
+[`Statistics`](api/python/pyspark.mllib.stat.Statistics-class.html) provides methods to +calculate correlations between series. Depending on the type of input, two `RDD[Double]`s or +an `RDD[Vector]`, the output will be a `Double` or the correlation `Matrix` respectively. + +{% highlight python %} +from pyspark.mllib.stat import Statistics + +sc = ... # SparkContext + +seriesX = ... # a series +seriesY = ... # must have the same number of partitions and cardinality as seriesX -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. +# Compute the correlation using Pearson's method. Enter "spearman" for Spearman's method. If a +# method is not specified, Pearson's method will be used by default. +print Statistics.corr(seriesX, seriesY, method="pearson") + +data = ... # an RDD of Vectors +# calculate the correlation matrix using Pearson's method. Use "spearman" for Spearman's method. +# If a method is not specified, Pearson's method will be used by default. +print Statistics.corr(data, method="pearson") + +{% endhighlight %} +
+ +
+ +## Stratified sampling + +Unlike the other statistics functions, which reside in MLLib, stratified sampling methods, +`sampleByKey` and `sampleByKeyExact`, can be performed on RDD's of key-value pairs. For stratified +sampling, the keys can be thought of as a label and the value as a specific attribute. For example +the key can be man or woman, or document ids, and the respective values can be the list of ages +of the people in the population or the list of words in the documents. The `sampleByKey` method +will flip a coin to decide whether an observation will be sampled or not, therefore requires one +pass over the data, and provides an *expected* sample size. `sampleByKeyExact` requires significant +more resources than the per-stratum simple random sampling used in `sampleByKey`, but will provide +the exact sampling size with 99.99% confidence. `sampleByKeyExact` is currently not supported in +python.
- -[`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. +[`sampleByKeyExact()`](api/scala/index.html#org.apache.spark.rdd.PairRDDFunctions) allows users to +sample exactly $\lceil f_k \cdot n_k \rceil \, \forall k \in K$ items, where $f_k$ is the desired +fraction for key $k$, $n_k$ is the number of key-value pairs for key $k$, and $K$ is the set of +keys. Sampling without replacement requires one additional pass over the RDD to guarantee sample +size, whereas sampling with replacement requires two additional passes. {% highlight scala %} -import org.apache.spark.mllib.linalg.Matrix -import org.apache.spark.mllib.linalg.distributed.RowMatrix -import org.apache.spark.mllib.stat.MultivariateStatisticalSummary +import org.apache.spark.SparkContext +import org.apache.spark.SparkContext._ +import org.apache.spark.rdd.PairRDDFunctions -val mat: RowMatrix = ... // a RowMatrix +val sc: SparkContext = ... -// 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 +val data = ... // an RDD[(K, V)] of any key value pairs +val fractions: Map[K, Double] = ... // specify the exact fraction desired from each key + +// Get an exact sample from each stratum +val approxSample = data.sampleByKey(withReplacement = false, fractions) +val exactSample = data.sampleByKeyExact(withReplacement = false, fractions) -// 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. +[`sampleByKeyExact()`](api/java/org/apache/spark/api/java/JavaPairRDD.html) allows users to +sample exactly $\lceil f_k \cdot n_k \rceil \, \forall k \in K$ items, where $f_k$ is the desired +fraction for key $k$, $n_k$ is the number of key-value pairs for key $k$, and $K$ is the set of +keys. Sampling without replacement requires one additional pass over the RDD to guarantee sample +size, whereas sampling with replacement requires two additional passes. {% highlight java %} -import org.apache.spark.mllib.linalg.Matrix; -import org.apache.spark.mllib.linalg.distributed.RowMatrix; -import org.apache.spark.mllib.stat.MultivariateStatisticalSummary; +import java.util.Map; -RowMatrix mat = ... // a RowMatrix +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaSparkContext; -// 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 +JavaSparkContext jsc = ... + +JavaPairRDD data = ... // an RDD of any key value pairs +Map fractions = ... // specify the exact fraction desired from each key + +// Get an exact sample from each stratum +JavaPairRDD approxSample = data.sampleByKey(false, fractions); +JavaPairRDD exactSample = data.sampleByKeyExact(false, fractions); + +{% endhighlight %} +
+
+[`sampleByKey()`](api/python/pyspark.rdd.RDD-class.html#sampleByKey) allows users to +sample approximately $\lceil f_k \cdot n_k \rceil \, \forall k \in K$ items, where $f_k$ is the +desired fraction for key $k$, $n_k$ is the number of key-value pairs for key $k$, and $K$ is the +set of keys. + +*Note:* `sampleByKeyExact()` is currently not supported in Python. + +{% highlight python %} + +sc = ... # SparkContext + +data = ... # an RDD of any key value pairs +fractions = ... # specify the exact fraction desired from each key as a dictionary + +approxSample = data.sampleByKey(False, fractions); -// Compute the covariance matrix. -Matrix cov = mat.computeCovariance(); {% endhighlight %}
+ +
+ +## Hypothesis testing + +Hypothesis testing is a powerful tool in statistics to determine whether a result is statistically +significant, whether this result occurred by chance or not. MLlib currently supports Pearson's +chi-squared ( $\chi^2$) tests for goodness of fit and independence. The input data types determine +whether the goodness of fit or the independence test is conducted. The goodness of fit test requires +an input type of `Vector`, whereas the independence test requires a `Matrix` as input. + +MLlib also supports the input type `RDD[LabeledPoint]` to enable feature selection via chi-squared +independence tests. + +
+
+[`Statistics`](api/scala/index.html#org.apache.spark.mllib.stat.Statistics$) provides methods to +run Pearson's chi-squared tests. The following example demonstrates how to run and interpret +hypothesis tests. + +{% highlight scala %} +import org.apache.spark.SparkContext +import org.apache.spark.mllib.linalg._ +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.stat.Statistics._ + +val sc: SparkContext = ... + +val vec: Vector = ... // a vector composed of the frequencies of events + +// compute the goodness of fit. If a second vector to test against is not supplied as a parameter, +// the test runs against a uniform distribution. +val goodnessOfFitTestResult = Statistics.chiSqTest(vec) +println(goodnessOfFitTestResult) // summary of the test including the p-value, degrees of freedom, + // test statistic, the method used, and the null hypothesis. + +val mat: Matrix = ... // a contingency matrix + +// conduct Pearson's independence test on the input contingency matrix +val independenceTestResult = Statistics.chiSqTest(mat) +println(independenceTestResult) // summary of the test including the p-value, degrees of freedom... + +val obs: RDD[LabeledPoint] = ... // (feature, label) pairs. + +// 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. +val featureTestResults: Array[ChiSqTestResult] = Statistics.chiSqTest(obs) +var i = 1 +featureTestResults.foreach { result => + println(s"Column $i:\n$result") + i += 1 +} // summary of the test + +{% endhighlight %}
+
+[`Statistics`](api/java/org/apache/spark/mllib/stat/Statistics.html) provides methods to +run Pearson's chi-squared tests. The following example demonstrates how to run and interpret +hypothesis tests. + +{% highlight java %} +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.mllib.linalg.*; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.stat.Statistics; +import org.apache.spark.mllib.stat.test.ChiSqTestResult; + +JavaSparkContext jsc = ... -## Hypothesis Testing +Vector vec = ... // a vector composed of the frequencies of events + +// compute the goodness of fit. If a second vector to test against is not supplied as a parameter, +// the test runs against a uniform distribution. +ChiSqTestResult goodnessOfFitTestResult = Statistics.chiSqTest(vec); +// summary of the test including the p-value, degrees of freedom, test statistic, the method used, +// and the null hypothesis. +System.out.println(goodnessOfFitTestResult); + +Matrix mat = ... // a contingency matrix + +// conduct Pearson's independence test on the input contingency matrix +ChiSqTestResult independenceTestResult = Statistics.chiSqTest(mat); +// summary of the test including the p-value, degrees of freedom... +System.out.println(independenceTestResult); + +JavaRDD obs = ... // an RDD of labeled points + +// 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. +ChiSqTestResult[] featureTestResults = Statistics.chiSqTest(obs.rdd()); +int i = 1; +for (ChiSqTestResult result : featureTestResults) { + System.out.println("Column " + i + ":"); + System.out.println(result); // summary of the test + i++; +} + +{% endhighlight %} +
+ +
diff --git a/docs/running-on-mesos.md b/docs/running-on-mesos.md index bd046cfc1837d..9998dddc652a6 100644 --- a/docs/running-on-mesos.md +++ b/docs/running-on-mesos.md @@ -107,7 +107,7 @@ cluster, or `mesos://zk://host:2181` for a multi-master Mesos cluster using ZooK The driver also needs some configuration in `spark-env.sh` to interact properly with Mesos: -1. In `spark.env.sh` set some environment variables: +1. In `spark-env.sh` set some environment variables: * `export MESOS_NATIVE_LIBRARY=`. This path is typically `/lib/libmesos.so` where the prefix is `/usr/local` by default. See Mesos installation instructions above. On Mac OS X, the library is called `libmesos.dylib` instead of diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 3a8c816cfffa1..77a246fffe6a7 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -141,6 +141,12 @@ def parse_args(): parser.add_option( "--security-group-prefix", type="string", default=None, help="Use this prefix for the security group rather than the cluster name.") + parser.add_option( + "--authorized-address", type="string", default="0.0.0.0/0", + help="Address to authorize on created security groups (default: 0.0.0.0/0)") + parser.add_option( + "--additional-security-group", type="string", default="", + help="Additional security group to place the machines in") (opts, args) = parser.parse_args() if len(args) != 2: @@ -293,28 +299,29 @@ def launch_cluster(conn, opts, cluster_name): else: master_group = get_or_make_group(conn, opts.security_group_prefix + "-master") slave_group = get_or_make_group(conn, opts.security_group_prefix + "-slaves") + authorized_address = opts.authorized_address if master_group.rules == []: # Group was just now created master_group.authorize(src_group=master_group) master_group.authorize(src_group=slave_group) - master_group.authorize('tcp', 22, 22, '0.0.0.0/0') - master_group.authorize('tcp', 8080, 8081, '0.0.0.0/0') - master_group.authorize('tcp', 18080, 18080, '0.0.0.0/0') - master_group.authorize('tcp', 19999, 19999, '0.0.0.0/0') - master_group.authorize('tcp', 50030, 50030, '0.0.0.0/0') - master_group.authorize('tcp', 50070, 50070, '0.0.0.0/0') - master_group.authorize('tcp', 60070, 60070, '0.0.0.0/0') - master_group.authorize('tcp', 4040, 4045, '0.0.0.0/0') + master_group.authorize('tcp', 22, 22, authorized_address) + master_group.authorize('tcp', 8080, 8081, authorized_address) + master_group.authorize('tcp', 18080, 18080, authorized_address) + master_group.authorize('tcp', 19999, 19999, authorized_address) + master_group.authorize('tcp', 50030, 50030, authorized_address) + master_group.authorize('tcp', 50070, 50070, authorized_address) + master_group.authorize('tcp', 60070, 60070, authorized_address) + master_group.authorize('tcp', 4040, 4045, authorized_address) if opts.ganglia: - master_group.authorize('tcp', 5080, 5080, '0.0.0.0/0') + master_group.authorize('tcp', 5080, 5080, authorized_address) if slave_group.rules == []: # Group was just now created slave_group.authorize(src_group=master_group) slave_group.authorize(src_group=slave_group) - slave_group.authorize('tcp', 22, 22, '0.0.0.0/0') - slave_group.authorize('tcp', 8080, 8081, '0.0.0.0/0') - slave_group.authorize('tcp', 50060, 50060, '0.0.0.0/0') - slave_group.authorize('tcp', 50075, 50075, '0.0.0.0/0') - slave_group.authorize('tcp', 60060, 60060, '0.0.0.0/0') - slave_group.authorize('tcp', 60075, 60075, '0.0.0.0/0') + slave_group.authorize('tcp', 22, 22, authorized_address) + slave_group.authorize('tcp', 8080, 8081, authorized_address) + slave_group.authorize('tcp', 50060, 50060, authorized_address) + slave_group.authorize('tcp', 50075, 50075, authorized_address) + slave_group.authorize('tcp', 60060, 60060, authorized_address) + slave_group.authorize('tcp', 60075, 60075, authorized_address) # Check if instances are already running with the cluster name existing_masters, existing_slaves = get_existing_cluster(conn, opts, cluster_name, @@ -326,6 +333,13 @@ def launch_cluster(conn, opts, cluster_name): # Figure out Spark AMI if opts.ami is None: opts.ami = get_spark_ami(opts) + + + additional_groups = [] + if opts.additional_security_group: + additional_groups = [sg + for sg in conn.get_all_security_groups() + if opts.additional_security_group in (sg.name, sg.id)] print "Launching instances..." try: @@ -360,7 +374,7 @@ def launch_cluster(conn, opts, cluster_name): placement=zone, count=num_slaves_this_zone, key_name=opts.key_pair, - security_groups=[slave_group], + security_groups=[slave_group] + additional_groups, instance_type=opts.instance_type, block_device_map=block_map, user_data=user_data_content) @@ -413,7 +427,7 @@ def launch_cluster(conn, opts, cluster_name): num_slaves_this_zone = get_partition(opts.slaves, num_zones, i) if num_slaves_this_zone > 0: slave_res = image.run(key_name=opts.key_pair, - security_groups=[slave_group], + security_groups=[slave_group] + additional_groups, instance_type=opts.instance_type, placement=zone, min_count=num_slaves_this_zone, @@ -439,7 +453,7 @@ def launch_cluster(conn, opts, cluster_name): if opts.zone == 'all': opts.zone = random.choice(conn.get_all_zones()).name master_res = image.run(key_name=opts.key_pair, - security_groups=[master_group], + security_groups=[master_group] + additional_groups, instance_type=master_type, placement=opts.zone, min_count=1, diff --git a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java index 7ea6df9c17245..c22506491fbff 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java @@ -96,7 +96,7 @@ public Double call(Iterable rs) { .flatMapToPair(new PairFlatMapFunction, Double>, String, Double>() { @Override public Iterable> call(Tuple2, Double> s) { - int urlCount = Iterables.size(s._1); + int urlCount = Iterables.size(s._1); List> results = new ArrayList>(); for (String n : s._1) { results.add(new Tuple2(n, s._2() / urlCount)); diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaDecisionTree.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaDecisionTree.java new file mode 100644 index 0000000000000..e4468e8bf1744 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaDecisionTree.java @@ -0,0 +1,116 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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 java.util.HashMap; + +import scala.Tuple2; + +import org.apache.spark.api.java.function.Function2; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.tree.DecisionTree; +import org.apache.spark.mllib.tree.model.DecisionTreeModel; +import org.apache.spark.mllib.util.MLUtils; +import org.apache.spark.SparkConf; + +/** + * Classification and regression using decision trees. + */ +public final class JavaDecisionTree { + + public static void main(String[] args) { + String datapath = "data/mllib/sample_libsvm_data.txt"; + if (args.length == 1) { + datapath = args[0]; + } else if (args.length > 1) { + System.err.println("Usage: JavaDecisionTree "); + System.exit(1); + } + SparkConf sparkConf = new SparkConf().setAppName("JavaDecisionTree"); + JavaSparkContext sc = new JavaSparkContext(sparkConf); + + JavaRDD data = MLUtils.loadLibSVMFile(sc.sc(), datapath).toJavaRDD().cache(); + + // Compute the number of classes from the data. + Integer numClasses = data.map(new Function() { + @Override public Double call(LabeledPoint p) { + return p.label(); + } + }).countByValue().size(); + + // Set parameters. + // Empty categoricalFeaturesInfo indicates all features are continuous. + HashMap categoricalFeaturesInfo = new HashMap(); + String impurity = "gini"; + Integer maxDepth = 5; + Integer maxBins = 100; + + // Train a DecisionTree model for classification. + final DecisionTreeModel model = DecisionTree.trainClassifier(data, numClasses, + categoricalFeaturesInfo, impurity, maxDepth, maxBins); + + // Evaluate model on training instances and compute training error + JavaPairRDD predictionAndLabel = + data.mapToPair(new PairFunction() { + @Override public Tuple2 call(LabeledPoint p) { + return new Tuple2(model.predict(p.features()), p.label()); + } + }); + Double trainErr = + 1.0 * predictionAndLabel.filter(new Function, Boolean>() { + @Override public Boolean call(Tuple2 pl) { + return !pl._1().equals(pl._2()); + } + }).count() / data.count(); + System.out.println("Training error: " + trainErr); + System.out.println("Learned classification tree model:\n" + model); + + // Train a DecisionTree model for regression. + impurity = "variance"; + final DecisionTreeModel regressionModel = DecisionTree.trainRegressor(data, + categoricalFeaturesInfo, impurity, maxDepth, maxBins); + + // Evaluate model on training instances and compute training error + JavaPairRDD regressorPredictionAndLabel = + data.mapToPair(new PairFunction() { + @Override public Tuple2 call(LabeledPoint p) { + return new Tuple2(regressionModel.predict(p.features()), p.label()); + } + }); + Double trainMSE = + regressorPredictionAndLabel.map(new Function, Double>() { + @Override public Double call(Tuple2 pl) { + Double diff = pl._1() - pl._2(); + return diff * diff; + } + }).reduce(new Function2() { + @Override public Double call(Double a, Double b) { + return a + b; + } + }) / data.count(); + System.out.println("Training Mean Squared Error: " + trainMSE); + System.out.println("Learned regression tree model:\n" + regressionModel); + + sc.stop(); + } +} diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index 0c68defa5e101..c1e8e65464fc1 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -25,19 +25,20 @@ ../../pom.xml + org.apache.spark 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 + ${flume.version} io.netty @@ -52,7 +53,7 @@ org.apache.flume flume-ng-core - 1.4.0 + ${flume.version} io.netty @@ -64,20 +65,22 @@ - - org.scala-lang - scala-library - org.scalatest scalatest_${scala.binary.version} + test - org.apache.spark - spark-streaming_${scala.binary.version} - ${project.version} - test-jar - test + + io.netty + netty + 3.4.0.Final + test @@ -91,7 +94,7 @@ org.apache.avro avro-maven-plugin - 1.7.3 + ${avro.version} ${project.basedir}/target/scala-${scala.binary.version}/src_managed/main/compiled_avro diff --git a/external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala b/external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala index 44b27edf85ce8..75a6668c6210b 100644 --- a/external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala +++ b/external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala @@ -30,14 +30,14 @@ import org.apache.avro.ipc.specific.SpecificRequestor import org.apache.flume.Context import org.apache.flume.channel.MemoryChannel import org.apache.flume.event.EventBuilder -import org.apache.spark.streaming.TestSuiteBase import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory +import org.scalatest.FunSuite -class SparkSinkSuite extends TestSuiteBase { +class SparkSinkSuite extends FunSuite { val eventsPerBatch = 1000 val channelCapacity = 5000 - test("Success") { + test("Success with ack") { val (channel, sink) = initializeChannelAndSink() channel.start() sink.start() @@ -57,7 +57,7 @@ class SparkSinkSuite extends TestSuiteBase { transceiver.close() } - test("Nack") { + test("Failure with nack") { val (channel, sink) = initializeChannelAndSink() channel.start() sink.start() @@ -76,7 +76,7 @@ class SparkSinkSuite extends TestSuiteBase { transceiver.close() } - test("Timeout") { + test("Failure with timeout") { val (channel, sink) = initializeChannelAndSink(Map(SparkSinkConfig .CONF_TRANSACTION_TIMEOUT -> 1.toString)) channel.start() diff --git a/external/flume/pom.xml b/external/flume/pom.xml index c532705f3950c..f71f6b6c4f931 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -40,6 +40,11 @@ spark-streaming_${scala.binary.version} ${project.version} + + org.apache.spark + spark-streaming-flume-sink_${scala.binary.version} + ${project.version} + org.apache.spark spark-streaming_${scala.binary.version} @@ -50,7 +55,7 @@ org.apache.flume flume-ng-sdk - 1.4.0 + ${flume.version} io.netty @@ -82,11 +87,6 @@ junit-interface test - - org.apache.spark - spark-streaming-flume-sink_2.10 - ${project.version} - target/scala-${scala.binary.version}/classes 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 index a8b907b241893..1a710d7b18c6f 100644 --- 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 @@ -130,10 +130,10 @@ public static void main(String[] args) { /* 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()) - ); + 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 */ 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 5823cb6e52e7f..12a3d91cd31a6 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 @@ -25,7 +25,7 @@ import org.apache.spark.mllib.linalg.Vector /** * A clustering model for K-means. Each point belongs to the cluster with the closest center. */ -class KMeansModel private[mllib] (val clusterCenters: Array[Vector]) extends Serializable { +class KMeansModel (val clusterCenters: Array[Vector]) extends Serializable { /** Total number of clusters. */ def k: Int = clusterCenters.length diff --git a/pom.xml b/pom.xml index 9cbf3ea5995c3..556b9da3d6d90 100644 --- a/pom.xml +++ b/pom.xml @@ -125,6 +125,7 @@ 2.4.1 ${hadoop.version} 0.94.6 + 1.4.0 3.4.5 0.12.0 1.4.3 diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py index c7f7c1fe591b0..6f4f62f23bc4d 100644 --- a/python/pyspark/java_gateway.py +++ b/python/pyspark/java_gateway.py @@ -54,12 +54,19 @@ def preexec_func(): gateway_port = proc.stdout.readline() gateway_port = int(gateway_port) except ValueError: + # Grab the remaining lines of stdout (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 += "(Warning: unexpected output detected.)\n\n" - error_msg += gateway_port + stdout + error_msg += " with exit code %d!\n" % exit_code if exit_code else "!\n" + error_msg += "Warning: Expected GatewayServer to output a port, but found " + if gateway_port == "" and stdout == "": + error_msg += "no output.\n" + else: + error_msg += "the following:\n\n" + error_msg += "--------------------------------------------------------------\n" + error_msg += gateway_port + stdout + error_msg += "--------------------------------------------------------------\n" raise Exception(error_msg) # Create a thread to echo output from the GatewayServer, which is required diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 8295b6f4729b0..46ae79d9c1f01 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -32,7 +32,7 @@ import heapq import bisect from random import Random -from math import sqrt, log +from math import sqrt, log, isinf, isnan from pyspark.serializers import NoOpSerializer, CartesianDeserializer, \ BatchedSerializer, CloudPickleSerializer, PairDeserializer, \ @@ -46,6 +46,7 @@ from pyspark.shuffle import Aggregator, InMemoryMerger, ExternalMerger, \ get_used_memory, ExternalSorter, ExternalGroupBy + from py4j.java_collections import ListConverter, MapConverter __all__ = ["RDD"] @@ -131,6 +132,22 @@ def __exit__(self, type, value, tb): self._context._jsc.setCallSite(None) +class BoundedFloat(float): + """ + Bounded value is generated by approximate job, with confidence and low + bound and high bound. + + >>> BoundedFloat(100.0, 0.95, 95.0, 105.0) + 100.0 + """ + def __new__(cls, mean, confidence, low, high): + obj = float.__new__(cls, mean) + obj.confidence = confidence + obj.low = low + obj.high = high + return obj + + class MaxHeapQ(object): """ @@ -815,23 +832,37 @@ def func(iterator): return self.mapPartitions(func).fold(zeroValue, combOp) - def max(self): + def max(self, key=None): """ Find the maximum item in this RDD. - >>> sc.parallelize([1.0, 5.0, 43.0, 10.0]).max() + @param key: A function used to generate key for comparing + + >>> rdd = sc.parallelize([1.0, 5.0, 43.0, 10.0]) + >>> rdd.max() 43.0 + >>> rdd.max(key=str) + 5.0 """ - return self.reduce(max) + if key is None: + return self.reduce(max) + return self.reduce(lambda a, b: max(a, b, key=key)) - def min(self): + def min(self, key=None): """ Find the minimum item in this RDD. - >>> sc.parallelize([1.0, 5.0, 43.0, 10.0]).min() - 1.0 + @param key: A function used to generate key for comparing + + >>> rdd = sc.parallelize([2.0, 5.0, 43.0, 10.0]) + >>> rdd.min() + 2.0 + >>> rdd.min(key=str) + 10.0 """ - return self.reduce(min) + if key is None: + return self.reduce(min) + return self.reduce(lambda a, b: min(a, b, key=key)) def sum(self): """ @@ -861,6 +892,133 @@ def redFunc(left_counter, right_counter): return self.mapPartitions(lambda i: [StatCounter(i)]).reduce(redFunc) + def histogram(self, buckets): + """ + Compute a histogram using the provided buckets. The buckets + are all open to the right except for the last which is closed. + e.g. [1,10,20,50] means the buckets are [1,10) [10,20) [20,50], + which means 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. + + 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). + + Buckets must be sorted and not contain any duplicates, must be + at least two elements. + + If `buckets` is a number, it will generates buckets which are + evenly spaced between the minimum and maximum of the RDD. For + example, if the min value is 0 and the max is 100, given buckets + as 2, the resulting buckets will be [0,50) [50,100]. buckets must + be at least 1 If the RDD contains infinity, NaN throws an exception + If the elements in RDD do not vary (max == min) always returns + a single bucket. + + It will return an tuple of buckets and histogram. + + >>> rdd = sc.parallelize(range(51)) + >>> rdd.histogram(2) + ([0, 25, 50], [25, 26]) + >>> rdd.histogram([0, 5, 25, 50]) + ([0, 5, 25, 50], [5, 20, 26]) + >>> rdd.histogram([0, 15, 30, 45, 60]) # evenly spaced buckets + ([0, 15, 30, 45, 60], [15, 15, 15, 6]) + >>> rdd = sc.parallelize(["ab", "ac", "b", "bd", "ef"]) + >>> rdd.histogram(("a", "b", "c")) + (('a', 'b', 'c'), [2, 2]) + """ + + if isinstance(buckets, (int, long)): + if buckets < 1: + raise ValueError("number of buckets must be >= 1") + + # filter out non-comparable elements + def comparable(x): + if x is None: + return False + if type(x) is float and isnan(x): + return False + return True + + filtered = self.filter(comparable) + + # faster than stats() + def minmax(a, b): + return min(a[0], b[0]), max(a[1], b[1]) + try: + minv, maxv = filtered.map(lambda x: (x, x)).reduce(minmax) + except TypeError as e: + if " empty " in str(e): + raise ValueError("can not generate buckets from empty RDD") + raise + + if minv == maxv or buckets == 1: + return [minv, maxv], [filtered.count()] + + try: + inc = (maxv - minv) / buckets + except TypeError: + raise TypeError("Can not generate buckets with non-number in RDD") + + if isinf(inc): + raise ValueError("Can not generate buckets with infinite value") + + # keep them as integer if possible + if inc * buckets != maxv - minv: + inc = (maxv - minv) * 1.0 / buckets + + buckets = [i * inc + minv for i in range(buckets)] + buckets.append(maxv) # fix accumulated error + even = True + + elif isinstance(buckets, (list, tuple)): + if len(buckets) < 2: + raise ValueError("buckets should have more than one value") + + if any(i is None or isinstance(i, float) and isnan(i) for i in buckets): + raise ValueError("can not have None or NaN in buckets") + + if sorted(buckets) != list(buckets): + raise ValueError("buckets should be sorted") + + if len(set(buckets)) != len(buckets): + raise ValueError("buckets should not contain duplicated values") + + minv = buckets[0] + maxv = buckets[-1] + even = False + inc = None + try: + steps = [buckets[i + 1] - buckets[i] for i in range(len(buckets) - 1)] + except TypeError: + pass # objects in buckets do not support '-' + else: + if max(steps) - min(steps) < 1e-10: # handle precision errors + even = True + inc = (maxv - minv) / (len(buckets) - 1) + + else: + raise TypeError("buckets should be a list or tuple or number(int or long)") + + def histogram(iterator): + counters = [0] * len(buckets) + for i in iterator: + if i is None or (type(i) is float and isnan(i)) or i > maxv or i < minv: + continue + t = (int((i - minv) / inc) if even + else bisect.bisect_right(buckets, i) - 1) + counters[t] += 1 + # add last two together + last = counters.pop() + counters[-1] += last + return [counters] + + def mergeCounters(a, b): + return [i + j for i, j in zip(a, b)] + + return buckets, self.mapPartitions(histogram).reduce(mergeCounters) + def mean(self): """ Compute the mean of this RDD's elements. @@ -929,7 +1087,7 @@ def mergeMaps(m1, m2): return m1 return self.mapPartitions(countPartition).reduce(mergeMaps) - def top(self, num): + def top(self, num, key=None): """ Get the top N elements from a RDD. @@ -938,20 +1096,16 @@ def top(self, num): [12] >>> sc.parallelize([2, 3, 4, 5, 6], 2).top(2) [6, 5] + >>> sc.parallelize([10, 4, 2, 12, 3]).top(3, key=str) + [4, 3, 2] """ def topIterator(iterator): - q = [] - for k in iterator: - if len(q) < num: - heapq.heappush(q, k) - else: - heapq.heappushpop(q, k) - yield q + yield heapq.nlargest(num, iterator, key=key) def merge(a, b): - return next(topIterator(a + b)) + return heapq.nlargest(num, a + b, key=key) - return sorted(self.mapPartitions(topIterator).reduce(merge), reverse=True) + return self.mapPartitions(topIterator).reduce(merge) def takeOrdered(self, num, key=None): """ @@ -1742,6 +1896,53 @@ def batch_as(rdd, batchSize): other._jrdd_deserializer) return RDD(pairRDD, self.ctx, deserializer) + def zipWithIndex(self): + """ + Zips this RDD with its element indices. + + The ordering is first based on the partition index and then the + ordering of items within each partition. So the first item in + the first partition gets index 0, and the last item in the last + partition receives the largest index. + + This method needs to trigger a spark job when this RDD contains + more than one partitions. + + >>> sc.parallelize(["a", "b", "c", "d"], 3).zipWithIndex().collect() + [('a', 0), ('b', 1), ('c', 2), ('d', 3)] + """ + starts = [0] + if self.getNumPartitions() > 1: + nums = self.mapPartitions(lambda it: [sum(1 for i in it)]).collect() + for i in range(len(nums) - 1): + starts.append(starts[-1] + nums[i]) + + def func(k, it): + for i, v in enumerate(it, starts[k]): + yield v, i + + return self.mapPartitionsWithIndex(func) + + def zipWithUniqueId(self): + """ + Zips this RDD with generated unique Long ids. + + Items in the kth partition will get ids k, n+k, 2*n+k, ..., where + n is the number of partitions. So there may exist gaps, but this + method won't trigger a spark job, which is different from + L{zipWithIndex} + + >>> sc.parallelize(["a", "b", "c", "d", "e"], 3).zipWithUniqueId().collect() + [('a', 0), ('b', 1), ('c', 4), ('d', 2), ('e', 5)] + """ + n = self.getNumPartitions() + + def func(k, it): + for i, v in enumerate(it): + yield v, i * n + k + + return self.mapPartitionsWithIndex(func) + def name(self): """ Return the name of this RDD. @@ -1809,6 +2010,71 @@ def _defaultReducePartitions(self): # keys in the pairs. This could be an expensive operation, since those # hashes aren't retained. + def _is_pickled(self): + """ Return this RDD is serialized by Pickle or not. """ + der = self._jrdd_deserializer + if isinstance(der, PickleSerializer): + return True + if isinstance(der, BatchedSerializer) and isinstance(der.serializer, PickleSerializer): + return True + return False + + def _to_jrdd(self): + """ Return an JavaRDD of Object by unpickling + + It will convert each Python object into Java object by Pyrolite, whenever the + RDD is serialized in batch or not. + """ + if not self._is_pickled(): + self = self._reserialize(BatchedSerializer(PickleSerializer(), 1024)) + batched = isinstance(self._jrdd_deserializer, BatchedSerializer) + return self.ctx._jvm.PythonRDD.pythonToJava(self._jrdd, batched) + + def countApprox(self, timeout, confidence=0.95): + """ + :: Experimental :: + Approximate version of count() that returns a potentially incomplete + result within a timeout, even if not all tasks have finished. + + >>> rdd = sc.parallelize(range(1000), 10) + >>> rdd.countApprox(1000, 1.0) + 1000 + """ + drdd = self.mapPartitions(lambda it: [float(sum(1 for i in it))]) + return int(drdd.sumApprox(timeout, confidence)) + + def sumApprox(self, timeout, confidence=0.95): + """ + :: Experimental :: + Approximate operation to return the sum within a timeout + or meet the confidence. + + >>> rdd = sc.parallelize(range(1000), 10) + >>> r = sum(xrange(1000)) + >>> (rdd.sumApprox(1000) - r) / r < 0.05 + True + """ + jrdd = self.mapPartitions(lambda it: [float(sum(it))])._to_jrdd() + jdrdd = self.ctx._jvm.JavaDoubleRDD.fromRDD(jrdd.rdd()) + r = jdrdd.sumApprox(timeout, confidence).getFinalValue() + return BoundedFloat(r.mean(), r.confidence(), r.low(), r.high()) + + def meanApprox(self, timeout, confidence=0.95): + """ + :: Experimental :: + Approximate operation to return the mean within a timeout + or meet the confidence. + + >>> rdd = sc.parallelize(range(1000), 10) + >>> r = sum(xrange(1000)) / 1000.0 + >>> (rdd.meanApprox(1000) - r) / r < 0.05 + True + """ + jrdd = self.map(float)._to_jrdd() + jdrdd = self.ctx._jvm.JavaDoubleRDD.fromRDD(jrdd.rdd()) + r = jdrdd.meanApprox(timeout, confidence).getFinalValue() + return BoundedFloat(r.mean(), r.confidence(), r.low(), r.high()) + class PipelinedRDD(RDD): diff --git a/python/pyspark/shuffle.py b/python/pyspark/shuffle.py index a28f6efde1c4f..7a5653a52d835 100644 --- a/python/pyspark/shuffle.py +++ b/python/pyspark/shuffle.py @@ -23,11 +23,13 @@ import gc import itertools import operator +import random import pyspark.heapq3 as heapq from pyspark.serializers import BatchedSerializer, PickleSerializer, FlattedValuesSerializer, \ CompressedSerializer + try: import psutil @@ -62,6 +64,10 @@ def _get_local_dirs(sub): """ Get all the directories """ path = os.environ.get("SPARK_LOCAL_DIRS", "/tmp") dirs = path.split(",") + if len(dirs) > 1: + # different order in different processes and instances + rnd = random.Random(os.getpid() + id(dirs)) + random.shuffle(dirs, rnd.random) return [os.path.join(d, "python", str(os.getpid()), sub) for d in dirs] @@ -205,7 +211,7 @@ def __init__(self, aggregator, memory_limit=512, serializer=None, Merger.__init__(self, aggregator) self.memory_limit = memory_limit # default serializer is only used for tests - self.serializer = serializer or PickleSerializer() + self.serializer = serializer or BatchedSerializer(PickleSerializer(), 1024) # add compression if isinstance(self.serializer, BatchedSerializer): if not isinstance(self.serializer.serializer, CompressedSerializer): @@ -215,6 +221,7 @@ def __init__(self, aggregator, memory_limit=512, serializer=None, else: if not isinstance(self.serializer, CompressedSerializer): self.serializer = CompressedSerializer(self.serializer) + self.localdirs = localdirs or _get_local_dirs(str(id(self))) # number of partitions when spill data into disks self.partitions = partitions @@ -702,6 +709,74 @@ def load_partition(j): return GroupByKey(sorted_items) +class ExternalSorter(object): + """ + ExtenalSorter will divide the elements into chunks, sort them in + memory and dump them into disks, finally merge them back. + + The spilling will only happen when the used memory goes above + the limit. + + >>> sorter = ExternalSorter(1) # 1M + >>> import random + >>> l = range(1024) + >>> random.shuffle(l) + >>> sorted(l) == list(sorter.sorted(l)) + True + >>> sorted(l) == list(sorter.sorted(l, key=lambda x: -x, reverse=True)) + True + """ + def __init__(self, memory_limit, serializer=None): + self.memory_limit = memory_limit + self.local_dirs = _get_local_dirs("sort") + self.serializer = serializer or BatchedSerializer(PickleSerializer(), 1024) + self._spilled_bytes = 0 + + def _get_path(self, n): + """ Choose one directory for spill by number n """ + d = self.local_dirs[n % len(self.local_dirs)] + if not os.path.exists(d): + os.makedirs(d) + return os.path.join(d, str(n)) + + def sorted(self, iterator, key=None, reverse=False): + """ + Sort the elements in iterator, do external sort when the memory + goes above the limit. + """ + batch = 10 + chunks, current_chunk = [], [] + iterator = iter(iterator) + while True: + # pick elements in batch + chunk = list(itertools.islice(iterator, batch)) + current_chunk.extend(chunk) + if len(chunk) < batch: + break + + if get_used_memory() > self.memory_limit: + # sort them inplace will save memory + current_chunk.sort(key=key, reverse=reverse) + path = self._get_path(len(chunks)) + with open(path, 'w') as f: + self.serializer.dump_stream(current_chunk, f) + self._spilled_bytes += os.path.getsize(path) + chunks.append(self.serializer.load_stream(open(path))) + current_chunk = [] + + elif not chunks: + batch = min(batch * 2, 10000) + + current_chunk.sort(key=key, reverse=reverse) + if not chunks: + return current_chunk + + if current_chunk: + chunks.append(iter(current_chunk)) + + return heapq.merge(chunks, key=key, reverse=reverse) + + if __name__ == "__main__": import doctest doctest.testmod() diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index d4ca0cc8f336e..0ff6a548a85f1 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -186,15 +186,15 @@ class ArrayType(DataType): """ - def __init__(self, elementType, containsNull=False): + def __init__(self, elementType, containsNull=True): """Creates an ArrayType :param elementType: the data type of elements. :param containsNull: indicates whether the list contains None values. - >>> ArrayType(StringType) == ArrayType(StringType, False) + >>> ArrayType(StringType) == ArrayType(StringType, True) True - >>> ArrayType(StringType, True) == ArrayType(StringType) + >>> ArrayType(StringType, False) == ArrayType(StringType) False """ self.elementType = elementType diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 31c9be555e289..541ca7dc78090 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -38,6 +38,7 @@ import unittest +from pyspark.conf import SparkConf from pyspark.context import SparkContext from pyspark.files import SparkFiles from pyspark.serializers import read_int, BatchedSerializer, MarshalSerializer, PickleSerializer @@ -134,10 +135,26 @@ def test_external_sort(self): random.shuffle(l) sorter = ExternalSorter(1) self.assertEquals(sorted(l), list(sorter.sorted(l))) + self.assertGreater(sorter._spilled_bytes, 0) + last = sorter._spilled_bytes self.assertEquals(sorted(l, reverse=True), list(sorter.sorted(l, reverse=True))) + self.assertGreater(sorter._spilled_bytes, last) + last = sorter._spilled_bytes self.assertEquals(sorted(l, key=lambda x: -x), list(sorter.sorted(l, key=lambda x: -x))) + self.assertGreater(sorter._spilled_bytes, last) + last = sorter._spilled_bytes self.assertEquals(sorted(l, key=lambda x: -x, reverse=True), list(sorter.sorted(l, key=lambda x: -x, reverse=True))) + self.assertGreater(sorter._spilled_bytes, last) + + def test_external_sort_in_rdd(self): + conf = SparkConf().set("spark.python.worker.memory", "1m") + sc = SparkContext(conf=conf) + l = range(10240) + random.shuffle(l) + rdd = sc.parallelize(l, 10) + self.assertEquals(sorted(l), rdd.sortBy(lambda x: x).collect()) + sc.stop() class SerializationTestCase(unittest.TestCase): @@ -387,6 +404,110 @@ def test_zip_with_different_number_of_items(self): self.assertEquals(a.count(), b.count()) self.assertRaises(Exception, lambda: a.zip(b).count()) + def test_histogram(self): + # empty + rdd = self.sc.parallelize([]) + self.assertEquals([0], rdd.histogram([0, 10])[1]) + self.assertEquals([0, 0], rdd.histogram([0, 4, 10])[1]) + self.assertRaises(ValueError, lambda: rdd.histogram(1)) + + # out of range + rdd = self.sc.parallelize([10.01, -0.01]) + self.assertEquals([0], rdd.histogram([0, 10])[1]) + self.assertEquals([0, 0], rdd.histogram((0, 4, 10))[1]) + + # in range with one bucket + rdd = self.sc.parallelize(range(1, 5)) + self.assertEquals([4], rdd.histogram([0, 10])[1]) + self.assertEquals([3, 1], rdd.histogram([0, 4, 10])[1]) + + # in range with one bucket exact match + self.assertEquals([4], rdd.histogram([1, 4])[1]) + + # out of range with two buckets + rdd = self.sc.parallelize([10.01, -0.01]) + self.assertEquals([0, 0], rdd.histogram([0, 5, 10])[1]) + + # out of range with two uneven buckets + rdd = self.sc.parallelize([10.01, -0.01]) + self.assertEquals([0, 0], rdd.histogram([0, 4, 10])[1]) + + # in range with two buckets + rdd = self.sc.parallelize([1, 2, 3, 5, 6]) + self.assertEquals([3, 2], rdd.histogram([0, 5, 10])[1]) + + # in range with two bucket and None + rdd = self.sc.parallelize([1, 2, 3, 5, 6, None, float('nan')]) + self.assertEquals([3, 2], rdd.histogram([0, 5, 10])[1]) + + # in range with two uneven buckets + rdd = self.sc.parallelize([1, 2, 3, 5, 6]) + self.assertEquals([3, 2], rdd.histogram([0, 5, 11])[1]) + + # mixed range with two uneven buckets + rdd = self.sc.parallelize([-0.01, 0.0, 1, 2, 3, 5, 6, 11.0, 11.01]) + self.assertEquals([4, 3], rdd.histogram([0, 5, 11])[1]) + + # mixed range with four uneven buckets + rdd = self.sc.parallelize([-0.01, 0.0, 1, 2, 3, 5, 6, 11.01, 12.0, 199.0, 200.0, 200.1]) + self.assertEquals([4, 2, 1, 3], rdd.histogram([0.0, 5.0, 11.0, 12.0, 200.0])[1]) + + # mixed range with uneven buckets and NaN + rdd = self.sc.parallelize([-0.01, 0.0, 1, 2, 3, 5, 6, 11.01, 12.0, + 199.0, 200.0, 200.1, None, float('nan')]) + self.assertEquals([4, 2, 1, 3], rdd.histogram([0.0, 5.0, 11.0, 12.0, 200.0])[1]) + + # out of range with infinite buckets + rdd = self.sc.parallelize([10.01, -0.01, float('nan'), float("inf")]) + self.assertEquals([1, 2], rdd.histogram([float('-inf'), 0, float('inf')])[1]) + + # invalid buckets + self.assertRaises(ValueError, lambda: rdd.histogram([])) + self.assertRaises(ValueError, lambda: rdd.histogram([1])) + self.assertRaises(ValueError, lambda: rdd.histogram(0)) + self.assertRaises(TypeError, lambda: rdd.histogram({})) + + # without buckets + rdd = self.sc.parallelize(range(1, 5)) + self.assertEquals(([1, 4], [4]), rdd.histogram(1)) + + # without buckets single element + rdd = self.sc.parallelize([1]) + self.assertEquals(([1, 1], [1]), rdd.histogram(1)) + + # without bucket no range + rdd = self.sc.parallelize([1] * 4) + self.assertEquals(([1, 1], [4]), rdd.histogram(1)) + + # without buckets basic two + rdd = self.sc.parallelize(range(1, 5)) + self.assertEquals(([1, 2.5, 4], [2, 2]), rdd.histogram(2)) + + # without buckets with more requested than elements + rdd = self.sc.parallelize([1, 2]) + buckets = [1 + 0.2 * i for i in range(6)] + hist = [1, 0, 0, 0, 1] + self.assertEquals((buckets, hist), rdd.histogram(5)) + + # invalid RDDs + rdd = self.sc.parallelize([1, float('inf')]) + self.assertRaises(ValueError, lambda: rdd.histogram(2)) + rdd = self.sc.parallelize([float('nan')]) + self.assertRaises(ValueError, lambda: rdd.histogram(2)) + + # string + rdd = self.sc.parallelize(["ab", "ac", "b", "bd", "ef"], 2) + self.assertEquals([2, 2], rdd.histogram(["a", "b", "c"])[1]) + self.assertEquals((["ab", "ef"], [5]), rdd.histogram(1)) + self.assertRaises(TypeError, lambda: rdd.histogram(2)) + + # mixed RDD + rdd = self.sc.parallelize([1, 4, "ab", "ac", "b"], 2) + self.assertEquals([1, 1], rdd.histogram([0, 4, 10])[1]) + self.assertEquals([2, 1], rdd.histogram(["a", "b", "c"])[1]) + self.assertEquals(([1, "b"], [5]), rdd.histogram(1)) + self.assertRaises(TypeError, lambda: rdd.histogram(2)) + class TestIO(PySparkTestCase): diff --git a/sbin/start-history-server.sh b/sbin/start-history-server.sh index e30493da32a7a..580ab471b8a79 100755 --- a/sbin/start-history-server.sh +++ b/sbin/start-history-server.sh @@ -27,6 +27,9 @@ sbin=`dirname "$0"` sbin=`cd "$sbin"; pwd` +. "$sbin/spark-config.sh" +. "$SPARK_PREFIX/bin/load-spark-env.sh" + if [ $# != 0 ]; then echo "Using command line arguments for setting the log directory is deprecated. Please " echo "set the spark.history.fs.logDirectory configuration option instead." diff --git a/sbin/start-thriftserver.sh b/sbin/start-thriftserver.sh index 2c4452473ccbc..c519a77df4a14 100755 --- a/sbin/start-thriftserver.sh +++ b/sbin/start-thriftserver.sh @@ -27,6 +27,7 @@ set -o posix FWDIR="$(cd `dirname $0`/..; pwd)" CLASS="org.apache.spark.sql.hive.thriftserver.HiveThriftServer2" +CLASS_NOT_FOUND_EXIT_STATUS=1 function usage { echo "Usage: ./sbin/start-thriftserver [options] [thrift server options]" @@ -43,36 +44,22 @@ function usage { $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 +if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then usage exit 0 fi -THRIFT_SERVER_ARGS=() -SUBMISSION_ARGS=() +source $FWDIR/bin/utils.sh +SUBMIT_USAGE_FUNCTION=usage +gatherSparkSubmitOpts "$@" -while (($#)); do - case $1 in - --hiveconf) - ensure_arg_number $# 2 - THRIFT_SERVER_ARGS+=("$1"); shift - THRIFT_SERVER_ARGS+=("$1"); shift - ;; +"$FWDIR"/bin/spark-submit --class $CLASS "${SUBMISSION_OPTS[@]}" spark-internal "${APPLICATION_OPTS[@]}" +exit_status=$? - *) - SUBMISSION_ARGS+=("$1"); shift - ;; - esac -done +if [[ exit_status -eq CLASS_NOT_FOUND_EXIT_STATUS ]]; then + echo + echo "Failed to load Hive Thrift server main class $CLASS." + echo "You need to build Spark with -Phive." +fi -exec "$FWDIR"/bin/spark-submit --class $CLASS "${SUBMISSION_ARGS[@]}" spark-internal "${THRIFT_SERVER_ARGS[@]}" +exit $exit_status 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 0d26b52a84695..6b6b636cd96dc 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 @@ -62,11 +62,14 @@ object ScalaReflection { sys.error(s"Only Array[Byte] supported now, use Seq instead of $t") case t if t <:< typeOf[Seq[_]] => val TypeRef(_, _, Seq(elementType)) = t - Schema(ArrayType(schemaFor(elementType).dataType), nullable = true) + val Schema(dataType, nullable) = schemaFor(elementType) + Schema(ArrayType(dataType, containsNull = nullable), nullable = true) case t if t <:< typeOf[Map[_,_]] => val TypeRef(_, _, Seq(keyType, valueType)) = t - Schema(MapType(schemaFor(keyType).dataType, schemaFor(valueType).dataType), nullable = true) - case t if t <:< typeOf[String] => Schema(StringType, nullable = true) + val Schema(valueDataType, valueNullable) = schemaFor(valueType) + Schema(MapType(schemaFor(keyType).dataType, + valueDataType, valueContainsNull = valueNullable), nullable = true) + case t if t <:< typeOf[String] => Schema(StringType, nullable = true) case t if t <:< typeOf[Timestamp] => Schema(TimestampType, nullable = true) case t if t <:< typeOf[BigDecimal] => Schema(DecimalType, nullable = true) case t if t <:< typeOf[java.lang.Integer] => Schema(IntegerType, nullable = true) 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 c18d7858f0a43..4a9524074132e 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 @@ -132,7 +132,7 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool 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 requiredAttributes = AttributeSet(resolved.collect { case a: Attribute => a }) val missingInProject = requiredAttributes -- p.output if (missingInProject.nonEmpty) { @@ -152,8 +152,8 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool ) logDebug(s"Grouping expressions: $groupingRelation") - val resolved = unresolved.flatMap(groupingRelation.resolve).toSet - val missingInAggs = resolved -- a.outputSet + val resolved = unresolved.flatMap(groupingRelation.resolve) + val missingInAggs = resolved.filterNot(a.outputSet.contains) logDebug(s"Resolved: $resolved Missing in aggs: $missingInAggs") if (missingInAggs.nonEmpty) { // Add missing grouping exprs and then project them away after the sort. 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 a0e25775da6dd..a2c61c65487cb 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 @@ -66,7 +66,6 @@ case class UnresolvedFunction(name: String, children: Seq[Expression]) extends E override def dataType = throw new UnresolvedException(this, "dataType") override def foldable = throw new UnresolvedException(this, "foldable") override def nullable = throw new UnresolvedException(this, "nullable") - override def references = children.flatMap(_.references).toSet override lazy val resolved = false // Unresolved functions are transient at compile time and don't get evaluated during execution. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala new file mode 100644 index 0000000000000..c3a08bbdb6bc7 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.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.sql.catalyst.expressions + +protected class AttributeEquals(val a: Attribute) { + override def hashCode() = a.exprId.hashCode() + override def equals(other: Any) = other match { + case otherReference: AttributeEquals => a.exprId == otherReference.a.exprId + case otherAttribute => false + } +} + +object AttributeSet { + /** Constructs a new [[AttributeSet]] given a sequence of [[Attribute Attributes]]. */ + def apply(baseSet: Seq[Attribute]) = { + new AttributeSet(baseSet.map(new AttributeEquals(_)).toSet) + } +} + +/** + * A Set designed to hold [[AttributeReference]] objects, that performs equality checking using + * expression id instead of standard java equality. Using expression id means that these + * sets will correctly test for membership, even when the AttributeReferences in question differ + * cosmetically (e.g., the names have different capitalizations). + * + * Note that we do not override equality for Attribute references as it is really weird when + * `AttributeReference("a"...) == AttrributeReference("b", ...)`. This tactic leads to broken tests, + * and also makes doing transformations hard (we always try keep older trees instead of new ones + * when the transformation was a no-op). + */ +class AttributeSet private (val baseSet: Set[AttributeEquals]) + extends Traversable[Attribute] with Serializable { + + /** Returns true if the members of this AttributeSet and other are the same. */ + override def equals(other: Any) = other match { + case otherSet: AttributeSet => baseSet.map(_.a).forall(otherSet.contains) + case _ => false + } + + /** Returns true if this set contains an Attribute with the same expression id as `elem` */ + def contains(elem: NamedExpression): Boolean = + baseSet.contains(new AttributeEquals(elem.toAttribute)) + + /** Returns a new [[AttributeSet]] that contains `elem` in addition to the current elements. */ + def +(elem: Attribute): AttributeSet = // scalastyle:ignore + new AttributeSet(baseSet + new AttributeEquals(elem)) + + /** Returns a new [[AttributeSet]] that does not contain `elem`. */ + def -(elem: Attribute): AttributeSet = + new AttributeSet(baseSet - new AttributeEquals(elem)) + + /** Returns an iterator containing all of the attributes in the set. */ + def iterator: Iterator[Attribute] = baseSet.map(_.a).iterator + + /** + * Returns true if the [[Attribute Attributes]] in this set are a subset of the Attributes in + * `other`. + */ + def subsetOf(other: AttributeSet) = baseSet.subsetOf(other.baseSet) + + /** + * Returns a new [[AttributeSet]] that does not contain any of the [[Attribute Attributes]] found + * in `other`. + */ + def --(other: Traversable[NamedExpression]) = + new AttributeSet(baseSet -- other.map(a => new AttributeEquals(a.toAttribute))) + + /** + * Returns a new [[AttributeSet]] that contains all of the [[Attribute Attributes]] found + * in `other`. + */ + def ++(other: AttributeSet) = new AttributeSet(baseSet ++ other.baseSet) + + /** + * Returns a new [[AttributeSet]] contain only the [[Attribute Attributes]] where `f` evaluates to + * true. + */ + override def filter(f: Attribute => Boolean) = new AttributeSet(baseSet.filter(ae => f(ae.a))) + + /** + * Returns a new [[AttributeSet]] that only contains [[Attribute Attributes]] that are found in + * `this` and `other`. + */ + def intersect(other: AttributeSet) = new AttributeSet(baseSet.intersect(other.baseSet)) + + override def foreach[U](f: (Attribute) => U): Unit = baseSet.map(_.a).foreach(f) + + // We must force toSeq to not be strict otherwise we end up with a [[Stream]] that captures all + // sorts of things in its closure. + override def toSeq: Seq[Attribute] = baseSet.map(_.a).toArray.toSeq +} 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 0913f15888780..54c6baf1af3bf 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 @@ -32,8 +32,6 @@ case class BoundReference(ordinal: Int, dataType: DataType, nullable: Boolean) type EvaluatedType = Any - override def references = Set.empty - override def toString = s"input[$ordinal]" override def eval(input: Row): Any = input(ordinal) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index ba62dabe3dd6a..70507e7ee2be8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -41,7 +41,7 @@ abstract class Expression extends TreeNode[Expression] { */ def foldable: Boolean = false def nullable: Boolean - def references: Set[Attribute] + def references: AttributeSet = AttributeSet(children.flatMap(_.references.iterator)) /** Returns the result of evaluating this expression on a given input Row */ def eval(input: Row = null): EvaluatedType @@ -230,8 +230,6 @@ abstract class BinaryExpression extends Expression with trees.BinaryNode[Express override def foldable = left.foldable && right.foldable - override def references = left.references ++ right.references - override def toString = s"($left $symbol $right)" } @@ -242,5 +240,5 @@ abstract class LeafExpression extends Expression with trees.LeafNode[Expression] abstract class UnaryExpression extends Expression with trees.UnaryNode[Expression] { self: Product => - override def references = child.references + } 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 8fc5896974438..ef1d12531f109 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 @@ -27,7 +27,8 @@ class InterpretedProjection(expressions: Seq[Expression]) extends Projection { def this(expressions: Seq[Expression], inputSchema: Seq[Attribute]) = this(expressions.map(BindReferences.bindReference(_, inputSchema))) - protected val exprArray = expressions.toArray + // null check is required for when Kryo invokes the no-arg constructor. + protected val exprArray = if (expressions != null) expressions.toArray else null def apply(input: Row): Row = { val outputArray = new Array[Any](exprArray.length) @@ -109,7 +110,346 @@ class JoinedRow extends Row { def apply(i: Int) = if (i < row1.size) row1(i) else row2(i - row1.size) - def isNullAt(i: Int) = apply(i) == null + def isNullAt(i: Int) = + if (i < row1.size) row1.isNullAt(i) else row2.isNullAt(i - row1.size) + + def getInt(i: Int): Int = + if (i < row1.size) row1.getInt(i) else row2.getInt(i - row1.size) + + def getLong(i: Int): Long = + if (i < row1.size) row1.getLong(i) else row2.getLong(i - row1.size) + + def getDouble(i: Int): Double = + if (i < row1.size) row1.getDouble(i) else row2.getDouble(i - row1.size) + + def getBoolean(i: Int): Boolean = + if (i < row1.size) row1.getBoolean(i) else row2.getBoolean(i - row1.size) + + def getShort(i: Int): Short = + if (i < row1.size) row1.getShort(i) else row2.getShort(i - row1.size) + + def getByte(i: Int): Byte = + if (i < row1.size) row1.getByte(i) else row2.getByte(i - row1.size) + + def getFloat(i: Int): Float = + if (i < row1.size) row1.getFloat(i) else row2.getFloat(i - row1.size) + + def getString(i: Int): String = + if (i < row1.size) row1.getString(i) else row2.getString(i - row1.size) + + def copy() = { + val totalSize = row1.size + row2.size + val copiedValues = new Array[Any](totalSize) + var i = 0 + while(i < totalSize) { + copiedValues(i) = apply(i) + i += 1 + } + new GenericRow(copiedValues) + } + + override def toString() = { + val row = (if (row1 != null) row1 else Seq[Any]()) ++ (if (row2 != null) row2 else Seq[Any]()) + s"[${row.mkString(",")}]" + } +} + +/** + * JIT HACK: Replace with macros + * The `JoinedRow` class is used in many performance critical situation. Unfortunately, since there + * are multiple different types of `Rows` that could be stored as `row1` and `row2` most of the + * calls in the critical path are polymorphic. By creating special versions of this class that are + * used in only a single location of the code, we increase the chance that only a single type of + * Row will be referenced, increasing the opportunity for the JIT to play tricks. This sounds + * crazy but in benchmarks it had noticeable effects. + */ +class JoinedRow2 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 + row2 = r2 + this + } + + /** Updates this JoinedRow by updating its left base row. Returns itself. */ + def withLeft(newLeft: Row): Row = { + row1 = newLeft + this + } + + /** Updates this JoinedRow by updating its right base row. Returns itself. */ + def withRight(newRight: Row): Row = { + row2 = newRight + this + } + + def iterator = row1.iterator ++ row2.iterator + + def length = row1.length + row2.length + + def apply(i: Int) = + if (i < row1.size) row1(i) else row2(i - row1.size) + + def isNullAt(i: Int) = + if (i < row1.size) row1.isNullAt(i) else row2.isNullAt(i - row1.size) + + def getInt(i: Int): Int = + if (i < row1.size) row1.getInt(i) else row2.getInt(i - row1.size) + + def getLong(i: Int): Long = + if (i < row1.size) row1.getLong(i) else row2.getLong(i - row1.size) + + def getDouble(i: Int): Double = + if (i < row1.size) row1.getDouble(i) else row2.getDouble(i - row1.size) + + def getBoolean(i: Int): Boolean = + if (i < row1.size) row1.getBoolean(i) else row2.getBoolean(i - row1.size) + + def getShort(i: Int): Short = + if (i < row1.size) row1.getShort(i) else row2.getShort(i - row1.size) + + def getByte(i: Int): Byte = + if (i < row1.size) row1.getByte(i) else row2.getByte(i - row1.size) + + def getFloat(i: Int): Float = + if (i < row1.size) row1.getFloat(i) else row2.getFloat(i - row1.size) + + def getString(i: Int): String = + if (i < row1.size) row1.getString(i) else row2.getString(i - row1.size) + + def copy() = { + val totalSize = row1.size + row2.size + val copiedValues = new Array[Any](totalSize) + var i = 0 + while(i < totalSize) { + copiedValues(i) = apply(i) + i += 1 + } + new GenericRow(copiedValues) + } + + override def toString() = { + val row = (if (row1 != null) row1 else Seq[Any]()) ++ (if (row2 != null) row2 else Seq[Any]()) + s"[${row.mkString(",")}]" + } +} + +/** + * JIT HACK: Replace with macros + */ +class JoinedRow3 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 + row2 = r2 + this + } + + /** Updates this JoinedRow by updating its left base row. Returns itself. */ + def withLeft(newLeft: Row): Row = { + row1 = newLeft + this + } + + /** Updates this JoinedRow by updating its right base row. Returns itself. */ + def withRight(newRight: Row): Row = { + row2 = newRight + this + } + + def iterator = row1.iterator ++ row2.iterator + + def length = row1.length + row2.length + + def apply(i: Int) = + if (i < row1.size) row1(i) else row2(i - row1.size) + + def isNullAt(i: Int) = + if (i < row1.size) row1.isNullAt(i) else row2.isNullAt(i - row1.size) + + def getInt(i: Int): Int = + if (i < row1.size) row1.getInt(i) else row2.getInt(i - row1.size) + + def getLong(i: Int): Long = + if (i < row1.size) row1.getLong(i) else row2.getLong(i - row1.size) + + def getDouble(i: Int): Double = + if (i < row1.size) row1.getDouble(i) else row2.getDouble(i - row1.size) + + def getBoolean(i: Int): Boolean = + if (i < row1.size) row1.getBoolean(i) else row2.getBoolean(i - row1.size) + + def getShort(i: Int): Short = + if (i < row1.size) row1.getShort(i) else row2.getShort(i - row1.size) + + def getByte(i: Int): Byte = + if (i < row1.size) row1.getByte(i) else row2.getByte(i - row1.size) + + def getFloat(i: Int): Float = + if (i < row1.size) row1.getFloat(i) else row2.getFloat(i - row1.size) + + def getString(i: Int): String = + if (i < row1.size) row1.getString(i) else row2.getString(i - row1.size) + + def copy() = { + val totalSize = row1.size + row2.size + val copiedValues = new Array[Any](totalSize) + var i = 0 + while(i < totalSize) { + copiedValues(i) = apply(i) + i += 1 + } + new GenericRow(copiedValues) + } + + override def toString() = { + val row = (if (row1 != null) row1 else Seq[Any]()) ++ (if (row2 != null) row2 else Seq[Any]()) + s"[${row.mkString(",")}]" + } +} + +/** + * JIT HACK: Replace with macros + */ +class JoinedRow4 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 + row2 = r2 + this + } + + /** Updates this JoinedRow by updating its left base row. Returns itself. */ + def withLeft(newLeft: Row): Row = { + row1 = newLeft + this + } + + /** Updates this JoinedRow by updating its right base row. Returns itself. */ + def withRight(newRight: Row): Row = { + row2 = newRight + this + } + + def iterator = row1.iterator ++ row2.iterator + + def length = row1.length + row2.length + + def apply(i: Int) = + if (i < row1.size) row1(i) else row2(i - row1.size) + + def isNullAt(i: Int) = + if (i < row1.size) row1.isNullAt(i) else row2.isNullAt(i - row1.size) + + def getInt(i: Int): Int = + if (i < row1.size) row1.getInt(i) else row2.getInt(i - row1.size) + + def getLong(i: Int): Long = + if (i < row1.size) row1.getLong(i) else row2.getLong(i - row1.size) + + def getDouble(i: Int): Double = + if (i < row1.size) row1.getDouble(i) else row2.getDouble(i - row1.size) + + def getBoolean(i: Int): Boolean = + if (i < row1.size) row1.getBoolean(i) else row2.getBoolean(i - row1.size) + + def getShort(i: Int): Short = + if (i < row1.size) row1.getShort(i) else row2.getShort(i - row1.size) + + def getByte(i: Int): Byte = + if (i < row1.size) row1.getByte(i) else row2.getByte(i - row1.size) + + def getFloat(i: Int): Float = + if (i < row1.size) row1.getFloat(i) else row2.getFloat(i - row1.size) + + def getString(i: Int): String = + if (i < row1.size) row1.getString(i) else row2.getString(i - row1.size) + + def copy() = { + val totalSize = row1.size + row2.size + val copiedValues = new Array[Any](totalSize) + var i = 0 + while(i < totalSize) { + copiedValues(i) = apply(i) + i += 1 + } + new GenericRow(copiedValues) + } + + override def toString() = { + val row = (if (row1 != null) row1 else Seq[Any]()) ++ (if (row2 != null) row2 else Seq[Any]()) + s"[${row.mkString(",")}]" + } +} + +/** + * JIT HACK: Replace with macros + */ +class JoinedRow5 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 + row2 = r2 + this + } + + /** Updates this JoinedRow by updating its left base row. Returns itself. */ + def withLeft(newLeft: Row): Row = { + row1 = newLeft + this + } + + /** Updates this JoinedRow by updating its right base row. Returns itself. */ + def withRight(newRight: Row): Row = { + row2 = newRight + this + } + + def iterator = row1.iterator ++ row2.iterator + + def length = row1.length + row2.length + + def apply(i: Int) = + if (i < row1.size) row1(i) else row2(i - row1.size) + + def isNullAt(i: Int) = + if (i < row1.size) row1.isNullAt(i) else row2.isNullAt(i - row1.size) def getInt(i: Int): Int = if (i < row1.size) row1.getInt(i) else row2.getInt(i - row1.size) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala index 38f836f0a1a0e..851db95b9177e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala @@ -24,7 +24,6 @@ import org.apache.spark.sql.catalyst.types.DoubleType case object Rand extends LeafExpression { override def dataType = DoubleType override def nullable = false - override def references = Set.empty private[this] lazy val rand = new Random 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 c9a63e201ef60..d68a4fabeac77 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 @@ -127,7 +127,7 @@ object EmptyRow extends Row { * the array is not copied, and thus could technically be mutated after creation, this is not * allowed. */ -class GenericRow(protected[catalyst] val values: Array[Any]) extends Row { +class GenericRow(protected[sql] val values: Array[Any]) extends Row { /** No-arg constructor for serialization. */ def this() = this(null) 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 95633dd0c9870..63ac2a608b6ff 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 @@ -24,7 +24,6 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi type EvaluatedType = Any - def references = children.flatMap(_.references).toSet def nullable = true /** This method has been generated by this script diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala index d2b7685e73065..d00b2ac09745c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala @@ -31,7 +31,6 @@ case object Descending extends SortDirection case class SortOrder(child: Expression, direction: SortDirection) extends Expression with trees.UnaryNode[Expression] { - override def references = child.references override def dataType = child.dataType override def nullable = child.nullable diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificRow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificRow.scala new file mode 100644 index 0000000000000..75ea0e8459df8 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificRow.scala @@ -0,0 +1,307 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.sql.catalyst.types._ + +/** + * A parent class for mutable container objects that are reused when the values are changed, + * resulting in less garbage. These values are held by a [[SpecificMutableRow]]. + * + * The following code was roughly used to generate these objects: + * {{{ + * val types = "Int,Float,Boolean,Double,Short,Long,Byte,Any".split(",") + * types.map {tpe => + * s""" + * final class Mutable$tpe extends MutableValue { + * var value: $tpe = 0 + * def boxed = if (isNull) null else value + * def update(v: Any) = value = { + * isNull = false + * v.asInstanceOf[$tpe] + * } + * def copy() = { + * val newCopy = new Mutable$tpe + * newCopy.isNull = isNull + * newCopy.value = value + * newCopy.asInstanceOf[this.type] + * } + * }""" + * }.foreach(println) + * + * types.map { tpe => + * s""" + * override def set$tpe(ordinal: Int, value: $tpe): Unit = { + * val currentValue = values(ordinal).asInstanceOf[Mutable$tpe] + * currentValue.isNull = false + * currentValue.value = value + * } + * + * override def get$tpe(i: Int): $tpe = { + * values(i).asInstanceOf[Mutable$tpe].value + * }""" + * }.foreach(println) + * }}} + */ +abstract class MutableValue extends Serializable { + var isNull: Boolean = true + def boxed: Any + def update(v: Any) + def copy(): this.type +} + +final class MutableInt extends MutableValue { + var value: Int = 0 + def boxed = if (isNull) null else value + def update(v: Any) = value = { + isNull = false + v.asInstanceOf[Int] + } + def copy() = { + val newCopy = new MutableInt + newCopy.isNull = isNull + newCopy.value = value + newCopy.asInstanceOf[this.type] + } +} + +final class MutableFloat extends MutableValue { + var value: Float = 0 + def boxed = if (isNull) null else value + def update(v: Any) = value = { + isNull = false + v.asInstanceOf[Float] + } + def copy() = { + val newCopy = new MutableFloat + newCopy.isNull = isNull + newCopy.value = value + newCopy.asInstanceOf[this.type] + } +} + +final class MutableBoolean extends MutableValue { + var value: Boolean = false + def boxed = if (isNull) null else value + def update(v: Any) = value = { + isNull = false + v.asInstanceOf[Boolean] + } + def copy() = { + val newCopy = new MutableBoolean + newCopy.isNull = isNull + newCopy.value = value + newCopy.asInstanceOf[this.type] + } +} + +final class MutableDouble extends MutableValue { + var value: Double = 0 + def boxed = if (isNull) null else value + def update(v: Any) = value = { + isNull = false + v.asInstanceOf[Double] + } + def copy() = { + val newCopy = new MutableDouble + newCopy.isNull = isNull + newCopy.value = value + newCopy.asInstanceOf[this.type] + } +} + +final class MutableShort extends MutableValue { + var value: Short = 0 + def boxed = if (isNull) null else value + def update(v: Any) = value = { + isNull = false + v.asInstanceOf[Short] + } + def copy() = { + val newCopy = new MutableShort + newCopy.isNull = isNull + newCopy.value = value + newCopy.asInstanceOf[this.type] + } +} + +final class MutableLong extends MutableValue { + var value: Long = 0 + def boxed = if (isNull) null else value + def update(v: Any) = value = { + isNull = false + v.asInstanceOf[Long] + } + def copy() = { + val newCopy = new MutableLong + newCopy.isNull = isNull + newCopy.value = value + newCopy.asInstanceOf[this.type] + } +} + +final class MutableByte extends MutableValue { + var value: Byte = 0 + def boxed = if (isNull) null else value + def update(v: Any) = value = { + isNull = false + v.asInstanceOf[Byte] + } + def copy() = { + val newCopy = new MutableByte + newCopy.isNull = isNull + newCopy.value = value + newCopy.asInstanceOf[this.type] + } +} + +final class MutableAny extends MutableValue { + var value: Any = 0 + def boxed = if (isNull) null else value + def update(v: Any) = value = { + isNull = false + v.asInstanceOf[Any] + } + def copy() = { + val newCopy = new MutableAny + newCopy.isNull = isNull + newCopy.value = value + newCopy.asInstanceOf[this.type] + } +} + +/** + * A row type that holds an array specialized container objects, of type [[MutableValue]], chosen + * based on the dataTypes of each column. The intent is to decrease garbage when modifying the + * values of primitive columns. + */ +final class SpecificMutableRow(val values: Array[MutableValue]) extends MutableRow { + + def this(dataTypes: Seq[DataType]) = + this( + dataTypes.map { + case IntegerType => new MutableInt + case ByteType => new MutableByte + case FloatType => new MutableFloat + case ShortType => new MutableShort + case DoubleType => new MutableDouble + case BooleanType => new MutableBoolean + case LongType => new MutableLong + case _ => new MutableAny + }.toArray) + + def this() = this(Seq.empty) + + override def length: Int = values.length + + override def setNullAt(i: Int): Unit = { + values(i).isNull = true + } + + override def apply(i: Int): Any = values(i).boxed + + override def isNullAt(i: Int): Boolean = values(i).isNull + + override def copy(): Row = { + val newValues = new Array[MutableValue](values.length) + var i = 0 + while (i < values.length) { + newValues(i) = values(i).copy() + i += 1 + } + new SpecificMutableRow(newValues) + } + + override def update(ordinal: Int, value: Any): Unit = values(ordinal).update(value) + + override def iterator: Iterator[Any] = values.map(_.boxed).iterator + + def setString(ordinal: Int, value: String) = update(ordinal, value) + + def getString(ordinal: Int) = apply(ordinal).asInstanceOf[String] + + override def setInt(ordinal: Int, value: Int): Unit = { + val currentValue = values(ordinal).asInstanceOf[MutableInt] + currentValue.isNull = false + currentValue.value = value + } + + override def getInt(i: Int): Int = { + values(i).asInstanceOf[MutableInt].value + } + + override def setFloat(ordinal: Int, value: Float): Unit = { + val currentValue = values(ordinal).asInstanceOf[MutableFloat] + currentValue.isNull = false + currentValue.value = value + } + + override def getFloat(i: Int): Float = { + values(i).asInstanceOf[MutableFloat].value + } + + override def setBoolean(ordinal: Int, value: Boolean): Unit = { + val currentValue = values(ordinal).asInstanceOf[MutableBoolean] + currentValue.isNull = false + currentValue.value = value + } + + override def getBoolean(i: Int): Boolean = { + values(i).asInstanceOf[MutableBoolean].value + } + + override def setDouble(ordinal: Int, value: Double): Unit = { + val currentValue = values(ordinal).asInstanceOf[MutableDouble] + currentValue.isNull = false + currentValue.value = value + } + + override def getDouble(i: Int): Double = { + values(i).asInstanceOf[MutableDouble].value + } + + override def setShort(ordinal: Int, value: Short): Unit = { + val currentValue = values(ordinal).asInstanceOf[MutableShort] + currentValue.isNull = false + currentValue.value = value + } + + override def getShort(i: Int): Short = { + values(i).asInstanceOf[MutableShort].value + } + + override def setLong(ordinal: Int, value: Long): Unit = { + val currentValue = values(ordinal).asInstanceOf[MutableLong] + currentValue.isNull = false + currentValue.value = value + } + + override def getLong(i: Int): Long = { + values(i).asInstanceOf[MutableLong].value + } + + override def setByte(ordinal: Int, value: Byte): Unit = { + val currentValue = values(ordinal).asInstanceOf[MutableByte] + currentValue.isNull = false + currentValue.value = value + } + + override def getByte(i: Int): Byte = { + values(i).asInstanceOf[MutableByte].value + } +} 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 eb8898900d6a5..1eb55715794a7 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 @@ -35,7 +35,7 @@ case class WrapDynamic(children: Seq[Attribute]) extends Expression { type EvaluatedType = DynamicRow def nullable = false - def references = children.toSet + def dataType = DynamicType override def eval(input: Row): DynamicRow = input match { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala index 01947273b6ccc..dbc0c2965a805 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala @@ -22,6 +22,7 @@ import com.clearspring.analytics.stream.cardinality.HyperLogLog import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.catalyst.trees import org.apache.spark.sql.catalyst.errors.TreeNodeException +import org.apache.spark.util.collection.OpenHashSet abstract class AggregateExpression extends Expression { self: Product => @@ -77,7 +78,7 @@ abstract class AggregateFunction /** Base should return the generic aggregate expression that this function is computing */ val base: AggregateExpression - override def references = base.references + override def nullable = base.nullable override def dataType = base.dataType @@ -88,7 +89,7 @@ abstract class AggregateFunction } case class Min(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { - override def references = child.references + override def nullable = true override def dataType = child.dataType override def toString = s"MIN($child)" @@ -118,7 +119,7 @@ case class MinFunction(expr: Expression, base: AggregateExpression) extends Aggr } case class Max(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { - override def references = child.references + override def nullable = true override def dataType = child.dataType override def toString = s"MAX($child)" @@ -148,7 +149,7 @@ case class MaxFunction(expr: Expression, base: AggregateExpression) extends Aggr } case class Count(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { - override def references = child.references + override def nullable = false override def dataType = LongType override def toString = s"COUNT($child)" @@ -161,18 +162,91 @@ case class Count(child: Expression) extends PartialAggregate with trees.UnaryNod override def newInstance() = new CountFunction(child, this) } -case class CountDistinct(expressions: Seq[Expression]) extends AggregateExpression { +case class CountDistinct(expressions: Seq[Expression]) extends PartialAggregate { + def this() = this(null) + override def children = expressions - override def references = expressions.flatMap(_.references).toSet + override def nullable = false override def dataType = LongType override def toString = s"COUNT(DISTINCT ${expressions.mkString(",")})" override def newInstance() = new CountDistinctFunction(expressions, this) + + override def asPartial = { + val partialSet = Alias(CollectHashSet(expressions), "partialSets")() + SplitEvaluation( + CombineSetsAndCount(partialSet.toAttribute), + partialSet :: Nil) + } +} + +case class CollectHashSet(expressions: Seq[Expression]) extends AggregateExpression { + def this() = this(null) + + override def children = expressions + override def nullable = false + override def dataType = ArrayType(expressions.head.dataType) + override def toString = s"AddToHashSet(${expressions.mkString(",")})" + override def newInstance() = new CollectHashSetFunction(expressions, this) +} + +case class CollectHashSetFunction( + @transient expr: Seq[Expression], + @transient base: AggregateExpression) + extends AggregateFunction { + + def this() = this(null, null) // Required for serialization. + + val seen = new OpenHashSet[Any]() + + @transient + val distinctValue = new InterpretedProjection(expr) + + override def update(input: Row): Unit = { + val evaluatedExpr = distinctValue(input) + if (!evaluatedExpr.anyNull) { + seen.add(evaluatedExpr) + } + } + + override def eval(input: Row): Any = { + seen + } +} + +case class CombineSetsAndCount(inputSet: Expression) extends AggregateExpression { + def this() = this(null) + + override def children = inputSet :: Nil + override def nullable = false + override def dataType = LongType + override def toString = s"CombineAndCount($inputSet)" + override def newInstance() = new CombineSetsAndCountFunction(inputSet, this) +} + +case class CombineSetsAndCountFunction( + @transient inputSet: Expression, + @transient base: AggregateExpression) + extends AggregateFunction { + + def this() = this(null, null) // Required for serialization. + + val seen = new OpenHashSet[Any]() + + override def update(input: Row): Unit = { + val inputSetEval = inputSet.eval(input).asInstanceOf[OpenHashSet[Any]] + val inputIterator = inputSetEval.iterator + while (inputIterator.hasNext) { + seen.add(inputIterator.next) + } + } + + override def eval(input: Row): Any = seen.size.toLong } case class ApproxCountDistinctPartition(child: Expression, relativeSD: Double) extends AggregateExpression with trees.UnaryNode[Expression] { - override def references = child.references + override def nullable = false override def dataType = child.dataType override def toString = s"APPROXIMATE COUNT(DISTINCT $child)" @@ -181,7 +255,7 @@ case class ApproxCountDistinctPartition(child: Expression, relativeSD: Double) case class ApproxCountDistinctMerge(child: Expression, relativeSD: Double) extends AggregateExpression with trees.UnaryNode[Expression] { - override def references = child.references + override def nullable = false override def dataType = LongType override def toString = s"APPROXIMATE COUNT(DISTINCT $child)" @@ -190,7 +264,7 @@ case class ApproxCountDistinctMerge(child: Expression, relativeSD: Double) case class ApproxCountDistinct(child: Expression, relativeSD: Double = 0.05) extends PartialAggregate with trees.UnaryNode[Expression] { - override def references = child.references + override def nullable = false override def dataType = LongType override def toString = s"APPROXIMATE COUNT(DISTINCT $child)" @@ -208,7 +282,7 @@ case class ApproxCountDistinct(child: Expression, relativeSD: Double = 0.05) } case class Average(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { - override def references = child.references + override def nullable = false override def dataType = DoubleType override def toString = s"AVG($child)" @@ -228,7 +302,7 @@ case class Average(child: Expression) extends PartialAggregate with trees.UnaryN } case class Sum(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { - override def references = child.references + override def nullable = false override def dataType = child.dataType override def toString = s"SUM($child)" @@ -246,7 +320,7 @@ case class Sum(child: Expression) extends PartialAggregate with trees.UnaryNode[ case class SumDistinct(child: Expression) extends AggregateExpression with trees.UnaryNode[Expression] { - override def references = child.references + override def nullable = false override def dataType = child.dataType override def toString = s"SUM(DISTINCT $child)" @@ -255,7 +329,6 @@ case class SumDistinct(child: Expression) } case class First(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { - override def references = child.references override def nullable = true override def dataType = child.dataType override def toString = s"FIRST($child)" @@ -379,17 +452,22 @@ case class SumDistinctFunction(expr: Expression, base: AggregateExpression) seen.reduceLeft(base.dataType.asInstanceOf[NumericType].numeric.asInstanceOf[Numeric[Any]].plus) } -case class CountDistinctFunction(expr: Seq[Expression], base: AggregateExpression) +case class CountDistinctFunction( + @transient expr: Seq[Expression], + @transient base: AggregateExpression) extends AggregateFunction { def this() = this(null, null) // Required for serialization. - val seen = new scala.collection.mutable.HashSet[Any]() + val seen = new OpenHashSet[Any]() + + @transient + val distinctValue = new InterpretedProjection(expr) override def update(input: Row): Unit = { - val evaluatedExpr = expr.map(_.eval(input)) - if (evaluatedExpr.map(_ != null).reduceLeft(_ && _)) { - seen += evaluatedExpr + val evaluatedExpr = distinctValue(input) + if (!evaluatedExpr.anyNull) { + seen.add(evaluatedExpr) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala index c79c1847cedf5..aae86a3628be1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala @@ -85,3 +85,34 @@ case class Remainder(left: Expression, right: Expression) extends BinaryArithmet override def eval(input: Row): Any = i2(input, left, right, _.rem(_, _)) } + +case class MaxOf(left: Expression, right: Expression) extends Expression { + type EvaluatedType = Any + + override def foldable = left.foldable && right.foldable + + override def nullable = left.nullable && right.nullable + + override def children = left :: right :: Nil + + override def dataType = left.dataType + + override def eval(input: Row): Any = { + val leftEval = left.eval(input) + val rightEval = right.eval(input) + if (leftEval == null) { + rightEval + } else if (rightEval == null) { + leftEval + } else { + val numeric = left.dataType.asInstanceOf[NumericType].numeric.asInstanceOf[Numeric[Any]] + if (numeric.compare(leftEval, rightEval) < 0) { + rightEval + } else { + leftEval + } + } + } + + override def toString = s"MaxOf($left, $right)" +} 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 index de2d67ce82ff1..5a3f013c34579 100644 --- 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 @@ -26,6 +26,10 @@ import org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.types._ +// These classes are here to avoid issues with serialization and integration with quasiquotes. +class IntegerHashSet extends org.apache.spark.util.collection.OpenHashSet[Int] +class LongHashSet extends org.apache.spark.util.collection.OpenHashSet[Long] + /** * 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 @@ -50,6 +54,11 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin private val curId = new java.util.concurrent.atomic.AtomicInteger() private val javaSeparator = "$" + /** + * Can be flipped on manually in the console to add (expensive) expression evaluation trace code. + */ + var debugLogging = false + /** * Generates a class for a given input expression. Called when there is not cached code * already available. @@ -71,7 +80,8 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin * 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 + * automatically, in order to constrain its memory footprint. Note that this cache does not use + * weak keys/values and thus does not respond to memory pressure. */ protected val cache = CacheBuilder.newBuilder() .maximumSize(1000) @@ -403,6 +413,78 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin $primitiveTerm = ${falseEval.primitiveTerm} } """.children + + case NewSet(elementType) => + q""" + val $nullTerm = false + val $primitiveTerm = new ${hashSetForType(elementType)}() + """.children + + case AddItemToSet(item, set) => + val itemEval = expressionEvaluator(item) + val setEval = expressionEvaluator(set) + + val ArrayType(elementType, _) = set.dataType + + itemEval.code ++ setEval.code ++ + q""" + if (!${itemEval.nullTerm}) { + ${setEval.primitiveTerm} + .asInstanceOf[${hashSetForType(elementType)}] + .add(${itemEval.primitiveTerm}) + } + + val $nullTerm = false + val $primitiveTerm = ${setEval.primitiveTerm} + """.children + + case CombineSets(left, right) => + val leftEval = expressionEvaluator(left) + val rightEval = expressionEvaluator(right) + + val ArrayType(elementType, _) = left.dataType + + leftEval.code ++ rightEval.code ++ + q""" + val $nullTerm = false + var $primitiveTerm: ${hashSetForType(elementType)} = null + + { + val leftSet = ${leftEval.primitiveTerm}.asInstanceOf[${hashSetForType(elementType)}] + val rightSet = ${rightEval.primitiveTerm}.asInstanceOf[${hashSetForType(elementType)}] + val iterator = rightSet.iterator + while (iterator.hasNext) { + leftSet.add(iterator.next()) + } + $primitiveTerm = leftSet + } + """.children + + case MaxOf(e1, e2) => + val eval1 = expressionEvaluator(e1) + val eval2 = expressionEvaluator(e2) + + eval1.code ++ eval2.code ++ + q""" + var $nullTerm = false + var $primitiveTerm: ${termForType(e1.dataType)} = ${defaultPrimitive(e1.dataType)} + + if (${eval1.nullTerm}) { + $nullTerm = ${eval2.nullTerm} + $primitiveTerm = ${eval2.primitiveTerm} + } else if (${eval2.nullTerm}) { + $nullTerm = ${eval1.nullTerm} + $primitiveTerm = ${eval1.primitiveTerm} + } else { + $nullTerm = false + if (${eval1.primitiveTerm} > ${eval2.primitiveTerm}) { + $primitiveTerm = ${eval1.primitiveTerm} + } else { + $primitiveTerm = ${eval2.primitiveTerm} + } + } + """.children + } // If there was no match in the partial function above, we fall back on calling the interpreted @@ -420,7 +502,7 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin // Only inject debugging code if debugging is turned on. val debugCode = - if (log.isDebugEnabled) { + if (debugLogging) { val localLogger = log val localLoggerTree = reify { localLogger } q""" @@ -454,6 +536,13 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin protected def accessorForType(dt: DataType) = newTermName(s"get${primitiveForType(dt)}") protected def mutatorForType(dt: DataType) = newTermName(s"set${primitiveForType(dt)}") + protected def hashSetForType(dt: DataType) = dt match { + case IntegerType => typeOf[IntegerHashSet] + case LongType => typeOf[LongHashSet] + case unsupportedType => + sys.error(s"Code generation not support for hashset of type $unsupportedType") + } + protected def primitiveForType(dt: DataType) = dt match { case IntegerType => "Int" case LongType => "Long" 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 index 77fa02c13de30..7871a62620478 100644 --- 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 @@ -69,8 +69,10 @@ object GenerateProjection extends CodeGenerator[Seq[Expression], Projection] { ..${evaluatedExpression.code} if(${evaluatedExpression.nullTerm}) setNullAt($iLit) - else + else { + nullBits($iLit) = false $elementName = ${evaluatedExpression.primitiveTerm} + } } """.children : Seq[Tree] } @@ -106,9 +108,10 @@ object GenerateProjection extends CodeGenerator[Seq[Expression], Projection] { if(value == null) { setNullAt(i) } else { + nullBits(i) = false $elementName = value.asInstanceOf[${termForType(e.dataType)}] - return } + return }""" } q"final def update(i: Int, value: Any): Unit = { ..$cases; $accessorFailure }" @@ -137,7 +140,7 @@ object GenerateProjection extends CodeGenerator[Seq[Expression], Projection] { 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 + q"if(i == $i) { nullBits($i) = false; $elementName = value; return }" :: Nil case _ => Nil } 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 c1154eb81c319..dafd745ec96c6 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 @@ -31,7 +31,7 @@ case class GetItem(child: Expression, ordinal: Expression) extends Expression { /** `Null` is returned for invalid ordinals. */ override def nullable = true 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 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 e99c5b452d183..9c865254e0be9 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 @@ -47,8 +47,6 @@ abstract class Generator extends Expression { override def nullable = false - override def references = children.flatMap(_.references).toSet - /** * Should be overridden by specific generators. Called only once for each instance to ensure * that rule application does not change the output schema of a generator. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala index e15e16d633365..a8c2396d62632 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala @@ -52,7 +52,7 @@ case class Literal(value: Any, dataType: DataType) extends LeafExpression { override def foldable = true def nullable = value == null - def references = Set.empty + override def toString = if (value != null) value.toString else "null" @@ -66,8 +66,6 @@ case class MutableLiteral(var value: Any, nullable: Boolean = true) extends Leaf val dataType = Literal(value).dataType - def references = Set.empty - def update(expression: Expression, input: Row) = { value = expression.eval(input) } 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 02d04762629f5..7c4b9d4847e26 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 @@ -62,7 +62,7 @@ abstract class Attribute extends NamedExpression { def toAttribute = this def newInstance: Attribute - override def references = Set(this) + } /** @@ -85,7 +85,7 @@ case class Alias(child: Expression, name: String) override def dataType = child.dataType override def nullable = child.nullable - override def references = child.references + override def toAttribute = { if (resolved) { @@ -116,6 +116,8 @@ case class AttributeReference(name: String, dataType: DataType, nullable: Boolea (val exprId: ExprId = NamedExpression.newExprId, val qualifiers: Seq[String] = Nil) extends Attribute with trees.LeafNode[Expression] { + override def references = AttributeSet(this :: Nil) + override def equals(other: Any) = other match { case ar: AttributeReference => exprId == ar.exprId && dataType == ar.dataType case _ => false 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 e88c5d4fa178a..086d0a3e073e5 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 @@ -26,7 +26,6 @@ case class Coalesce(children: Seq[Expression]) extends Expression { /** Coalesce is nullable if all of its children are nullable, or if it has no children. */ def nullable = !children.exists(!_.nullable) - def references = children.flatMap(_.references).toSet // Coalesce is foldable if all children are foldable. override def foldable = !children.exists(!_.foldable) @@ -53,7 +52,6 @@ case class Coalesce(children: Seq[Expression]) extends Expression { } case class IsNull(child: Expression) extends Predicate with trees.UnaryNode[Expression] { - def references = child.references override def foldable = child.foldable def nullable = false @@ -65,7 +63,6 @@ case class IsNull(child: Expression) extends Predicate with trees.UnaryNode[Expr } case class IsNotNull(child: Expression) extends Predicate with trees.UnaryNode[Expression] { - def references = child.references override def foldable = child.foldable def nullable = false override def toString = s"IS NOT NULL $child" 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 5976b0ddf3e03..1313ccd120c1f 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 @@ -85,7 +85,7 @@ case class Not(child: Expression) extends UnaryExpression with Predicate { */ case class In(value: Expression, list: Seq[Expression]) extends Predicate { def children = value +: list - def references = children.flatMap(_.references).toSet + def nullable = true // TODO: Figure out correct nullability semantics of IN. override def toString = s"$value IN ${list.mkString("(", ",", ")")}" @@ -197,7 +197,7 @@ case class If(predicate: Expression, trueValue: Expression, falseValue: Expressi def children = predicate :: trueValue :: falseValue :: Nil override def nullable = trueValue.nullable || falseValue.nullable - def references = children.flatMap(_.references).toSet + override lazy val resolved = childrenResolved && trueValue.dataType == falseValue.dataType def dataType = { if (!resolved) { @@ -239,7 +239,7 @@ case class If(predicate: Expression, trueValue: Expression, falseValue: Expressi case class CaseWhen(branches: Seq[Expression]) extends Expression { type EvaluatedType = Any def children = branches - def references = children.flatMap(_.references).toSet + def dataType = { if (!resolved) { throw new UnresolvedException(this, "cannot resolve due to differing types in some branches") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala new file mode 100644 index 0000000000000..3d4c4a8853c12 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala @@ -0,0 +1,124 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.sql.catalyst.types._ +import org.apache.spark.util.collection.OpenHashSet + +/** + * Creates a new set of the specified type + */ +case class NewSet(elementType: DataType) extends LeafExpression { + type EvaluatedType = Any + + def nullable = false + + // We are currently only using these Expressions internally for aggregation. However, if we ever + // expose these to users we'll want to create a proper type instead of hijacking ArrayType. + def dataType = ArrayType(elementType) + + def eval(input: Row): Any = { + new OpenHashSet[Any]() + } + + override def toString = s"new Set($dataType)" +} + +/** + * Adds an item to a set. + * For performance, this expression mutates its input during evaluation. + */ +case class AddItemToSet(item: Expression, set: Expression) extends Expression { + type EvaluatedType = Any + + def children = item :: set :: Nil + + def nullable = set.nullable + + def dataType = set.dataType + def eval(input: Row): Any = { + val itemEval = item.eval(input) + val setEval = set.eval(input).asInstanceOf[OpenHashSet[Any]] + + if (itemEval != null) { + if (setEval != null) { + setEval.add(itemEval) + setEval + } else { + null + } + } else { + setEval + } + } + + override def toString = s"$set += $item" +} + +/** + * Combines the elements of two sets. + * For performance, this expression mutates its left input set during evaluation. + */ +case class CombineSets(left: Expression, right: Expression) extends BinaryExpression { + type EvaluatedType = Any + + def nullable = left.nullable || right.nullable + + def dataType = left.dataType + + def symbol = "++=" + + def eval(input: Row): Any = { + val leftEval = left.eval(input).asInstanceOf[OpenHashSet[Any]] + if(leftEval != null) { + val rightEval = right.eval(input).asInstanceOf[OpenHashSet[Any]] + if (rightEval != null) { + val iterator = rightEval.iterator + while(iterator.hasNext) { + val rightValue = iterator.next() + leftEval.add(rightValue) + } + leftEval + } else { + null + } + } else { + null + } + } +} + +/** + * Returns the number of elements in the input set. + */ +case class CountSet(child: Expression) extends UnaryExpression { + type EvaluatedType = Any + + def nullable = child.nullable + + def dataType = LongType + + def eval(input: Row): Any = { + val childEval = child.eval(input).asInstanceOf[OpenHashSet[Any]] + if (childEval != null) { + childEval.size.toLong + } + } + + override def toString = s"$child.count()" +} 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 97fc3a3b14b88..c2a3a5ca3ca8b 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 @@ -226,8 +226,6 @@ case class Substring(str: Expression, pos: Expression, len: Expression) extends if (str.dataType == BinaryType) str.dataType else StringType } - def references = children.flatMap(_.references).toSet - override def children = str :: pos :: len :: Nil @inline 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 5f86d6047cb9c..ddd4b3755d629 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 @@ -65,8 +65,10 @@ object ColumnPruning extends Rule[LogicalPlan] { // Eliminate unneeded attributes from either side of a Join. case Project(projectList, Join(left, right, joinType, condition)) => // Collect the list of all references required either above or to evaluate the condition. - val allReferences: Set[Attribute] = - projectList.flatMap(_.references).toSet ++ condition.map(_.references).getOrElse(Set.empty) + val allReferences: AttributeSet = + AttributeSet( + projectList.flatMap(_.references.iterator)) ++ + condition.map(_.references).getOrElse(AttributeSet(Seq.empty)) /** Applies a projection only when the child is producing unnecessary attributes */ def pruneJoinChild(c: LogicalPlan) = prunedChild(c, allReferences) @@ -76,8 +78,8 @@ object ColumnPruning extends Rule[LogicalPlan] { // Eliminate unneeded attributes from right side of a LeftSemiJoin. case Join(left, right, LeftSemi, condition) => // Collect the list of all references required to evaluate the condition. - val allReferences: Set[Attribute] = - condition.map(_.references).getOrElse(Set.empty) + val allReferences: AttributeSet = + condition.map(_.references).getOrElse(AttributeSet(Seq.empty)) Join(left, prunedChild(right, allReferences), LeftSemi, condition) @@ -104,7 +106,7 @@ object ColumnPruning extends Rule[LogicalPlan] { } /** Applies a projection only when the child is producing unnecessary attributes */ - private def prunedChild(c: LogicalPlan, allReferences: Set[Attribute]) = + private def prunedChild(c: LogicalPlan, allReferences: AttributeSet) = if ((c.outputSet -- allReferences.filter(c.outputSet.contains)).nonEmpty) { Project(allReferences.filter(c.outputSet.contains).toSeq, c) } else { 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 0988b0c6d990c..1e177e28f80b3 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 @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.plans -import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeSet, Expression} import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.catalyst.types.{ArrayType, DataType, StructField, StructType} @@ -29,7 +29,7 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy /** * Returns the set of attributes that are output by this node. */ - def outputSet: Set[Attribute] = output.toSet + def outputSet: AttributeSet = AttributeSet(output) /** * Runs [[transform]] with `rule` on all expressions present in this query operator. 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 278569f0cb14a..8616ac45b0e95 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 @@ -45,17 +45,11 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] { sizeInBytes = children.map(_.statistics).map(_.sizeInBytes).product ) - /** - * Returns the set of attributes that are referenced by this node - * during evaluation. - */ - def references: Set[Attribute] - /** * Returns the set of attributes that this node takes as * input from its children. */ - lazy val inputSet: Set[Attribute] = children.flatMap(_.output).toSet + lazy val inputSet: AttributeSet = AttributeSet(children.flatMap(_.output)) /** * Returns true if this expression and all its children have been resolved to a specific schema @@ -126,9 +120,6 @@ abstract class LeafNode extends LogicalPlan with trees.LeafNode[LogicalPlan] { 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/ScriptTransformation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala index d3f9d0fb93237..4460c86ed9026 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala @@ -30,6 +30,4 @@ case class ScriptTransformation( input: Seq[Expression], script: String, output: Seq[Attribute], - child: LogicalPlan) extends UnaryNode { - def references = input.flatMap(_.references).toSet -} + child: LogicalPlan) extends UnaryNode 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 3cb407217c4c3..4adfb189372d6 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 @@ -23,7 +23,6 @@ import org.apache.spark.sql.catalyst.types._ case class Project(projectList: Seq[NamedExpression], child: LogicalPlan) extends UnaryNode { def output = projectList.map(_.toAttribute) - def references = projectList.flatMap(_.references).toSet } /** @@ -59,14 +58,10 @@ case class Generate( override def output = if (join) child.output ++ generatorOutput else generatorOutput - - override def references = - if (join) child.outputSet else generator.references } case class Filter(condition: Expression, child: LogicalPlan) extends UnaryNode { override def output = child.output - override def references = condition.references } case class Union(left: LogicalPlan, right: LogicalPlan) extends BinaryNode { @@ -76,8 +71,6 @@ case class Union(left: LogicalPlan, right: LogicalPlan) extends BinaryNode { override lazy val resolved = childrenResolved && !left.output.zip(right.output).exists { case (l,r) => l.dataType != r.dataType } - - override def references = Set.empty } case class Join( @@ -86,8 +79,6 @@ case class Join( joinType: JoinType, condition: Option[Expression]) extends BinaryNode { - override def references = condition.map(_.references).getOrElse(Set.empty) - override def output = { joinType match { case LeftSemi => @@ -106,8 +97,6 @@ case class Join( case class Except(left: LogicalPlan, right: LogicalPlan) extends BinaryNode { def output = left.output - - def references = Set.empty } case class InsertIntoTable( @@ -118,7 +107,6 @@ case class InsertIntoTable( extends LogicalPlan { // The table being inserted into is a child for the purposes of transformations. override def children = table :: child :: Nil - override def references = Set.empty override def output = child.output override lazy val resolved = childrenResolved && child.output.zip(table.output).forall { @@ -130,20 +118,17 @@ case class InsertIntoCreatedTable( databaseName: Option[String], tableName: String, child: LogicalPlan) extends UnaryNode { - override def references = Set.empty override def output = child.output } case class WriteToFile( path: String, child: LogicalPlan) extends UnaryNode { - override def references = Set.empty override def output = child.output } case class Sort(order: Seq[SortOrder], child: LogicalPlan) extends UnaryNode { override def output = child.output - override def references = order.flatMap(_.references).toSet } case class Aggregate( @@ -152,19 +137,20 @@ case class Aggregate( child: LogicalPlan) extends UnaryNode { + /** The set of all AttributeReferences required for this aggregation. */ + def references = + AttributeSet( + groupingExpressions.flatMap(_.references) ++ aggregateExpressions.flatMap(_.references)) + override def output = aggregateExpressions.map(_.toAttribute) - override def references = - (groupingExpressions ++ aggregateExpressions).flatMap(_.references).toSet } case class Limit(limitExpr: Expression, child: LogicalPlan) extends UnaryNode { override def output = child.output - override def references = limitExpr.references } case class Subquery(alias: String, child: LogicalPlan) extends UnaryNode { override def output = child.output.map(_.withQualifiers(alias :: Nil)) - override def references = Set.empty } /** @@ -191,20 +177,16 @@ case class LowerCaseSchema(child: LogicalPlan) extends UnaryNode { a.qualifiers) case other => other } - - override def references = Set.empty } case class Sample(fraction: Double, withReplacement: Boolean, seed: Long, child: LogicalPlan) extends UnaryNode { override def output = child.output - override def references = Set.empty } case class Distinct(child: LogicalPlan) extends UnaryNode { override def output = child.output - override def references = child.outputSet } case object NoRelation extends LeafNode { @@ -213,5 +195,4 @@ case object NoRelation extends LeafNode { case class Intersect(left: LogicalPlan, right: LogicalPlan) extends BinaryNode { override def output = left.output - override def references = Set.empty } 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 481a5a4f212b2..a01809c1fc5e2 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 @@ -50,7 +50,7 @@ case class SetCommand(key: Option[String], value: Option[String]) extends Comman * Returned by a parser when the users only wants to see what query plan would be executed, without * actually performing the execution. */ -case class ExplainCommand(plan: LogicalPlan) extends Command { +case class ExplainCommand(plan: LogicalPlan, extended: Boolean = false) extends Command { override def output = Seq(AttributeReference("plan", StringType, nullable = false)()) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala index 7146fbd540f29..72b0c5c8e7a26 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala @@ -31,13 +31,9 @@ abstract class RedistributeData extends UnaryNode { case class SortPartitions(sortExpressions: Seq[SortOrder], child: LogicalPlan) extends RedistributeData { - - def references = sortExpressions.flatMap(_.references).toSet } case class Repartition(partitionExpressions: Seq[Expression], child: LogicalPlan) extends RedistributeData { - - def references = partitionExpressions.flatMap(_.references).toSet } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala index 4bb022cf238af..ccb0df113c063 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala @@ -71,6 +71,7 @@ case class OrderedDistribution(ordering: Seq[SortOrder]) extends Distribution { "An AllTuples should be used to represent a distribution that only has " + "a single partition.") + // TODO: This is not really valid... def clustering = ordering.map(_.child).toSet } @@ -139,7 +140,6 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int) with Partitioning { override def children = expressions - override def references = expressions.flatMap(_.references).toSet override def nullable = false override def dataType = IntegerType @@ -179,7 +179,6 @@ case class RangePartitioning(ordering: Seq[SortOrder], numPartitions: Int) with Partitioning { override def children = ordering - override def references = ordering.flatMap(_.references).toSet override def nullable = false override def dataType = IntegerType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala index cd04bdf02cf84..96ce35939e2cc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala @@ -280,7 +280,8 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { */ def makeCopy(newArgs: Array[AnyRef]): this.type = attachTree(this, "makeCopy") { try { - val defaultCtor = getClass.getConstructors.head + // Skip no-arg constructors that are just there for kryo. + val defaultCtor = getClass.getConstructors.find(_.getParameterTypes.size != 0).head if (otherCopyArgs.isEmpty) { defaultCtor.newInstance(newArgs: _*).asInstanceOf[this.type] } else { 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 b52ee6d3378a3..70c6d06cf2534 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 @@ -270,8 +270,8 @@ case object FloatType extends FractionalType { } object ArrayType { - /** Construct a [[ArrayType]] object with the given element type. The `containsNull` is false. */ - def apply(elementType: DataType): ArrayType = ArrayType(elementType, false) + /** Construct a [[ArrayType]] object with the given element type. The `containsNull` is true. */ + def apply(elementType: DataType): ArrayType = ArrayType(elementType, true) } /** 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 e75373d5a74a7..428607d8c8253 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 @@ -57,7 +57,9 @@ case class OptionalData( case class ComplexData( arrayField: Seq[Int], - mapField: Map[Int, String], + arrayFieldContainsNull: Seq[java.lang.Integer], + mapField: Map[Int, Long], + mapFieldValueContainsNull: Map[Int, java.lang.Long], structField: PrimitiveData) case class GenericData[A]( @@ -116,8 +118,22 @@ class ScalaReflectionSuite extends FunSuite { val schema = schemaFor[ComplexData] assert(schema === Schema( StructType(Seq( - StructField("arrayField", ArrayType(IntegerType), nullable = true), - StructField("mapField", MapType(IntegerType, StringType), nullable = true), + StructField( + "arrayField", + ArrayType(IntegerType, containsNull = false), + nullable = true), + StructField( + "arrayFieldContainsNull", + ArrayType(IntegerType, containsNull = true), + nullable = true), + StructField( + "mapField", + MapType(IntegerType, LongType, valueContainsNull = false), + nullable = true), + StructField( + "mapFieldValueContainsNull", + MapType(IntegerType, LongType, valueContainsNull = true), + nullable = true), StructField( "structField", StructType(Seq( 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 999c9fff38d60..f1df817c41362 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 @@ -136,6 +136,16 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(In(Literal(1), Seq(Literal(1), Literal(2))) && In(Literal(2), Seq(Literal(1), Literal(2))), true) } + test("MaxOf") { + checkEvaluation(MaxOf(1, 2), 2) + checkEvaluation(MaxOf(2, 1), 2) + checkEvaluation(MaxOf(1L, 2L), 2L) + checkEvaluation(MaxOf(2L, 1L), 2L) + + checkEvaluation(MaxOf(Literal(null, IntegerType), 2), 2) + checkEvaluation(MaxOf(2, Literal(null, IntegerType)), 2) + } + test("LIKE literal Regular Expression") { checkEvaluation(Literal(null, StringType).like("a"), null) checkEvaluation(Literal("a", StringType).like(Literal(null, StringType)), null) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala index 6344874538d67..296202543e2ca 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala @@ -26,7 +26,6 @@ import org.apache.spark.sql.catalyst.types.{StringType, NullType} case class Dummy(optKey: Option[Expression]) extends Expression { def children = optKey.toSeq - def references = Set.empty[Attribute] def nullable = true def dataType = NullType override lazy val resolved = true 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 index 3eccddef88134..37b4c8ffcba0b 100644 --- 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 @@ -86,14 +86,14 @@ public abstract class DataType { /** * Creates an ArrayType by specifying the data type of elements ({@code elementType}). - * The field of {@code containsNull} is set to {@code false}. + * The field of {@code containsNull} is set to {@code true}. */ public static ArrayType createArrayType(DataType elementType) { if (elementType == null) { throw new IllegalArgumentException("elementType should not be null."); } - return new ArrayType(elementType, false); + return new ArrayType(elementType, true); } /** 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 5cc41a83cc792..f0df19112ae37 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 @@ -33,6 +33,7 @@ private[spark] object SQLConf { val DIALECT = "spark.sql.dialect" val PARQUET_BINARY_AS_STRING = "spark.sql.parquet.binaryAsString" val PARQUET_CACHE_METADATA = "spark.sql.parquet.cacheMetadata" + val PARQUET_COMPRESSION = "spark.sql.parquet.compression.codec" // This is only used for the thriftserver val THRIFTSERVER_POOL = "spark.sql.thriftserver.scheduler.pool" @@ -78,6 +79,9 @@ trait SQLConf { /** When true tables cached using the in-memory columnar caching will be compressed. */ private[spark] def useCompression: Boolean = getConf(COMPRESS_CACHED, "false").toBoolean + /** The compression codec for writing to a Parquetfile */ + private[spark] def parquetCompressionCodec: String = getConf(PARQUET_COMPRESSION, "snappy") + /** The number of rows that will be */ private[spark] def columnBatchSize: Int = getConf(COLUMN_BATCH_SIZE, "1000").toInt 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 af9f7c62a1d25..6f0eed3f63c41 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 @@ -344,8 +344,8 @@ class SQLContext(@transient val sparkContext: SparkContext) prunePushedDownFilters: Seq[Expression] => Seq[Expression], scanBuilder: Seq[Attribute] => SparkPlan): SparkPlan = { - val projectSet = projectList.flatMap(_.references).toSet - val filterSet = filterPredicates.flatMap(_.references).toSet + val projectSet = AttributeSet(projectList.flatMap(_.references)) + val filterSet = AttributeSet(filterPredicates.flatMap(_.references)) val filterCondition = prunePushedDownFilters(filterPredicates).reduceLeftOption(And) // Right now we still use a projection even if the only evaluation is applying an alias @@ -354,7 +354,8 @@ class SQLContext(@transient val sparkContext: SparkContext) // TODO: Decouple final output schema from expression evaluation so this copy can be // avoided safely. - if (projectList.toSet == projectSet && filterSet.subsetOf(projectSet)) { + if (AttributeSet(projectList.map(_.toAttribute)) == projectSet && + filterSet.subsetOf(projectSet)) { // When it is possible to just use column pruning to get the right projection and // when the columns of this projection are enough to evaluate all filter conditions, // just do a scan followed by a filter, with no extra project. @@ -408,10 +409,18 @@ class SQLContext(@transient val sparkContext: SparkContext) protected def stringOrError[A](f: => A): String = try f.toString catch { case e: Throwable => e.toString } - def simpleString: String = stringOrError(executedPlan) + def simpleString: String = + s"""== Physical Plan == + |${stringOrError(executedPlan)} + """ override def toString: String = - s"""== Logical Plan == + // TODO previously will output RDD details by run (${stringOrError(toRdd.toDebugString)}) + // however, the `toRdd` will cause the real execution, which is not what we want. + // We need to think about how to avoid the side effect. + s"""== Parsed Logical Plan == + |${stringOrError(logical)} + |== Analyzed Logical Plan == |${stringOrError(analyzed)} |== Optimized Logical Plan == |${stringOrError(optimizedPlan)} @@ -419,7 +428,6 @@ class SQLContext(@transient val sparkContext: SparkContext) |${stringOrError(executedPlan)} |Code Generation: ${executedPlan.codegenEnabled} |== RDD == - |${stringOrError(toRdd.toDebugString)} """.stripMargin.trim } 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 e63b4903041f6..24e88eea3189e 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 @@ -79,8 +79,6 @@ private[sql] case class InMemoryRelation( override def children = Seq.empty - override def references = Set.empty - override def newInstance() = { new InMemoryRelation( output.map(_.newInstance), 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 463a1d32d7fd7..be9f155253d77 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 @@ -175,7 +175,7 @@ case class Aggregate( private[this] val resultProjection = new InterpretedMutableProjection( resultExpressions, computedSchema ++ namedGroups.map(_._2)) - private[this] val joinedRow = new JoinedRow + private[this] val joinedRow = new JoinedRow4 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 09c34b7059fc3..4802e40595807 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 @@ -46,12 +46,15 @@ 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 => - @transient val hashExpressions = - newMutableProjection(expressions, child.output)() - if (sortBasedShuffleOn) { + @transient val hashExpressions = + newProjection(expressions, child.output) + iter.map(r => (hashExpressions(r), r.copy())) } else { + @transient val hashExpressions = + newMutableProjection(expressions, child.output)() + val mutablePair = new MutablePair[Row, Row]() iter.map(r => mutablePair.update(hashExpressions(r), r)) } 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 index 4a26934c49c93..31ad5e8aabb0e 100644 --- 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 @@ -103,6 +103,40 @@ case class GeneratedAggregate( updateCount :: updateSum :: Nil, result ) + + case m @ Max(expr) => + val currentMax = AttributeReference("currentMax", expr.dataType, nullable = true)() + val initialValue = Literal(null, expr.dataType) + val updateMax = MaxOf(currentMax, expr) + + AggregateEvaluation( + currentMax :: Nil, + initialValue :: Nil, + updateMax :: Nil, + currentMax) + + case CollectHashSet(Seq(expr)) => + val set = AttributeReference("hashSet", ArrayType(expr.dataType), nullable = false)() + val initialValue = NewSet(expr.dataType) + val addToSet = AddItemToSet(expr, set) + + AggregateEvaluation( + set :: Nil, + initialValue :: Nil, + addToSet :: Nil, + set) + + case CombineSetsAndCount(inputSet) => + val ArrayType(inputType, _) = inputSet.dataType + val set = AttributeReference("hashSet", inputSet.dataType, nullable = false)() + val initialValue = NewSet(inputType) + val collectSets = CombineSets(set, inputSet) + + AggregateEvaluation( + set :: Nil, + initialValue :: Nil, + collectSets :: Nil, + CountSet(set)) } val computationSchema = computeFunctions.flatMap(_.schema) @@ -151,7 +185,7 @@ case class GeneratedAggregate( (namedGroups.map(_._2.toAttribute) ++ computationSchema).toSeq) log.info(s"Result Projection: ${resultExpressions.mkString(",")}") - val joinedRow = new JoinedRow + val joinedRow = new JoinedRow3 if (groupingExpressions.isEmpty) { // TODO: Codegening anything other than the updateProjection is probably over kill. 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 21cbbc9772a00..7d33ea5b021e2 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 @@ -141,10 +141,9 @@ case class SparkLogicalPlan(alreadyPlanned: SparkPlan)(@transient sqlContext: SQ extends LogicalPlan with MultiInstanceRelation { def output = alreadyPlanned.output - override def references = Set.empty override def children = Nil - override final def newInstance: this.type = { + override final def newInstance(): this.type = { SparkLogicalPlan( alreadyPlanned match { case ExistingRdd(output, rdd) => ExistingRdd(output.map(_.newInstance), rdd) 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 34654447a5f4b..077e6ebc5f11e 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 @@ -28,9 +28,13 @@ import com.twitter.chill.{AllScalaRegistrar, ResourcePool} import org.apache.spark.{SparkEnv, SparkConf} import org.apache.spark.serializer.{SerializerInstance, KryoSerializer} +import org.apache.spark.sql.catalyst.expressions.GenericRow +import org.apache.spark.util.collection.OpenHashSet import org.apache.spark.util.MutablePair import org.apache.spark.util.Utils +import org.apache.spark.sql.catalyst.expressions.codegen.{IntegerHashSet, LongHashSet} + private[sql] class SparkSqlSerializer(conf: SparkConf) extends KryoSerializer(conf) { override def newKryo(): Kryo = { val kryo = new Kryo() @@ -41,6 +45,13 @@ private[sql] class SparkSqlSerializer(conf: SparkConf) extends KryoSerializer(co kryo.register(classOf[com.clearspring.analytics.stream.cardinality.HyperLogLog], new HyperLogLogSerializer) kryo.register(classOf[scala.math.BigDecimal], new BigDecimalSerializer) + + // Specific hashsets must come first TODO: Move to core. + kryo.register(classOf[IntegerHashSet], new IntegerHashSetSerializer) + kryo.register(classOf[LongHashSet], new LongHashSetSerializer) + kryo.register(classOf[org.apache.spark.util.collection.OpenHashSet[_]], + new OpenHashSetSerializer) + kryo.setReferences(false) kryo.setClassLoader(Utils.getSparkClassLoader) new AllScalaRegistrar().apply(kryo) @@ -109,3 +120,78 @@ private[sql] class HyperLogLogSerializer extends Serializer[HyperLogLog] { HyperLogLog.Builder.build(bytes) } } + +private[sql] class OpenHashSetSerializer extends Serializer[OpenHashSet[_]] { + def write(kryo: Kryo, output: Output, hs: OpenHashSet[_]) { + val rowSerializer = kryo.getDefaultSerializer(classOf[Array[Any]]).asInstanceOf[Serializer[Any]] + output.writeInt(hs.size) + val iterator = hs.iterator + while(iterator.hasNext) { + val row = iterator.next() + rowSerializer.write(kryo, output, row.asInstanceOf[GenericRow].values) + } + } + + def read(kryo: Kryo, input: Input, tpe: Class[OpenHashSet[_]]): OpenHashSet[_] = { + val rowSerializer = kryo.getDefaultSerializer(classOf[Array[Any]]).asInstanceOf[Serializer[Any]] + val numItems = input.readInt() + val set = new OpenHashSet[Any](numItems + 1) + var i = 0 + while (i < numItems) { + val row = + new GenericRow(rowSerializer.read( + kryo, + input, + classOf[Array[Any]].asInstanceOf[Class[Any]]).asInstanceOf[Array[Any]]) + set.add(row) + i += 1 + } + set + } +} + +private[sql] class IntegerHashSetSerializer extends Serializer[IntegerHashSet] { + def write(kryo: Kryo, output: Output, hs: IntegerHashSet) { + output.writeInt(hs.size) + val iterator = hs.iterator + while(iterator.hasNext) { + val value: Int = iterator.next() + output.writeInt(value) + } + } + + def read(kryo: Kryo, input: Input, tpe: Class[IntegerHashSet]): IntegerHashSet = { + val numItems = input.readInt() + val set = new IntegerHashSet + var i = 0 + while (i < numItems) { + val value = input.readInt() + set.add(value) + i += 1 + } + set + } +} + +private[sql] class LongHashSetSerializer extends Serializer[LongHashSet] { + def write(kryo: Kryo, output: Output, hs: LongHashSet) { + output.writeInt(hs.size) + val iterator = hs.iterator + while(iterator.hasNext) { + val value = iterator.next() + output.writeLong(value) + } + } + + def read(kryo: Kryo, input: Input, tpe: Class[LongHashSet]): LongHashSet = { + val numItems = input.readInt() + val set = new LongHashSet + var i = 0 + while (i < numItems) { + val value = input.readLong() + set.add(value) + i += 1 + } + set + } +} 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 f0c958fdb537f..8dacb84c8a17e 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 @@ -23,6 +23,7 @@ import org.apache.spark.sql.catalyst.planning._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.plans.physical._ +import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.columnar.{InMemoryRelation, InMemoryColumnarTableScan} import org.apache.spark.sql.parquet._ @@ -148,7 +149,10 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { } def canBeCodeGened(aggs: Seq[AggregateExpression]) = !aggs.exists { - case _: Sum | _: Count => false + case _: Sum | _: Count | _: Max | _: CombineSetsAndCount => false + // The generated set implementation is pretty limited ATM. + case CollectHashSet(exprs) if exprs.size == 1 && + Seq(IntegerType, LongType).contains(exprs.head.dataType) => false case _ => true } @@ -297,8 +301,8 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case logical.SetCommand(key, value) => Seq(execution.SetCommand(key, value, plan.output)(context)) - case logical.ExplainCommand(logicalPlan) => - Seq(execution.ExplainCommand(logicalPlan, plan.output)(context)) + case logical.ExplainCommand(logicalPlan, extended) => + Seq(execution.ExplainCommand(logicalPlan, plan.output, extended)(context)) case logical.CacheCommand(tableName, cache) => Seq(execution.CacheCommand(tableName, cache)(context)) 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 f9dfa3c92f1eb..374af48b820c2 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 @@ -206,7 +206,8 @@ case class Sort( object ExistingRdd { def convertToCatalyst(a: Any): Any = a match { case o: Option[_] => o.orNull - case s: Seq[Any] => s.map(convertToCatalyst) + case s: Seq[_] => s.map(convertToCatalyst) + case m: Map[_, _] => m.map { case (k, v) => convertToCatalyst(k) -> convertToCatalyst(v) } case p: Product => new GenericRow(p.productIterator.map(convertToCatalyst).toArray) case other => other } 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 38f37564f1788..031b695169cea 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 @@ -108,15 +108,19 @@ case class SetCommand( */ @DeveloperApi case class ExplainCommand( - logicalPlan: LogicalPlan, output: Seq[Attribute])( + logicalPlan: LogicalPlan, output: Seq[Attribute], extended: Boolean)( @transient context: SQLContext) extends LeafNode with Command { // Run through the optimizer to generate the physical plan. override protected[sql] lazy val sideEffectResult: Seq[String] = try { - "Physical execution plan:" +: context.executePlan(logicalPlan).executedPlan.toString.split("\n") + // TODO in Hive, the "extended" ExplainCommand prints the AST as well, and detailed properties. + val queryExecution = context.executePlan(logicalPlan) + val outputString = if (extended) queryExecution.toString else queryExecution.simpleString + + outputString.split("\n") } catch { case cause: TreeNodeException[_] => - "Error occurred during query planning: " +: cause.getMessage.split("\n") + ("Error occurred during query planning: \n" + cause.getMessage).split("\n") } def execute(): RDD[Row] = { 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 f31df051824d7..5b896c55b7393 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 @@ -58,8 +58,6 @@ package object debug { } private[sql] case class DebugNode(child: SparkPlan) extends UnaryNode { - def references = Set.empty - def output = child.output implicit object SetAccumulatorParam extends AccumulatorParam[HashSet[String]] { 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 b08f9aacc1fcb..2890a563bed48 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 @@ -92,7 +92,7 @@ trait HashJoin { private[this] var currentMatchPosition: Int = -1 // Mutable per row objects. - private[this] val joinRow = new JoinedRow + private[this] val joinRow = new JoinedRow2 private[this] val joinKeys = streamSideKeyGenerator() 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 index b92091b560b1c..aef6ebf86b1eb 100644 --- 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 @@ -49,7 +49,6 @@ private[spark] case class PythonUDF( 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.") } @@ -113,7 +112,6 @@ private[spark] object ExtractPythonUdfs extends Rule[LogicalPlan] { 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 } 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 0a3b59cbc233a..ef4526ec03439 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 @@ -23,7 +23,7 @@ import parquet.io.api.{PrimitiveConverter, GroupConverter, Binary, Converter} import parquet.schema.MessageType import org.apache.spark.sql.catalyst.types._ -import org.apache.spark.sql.catalyst.expressions.{GenericRow, Row, Attribute} +import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.parquet.CatalystConverter.FieldType /** @@ -278,14 +278,14 @@ private[parquet] class CatalystGroupConverter( */ private[parquet] class CatalystPrimitiveRowConverter( protected[parquet] val schema: Array[FieldType], - protected[parquet] var current: ParquetRelation.RowType) + protected[parquet] var current: MutableRow) extends CatalystConverter { // This constructor is used for the root converter only def this(attributes: Array[Attribute]) = this( attributes.map(a => new FieldType(a.name, a.dataType, a.nullable)), - new ParquetRelation.RowType(attributes.length)) + new SpecificMutableRow(attributes.map(_.dataType))) protected [parquet] val converters: Array[Converter] = schema.zipWithIndex.map { @@ -299,7 +299,7 @@ private[parquet] class CatalystPrimitiveRowConverter( override val parent = null // Should be only called in root group converter! - override def getCurrentRecord: ParquetRelation.RowType = current + override def getCurrentRecord: Row = current override def getConverter(fieldIndex: Int): Converter = converters(fieldIndex) 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 1713ae6fb5d93..5ae768293a22e 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 @@ -100,8 +100,13 @@ private[sql] object ParquetRelation { // The compression type type CompressionType = parquet.hadoop.metadata.CompressionCodecName - // The default compression - val defaultCompression = CompressionCodecName.GZIP + // The parquet compression short names + val shortParquetCompressionCodecNames = Map( + "NONE" -> CompressionCodecName.UNCOMPRESSED, + "UNCOMPRESSED" -> CompressionCodecName.UNCOMPRESSED, + "SNAPPY" -> CompressionCodecName.SNAPPY, + "GZIP" -> CompressionCodecName.GZIP, + "LZO" -> CompressionCodecName.LZO) /** * Creates a new ParquetRelation and underlying Parquetfile for the given LogicalPlan. Note that @@ -141,9 +146,8 @@ private[sql] object 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()) - } + conf.set(ParquetOutputFormat.COMPRESSION, shortParquetCompressionCodecNames.getOrElse( + sqlContext.parquetCompressionCodec.toUpperCase, CompressionCodecName.UNCOMPRESSED).name()) ParquetRelation.enableLogForwarding() ParquetTypesConverter.writeMetaData(attributes, path, conf) new ParquetRelation(path.toString, Some(conf), sqlContext) { 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 f6cfab736d98a..a5a5d139a65cb 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 @@ -139,7 +139,7 @@ case class ParquetTableScan( partOutput.map(a => Cast(Literal(partValues(a.name)), a.dataType).eval(EmptyRow)) new Iterator[Row] { - private[this] val joinedRow = new JoinedRow(Row(partitionRowValues:_*), null) + private[this] val joinedRow = new JoinedRow5(Row(partitionRowValues:_*), null) def hasNext = iter.hasNext 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 c79a9ac2dad81..af8cd0a73b674 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 @@ -378,7 +378,7 @@ private[parquet] object ParquetTypesConverter extends Logging { val children = fs.listStatus(path).filterNot { status => val name = status.getPath.getName - name(0) == '.' || name == FileOutputCommitter.SUCCEEDED_FILE_NAME + (name(0) == '.' || name(0) == '_') && name != ParquetFileWriter.PARQUET_METADATA_FILE } // NOTE (lian): Parquet "_metadata" file can be very slow if the file consists of lots of row 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 index cf7d79f42db1d..8fb59c5830f6d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataTypeSuite.scala @@ -24,7 +24,7 @@ class DataTypeSuite extends FunSuite { test("construct an ArrayType") { val array = ArrayType(StringType) - assert(ArrayType(StringType, false) === array) + assert(ArrayType(StringType, true) === array) } test("construct an MapType") { 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 5b84c658db942..e24c521d24c7a 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 @@ -21,6 +21,7 @@ import java.sql.Timestamp import org.scalatest.FunSuite +import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.test.TestSQLContext._ case class ReflectData( @@ -56,6 +57,22 @@ case class OptionalReflectData( case class ReflectBinary(data: Array[Byte]) +case class Nested(i: Option[Int], s: String) + +case class Data( + array: Seq[Int], + arrayContainsNull: Seq[Option[Int]], + map: Map[Int, Long], + mapContainsNul: Map[Int, Option[Long]], + nested: Nested) + +case class ComplexReflectData( + arrayField: Seq[Int], + arrayFieldContainsNull: Seq[Option[Int]], + mapField: Map[Int, Long], + mapFieldContainsNull: Map[Int, Option[Long]], + dataField: Data) + class ScalaReflectionRelationSuite extends FunSuite { test("query case class RDD") { val data = ReflectData("a", 1, 1L, 1.toFloat, 1.toDouble, 1.toShort, 1.toByte, true, @@ -90,4 +107,33 @@ class ScalaReflectionRelationSuite extends FunSuite { val result = sql("SELECT data FROM reflectBinary").collect().head(0).asInstanceOf[Array[Byte]] assert(result.toSeq === Seq[Byte](1)) } + + test("query complex data") { + val data = ComplexReflectData( + Seq(1, 2, 3), + Seq(Some(1), Some(2), None), + Map(1 -> 10L, 2 -> 20L), + Map(1 -> Some(10L), 2 -> Some(20L), 3 -> None), + Data( + Seq(10, 20, 30), + Seq(Some(10), Some(20), None), + Map(10 -> 100L, 20 -> 200L), + Map(10 -> Some(100L), 20 -> Some(200L), 30 -> None), + Nested(None, "abc"))) + val rdd = sparkContext.parallelize(data :: Nil) + rdd.registerTempTable("reflectComplexData") + + assert(sql("SELECT * FROM reflectComplexData").collect().head === + new GenericRow(Array[Any]( + Seq(1, 2, 3), + Seq(1, 2, null), + Map(1 -> 10L, 2 -> 20L), + Map(1 -> 10L, 2 -> 20L, 3 -> null), + new GenericRow(Array[Any]( + Seq(10, 20, 30), + Seq(10, 20, null), + Map(10 -> 100L, 20 -> 200L), + Map(10 -> 100L, 20 -> 200L, 30 -> null), + new GenericRow(Array[Any](null, "abc"))))))) + } } 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 76b1724471442..37d64f0de7bab 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 @@ -45,16 +45,16 @@ class PlannerSuite extends FunSuite { assert(aggregations.size === 2) } - test("count distinct is not partially aggregated") { + test("count distinct is partially aggregated") { val query = testData.groupBy('value)(CountDistinct('key :: Nil)).queryExecution.analyzed val planned = HashAggregation(query) - assert(planned.isEmpty) + assert(planned.nonEmpty) } - test("mixed aggregates are not partially aggregated") { + test("mixed aggregates are partially aggregated") { val query = testData.groupBy('value)(Count('value), CountDistinct('key :: Nil)).queryExecution.analyzed val planned = HashAggregation(query) - assert(planned.isEmpty) + assert(planned.nonEmpty) } } 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 58b1e23891a3b..05513a127150c 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 @@ -130,11 +130,11 @@ class JsonSuite extends QueryTest { checkDataType( ArrayType(IntegerType, true), ArrayType(IntegerType, true), ArrayType(IntegerType, true)) checkDataType( - ArrayType(IntegerType, false), ArrayType(IntegerType), ArrayType(IntegerType, false)) + ArrayType(IntegerType, false), ArrayType(IntegerType), ArrayType(IntegerType, true)) checkDataType( ArrayType(IntegerType, false), ArrayType(IntegerType, false), ArrayType(IntegerType, false)) checkDataType( - ArrayType(IntegerType, false), ArrayType(IntegerType, false), ArrayType(IntegerType)) + ArrayType(IntegerType, false), ArrayType(IntegerType, true), ArrayType(IntegerType, true)) // StructType checkDataType(StructType(Nil), StructType(Nil), StructType(Nil)) @@ -201,26 +201,26 @@ class JsonSuite extends QueryTest { val jsonSchemaRDD = jsonRDD(complexFieldAndType) 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("arrayOfArray1", ArrayType(ArrayType(StringType, false), false), true) :: + StructField("arrayOfArray2", ArrayType(ArrayType(DoubleType, false), false), true) :: + StructField("arrayOfBigInteger", ArrayType(DecimalType, false), true) :: + StructField("arrayOfBoolean", ArrayType(BooleanType, false), true) :: + StructField("arrayOfDouble", ArrayType(DoubleType, false), true) :: + StructField("arrayOfInteger", ArrayType(IntegerType, false), true) :: + StructField("arrayOfLong", ArrayType(LongType, false), true) :: StructField("arrayOfNull", ArrayType(StringType, true), true) :: - StructField("arrayOfString", ArrayType(StringType), true) :: + StructField("arrayOfString", ArrayType(StringType, false), true) :: StructField("arrayOfStruct", ArrayType( StructType( StructField("field1", BooleanType, true) :: StructField("field2", StringType, true) :: - StructField("field3", StringType, true) :: Nil)), true) :: + StructField("field3", StringType, true) :: Nil), false), 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("field1", ArrayType(IntegerType, false), true) :: + StructField("field2", ArrayType(StringType, false), true) :: Nil), true) :: Nil) assert(expectedSchema === jsonSchemaRDD.schema) @@ -441,7 +441,7 @@ class JsonSuite extends QueryTest { val jsonSchemaRDD = jsonRDD(complexFieldValueTypeConflict) val expectedSchema = StructType( - StructField("array", ArrayType(IntegerType), true) :: + StructField("array", ArrayType(IntegerType, false), true) :: StructField("num_struct", StringType, true) :: StructField("str_array", StringType, true) :: StructField("struct", StructType( @@ -467,7 +467,7 @@ class JsonSuite extends QueryTest { val expectedSchema = StructType( StructField("array1", ArrayType(StringType, true), true) :: StructField("array2", ArrayType(StructType( - StructField("field", LongType, true) :: Nil)), true) :: Nil) + StructField("field", LongType, true) :: Nil), false), true) :: Nil) assert(expectedSchema === jsonSchemaRDD.schema) @@ -492,7 +492,7 @@ class JsonSuite extends QueryTest { val expectedSchema = StructType( StructField("a", BooleanType, true) :: StructField("b", LongType, true) :: - StructField("c", ArrayType(IntegerType), true) :: + StructField("c", ArrayType(IntegerType, false), true) :: StructField("d", StructType( StructField("field", BooleanType, true) :: Nil), true) :: StructField("e", StringType, true) :: 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 172dcd6aa0ee3..28f43b36832ac 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 @@ -186,6 +186,100 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA TestSQLContext.setConf(SQLConf.PARQUET_BINARY_AS_STRING, oldIsParquetBinaryAsString.toString) } + test("Compression options for writing to a Parquetfile") { + val defaultParquetCompressionCodec = TestSQLContext.parquetCompressionCodec + import scala.collection.JavaConversions._ + + val file = getTempFilePath("parquet") + val path = file.toString + val rdd = TestSQLContext.sparkContext.parallelize((1 to 100)) + .map(i => TestRDDEntry(i, s"val_$i")) + + // test default compression codec + rdd.saveAsParquetFile(path) + var actualCodec = ParquetTypesConverter.readMetaData(new Path(path), Some(TestSQLContext.sparkContext.hadoopConfiguration)) + .getBlocks.flatMap(block => block.getColumns).map(column => column.getCodec.name()).distinct + assert(actualCodec === TestSQLContext.parquetCompressionCodec.toUpperCase :: Nil) + + parquetFile(path).registerTempTable("tmp") + checkAnswer( + sql("SELECT key, value FROM tmp WHERE value = 'val_5' OR value = 'val_7'"), + (5, "val_5") :: + (7, "val_7") :: Nil) + + Utils.deleteRecursively(file) + + // test uncompressed parquet file with property value "UNCOMPRESSED" + TestSQLContext.setConf(SQLConf.PARQUET_COMPRESSION, "UNCOMPRESSED") + + rdd.saveAsParquetFile(path) + actualCodec = ParquetTypesConverter.readMetaData(new Path(path), Some(TestSQLContext.sparkContext.hadoopConfiguration)) + .getBlocks.flatMap(block => block.getColumns).map(column => column.getCodec.name()).distinct + assert(actualCodec === TestSQLContext.parquetCompressionCodec.toUpperCase :: Nil) + + parquetFile(path).registerTempTable("tmp") + checkAnswer( + sql("SELECT key, value FROM tmp WHERE value = 'val_5' OR value = 'val_7'"), + (5, "val_5") :: + (7, "val_7") :: Nil) + + Utils.deleteRecursively(file) + + // test uncompressed parquet file with property value "none" + TestSQLContext.setConf(SQLConf.PARQUET_COMPRESSION, "none") + + rdd.saveAsParquetFile(path) + actualCodec = ParquetTypesConverter.readMetaData(new Path(path), Some(TestSQLContext.sparkContext.hadoopConfiguration)) + .getBlocks.flatMap(block => block.getColumns).map(column => column.getCodec.name()).distinct + assert(actualCodec === "UNCOMPRESSED" :: Nil) + + parquetFile(path).registerTempTable("tmp") + checkAnswer( + sql("SELECT key, value FROM tmp WHERE value = 'val_5' OR value = 'val_7'"), + (5, "val_5") :: + (7, "val_7") :: Nil) + + Utils.deleteRecursively(file) + + // test gzip compression codec + TestSQLContext.setConf(SQLConf.PARQUET_COMPRESSION, "gzip") + + rdd.saveAsParquetFile(path) + actualCodec = ParquetTypesConverter.readMetaData(new Path(path), Some(TestSQLContext.sparkContext.hadoopConfiguration)) + .getBlocks.flatMap(block => block.getColumns).map(column => column.getCodec.name()).distinct + assert(actualCodec === TestSQLContext.parquetCompressionCodec.toUpperCase :: Nil) + + parquetFile(path).registerTempTable("tmp") + checkAnswer( + sql("SELECT key, value FROM tmp WHERE value = 'val_5' OR value = 'val_7'"), + (5, "val_5") :: + (7, "val_7") :: Nil) + + Utils.deleteRecursively(file) + + // test snappy compression codec + TestSQLContext.setConf(SQLConf.PARQUET_COMPRESSION, "snappy") + + rdd.saveAsParquetFile(path) + actualCodec = ParquetTypesConverter.readMetaData(new Path(path), Some(TestSQLContext.sparkContext.hadoopConfiguration)) + .getBlocks.flatMap(block => block.getColumns).map(column => column.getCodec.name()).distinct + assert(actualCodec === TestSQLContext.parquetCompressionCodec.toUpperCase :: Nil) + + parquetFile(path).registerTempTable("tmp") + checkAnswer( + sql("SELECT key, value FROM tmp WHERE value = 'val_5' OR value = 'val_7'"), + (5, "val_5") :: + (7, "val_7") :: Nil) + + Utils.deleteRecursively(file) + + // TODO: Lzo requires additional external setup steps so leave it out for now + // ref.: https://github.com/Parquet/parquet-mr/blob/parquet-1.5.0/parquet-hadoop/src/test/java/parquet/hadoop/example/TestInputOutputFormat.java#L169 + + // Set it back. + TestSQLContext.setConf(SQLConf.PARQUET_COMPRESSION, defaultParquetCompressionCodec) + } + test("Read/Write All Types with non-primitive type") { val tempDir = getTempFilePath("parquetTest").getCanonicalPath val range = (0 to 255) 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 index 699a1103f3248..6eccb1ba6d4dc 100644 --- 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 @@ -151,7 +151,7 @@ class SparkSQLOperationManager(hiveContext: HiveContext) extends OperationManage } def getResultSetSchema: TableSchema = { - logWarning(s"Result Schema: ${result.queryExecution.analyzed.output}") + logInfo(s"Result Schema: ${result.queryExecution.analyzed.output}") if (result.queryExecution.analyzed.output.size == 0) { new TableSchema(new FieldSchema("Result", "string", "") :: Nil) } else { 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 index 70bea1ed80fda..3475c2c9db080 100644 --- 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 @@ -18,41 +18,112 @@ package org.apache.spark.sql.hive.thriftserver -import java.io.{BufferedReader, InputStreamReader, PrintWriter} +import scala.collection.mutable.ArrayBuffer +import scala.concurrent.ExecutionContext.Implicits.global +import scala.concurrent.duration._ +import scala.concurrent.{Await, Future, Promise} +import scala.sys.process.{Process, ProcessLogger} + +import java.io._ +import java.util.concurrent.atomic.AtomicInteger 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") +import org.apache.spark.Logging +import org.apache.spark.sql.catalyst.util.getTempFilePath + +class CliSuite extends FunSuite with BeforeAndAfterAll with Logging { + def runCliWithin( + timeout: FiniteDuration, + extraArgs: Seq[String] = Seq.empty)( + queriesAndExpectedAnswers: (String, String)*) { + + val (queries, expectedAnswers) = queriesAndExpectedAnswers.unzip + val warehousePath = getTempFilePath("warehouse") + val metastorePath = getTempFilePath("metastore") + val cliScript = "../../bin/spark-sql".split("/").mkString(File.separator) - override def beforeAll() { - val jdbcUrl = s"jdbc:derby:;databaseName=$METASTORE_PATH;create=true" - val commands = - s"""../../bin/spark-sql + val command = { + val jdbcUrl = s"jdbc:derby:;databaseName=$metastorePath;create=true" + s"""$cliScript | --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>") + | --hiveconf ${ConfVars.METASTOREWAREHOUSE}=$warehousePath + """.stripMargin.split("\\s+").toSeq ++ extraArgs + } + + // AtomicInteger is needed because stderr and stdout of the forked process are handled in + // different threads. + val next = new AtomicInteger(0) + val foundAllExpectedAnswers = Promise.apply[Unit]() + val queryStream = new ByteArrayInputStream(queries.mkString("\n").getBytes) + val buffer = new ArrayBuffer[String]() + + def captureOutput(source: String)(line: String) { + buffer += s"$source> $line" + if (line.contains(expectedAnswers(next.get()))) { + if (next.incrementAndGet() == expectedAnswers.size) { + foundAllExpectedAnswers.trySuccess(()) + } + } + } + + // Searching expected output line from both stdout and stderr of the CLI process + val process = (Process(command) #< queryStream).run( + ProcessLogger(captureOutput("stdout"), captureOutput("stderr"))) + + Future { + val exitValue = process.exitValue() + logInfo(s"Spark SQL CLI process exit value: $exitValue") + } + + try { + Await.result(foundAllExpectedAnswers.future, timeout) + } catch { case cause: Throwable => + logError( + s""" + |======================= + |CliSuite failure output + |======================= + |Spark SQL CLI command line: ${command.mkString(" ")} + | + |Executed query ${next.get()} "${queries(next.get())}", + |But failed to capture expected output "${expectedAnswers(next.get())}" within $timeout. + | + |${buffer.mkString("\n")} + |=========================== + |End CliSuite failure output + |=========================== + """.stripMargin, cause) + } finally { + warehousePath.delete() + metastorePath.delete() + process.destroy() + } } - override def afterAll() { - process.destroy() - process.waitFor() + test("Simple commands") { + val dataFilePath = + Thread.currentThread().getContextClassLoader.getResource("data/files/small_kv.txt") + + runCliWithin(1.minute)( + "CREATE TABLE hive_test(key INT, val STRING);" + -> "OK", + "SHOW TABLES;" + -> "hive_test", + s"LOAD DATA LOCAL INPATH '$dataFilePath' OVERWRITE INTO TABLE hive_test;" + -> "OK", + "CACHE TABLE hive_test;" + -> "Time taken: ", + "SELECT COUNT(*) FROM hive_test;" + -> "5", + "DROP TABLE hive_test" + -> "Time taken: " + ) } - 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") + test("Single command with -e") { + runCliWithin(1.minute, Seq("-e", "SHOW TABLES;"))("" -> "OK") } } 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 index 326b0a7275b34..38977ff162097 100644 --- 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 @@ -17,32 +17,32 @@ package org.apache.spark.sql.hive.thriftserver -import scala.collection.JavaConversions._ +import scala.collection.mutable.ArrayBuffer import scala.concurrent.ExecutionContext.Implicits.global -import scala.concurrent._ +import scala.concurrent.duration._ +import scala.concurrent.{Await, Future, Promise} +import scala.sys.process.{Process, ProcessLogger} -import java.io.{BufferedReader, InputStreamReader} +import java.io.File import java.net.ServerSocket -import java.sql.{Connection, DriverManager, Statement} +import java.sql.{DriverManager, Statement} +import java.util.concurrent.TimeoutException import org.apache.hadoop.hive.conf.HiveConf.ConfVars -import org.scalatest.{BeforeAndAfterAll, FunSuite} +import org.apache.hive.jdbc.HiveDriver +import org.scalatest.FunSuite import org.apache.spark.Logging import org.apache.spark.sql.catalyst.util.getTempFilePath /** - * Test for the HiveThriftServer2 using JDBC. + * Tests for the HiveThriftServer2 using JDBC. */ -class HiveThriftServer2Suite extends FunSuite with BeforeAndAfterAll with TestUtils with Logging { +class HiveThriftServer2Suite extends FunSuite with Logging { + Class.forName(classOf[HiveDriver].getCanonicalName) - 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 = { + private val listeningHost = "localhost" + private val listeningPort = { // Let the system to choose a random available port to avoid collision with other parallel // builds. val socket = new ServerSocket(0) @@ -51,96 +51,126 @@ class HiveThriftServer2Suite extends FunSuite with BeforeAndAfterAll with TestUt port } - Class.forName(DRIVER_NAME) - - override def beforeAll() { launchServer() } + private val warehousePath = getTempFilePath("warehouse") + private val metastorePath = getTempFilePath("metastore") + private val metastoreJdbcUri = s"jdbc:derby:;databaseName=$metastorePath;create=true" - override def afterAll() { stopServer() } + def startThriftServerWithin(timeout: FiniteDuration = 30.seconds)(f: Statement => Unit) { + val serverScript = "../../sbin/start-thriftserver.sh".split("/").mkString(File.separator) - 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 + s"""$serverScript | --master local - | --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() - process = pb.start() - inputReader = new BufferedReader(new InputStreamReader(process.getInputStream)) - errorReader = new BufferedReader(new InputStreamReader(process.getErrorStream)) - waitForOutput(inputReader, "ThriftBinaryCLIService listening on", 300000) - - // 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) - print(stdout) - print(stderr) - Thread.sleep(50) + | --hiveconf hive.root.logger=INFO,console + | --hiveconf ${ConfVars.METASTORECONNECTURLKEY}=$metastoreJdbcUri + | --hiveconf ${ConfVars.METASTOREWAREHOUSE}=$warehousePath + | --hiveconf ${ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST}=$listeningHost + | --hiveconf ${ConfVars.HIVE_SERVER2_THRIFT_PORT}=$listeningPort + """.stripMargin.split("\\s+").toSeq + + val serverStarted = Promise[Unit]() + val buffer = new ArrayBuffer[String]() + + def captureOutput(source: String)(line: String) { + buffer += s"$source> $line" + if (line.contains("ThriftBinaryCLIService listening on")) { + serverStarted.success(()) } } - } - private def stopServer() { - process.destroy() - process.waitFor() + val process = Process(command).run( + ProcessLogger(captureOutput("stdout"), captureOutput("stderr"))) + + Future { + val exitValue = process.exitValue() + logInfo(s"Spark SQL Thrift server process exit value: $exitValue") + } + + val jdbcUri = s"jdbc:hive2://$listeningHost:$listeningPort/" + val user = System.getProperty("user.name") + + try { + Await.result(serverStarted.future, timeout) + + val connection = DriverManager.getConnection(jdbcUri, user, "") + val statement = connection.createStatement() + + try { + f(statement) + } finally { + statement.close() + connection.close() + } + } catch { + case cause: Exception => + cause match { + case _: TimeoutException => + logError(s"Failed to start Hive Thrift server within $timeout", cause) + case _ => + } + logError( + s""" + |===================================== + |HiveThriftServer2Suite failure output + |===================================== + |HiveThriftServer2 command line: ${command.mkString(" ")} + |JDBC URI: $jdbcUri + |User: $user + | + |${buffer.mkString("\n")} + |========================================= + |End HiveThriftServer2Suite failure output + |========================================= + """.stripMargin, cause) + } finally { + warehousePath.delete() + metastorePath.delete() + process.destroy() + } } - 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("Test JDBC query execution") { + startThriftServerWithin() { statement => + val dataFilePath = + Thread.currentThread().getContextClassLoader.getResource("data/files/small_kv.txt") + + val queries = Seq( + "CREATE TABLE test(key INT, val STRING)", + s"LOAD DATA LOCAL INPATH '$dataFilePath' OVERWRITE INTO TABLE test", + "CACHE TABLE test") + + queries.foreach(statement.execute) + + assertResult(5, "Row count mismatch") { + val resultSet = statement.executeQuery("SELECT COUNT(*) FROM test") + resultSet.next() + resultSet.getInt(1) + } + } } 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) + startThriftServerWithin() { statement => + val dataFilePath = + Thread.currentThread().getContextClassLoader.getResource( + "data/files/small_kv_with_null.txt") - stmt.close() - } + val queries = Seq( + "DROP TABLE IF EXISTS test_null", + "CREATE TABLE test_null(key INT, val STRING)", + s"LOAD DATA LOCAL INPATH '$dataFilePath' OVERWRITE INTO TABLE test_null") - def getConnection: Connection = { - val connectURI = s"jdbc:hive2://localhost:$PORT/" - DriverManager.getConnection(connectURI, System.getProperty("user.name"), "") - } + queries.foreach(statement.execute) + + val resultSet = statement.executeQuery("SELECT * FROM test_null WHERE key IS NULL") + + (0 until 5).foreach { _ => + resultSet.next() + assert(resultSet.getInt(1) === 0) + assert(resultSet.wasNull()) + } - def createStatement(): Statement = getConnection.createStatement() + assert(!resultSet.next()) + } + } } 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 deleted file mode 100644 index bb2242618fbef..0000000000000 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/TestUtils.scala +++ /dev/null @@ -1,108 +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.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/compatibility/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 index 210753efe7678..66243879b9019 100644 --- a/sql/hive/compatibility/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 @@ -310,6 +310,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "create_nested_type", "create_skewed_table1", "create_struct_table", + "cross_join", "ct_case_insensitive", "database_location", "database_properties", 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 ff32c7c90a0d2..d9b2bc7348ad2 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 @@ -255,6 +255,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { } // Note that HiveUDFs will be overridden by functions registered in this context. + @transient override protected[sql] lazy val functionRegistry = new HiveFunctionRegistry with OverrideFunctionRegistry @@ -423,7 +424,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { logical match { case _: NativeCommand => "" case _: SetCommand => "" - case _ => executedPlan.toString + case _ => super.simpleString } } } 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 3b371211e14cd..6571c35499ef4 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 @@ -265,9 +265,9 @@ private[hive] case class MetastoreRelation // org.apache.hadoop.hive.ql.metadata.Partition will cause a NotSerializableException // which indicates the SerDe we used is not Serializable. - @transient lazy val hiveQlTable = new Table(table) + @transient val hiveQlTable = new Table(table) - def hiveQlPartitions = partitions.map { p => + @transient val hiveQlPartitions = partitions.map { p => new Partition(hiveQlTable, p) } 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 1d9ba1b24a7a4..581332e600183 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 @@ -409,10 +409,9 @@ private[hive] object HiveQl { ExplainCommand(NoRelation) case Token("TOK_EXPLAIN", explainArgs) => // Ignore FORMATTED if present. - val Some(query) :: _ :: _ :: Nil = + val Some(query) :: _ :: extended :: Nil = getClauses(Seq("TOK_QUERY", "FORMATTED", "EXTENDED"), explainArgs) - // TODO: support EXTENDED? - ExplainCommand(nodeToPlan(query)) + ExplainCommand(nodeToPlan(query), extended != None) case Token("TOK_DESCTABLE", describeArgs) => // Reference: https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL @@ -773,6 +772,7 @@ private[hive] object HiveQl { val joinType = joinToken match { case "TOK_JOIN" => Inner + case "TOK_CROSSJOIN" => Inner case "TOK_RIGHTOUTERJOIN" => RightOuter case "TOK_LEFTOUTERJOIN" => LeftOuter case "TOK_FULLOUTERJOIN" => FullOuter 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 389ace726d205..10fa8314c9156 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 @@ -79,9 +79,9 @@ private[hive] trait HiveStrategies { hiveContext.convertMetastoreParquet => // Filter out all predicates that only deal with partition keys - val partitionKeyIds = relation.partitionKeys.map(_.exprId).toSet + val partitionsKeys = AttributeSet(relation.partitionKeys) val (pruningPredicates, otherPredicates) = predicates.partition { - _.references.map(_.exprId).subsetOf(partitionKeyIds) + _.references.subsetOf(partitionsKeys) } // We are going to throw the predicates and projection back at the whole optimization @@ -176,9 +176,9 @@ private[hive] trait HiveStrategies { case PhysicalOperation(projectList, predicates, relation: MetastoreRelation) => // Filter out all predicates that only deal with partition keys, these are given to the // hive table scan operator to be used for partition pruning. - val partitionKeyIds = relation.partitionKeys.map(_.exprId).toSet + val partitionKeyIds = AttributeSet(relation.partitionKeys) val (pruningPredicates, otherPredicates) = predicates.partition { - _.references.map(_.exprId).subsetOf(partitionKeyIds) + _.references.subsetOf(partitionKeyIds) } pruneFilterProject( 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 c6497a15efa0c..7d1ad53d8bdb3 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 @@ -88,7 +88,6 @@ private[hive] abstract class HiveUdf extends Expression with Logging with HiveFu type EvaluatedType = Any def nullable = true - def references = children.flatMap(_.references).toSet lazy val function = createFunction[UDFType]() @@ -229,8 +228,6 @@ private[hive] case class HiveGenericUdaf( def nullable: Boolean = true - def references: Set[Attribute] = children.map(_.references).flatten.toSet - override def toString = s"$nodeName#$functionClassName(${children.mkString(",")})" def newInstance() = new HiveUdafFunction(functionClassName, children, this) @@ -253,8 +250,6 @@ private[hive] case class HiveGenericUdtf( children: Seq[Expression]) extends Generator with HiveInspectors with HiveFunctionFactory { - override def references = children.flatMap(_.references).toSet - @transient protected lazy val function: GenericUDTF = createFunction() diff --git a/sql/hive/src/test/resources/golden/count distinct 0 values-0-1843b7947729b771fee3a4abd050bfdc b/sql/hive/src/test/resources/golden/count distinct 0 values-0-1843b7947729b771fee3a4abd050bfdc new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/count distinct 0 values-0-1843b7947729b771fee3a4abd050bfdc @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/count distinct 1 value + null long-0-89b850197b326239d60a5e1d5db7c9c9 b/sql/hive/src/test/resources/golden/count distinct 1 value + null long-0-89b850197b326239d60a5e1d5db7c9c9 new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/count distinct 1 value + null long-0-89b850197b326239d60a5e1d5db7c9c9 @@ -0,0 +1 @@ +1 diff --git a/sql/hive/src/test/resources/golden/count distinct 1 value + null-0-a014038c00fb81e88041ed4a8368e6f7 b/sql/hive/src/test/resources/golden/count distinct 1 value + null-0-a014038c00fb81e88041ed4a8368e6f7 new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/count distinct 1 value + null-0-a014038c00fb81e88041ed4a8368e6f7 @@ -0,0 +1 @@ +1 diff --git a/sql/hive/src/test/resources/golden/count distinct 1 value long-0-77b9ed1d7ae65fa53830a3bc586856ff b/sql/hive/src/test/resources/golden/count distinct 1 value long-0-77b9ed1d7ae65fa53830a3bc586856ff new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/count distinct 1 value long-0-77b9ed1d7ae65fa53830a3bc586856ff @@ -0,0 +1 @@ +1 diff --git a/sql/hive/src/test/resources/golden/count distinct 1 value strings-0-c68e75ec4c884b93765a466e992e391d b/sql/hive/src/test/resources/golden/count distinct 1 value strings-0-c68e75ec4c884b93765a466e992e391d new file mode 100644 index 0000000000000..0cfbf08886fca --- /dev/null +++ b/sql/hive/src/test/resources/golden/count distinct 1 value strings-0-c68e75ec4c884b93765a466e992e391d @@ -0,0 +1 @@ +2 diff --git a/sql/hive/src/test/resources/golden/count distinct 1 value-0-a4047b06a324fb5ea400c94350c9e038 b/sql/hive/src/test/resources/golden/count distinct 1 value-0-a4047b06a324fb5ea400c94350c9e038 new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/count distinct 1 value-0-a4047b06a324fb5ea400c94350c9e038 @@ -0,0 +1 @@ +1 diff --git a/sql/hive/src/test/resources/golden/count distinct 2 values including null-0-75672236a30e10dab13b9b246c5a3a1e b/sql/hive/src/test/resources/golden/count distinct 2 values including null-0-75672236a30e10dab13b9b246c5a3a1e new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/count distinct 2 values including null-0-75672236a30e10dab13b9b246c5a3a1e @@ -0,0 +1 @@ +1 diff --git a/sql/hive/src/test/resources/golden/count distinct 2 values long-0-f4ec7d767ba8c49d41edf5d6f58cf6d1 b/sql/hive/src/test/resources/golden/count distinct 2 values long-0-f4ec7d767ba8c49d41edf5d6f58cf6d1 new file mode 100644 index 0000000000000..0cfbf08886fca --- /dev/null +++ b/sql/hive/src/test/resources/golden/count distinct 2 values long-0-f4ec7d767ba8c49d41edf5d6f58cf6d1 @@ -0,0 +1 @@ +2 diff --git a/sql/hive/src/test/resources/golden/count distinct 2 values-0-c61df65af167acaf7edb174e77898f3e b/sql/hive/src/test/resources/golden/count distinct 2 values-0-c61df65af167acaf7edb174e77898f3e new file mode 100644 index 0000000000000..0cfbf08886fca --- /dev/null +++ b/sql/hive/src/test/resources/golden/count distinct 2 values-0-c61df65af167acaf7edb174e77898f3e @@ -0,0 +1 @@ +2 diff --git a/sql/hive/src/test/resources/golden/cross_join-0-7e4af1870bc73decae43b3383c7d2046 b/sql/hive/src/test/resources/golden/cross_join-0-7e4af1870bc73decae43b3383c7d2046 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/cross_join-1-1a96761bf3e47ace9a422ed58273ff35 b/sql/hive/src/test/resources/golden/cross_join-1-1a96761bf3e47ace9a422ed58273ff35 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/cross_join-2-85c93a81eae05bf56a04a904bb80a229 b/sql/hive/src/test/resources/golden/cross_join-2-85c93a81eae05bf56a04a904bb80a229 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_create_table_delimited-0-52b0e534c7df544258a1c59df9f816ce b/sql/hive/src/test/resources/golden/show_create_table_delimited-0-52b0e534c7df544258a1c59df9f816ce new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala index 7c82964b5ecdc..8d6ca9939a730 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.hive +import org.scalatest.BeforeAndAfterAll + import scala.reflect.ClassTag @@ -26,7 +28,9 @@ 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 { +class StatisticsSuite extends QueryTest with BeforeAndAfterAll { + TestHive.reset() + TestHive.cacheTables = false test("parse analyze commands") { def assertAnalyzeCommand(analyzeCommand: String, c: Class[_]) { @@ -126,7 +130,7 @@ class StatisticsSuite extends QueryTest { val sizes = rdd.queryExecution.analyzed.collect { case mr: MetastoreRelation => mr.statistics.sizeInBytes } - assert(sizes.size === 1) + assert(sizes.size === 1, s"Size wrong for:\n ${rdd.queryExecution}") assert(sizes(0).equals(BigInt(5812)), s"expected exact size 5812 for test table 'src', got: ${sizes(0)}") } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala new file mode 100644 index 0000000000000..4ed58f4be1167 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.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.sql.hive.execution + +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.hive.test.TestHive +import org.apache.spark.sql.hive.test.TestHive._ +import org.apache.spark.sql.Row + +/** + * A set of tests that validates support for Hive Explain command. + */ +class HiveExplainSuite extends QueryTest { + private def check(sqlCmd: String, exists: Boolean, keywords: String*) { + val outputs = sql(sqlCmd).collect().map(_.getString(0)).mkString + for (key <- keywords) { + if (exists) { + assert(outputs.contains(key), s"Failed for $sqlCmd ($key doens't exist in result)") + } else { + assert(!outputs.contains(key), s"Failed for $sqlCmd ($key existed in the result)") + } + } + } + + test("explain extended command") { + check(" explain select * from src where key=123 ", true, + "== Physical Plan ==") + check(" explain select * from src where key=123 ", false, + "== Parsed Logical Plan ==", + "== Analyzed Logical Plan ==", + "== Optimized Logical Plan ==") + check(" explain extended select * from src where key=123 ", true, + "== Parsed Logical Plan ==", + "== Analyzed Logical Plan ==", + "== Optimized Logical Plan ==", + "== Physical Plan ==", + "Code Generation", "== RDD ==") + } +} 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 fdb2f41f5a5b6..6d925e56e6838 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 @@ -32,6 +32,71 @@ case class TestData(a: Int, b: String) */ class HiveQuerySuite extends HiveComparisonTest { + createQueryTest("count distinct 0 values", + """ + |SELECT COUNT(DISTINCT a) FROM ( + | SELECT 'a' AS a FROM src LIMIT 0) table + """.stripMargin) + + createQueryTest("count distinct 1 value strings", + """ + |SELECT COUNT(DISTINCT a) FROM ( + | SELECT 'a' AS a FROM src LIMIT 1 UNION ALL + | SELECT 'b' AS a FROM src LIMIT 1) table + """.stripMargin) + + createQueryTest("count distinct 1 value", + """ + |SELECT COUNT(DISTINCT a) FROM ( + | SELECT 1 AS a FROM src LIMIT 1 UNION ALL + | SELECT 1 AS a FROM src LIMIT 1) table + """.stripMargin) + + createQueryTest("count distinct 2 values", + """ + |SELECT COUNT(DISTINCT a) FROM ( + | SELECT 1 AS a FROM src LIMIT 1 UNION ALL + | SELECT 2 AS a FROM src LIMIT 1) table + """.stripMargin) + + createQueryTest("count distinct 2 values including null", + """ + |SELECT COUNT(DISTINCT a, 1) FROM ( + | SELECT 1 AS a FROM src LIMIT 1 UNION ALL + | SELECT 1 AS a FROM src LIMIT 1 UNION ALL + | SELECT null AS a FROM src LIMIT 1) table + """.stripMargin) + + createQueryTest("count distinct 1 value + null", + """ + |SELECT COUNT(DISTINCT a) FROM ( + | SELECT 1 AS a FROM src LIMIT 1 UNION ALL + | SELECT 1 AS a FROM src LIMIT 1 UNION ALL + | SELECT null AS a FROM src LIMIT 1) table + """.stripMargin) + + createQueryTest("count distinct 1 value long", + """ + |SELECT COUNT(DISTINCT a) FROM ( + | SELECT 1L AS a FROM src LIMIT 1 UNION ALL + | SELECT 1L AS a FROM src LIMIT 1) table + """.stripMargin) + + createQueryTest("count distinct 2 values long", + """ + |SELECT COUNT(DISTINCT a) FROM ( + | SELECT 1L AS a FROM src LIMIT 1 UNION ALL + | SELECT 2L AS a FROM src LIMIT 1) table + """.stripMargin) + + createQueryTest("count distinct 1 value + null long", + """ + |SELECT COUNT(DISTINCT a) FROM ( + | SELECT 1L AS a FROM src LIMIT 1 UNION ALL + | SELECT 1L AS a FROM src LIMIT 1 UNION ALL + | SELECT null AS a FROM src LIMIT 1) table + """.stripMargin) + createQueryTest("null case", "SELECT case when(true) then 1 else null end FROM src LIMIT 1") @@ -262,7 +327,7 @@ class HiveQuerySuite extends HiveComparisonTest { def isExplanation(result: SchemaRDD) = { val explanation = result.select('plan).collect().map { case Row(plan: String) => plan } - explanation.size > 1 && explanation.head.startsWith("Physical execution plan") + explanation.exists(_ == "== Physical Plan ==") } test("SPARK-1704: Explain commands as a SchemaRDD") { 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 6b3ffd1c0ffe2..b6be6bc1bfefe 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,8 +20,8 @@ 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, nestedArray: Seq[Nested]) case class Nested(a: Int, B: Int) +case class Data(a: Int, B: Int, n: Nested, nestedArray: Seq[Nested]) /** * A set of test cases expressed in Hive QL that are not covered by the tests included in the hive distribution. @@ -57,6 +57,13 @@ class HiveResolutionSuite extends HiveComparisonTest { .registerTempTable("caseSensitivityTest") sql("SELECT a, b, A, B, n.a, n.b, n.A, n.B FROM caseSensitivityTest") + + println(sql("SELECT * FROM casesensitivitytest one JOIN casesensitivitytest two ON one.a = two.a").queryExecution) + + sql("SELECT * FROM casesensitivitytest one JOIN casesensitivitytest two ON one.a = two.a").collect() + + // TODO: sql("SELECT * FROM casesensitivitytest a JOIN casesensitivitytest b ON a.a = b.a") + } test("nested repeated resolution") { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala index df9bae96494d5..8bc72384a64ee 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala @@ -17,10 +17,19 @@ package org.apache.spark.sql.hive.execution +import org.scalatest.BeforeAndAfterAll + +import org.apache.spark.sql.hive.test.TestHive + /** * A set of tests that validates support for Hive SerDe. */ -class HiveSerDeSuite extends HiveComparisonTest { +class HiveSerDeSuite extends HiveComparisonTest with BeforeAndAfterAll { + + override def beforeAll() = { + TestHive.cacheTables = false + } + createQueryTest( "Read and write with LazySimpleSerDe (tab separated)", "SELECT * from serdeins") 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 1a6dbc0ce0c0d..8275e2d3bcce3 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.hive.execution +import org.scalatest.BeforeAndAfter + import org.apache.spark.sql.hive.test.TestHive /* Implicit conversions */ @@ -25,9 +27,10 @@ import scala.collection.JavaConversions._ /** * A set of test cases that validate partition and column pruning. */ -class PruningSuite extends HiveComparisonTest { +class PruningSuite extends HiveComparisonTest with BeforeAndAfter { // MINOR HACK: You must run a query before calling reset the first time. TestHive.sql("SHOW TABLES") + TestHive.cacheTables = false // 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/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala index d934b9cbfc3e8..53a3e6200e340 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 @@ -20,22 +20,21 @@ package org.apache.spark.streaming.receiver import java.nio.ByteBuffer import java.util.concurrent.atomic.AtomicLong -import scala.collection.mutable.{SynchronizedBuffer, ArrayBuffer} +import scala.collection.mutable.ArrayBuffer import scala.concurrent.Await import akka.actor.{Actor, Props} import akka.pattern.ask +import com.google.common.base.Throwables + import org.apache.spark.{Logging, SparkEnv} -import org.apache.spark.storage.StreamBlockId import org.apache.spark.streaming.scheduler._ import org.apache.spark.util.{Utils, AkkaUtils} import org.apache.spark.storage.StreamBlockId import org.apache.spark.streaming.scheduler.DeregisterReceiver import org.apache.spark.streaming.scheduler.AddBlock -import scala.Some import org.apache.spark.streaming.scheduler.RegisterReceiver -import com.google.common.base.Throwables /** * Concrete implementation of [[org.apache.spark.streaming.receiver.ReceiverSupervisor]] @@ -56,7 +55,8 @@ private[streaming] class ReceiverSupervisorImpl( private val trackerActor = { val ip = env.conf.get("spark.driver.host", "localhost") val port = env.conf.getInt("spark.driver.port", 7077) - val url = "akka.tcp://spark@%s:%s/user/ReceiverTracker".format(ip, port) + val url = "akka.tcp://%s@%s:%s/user/ReceiverTracker".format( + SparkEnv.driverActorSystemName, ip, port) env.actorSystem.actorSelection(url) } 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 c3310fbc24a98..155dd88aa2b81 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,7 +28,7 @@ import org.apache.hadoop.yarn.ipc.YarnRPC import org.apache.hadoop.yarn.util.{ConverterUtils, Records} import akka.actor._ import akka.remote._ -import org.apache.spark.{Logging, SecurityManager, SparkConf} +import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkEnv} import org.apache.spark.util.{Utils, AkkaUtils} import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.AddWebUIFilter @@ -210,8 +210,11 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp sparkConf.set("spark.driver.host", driverHost) sparkConf.set("spark.driver.port", driverPort.toString) - val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( - driverHost, driverPort.toString, CoarseGrainedSchedulerBackend.ACTOR_NAME) + val driverUrl = "akka.tcp://%s@%s:%s/user/%s".format( + SparkEnv.driverActorSystemName, + driverHost, + driverPort.toString, + CoarseGrainedSchedulerBackend.ACTOR_NAME) actor = actorSystem.actorOf(Props(new MonitorActor(driverUrl)), name = "YarnAM") } diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index 80e0162e9f277..568a6ef932bbd 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -26,7 +26,7 @@ import scala.collection import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.{Logging, SparkConf, SparkEnv} import org.apache.spark.scheduler.{SplitInfo,TaskSchedulerImpl} import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.util.Utils @@ -245,8 +245,10 @@ private[yarn] class YarnAllocationHandler( // Deallocate + allocate can result in reusing id's wrongly - so use a different counter // (executorIdCounter) val executorId = executorIdCounter.incrementAndGet().toString - val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( - sparkConf.get("spark.driver.host"), sparkConf.get("spark.driver.port"), + val driverUrl = "akka.tcp://%s@%s:%s/user/%s".format( + SparkEnv.driverActorSystemName, + sparkConf.get("spark.driver.host"), + sparkConf.get("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) logInfo("launching container on " + containerId + " host " + executorHostname) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala index 62f9b3cf5ab88..afa4fd4c6959e 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala @@ -39,7 +39,6 @@ class ClientArguments(val args: Array[String], val sparkConf: SparkConf) { var amMemory: Int = 512 // MB var amClass: String = "org.apache.spark.deploy.yarn.ApplicationMaster" var appName: String = "Spark" - var inputFormatInfo: List[InputFormatInfo] = null var priority = 0 parseArgs(args.toList) @@ -58,8 +57,7 @@ class ClientArguments(val args: Array[String], val sparkConf: SparkConf) { private def parseArgs(inputArgs: List[String]): Unit = { val userArgsBuffer: ArrayBuffer[String] = new ArrayBuffer[String]() - val inputFormatMap: HashMap[String, InputFormatInfo] = new HashMap[String, InputFormatInfo]() - + var args = inputArgs while (!args.isEmpty) { @@ -145,7 +143,6 @@ class ClientArguments(val args: Array[String], val sparkConf: SparkConf) { } userArgs = userArgsBuffer.readOnly - inputFormatInfo = inputFormatMap.values.toList } 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 45925f1fea005..e093fe4ae6ff8 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 @@ -25,7 +25,7 @@ import org.apache.hadoop.yarn.api.protocolrecords._ import org.apache.hadoop.yarn.conf.YarnConfiguration import akka.actor._ import akka.remote._ -import org.apache.spark.{Logging, SecurityManager, SparkConf} +import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkEnv} import org.apache.spark.util.{Utils, AkkaUtils} import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.AddWebUIFilter @@ -174,8 +174,11 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp sparkConf.set("spark.driver.host", driverHost) sparkConf.set("spark.driver.port", driverPort.toString) - val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( - driverHost, driverPort.toString, CoarseGrainedSchedulerBackend.ACTOR_NAME) + val driverUrl = "akka.tcp://%s@%s:%s/user/%s".format( + SparkEnv.driverActorSystemName, + driverHost, + driverPort.toString, + CoarseGrainedSchedulerBackend.ACTOR_NAME) actor = actorSystem.actorOf(Props(new MonitorActor(driverUrl)), name = "YarnAM") } diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index 29ccec2adcac3..0a461749c819d 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -26,7 +26,7 @@ import scala.collection import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.{Logging, SparkConf, SparkEnv} import org.apache.spark.scheduler.{SplitInfo,TaskSchedulerImpl} import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.util.Utils @@ -262,7 +262,8 @@ private[yarn] class YarnAllocationHandler( numExecutorsRunning.decrementAndGet() } else { val executorId = executorIdCounter.incrementAndGet().toString - val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( + val driverUrl = "akka.tcp://%s@%s:%s/user/%s".format( + SparkEnv.driverActorSystemName, sparkConf.get("spark.driver.host"), sparkConf.get("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME)