diff --git a/.gitignore b/.gitignore index 857e9feb953bd..32b603f1bc84f 100644 --- a/.gitignore +++ b/.gitignore @@ -49,3 +49,9 @@ unit-tests.log /lib/ rat-results.txt scalastyle.txt + +# For Hive +metastore_db/ +metastore/ +warehouse/ +TempStatsStore/ diff --git a/assembly/pom.xml b/assembly/pom.xml index bdb38806492a6..7d123fb1d7f02 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -40,14 +40,6 @@ root - - - - lib - file://${project.basedir}/lib - - - org.apache.spark @@ -84,11 +76,6 @@ spark-sql_${scala.binary.version} ${project.version} - - net.sf.py4j - py4j - 0.8.1 - diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index b0218531e9eb8..7df43a555d562 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -32,6 +32,12 @@ CLASSPATH="$SPARK_CLASSPATH:$SPARK_SUBMIT_CLASSPATH:$FWDIR/conf" ASSEMBLY_DIR="$FWDIR/assembly/target/scala-$SCALA_VERSION" +if [ -n "$JAVA_HOME" ]; then + JAR_CMD="$JAVA_HOME/bin/jar" +else + JAR_CMD="jar" +fi + # First check if we have a dependencies jar. If so, include binary classes with the deps jar if [ -f "$ASSEMBLY_DIR"/spark-assembly*hadoop*-deps.jar ]; then CLASSPATH="$CLASSPATH:$FWDIR/core/target/scala-$SCALA_VERSION/classes" @@ -44,33 +50,50 @@ if [ -f "$ASSEMBLY_DIR"/spark-assembly*hadoop*-deps.jar ]; then CLASSPATH="$CLASSPATH:$FWDIR/sql/catalyst/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/sql/core/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/sql/hive/target/scala-$SCALA_VERSION/classes" + CLASSPATH="$CLASSPATH:$FWDIR/yarn/stable/target/scala-$SCALA_VERSION/classes" - DEPS_ASSEMBLY_JAR=`ls "$ASSEMBLY_DIR"/spark-assembly*hadoop*-deps.jar` - CLASSPATH="$CLASSPATH:$DEPS_ASSEMBLY_JAR" + ASSEMBLY_JAR=$(ls "$ASSEMBLY_DIR"/spark-assembly*hadoop*-deps.jar 2>/dev/null) else # Else use spark-assembly jar from either RELEASE or assembly directory if [ -f "$FWDIR/RELEASE" ]; then - ASSEMBLY_JAR=`ls "$FWDIR"/lib/spark-assembly*hadoop*.jar` + ASSEMBLY_JAR=$(ls "$FWDIR"/lib/spark-assembly*hadoop*.jar 2>/dev/null) else - ASSEMBLY_JAR=`ls "$ASSEMBLY_DIR"/spark-assembly*hadoop*.jar` + ASSEMBLY_JAR=$(ls "$ASSEMBLY_DIR"/spark-assembly*hadoop*.jar 2>/dev/null) fi - CLASSPATH="$CLASSPATH:$ASSEMBLY_JAR" fi +# Verify that versions of java used to build the jars and run Spark are compatible +jar_error_check=$("$JAR_CMD" -tf "$ASSEMBLY_JAR" nonexistent/class/path 2>&1) +if [[ "$jar_error_check" =~ "invalid CEN header" ]]; then + echo "Loading Spark jar with '$JAR_CMD' failed. " + echo "This is likely because Spark was compiled with Java 7 and run " + echo "with Java 6. (see SPARK-1703). Please use Java 7 to run Spark " + echo "or build Spark with Java 6." + exit 1 +fi + +CLASSPATH="$CLASSPATH:$ASSEMBLY_JAR" + # When Hive support is needed, Datanucleus jars must be included on the classpath. -# Datanucleus jars do not work if only included in the uber jar as plugin.xml metadata is lost. +# Datanucleus jars do not work if only included in the uber jar as plugin.xml metadata is lost. # Both sbt and maven will populate "lib_managed/jars/" with the datanucleus jars when Spark is # built with Hive, so first check if the datanucleus jars exist, and then ensure the current Spark # assembly is built for Hive, before actually populating the CLASSPATH with the jars. # Note that this check order is faster (by up to half a second) in the case where Hive is not used. -num_datanucleus_jars=$(ls "$FWDIR"/lib_managed/jars/ 2>/dev/null | grep "datanucleus-.*\\.jar" | wc -l) -if [ $num_datanucleus_jars -gt 0 ]; then - AN_ASSEMBLY_JAR=${ASSEMBLY_JAR:-$DEPS_ASSEMBLY_JAR} - num_hive_files=$(jar tvf "$AN_ASSEMBLY_JAR" org/apache/hadoop/hive/ql/exec 2>/dev/null | wc -l) - if [ $num_hive_files -gt 0 ]; then +if [ -f "$FWDIR/RELEASE" ]; then + datanucleus_dir="$FWDIR"/lib +else + datanucleus_dir="$FWDIR"/lib_managed/jars +fi + +datanucleus_jars=$(find "$datanucleus_dir" 2>/dev/null | grep "datanucleus-.*\\.jar") +datanucleus_jars=$(echo "$datanucleus_jars" | tr "\n" : | sed s/:$//g) + +if [ -n "$datanucleus_jars" ]; then + hive_files=$("$JAR_CMD" -tf "$ASSEMBLY_JAR" org/apache/hadoop/hive/ql/exec 2>/dev/null) + if [ -n "$hive_files" ]; then echo "Spark assembly has been built with Hive, including Datanucleus jars on classpath" 1>&2 - DATANUCLEUSJARS=$(echo "$FWDIR/lib_managed/jars"/datanucleus-*.jar | tr " " :) - CLASSPATH=$CLASSPATH:$DATANUCLEUSJARS + CLASSPATH="$CLASSPATH:$datanucleus_jars" fi fi @@ -90,10 +113,10 @@ fi # Add hadoop conf dir if given -- otherwise FileSystem.*, etc fail ! # Note, this assumes that there is either a HADOOP_CONF_DIR or YARN_CONF_DIR which hosts # the configurtion files. -if [ "x" != "x$HADOOP_CONF_DIR" ]; then +if [ -n "$HADOOP_CONF_DIR" ]; then CLASSPATH="$CLASSPATH:$HADOOP_CONF_DIR" fi -if [ "x" != "x$YARN_CONF_DIR" ]; then +if [ -n "$YARN_CONF_DIR" ]; then CLASSPATH="$CLASSPATH:$YARN_CONF_DIR" fi diff --git a/bin/pyspark b/bin/pyspark index cad982bc33477..f5558853e8a4e 100755 --- a/bin/pyspark +++ b/bin/pyspark @@ -46,6 +46,7 @@ export PYSPARK_PYTHON # Add the PySpark classes to the Python path: export PYTHONPATH=$SPARK_HOME/python/:$PYTHONPATH +export PYTHONPATH=$SPARK_HOME/python/lib/py4j-0.8.1-src.zip:$PYTHONPATH # Load the PySpark shell.py script when ./pyspark is used interactively: export OLD_PYTHONSTARTUP=$PYTHONSTARTUP diff --git a/bin/pyspark2.cmd b/bin/pyspark2.cmd index 95791095ec932..d7cfd5eec501c 100644 --- a/bin/pyspark2.cmd +++ b/bin/pyspark2.cmd @@ -45,6 +45,7 @@ rem Figure out which Python to use. if "x%PYSPARK_PYTHON%"=="x" set PYSPARK_PYTHON=python set PYTHONPATH=%FWDIR%python;%PYTHONPATH% +set PYTHONPATH=%FWDIR%python\lib\py4j-0.8.1-src.zip;%PYTHONPATH% set OLD_PYTHONSTARTUP=%PYTHONSTARTUP% set PYTHONSTARTUP=%FWDIR%python\pyspark\shell.py diff --git a/bin/spark-class b/bin/spark-class index e8160c8af64c1..72f8b9bf9a495 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -138,7 +138,14 @@ if [ -e "$TOOLS_DIR"/target/spark-tools*[0-9Tg].jar ]; then fi # Compute classpath using external script -CLASSPATH=`$FWDIR/bin/compute-classpath.sh` +classpath_output=$($FWDIR/bin/compute-classpath.sh) +if [[ "$?" != "0" ]]; then + echo "$classpath_output" + exit 1 +else + CLASSPATH=$classpath_output +fi + if [[ "$1" =~ org.apache.spark.tools.* ]]; then CLASSPATH="$CLASSPATH:$SPARK_TOOLS_JAR" fi diff --git a/bin/spark-submit b/bin/spark-submit index dd0d95d9d4002..49bc26252cadf 100755 --- a/bin/spark-submit +++ b/bin/spark-submit @@ -18,7 +18,7 @@ # export SPARK_HOME="$(cd `dirname $0`/..; pwd)" -ORIG_ARGS=$@ +ORIG_ARGS=("$@") while (($#)); do if [ "$1" = "--deploy-mode" ]; then @@ -39,5 +39,5 @@ if [ ! -z $DRIVER_MEMORY ] && [ ! -z $DEPLOY_MODE ] && [ $DEPLOY_MODE = "client" export SPARK_MEM=$DRIVER_MEMORY fi -$SPARK_HOME/bin/spark-class org.apache.spark.deploy.SparkSubmit $ORIG_ARGS +$SPARK_HOME/bin/spark-class org.apache.spark.deploy.SparkSubmit "${ORIG_ARGS[@]}" diff --git a/core/pom.xml b/core/pom.xml index 73f573a414050..8fe215ab24289 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -38,12 +38,6 @@ net.java.dev.jets3t jets3t - - - commons-logging - commons-logging - - org.apache.curator @@ -69,6 +63,10 @@ com.google.guava guava + + org.apache.commons + commons-lang3 + com.google.code.findbugs jsr305 @@ -249,6 +247,11 @@ pyrolite 2.0.1 + + net.sf.py4j + py4j + 0.8.1 + target/scala-${scala.binary.version}/classes @@ -294,6 +297,48 @@ + + + org.codehaus.mojo + exec-maven-plugin + 1.2.1 + + + generate-resources + + exec + + + + + unzip + ../python + + -o + lib/py4j*.zip + -d + build + + + + + + + src/main/resources + + + ../python + + pyspark/*.py + + + + ../python/build + + py4j/*.py + + + diff --git a/core/src/main/scala/org/apache/spark/SecurityManager.scala b/core/src/main/scala/org/apache/spark/SecurityManager.scala index b4b0067801259..74aa441619bd2 100644 --- a/core/src/main/scala/org/apache/spark/SecurityManager.scala +++ b/core/src/main/scala/org/apache/spark/SecurityManager.scala @@ -146,8 +146,9 @@ private[spark] class SecurityManager(sparkConf: SparkConf) extends Logging { setViewAcls(defaultAclUsers, sparkConf.get("spark.ui.view.acls", "")) private val secretKey = generateSecretKey() - logInfo("SecurityManager, is authentication enabled: " + authOn + - " are ui acls enabled: " + uiAclsOn + " users with view permissions: " + viewAcls.toString()) + logInfo("SecurityManager: authentication " + (if (authOn) "enabled" else "disabled") + + "; ui acls " + (if (uiAclsOn) "enabled" else "disabled") + + "; users with view permissions: " + viewAcls.toString()) // Set our own authenticator to properly negotiate user/password for HTTP connections. // This is needed by the HTTP client fetching from the HttpServer. Put here so its diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index bea435ec34ce9..806e77d98fc5f 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -17,6 +17,8 @@ package org.apache.spark +import java.io.File + import scala.collection.JavaConversions._ import scala.collection.mutable import scala.concurrent.Await @@ -156,13 +158,11 @@ object SparkEnv extends Logging { conf.set("spark.driver.port", boundPort.toString) } - val classLoader = Thread.currentThread.getContextClassLoader - // Create an instance of the class named by the given Java system property, or by // defaultClassName if the property is not set, and return it as a T def instantiateClass[T](propertyName: String, defaultClassName: String): T = { val name = conf.get(propertyName, defaultClassName) - val cls = Class.forName(name, true, classLoader) + val cls = Class.forName(name, true, Utils.getContextOrSparkClassLoader) // First try with the constructor that takes SparkConf. If we can't find one, // use a no-arg constructor instead. try { @@ -306,7 +306,7 @@ object SparkEnv extends Logging { k == "java.class.path" }.getOrElse(("", "")) val classPathEntries = classPathProperty._2 - .split(conf.get("path.separator", ":")) + .split(File.pathSeparator) .filterNot(e => e.isEmpty) .map(e => (e, "System Classpath")) val addedJarsAndFiles = (addedJars ++ addedFiles).map((_, "Added By User")) diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala index dc012cc381346..fc4812753d005 100644 --- a/core/src/main/scala/org/apache/spark/TaskContext.scala +++ b/core/src/main/scala/org/apache/spark/TaskContext.scala @@ -42,9 +42,13 @@ class TaskContext( // List of callback functions to execute when the task completes. @transient private val onCompleteCallbacks = new ArrayBuffer[() => Unit] + // Set to true when the task is completed, before the onCompleteCallbacks are executed. + @volatile var completed: Boolean = false + /** * Add a callback function to be executed on task completion. An example use * is for HadoopRDD to register a callback to close the input stream. + * Will be called in any situation - success, failure, or cancellation. * @param f Callback function. */ def addOnCompleteCallback(f: () => Unit) { @@ -52,6 +56,7 @@ class TaskContext( } def executeOnCompleteCallbacks() { + completed = true // Process complete callbacks in the reverse order of registration onCompleteCallbacks.reverse.foreach{_()} } diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index 574a98636a619..619bfd75be8eb 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -18,7 +18,7 @@ package org.apache.spark.api.java import java.util.{Comparator, List => JList, Iterator => JIterator} -import java.lang.{Iterable => JIterable} +import java.lang.{Iterable => JIterable, Long => JLong} import scala.collection.JavaConversions._ import scala.reflect.ClassTag @@ -75,11 +75,11 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * Return a new RDD by applying a function to each partition of this RDD, while tracking the index * of the original partition. */ - def mapPartitionsWithIndex[R: ClassTag]( + def mapPartitionsWithIndex[R]( f: JFunction2[java.lang.Integer, java.util.Iterator[T], java.util.Iterator[R]], preservesPartitioning: Boolean = false): JavaRDD[R] = new JavaRDD(rdd.mapPartitionsWithIndex(((a,b) => f(a,asJavaIterator(b))), - preservesPartitioning)) + preservesPartitioning)(fakeClassTag))(fakeClassTag) /** * Return a new RDD by applying a function to all elements of this RDD. @@ -264,6 +264,26 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { rdd.zipPartitions(other.rdd)(fn)(other.classTag, fakeClassTag[V]))(fakeClassTag[V]) } + /** + * 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 [[org.apache.spark.rdd.RDD#zipWithIndex]]. + */ + def zipWithUniqueId(): JavaPairRDD[T, JLong] = { + JavaPairRDD.fromRDD(rdd.zipWithUniqueId()).asInstanceOf[JavaPairRDD[T, JLong]] + } + + /** + * 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 is similar to Scala's zipWithIndex but it uses Long instead of Int as the index type. + * This method needs to trigger a spark job when this RDD contains more than one partitions. + */ + def zipWithIndex(): JavaPairRDD[T, JLong] = { + JavaPairRDD.fromRDD(rdd.zipWithIndex()).asInstanceOf[JavaPairRDD[T, JLong]] + } + // Actions (launch a job to return a value to the user program) /** 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 672c344a56597..fecd9762f3f60 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 @@ -54,115 +54,39 @@ private[spark] class PythonRDD[T: ClassTag]( override def compute(split: Partition, context: TaskContext): Iterator[Array[Byte]] = { val startTime = System.currentTimeMillis val env = SparkEnv.get - val worker = env.createPythonWorker(pythonExec, envVars.toMap) - - @volatile var readerException: Exception = null + val worker: Socket = env.createPythonWorker(pythonExec, envVars.toMap) // Start a thread to feed the process input from our parent's iterator - new Thread("stdin writer for " + pythonExec) { - override def run() { - try { - SparkEnv.set(env) - val stream = new BufferedOutputStream(worker.getOutputStream, bufferSize) - val dataOut = new DataOutputStream(stream) - // Partition index - dataOut.writeInt(split.index) - // sparkFilesDir - PythonRDD.writeUTF(SparkFiles.getRootDirectory, dataOut) - // Broadcast variables - dataOut.writeInt(broadcastVars.length) - for (broadcast <- broadcastVars) { - dataOut.writeLong(broadcast.id) - dataOut.writeInt(broadcast.value.length) - dataOut.write(broadcast.value) - } - // Python includes (*.zip and *.egg files) - dataOut.writeInt(pythonIncludes.length) - for (include <- pythonIncludes) { - PythonRDD.writeUTF(include, dataOut) - } - dataOut.flush() - // Serialized command: - dataOut.writeInt(command.length) - dataOut.write(command) - // Data values - PythonRDD.writeIteratorToStream(parent.iterator(split, context), dataOut) - dataOut.flush() - worker.shutdownOutput() - } catch { - - case e: java.io.FileNotFoundException => - readerException = e - Try(worker.shutdownOutput()) // kill Python worker process + val writerThread = new WriterThread(env, worker, split, context) - case e: IOException => - // This can happen for legitimate reasons if the Python code stops returning data - // before we are done passing elements through, e.g., for take(). Just log a message to - // say it happened (as it could also be hiding a real IOException from a data source). - logInfo("stdin writer to Python finished early (may not be an error)", e) + context.addOnCompleteCallback { () => + writerThread.shutdownOnTaskCompletion() - case e: Exception => - // We must avoid throwing exceptions here, because the thread uncaught exception handler - // will kill the whole executor (see Executor). - readerException = e - Try(worker.shutdownOutput()) // kill Python worker process - } + // Cleanup the worker socket. This will also cause the Python worker to exit. + try { + worker.close() + } catch { + case e: Exception => logWarning("Failed to close worker socket", e) } - }.start() - - // Necessary to distinguish between a task that has failed and a task that is finished - @volatile var complete: Boolean = false - - // It is necessary to have a monitor thread for python workers if the user cancels with - // interrupts disabled. In that case we will need to explicitly kill the worker, otherwise the - // threads can block indefinitely. - new Thread(s"Worker Monitor for $pythonExec") { - override def run() { - // Kill the worker if it is interrupted or completed - // When a python task completes, the context is always set to interupted - while (!context.interrupted) { - Thread.sleep(2000) - } - if (!complete) { - try { - logWarning("Incomplete task interrupted: Attempting to kill Python Worker") - env.destroyPythonWorker(pythonExec, envVars.toMap) - } catch { - case e: Exception => - logError("Exception when trying to kill worker", e) - } - } - } - }.start() - - /* - * Partial fix for SPARK-1019: Attempts to stop reading the input stream since - * other completion callbacks might invalidate the input. Because interruption - * is not synchronous this still leaves a potential race where the interruption is - * processed only after the stream becomes invalid. - */ - context.addOnCompleteCallback{ () => - complete = true // Indicate that the task has completed successfully - context.interrupted = true } + writerThread.start() + new MonitorThread(env, worker, context).start() + // Return an iterator that read lines from the process's stdout val stream = new DataInputStream(new BufferedInputStream(worker.getInputStream, bufferSize)) val stdoutIterator = new Iterator[Array[Byte]] { def next(): Array[Byte] = { val obj = _nextObj if (hasNext) { - // FIXME: can deadlock if worker is waiting for us to - // respond to current message (currently irrelevant because - // output is shutdown before we read any input) _nextObj = read() } obj } private def read(): Array[Byte] = { - if (readerException != null) { - throw readerException + if (writerThread.exception.isDefined) { + throw writerThread.exception.get } try { stream.readInt() match { @@ -181,13 +105,14 @@ private[spark] class PythonRDD[T: ClassTag]( val total = finishTime - startTime logInfo("Times: total = %s, boot = %s, init = %s, finish = %s".format(total, boot, init, finish)) - read + read() case SpecialLengths.PYTHON_EXCEPTION_THROWN => // Signals that an exception has been thrown in python val exLength = stream.readInt() val obj = new Array[Byte](exLength) stream.readFully(obj) - throw new PythonException(new String(obj, "utf-8"), readerException) + throw new PythonException(new String(obj, "utf-8"), + writerThread.exception.getOrElse(null)) case SpecialLengths.END_OF_DATA_SECTION => // We've finished the data section of the output, but we can still // read some accumulator updates: @@ -201,10 +126,15 @@ private[spark] class PythonRDD[T: ClassTag]( Array.empty[Byte] } } catch { - case e: Exception if readerException != null => + + case e: Exception if context.interrupted => + logDebug("Exception thrown after task interruption", e) + throw new TaskKilledException + + case e: Exception if writerThread.exception.isDefined => logError("Python worker exited unexpectedly (crashed)", e) - logError("Python crash may have been caused by prior exception:", readerException) - throw readerException + logError("This may have been caused by a prior exception:", writerThread.exception.get) + throw writerThread.exception.get case eof: EOFException => throw new SparkException("Python worker exited unexpectedly (crashed)", eof) @@ -215,10 +145,100 @@ private[spark] class PythonRDD[T: ClassTag]( def hasNext = _nextObj.length != 0 } - stdoutIterator + new InterruptibleIterator(context, stdoutIterator) } val asJavaRDD : JavaRDD[Array[Byte]] = JavaRDD.fromRDD(this) + + /** + * The thread responsible for writing the data from the PythonRDD's parent iterator to the + * Python process. + */ + class WriterThread(env: SparkEnv, worker: Socket, split: Partition, context: TaskContext) + extends Thread(s"stdout writer for $pythonExec") { + + @volatile private var _exception: Exception = null + + setDaemon(true) + + /** Contains the exception thrown while writing the parent iterator to the Python process. */ + def exception: Option[Exception] = Option(_exception) + + /** Terminates the writer thread, ignoring any exceptions that may occur due to cleanup. */ + def shutdownOnTaskCompletion() { + assert(context.completed) + this.interrupt() + } + + override def run() { + try { + SparkEnv.set(env) + val stream = new BufferedOutputStream(worker.getOutputStream, bufferSize) + val dataOut = new DataOutputStream(stream) + // Partition index + dataOut.writeInt(split.index) + // sparkFilesDir + PythonRDD.writeUTF(SparkFiles.getRootDirectory, dataOut) + // Broadcast variables + dataOut.writeInt(broadcastVars.length) + for (broadcast <- broadcastVars) { + dataOut.writeLong(broadcast.id) + dataOut.writeInt(broadcast.value.length) + dataOut.write(broadcast.value) + } + // Python includes (*.zip and *.egg files) + dataOut.writeInt(pythonIncludes.length) + for (include <- pythonIncludes) { + PythonRDD.writeUTF(include, dataOut) + } + dataOut.flush() + // Serialized command: + dataOut.writeInt(command.length) + dataOut.write(command) + // Data values + PythonRDD.writeIteratorToStream(parent.iterator(split, context), dataOut) + dataOut.flush() + } catch { + case e: Exception if context.completed || context.interrupted => + logDebug("Exception thrown after task completion (likely due to cleanup)", e) + + case e: Exception => + // We must avoid throwing exceptions here, because the thread uncaught exception handler + // will kill the whole executor (see org.apache.spark.executor.Executor). + _exception = e + } finally { + Try(worker.shutdownOutput()) // kill Python worker process + } + } + } + + /** + * It is necessary to have a monitor thread for python workers if the user cancels with + * interrupts disabled. In that case we will need to explicitly kill the worker, otherwise the + * threads can block indefinitely. + */ + class MonitorThread(env: SparkEnv, worker: Socket, context: TaskContext) + extends Thread(s"Worker Monitor for $pythonExec") { + + setDaemon(true) + + override def run() { + // Kill the worker if it is interrupted, checking until task completion. + // TODO: This has a race condition if interruption occurs, as completed may still become true. + while (!context.interrupted && !context.completed) { + Thread.sleep(2000) + } + if (!context.completed) { + try { + logWarning("Incomplete task interrupted: Attempting to kill Python Worker") + env.destroyPythonWorker(pythonExec, envVars.toMap) + } catch { + case e: Exception => + logError("Exception when trying to kill worker", e) + } + } + } + } } /** Thrown for exceptions in user Python code. */ diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala b/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala new file mode 100644 index 0000000000000..6d3e257c4d5df --- /dev/null +++ b/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.api.python + +import java.io.{File, InputStream, IOException, OutputStream} + +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.SparkContext + +private[spark] object PythonUtils { + /** Get the PYTHONPATH for PySpark, either from SPARK_HOME, if it is set, or from our JAR */ + def sparkPythonPath: String = { + val pythonPath = new ArrayBuffer[String] + for (sparkHome <- sys.env.get("SPARK_HOME")) { + pythonPath += Seq(sparkHome, "python").mkString(File.separator) + pythonPath += Seq(sparkHome, "python", "lib", "py4j-0.8.1-src.zip").mkString(File.separator) + } + pythonPath ++= SparkContext.jarOfObject(this) + pythonPath.mkString(File.pathSeparator) + } + + /** Merge PYTHONPATHS with the appropriate separator. Ignores blank strings. */ + def mergePythonPaths(paths: String*): String = { + paths.filter(_ != "").mkString(File.pathSeparator) + } +} + + +/** + * A utility class to redirect the child process's stdout or stderr. + */ +private[spark] class RedirectThread( + in: InputStream, + out: OutputStream, + name: String) + extends Thread(name) { + + setDaemon(true) + override def run() { + scala.util.control.Exception.ignoring(classOf[IOException]) { + // FIXME: We copy the stream on the level of bytes to avoid encoding problems. + val buf = new Array[Byte](1024) + var len = in.read(buf) + while (len != -1) { + out.write(buf, 0, len) + out.flush() + len = in.read(buf) + } + } + } +} diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala index a5f0f3d5e7eae..002f2acd94dee 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala @@ -17,15 +17,18 @@ package org.apache.spark.api.python -import java.io.{DataInputStream, File, IOException, OutputStreamWriter} +import java.io.{DataInputStream, InputStream, OutputStreamWriter} import java.net.{InetAddress, ServerSocket, Socket, SocketException} import scala.collection.JavaConversions._ import org.apache.spark._ +import org.apache.spark.util.Utils private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String, String]) - extends Logging { + extends Logging { + + import PythonWorkerFactory._ // Because forking processes from Java is expensive, we prefer to launch a single Python daemon // (pyspark/daemon.py) and tell it to fork new workers for our tasks. This daemon currently @@ -37,6 +40,11 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String val daemonHost = InetAddress.getByAddress(Array(127, 0, 0, 1)) var daemonPort: Int = 0 + val pythonPath = PythonUtils.mergePythonPaths( + PythonUtils.sparkPythonPath, + envVars.getOrElse("PYTHONPATH", ""), + sys.env.getOrElse("PYTHONPATH", "")) + def create(): Socket = { if (useDaemon) { createThroughDaemon() @@ -58,12 +66,11 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String try { new Socket(daemonHost, daemonPort) } catch { - case exc: SocketException => { + case exc: SocketException => logWarning("Python daemon unexpectedly quit, attempting to restart") stopDaemon() startDaemon() new Socket(daemonHost, daemonPort) - } case e: Throwable => throw e } } @@ -78,47 +85,14 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String serverSocket = new ServerSocket(0, 1, InetAddress.getByAddress(Array(127, 0, 0, 1))) // Create and start the worker - val sparkHome = new ProcessBuilder().environment().get("SPARK_HOME") - val pb = new ProcessBuilder(Seq(pythonExec, sparkHome + "/python/pyspark/worker.py")) + val pb = new ProcessBuilder(Seq(pythonExec, "-u", "-m", "pyspark.worker")) val workerEnv = pb.environment() workerEnv.putAll(envVars) - val pythonPath = sparkHome + "/python/" + File.pathSeparator + workerEnv.get("PYTHONPATH") workerEnv.put("PYTHONPATH", pythonPath) val worker = pb.start() - // Redirect the worker's stderr to ours - new Thread("stderr reader for " + pythonExec) { - setDaemon(true) - override def run() { - scala.util.control.Exception.ignoring(classOf[IOException]) { - // FIXME: We copy the stream on the level of bytes to avoid encoding problems. - val in = worker.getErrorStream - val buf = new Array[Byte](1024) - var len = in.read(buf) - while (len != -1) { - System.err.write(buf, 0, len) - len = in.read(buf) - } - } - } - }.start() - - // Redirect worker's stdout to our stderr - new Thread("stdout reader for " + pythonExec) { - setDaemon(true) - override def run() { - scala.util.control.Exception.ignoring(classOf[IOException]) { - // FIXME: We copy the stream on the level of bytes to avoid encoding problems. - val in = worker.getInputStream - val buf = new Array[Byte](1024) - var len = in.read(buf) - while (len != -1) { - System.err.write(buf, 0, len) - len = in.read(buf) - } - } - } - }.start() + // Redirect worker stdout and stderr + redirectStreamsToStderr(worker.getInputStream, worker.getErrorStream) // Tell the worker our port val out = new OutputStreamWriter(worker.getOutputStream) @@ -141,10 +115,6 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String null } - def stop() { - stopDaemon() - } - private def startDaemon() { synchronized { // Is it already running? @@ -154,54 +124,44 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String try { // Create and start the daemon - val sparkHome = new ProcessBuilder().environment().get("SPARK_HOME") - val pb = new ProcessBuilder(Seq(pythonExec, sparkHome + "/python/pyspark/daemon.py")) + val pb = new ProcessBuilder(Seq(pythonExec, "-u", "-m", "pyspark.daemon")) val workerEnv = pb.environment() workerEnv.putAll(envVars) - val pythonPath = sparkHome + "/python/" + File.pathSeparator + workerEnv.get("PYTHONPATH") workerEnv.put("PYTHONPATH", pythonPath) daemon = pb.start() - // Redirect the stderr to ours - new Thread("stderr reader for " + pythonExec) { - setDaemon(true) - override def run() { - scala.util.control.Exception.ignoring(classOf[IOException]) { - // FIXME: We copy the stream on the level of bytes to avoid encoding problems. - val in = daemon.getErrorStream - val buf = new Array[Byte](1024) - var len = in.read(buf) - while (len != -1) { - System.err.write(buf, 0, len) - len = in.read(buf) - } - } - } - }.start() - val in = new DataInputStream(daemon.getInputStream) daemonPort = in.readInt() - // Redirect further stdout output to our stderr - new Thread("stdout reader for " + pythonExec) { - setDaemon(true) - override def run() { - scala.util.control.Exception.ignoring(classOf[IOException]) { - // FIXME: We copy the stream on the level of bytes to avoid encoding problems. - val buf = new Array[Byte](1024) - var len = in.read(buf) - while (len != -1) { - System.err.write(buf, 0, len) - len = in.read(buf) - } - } - } - }.start() + // Redirect daemon stdout and stderr + redirectStreamsToStderr(in, daemon.getErrorStream) + } catch { - case e: Throwable => { + case e: Exception => + + // If the daemon exists, wait for it to finish and get its stderr + val stderr = Option(daemon) + .flatMap { d => Utils.getStderr(d, PROCESS_WAIT_TIMEOUT_MS) } + .getOrElse("") + stopDaemon() - throw e - } + + if (stderr != "") { + val formattedStderr = stderr.replace("\n", "\n ") + val errorMessage = s""" + |Error from python worker: + | $formattedStderr + |PYTHONPATH was: + | $pythonPath + |$e""" + + // Append error message from python daemon, but keep original stack trace + val wrappedException = new SparkException(errorMessage.stripMargin) + wrappedException.setStackTrace(e.getStackTrace) + throw wrappedException + } else { + throw e + } } // Important: don't close daemon's stdin (daemon.getOutputStream) so it can correctly @@ -209,6 +169,19 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String } } + /** + * Redirect the given streams to our stderr in separate threads. + */ + private def redirectStreamsToStderr(stdout: InputStream, stderr: InputStream) { + try { + new RedirectThread(stdout, System.err, "stdout reader for " + pythonExec).start() + new RedirectThread(stderr, System.err, "stderr reader for " + pythonExec).start() + } catch { + case e: Exception => + logError("Exception in redirecting streams", e) + } + } + private def stopDaemon() { synchronized { // Request shutdown of existing daemon by sending SIGTERM @@ -220,4 +193,12 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String daemonPort = 0 } } + + def stop() { + stopDaemon() + } +} + +private object PythonWorkerFactory { + val PROCESS_WAIT_TIMEOUT_MS = 10000 } diff --git a/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala b/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala new file mode 100644 index 0000000000000..e20d4486c8f0c --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy + +import scala.collection.mutable.ArrayBuffer +import scala.collection.JavaConversions._ + +import org.apache.spark.api.python.{PythonUtils, RedirectThread} + +/** + * A main class used by spark-submit to launch Python applications. It executes python as a + * subprocess and then has it connect back to the JVM to access system properties, etc. + */ +object PythonRunner { + def main(args: Array[String]) { + val primaryResource = args(0) + val pyFiles = args(1) + val otherArgs = args.slice(2, args.length) + + val pythonExec = sys.env.get("PYSPARK_PYTHON").getOrElse("python") // TODO: get this from conf + + // Launch a Py4J gateway server for the process to connect to; this will let it see our + // Java system properties and such + val gatewayServer = new py4j.GatewayServer(null, 0) + gatewayServer.start() + + // Build up a PYTHONPATH that includes the Spark assembly JAR (where this class is), the + // python directories in SPARK_HOME (if set), and any files in the pyFiles argument + val pathElements = new ArrayBuffer[String] + pathElements ++= pyFiles.split(",") + pathElements += PythonUtils.sparkPythonPath + pathElements += sys.env.getOrElse("PYTHONPATH", "") + val pythonPath = PythonUtils.mergePythonPaths(pathElements: _*) + + // Launch Python process + val builder = new ProcessBuilder(Seq(pythonExec, "-u", primaryResource) ++ otherArgs) + val env = builder.environment() + env.put("PYTHONPATH", pythonPath) + env.put("PYSPARK_GATEWAY_PORT", "" + gatewayServer.getListeningPort) + builder.redirectErrorStream(true) // Ugly but needed for stdout and stderr to synchronize + val process = builder.start() + + new RedirectThread(process.getInputStream, System.out, "redirect output").start() + + System.exit(process.waitFor()) + } +} diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index 498fcc520ac5e..e2df1b8954124 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -24,25 +24,36 @@ import org.apache.hadoop.mapred.JobConf import org.apache.hadoop.security.Credentials import org.apache.hadoop.security.UserGroupInformation -import org.apache.spark.{SparkContext, SparkException} +import org.apache.spark.{Logging, SparkContext, SparkException} import scala.collection.JavaConversions._ /** * Contains util methods to interact with Hadoop from Spark. */ -class SparkHadoopUtil { +class SparkHadoopUtil extends Logging { val conf: Configuration = newConfiguration() UserGroupInformation.setConfiguration(conf) - def runAsUser(user: String)(func: () => Unit) { + /** + * Runs the given function with a Hadoop UserGroupInformation as a thread local variable + * (distributed to child threads), used for authenticating HDFS and YARN calls. + * + * IMPORTANT NOTE: If this function is going to be called repeated in the same process + * you need to look https://issues.apache.org/jira/browse/HDFS-3545 and possibly + * do a FileSystem.closeAllForUGI in order to avoid leaking Filesystems + */ + def runAsSparkUser(func: () => Unit) { + val user = Option(System.getenv("SPARK_USER")).getOrElse(SparkContext.SPARK_UNKNOWN_USER) if (user != SparkContext.SPARK_UNKNOWN_USER) { + logDebug("running as user: " + user) val ugi = UserGroupInformation.createRemoteUser(user) transferCredentials(UserGroupInformation.getCurrentUser(), ugi) ugi.doAs(new PrivilegedExceptionAction[Unit] { def run: Unit = func() }) } else { + logDebug("running as SPARK_UNKNOWN_USER") func() } } 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 d131f1809c99e..e39723f38347c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -18,6 +18,7 @@ package org.apache.spark.deploy import java.io.{File, PrintStream} +import java.lang.reflect.InvocationTargetException import java.net.{URI, URL} import scala.collection.mutable.{ArrayBuffer, HashMap, Map} @@ -59,11 +60,11 @@ object SparkSubmit { private[spark] var exitFn: () => Unit = () => System.exit(-1) private[spark] def printErrorAndExit(str: String) = { - printStream.println("error: " + str) - printStream.println("run with --help for more information or --verbose for debugging output") + printStream.println("Error: " + str) + printStream.println("Run with --help for usage help or --verbose for debug output") exitFn() } - private[spark] def printWarning(str: String) = printStream.println("warning: " + str) + private[spark] def printWarning(str: String) = printStream.println("Warning: " + str) /** * @return @@ -71,43 +72,43 @@ object SparkSubmit { * entries for the child, a list of system propertes, a list of env vars * and the main class for the child */ - private[spark] def createLaunchEnv(appArgs: SparkSubmitArguments): (ArrayBuffer[String], + private[spark] def createLaunchEnv(args: SparkSubmitArguments): (ArrayBuffer[String], ArrayBuffer[String], Map[String, String], String) = { - if (appArgs.master.startsWith("local")) { + if (args.master.startsWith("local")) { clusterManager = LOCAL - } else if (appArgs.master.startsWith("yarn")) { + } else if (args.master.startsWith("yarn")) { clusterManager = YARN - } else if (appArgs.master.startsWith("spark")) { + } else if (args.master.startsWith("spark")) { clusterManager = STANDALONE - } else if (appArgs.master.startsWith("mesos")) { + } else if (args.master.startsWith("mesos")) { clusterManager = MESOS } else { - printErrorAndExit("master must start with yarn, mesos, spark, or local") + printErrorAndExit("Master must start with yarn, mesos, spark, or local") } // Because "yarn-cluster" and "yarn-client" encapsulate both the master // and deploy mode, we have some logic to infer the master and deploy mode // from each other if only one is specified, or exit early if they are at odds. - if (appArgs.deployMode == null && - (appArgs.master == "yarn-standalone" || appArgs.master == "yarn-cluster")) { - appArgs.deployMode = "cluster" + if (args.deployMode == null && + (args.master == "yarn-standalone" || args.master == "yarn-cluster")) { + args.deployMode = "cluster" } - if (appArgs.deployMode == "cluster" && appArgs.master == "yarn-client") { + if (args.deployMode == "cluster" && args.master == "yarn-client") { printErrorAndExit("Deploy mode \"cluster\" and master \"yarn-client\" are not compatible") } - if (appArgs.deployMode == "client" && - (appArgs.master == "yarn-standalone" || appArgs.master == "yarn-cluster")) { - printErrorAndExit("Deploy mode \"client\" and master \"" + appArgs.master + if (args.deployMode == "client" && + (args.master == "yarn-standalone" || args.master == "yarn-cluster")) { + printErrorAndExit("Deploy mode \"client\" and master \"" + args.master + "\" are not compatible") } - if (appArgs.deployMode == "cluster" && appArgs.master.startsWith("yarn")) { - appArgs.master = "yarn-cluster" + if (args.deployMode == "cluster" && args.master.startsWith("yarn")) { + args.master = "yarn-cluster" } - if (appArgs.deployMode != "cluster" && appArgs.master.startsWith("yarn")) { - appArgs.master = "yarn-client" + if (args.deployMode != "cluster" && args.master.startsWith("yarn")) { + args.master = "yarn-client" } - val deployOnCluster = Option(appArgs.deployMode).getOrElse("client") == "cluster" + val deployOnCluster = Option(args.deployMode).getOrElse("client") == "cluster" val childClasspath = new ArrayBuffer[String]() val childArgs = new ArrayBuffer[String]() @@ -115,76 +116,93 @@ object SparkSubmit { var childMainClass = "" if (clusterManager == MESOS && deployOnCluster) { - printErrorAndExit("Mesos does not support running the driver on the cluster") + printErrorAndExit("Cannot currently run driver on the cluster in Mesos") } + // If we're running a Python app, set the Java class to run to be our PythonRunner, add + // Python files to deployment list, and pass the main file and Python path to PythonRunner + if (args.isPython) { + if (deployOnCluster) { + printErrorAndExit("Cannot currently run Python driver programs on cluster") + } + args.mainClass = "org.apache.spark.deploy.PythonRunner" + args.files = mergeFileLists(args.files, args.pyFiles, args.primaryResource) + val pyFiles = Option(args.pyFiles).getOrElse("") + args.childArgs = ArrayBuffer(args.primaryResource, pyFiles) ++ args.childArgs + args.primaryResource = RESERVED_JAR_NAME + sysProps("spark.submit.pyFiles") = pyFiles + } + + // If we're deploying into YARN, use yarn.Client as a wrapper around the user class if (!deployOnCluster) { - childMainClass = appArgs.mainClass - if (appArgs.primaryResource != RESERVED_JAR_NAME) { - childClasspath += appArgs.primaryResource + childMainClass = args.mainClass + if (args.primaryResource != RESERVED_JAR_NAME) { + childClasspath += args.primaryResource } } else if (clusterManager == YARN) { childMainClass = "org.apache.spark.deploy.yarn.Client" - childArgs += ("--jar", appArgs.primaryResource) - childArgs += ("--class", appArgs.mainClass) + childArgs += ("--jar", args.primaryResource) + childArgs += ("--class", args.mainClass) } + // Make sure YARN is included in our build if we're trying to use it if (clusterManager == YARN) { - // The choice of class is arbitrary, could use any spark-yarn class if (!Utils.classIsLoadable("org.apache.spark.deploy.yarn.Client") && !Utils.isTesting) { - val msg = "Could not load YARN classes. This copy of Spark may not have been compiled " + - "with YARN support." - throw new Exception(msg) + printErrorAndExit("Could not load YARN classes. " + + "This copy of Spark may not have been compiled with YARN support.") } } - + // Special flag to avoid deprecation warnings at the client sysProps("SPARK_SUBMIT") = "true" + // A list of rules to map each argument to system properties or command-line options in + // each deploy mode; we iterate through these below val options = List[OptionAssigner]( - new OptionAssigner(appArgs.master, ALL_CLUSTER_MGRS, false, sysProp = "spark.master"), - new OptionAssigner(appArgs.driverExtraClassPath, STANDALONE | YARN, true, + OptionAssigner(args.master, ALL_CLUSTER_MGRS, false, sysProp = "spark.master"), + OptionAssigner(args.driverExtraClassPath, STANDALONE | YARN, true, sysProp = "spark.driver.extraClassPath"), - new OptionAssigner(appArgs.driverExtraJavaOptions, STANDALONE | YARN, true, + OptionAssigner(args.driverExtraJavaOptions, STANDALONE | YARN, true, sysProp = "spark.driver.extraJavaOptions"), - new OptionAssigner(appArgs.driverExtraLibraryPath, STANDALONE | YARN, true, + OptionAssigner(args.driverExtraLibraryPath, STANDALONE | YARN, true, sysProp = "spark.driver.extraLibraryPath"), - new OptionAssigner(appArgs.driverMemory, YARN, true, clOption = "--driver-memory"), - new OptionAssigner(appArgs.name, YARN, true, clOption = "--name"), - new OptionAssigner(appArgs.queue, YARN, true, clOption = "--queue"), - new OptionAssigner(appArgs.queue, YARN, false, sysProp = "spark.yarn.queue"), - new OptionAssigner(appArgs.numExecutors, YARN, true, clOption = "--num-executors"), - new OptionAssigner(appArgs.numExecutors, YARN, false, sysProp = "spark.executor.instances"), - new OptionAssigner(appArgs.executorMemory, YARN, true, clOption = "--executor-memory"), - new OptionAssigner(appArgs.executorMemory, STANDALONE | MESOS | YARN, false, + OptionAssigner(args.driverMemory, YARN, true, clOption = "--driver-memory"), + OptionAssigner(args.name, YARN, true, clOption = "--name"), + OptionAssigner(args.queue, YARN, true, clOption = "--queue"), + OptionAssigner(args.queue, YARN, false, sysProp = "spark.yarn.queue"), + OptionAssigner(args.numExecutors, YARN, true, clOption = "--num-executors"), + OptionAssigner(args.numExecutors, YARN, false, sysProp = "spark.executor.instances"), + OptionAssigner(args.executorMemory, YARN, true, clOption = "--executor-memory"), + OptionAssigner(args.executorMemory, STANDALONE | MESOS | YARN, false, sysProp = "spark.executor.memory"), - new OptionAssigner(appArgs.driverMemory, STANDALONE, true, clOption = "--memory"), - new OptionAssigner(appArgs.driverCores, STANDALONE, true, clOption = "--cores"), - new OptionAssigner(appArgs.executorCores, YARN, true, clOption = "--executor-cores"), - new OptionAssigner(appArgs.executorCores, YARN, false, sysProp = "spark.executor.cores"), - new OptionAssigner(appArgs.totalExecutorCores, STANDALONE | MESOS, false, + OptionAssigner(args.driverMemory, STANDALONE, true, clOption = "--memory"), + OptionAssigner(args.driverCores, STANDALONE, true, clOption = "--cores"), + OptionAssigner(args.executorCores, YARN, true, clOption = "--executor-cores"), + OptionAssigner(args.executorCores, YARN, false, sysProp = "spark.executor.cores"), + OptionAssigner(args.totalExecutorCores, STANDALONE | MESOS, false, sysProp = "spark.cores.max"), - new OptionAssigner(appArgs.files, YARN, false, sysProp = "spark.yarn.dist.files"), - new OptionAssigner(appArgs.files, YARN, true, clOption = "--files"), - new OptionAssigner(appArgs.archives, YARN, false, sysProp = "spark.yarn.dist.archives"), - new OptionAssigner(appArgs.archives, YARN, true, clOption = "--archives"), - new OptionAssigner(appArgs.jars, YARN, true, clOption = "--addJars"), - new OptionAssigner(appArgs.files, LOCAL | STANDALONE | MESOS, true, sysProp = "spark.files"), - new OptionAssigner(appArgs.jars, LOCAL | STANDALONE | MESOS, false, sysProp = "spark.jars"), - new OptionAssigner(appArgs.name, LOCAL | STANDALONE | MESOS, false, - sysProp = "spark.app.name") + OptionAssigner(args.files, YARN, false, sysProp = "spark.yarn.dist.files"), + OptionAssigner(args.files, YARN, true, clOption = "--files"), + OptionAssigner(args.archives, YARN, false, sysProp = "spark.yarn.dist.archives"), + OptionAssigner(args.archives, YARN, true, clOption = "--archives"), + OptionAssigner(args.jars, YARN, true, clOption = "--addJars"), + OptionAssigner(args.files, LOCAL | STANDALONE | MESOS, false, sysProp = "spark.files"), + OptionAssigner(args.files, LOCAL | STANDALONE | MESOS, true, sysProp = "spark.files"), + OptionAssigner(args.jars, LOCAL | STANDALONE | MESOS, false, sysProp = "spark.jars"), + OptionAssigner(args.name, LOCAL | STANDALONE | MESOS, false, sysProp = "spark.app.name") ) // For client mode make any added jars immediately visible on the classpath - if (appArgs.jars != null && !deployOnCluster) { - for (jar <- appArgs.jars.split(",")) { + if (args.jars != null && !deployOnCluster) { + for (jar <- args.jars.split(",")) { childClasspath += jar } } + // Map all arguments to command-line options or system properties for our chosen mode for (opt <- options) { if (opt.value != null && deployOnCluster == opt.deployOnCluster && - (clusterManager & opt.clusterManager) != 0) { + (clusterManager & opt.clusterManager) != 0) { if (opt.clOption != null) { childArgs += (opt.clOption, opt.value) } else if (opt.sysProp != null) { @@ -196,32 +214,35 @@ object SparkSubmit { // For standalone mode, add the application jar automatically so the user doesn't have to // call sc.addJar. TODO: Standalone mode in the cluster if (clusterManager == STANDALONE) { - val existingJars = sysProps.get("spark.jars").map(x => x.split(",").toSeq).getOrElse(Seq()) - sysProps.put("spark.jars", (existingJars ++ Seq(appArgs.primaryResource)).mkString(",")) + var jars = sysProps.get("spark.jars").map(x => x.split(",").toSeq).getOrElse(Seq()) + if (args.primaryResource != RESERVED_JAR_NAME) { + jars = jars ++ Seq(args.primaryResource) + } + sysProps.put("spark.jars", jars.mkString(",")) } if (deployOnCluster && clusterManager == STANDALONE) { - if (appArgs.supervise) { + if (args.supervise) { childArgs += "--supervise" } childMainClass = "org.apache.spark.deploy.Client" childArgs += "launch" - childArgs += (appArgs.master, appArgs.primaryResource, appArgs.mainClass) + childArgs += (args.master, args.primaryResource, args.mainClass) } // Arguments to be passed to user program - if (appArgs.childArgs != null) { + if (args.childArgs != null) { if (!deployOnCluster || clusterManager == STANDALONE) { - childArgs ++= appArgs.childArgs + childArgs ++= args.childArgs } else if (clusterManager == YARN) { - for (arg <- appArgs.childArgs) { + for (arg <- args.childArgs) { childArgs += ("--arg", arg) } } } - for ((k, v) <- appArgs.getDefaultSparkProperties) { + for ((k, v) <- args.getDefaultSparkProperties) { if (!sysProps.contains(k)) sysProps(k) = v } @@ -229,8 +250,8 @@ object SparkSubmit { } private def launch(childArgs: ArrayBuffer[String], childClasspath: ArrayBuffer[String], - sysProps: Map[String, String], childMainClass: String, verbose: Boolean = false) { - + sysProps: Map[String, String], childMainClass: String, verbose: Boolean = false) + { if (verbose) { printStream.println(s"Main class:\n$childMainClass") printStream.println(s"Arguments:\n${childArgs.mkString("\n")}") @@ -253,7 +274,14 @@ object SparkSubmit { val mainClass = Class.forName(childMainClass, true, loader) val mainMethod = mainClass.getMethod("main", new Array[String](0).getClass) - mainMethod.invoke(null, childArgs.toArray) + try { + mainMethod.invoke(null, childArgs.toArray) + } catch { + case e: InvocationTargetException => e.getCause match { + case cause: Throwable => throw cause + case null => throw e + } + } } private def addJarToClasspath(localJar: String, loader: ExecutorURLClassLoader) { @@ -265,15 +293,26 @@ object SparkSubmit { val url = localJarFile.getAbsoluteFile.toURI.toURL loader.addURL(url) } + + /** + * Merge a sequence of comma-separated file lists, some of which may be null to indicate + * no files, into a single comma-separated string. + */ + private[spark] def mergeFileLists(lists: String*): String = { + val merged = lists.filter(_ != null) + .flatMap(_.split(",")) + .mkString(",") + if (merged == "") null else merged + } } /** * Provides an indirection layer for passing arguments as system properties or flags to * the user's driver program or to downstream launcher tools. */ -private[spark] class OptionAssigner(val value: String, - val clusterManager: Int, - val deployOnCluster: Boolean, - val clOption: String = null, - val sysProp: String = null -) { } +private[spark] case class OptionAssigner( + value: String, + clusterManager: Int, + deployOnCluster: Boolean, + clOption: String = null, + sysProp: String = null) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 5834dc40f1f95..2d327aa3fb27f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -19,6 +19,7 @@ package org.apache.spark.deploy import java.io.{File, FileInputStream, IOException} import java.util.Properties +import java.util.jar.JarFile import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, HashMap} @@ -52,6 +53,8 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { var childArgs: ArrayBuffer[String] = new ArrayBuffer[String]() var jars: String = null var verbose: Boolean = false + var isPython: Boolean = false + var pyFiles: String = null parseOpts(args.toList) loadDefaults() @@ -76,7 +79,7 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { } /** Fill in any undefined values based on the current properties file or built-in defaults. */ - private def loadDefaults() = { + private def loadDefaults(): Unit = { // Use common defaults file, if not specified by user if (propertiesFile == null) { @@ -107,15 +110,43 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { master = Option(master).getOrElse(System.getenv("MASTER")) deployMode = Option(deployMode).getOrElse(System.getenv("DEPLOY_MODE")) + // Try to set main class from JAR if no --class argument is given + if (mainClass == null && !isPython && primaryResource != null) { + try { + val jar = new JarFile(primaryResource) + // Note that this might still return null if no main-class is set; we catch that later + mainClass = jar.getManifest.getMainAttributes.getValue("Main-Class") + } catch { + case e: Exception => + SparkSubmit.printErrorAndExit("Failed to read JAR: " + primaryResource) + return + } + } + // Global defaults. These should be keep to minimum to avoid confusing behavior. master = Option(master).getOrElse("local[*]") + + // Set name from main class if not given + name = Option(name).orElse(Option(mainClass)).orNull + if (name == null && primaryResource != null) { + name = Utils.stripDirectory(primaryResource) + } } /** Ensure that required fields exists. Call this only once all defaults are loaded. */ private def checkRequiredArguments() = { - if (args.length == 0) printUsageAndExit(-1) - if (primaryResource == null) SparkSubmit.printErrorAndExit("Must specify a primary resource") - if (mainClass == null) SparkSubmit.printErrorAndExit("Must specify a main class with --class") + if (args.length == 0) { + printUsageAndExit(-1) + } + if (primaryResource == null) { + SparkSubmit.printErrorAndExit("Must specify a primary resource (JAR or Python file)") + } + if (mainClass == null && !isPython) { + SparkSubmit.printErrorAndExit("No main class set in JAR; please specify one with --class") + } + if (pyFiles != null && !isPython) { + SparkSubmit.printErrorAndExit("--py-files given but primary resource is not a Python script") + } if (master.startsWith("yarn")) { val hasHadoopEnv = sys.env.contains("HADOOP_CONF_DIR") || sys.env.contains("YARN_CONF_DIR") @@ -143,6 +174,7 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { | queue $queue | numExecutors $numExecutors | files $files + | pyFiles $pyFiles | archives $archives | mainClass $mainClass | primaryResource $primaryResource @@ -234,6 +266,10 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { files = value parse(tail) + case ("--py-files") :: value :: tail => + pyFiles = value + parse(tail) + case ("--archives") :: value :: tail => archives = value parse(tail) @@ -260,9 +296,10 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { val errMessage = s"Unrecognized option '$value'." SparkSubmit.printErrorAndExit(errMessage) case v => - primaryResource = v - inSparkOpts = false - parse(tail) + primaryResource = v + inSparkOpts = false + isPython = v.endsWith(".py") + parse(tail) } } else { childArgs += value @@ -270,7 +307,7 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { } case Nil => - } + } } private def printUsageAndExit(exitCode: Int, unknownParam: Any = null) { @@ -279,25 +316,26 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { outStream.println("Unknown/unsupported param " + unknownParam) } outStream.println( - """Usage: spark-submit [options] [app options] + """Usage: spark-submit [options] [app options] |Options: | --master MASTER_URL spark://host:port, mesos://host:port, yarn, or local. - | --deploy-mode DEPLOY_MODE Mode to deploy the app in, either 'client' or 'cluster'. - | --class CLASS_NAME Name of your app's main class (required for Java apps). - | --arg ARG Argument to be passed to your application's main class. This - | option can be specified multiple times for multiple args. - | --name NAME The name of your application (Default: 'Spark'). - | --jars JARS A comma-separated list of local jars to include on the - | driver classpath and that SparkContext.addJar will work - | with. Doesn't work on standalone with 'cluster' deploy mode. - | --files FILES Comma separated list of files to be placed in the working dir - | of each executor. + | --deploy-mode DEPLOY_MODE Where to run the driver program: either "client" to run + | on the local machine, or "cluster" to run inside cluster. + | --class CLASS_NAME Your application's main class (for Java / Scala apps). + | --name NAME A name of your application. + | --jars JARS Comma-separated list of local jars to include on the driver + | and executor classpaths. Doesn't work for drivers in + | standalone mode with "cluster" deploy mode. + | --py-files PY_FILES Comma-separated list of .zip or .egg files to place on the + | PYTHONPATH for Python apps. + | --files FILES Comma-separated list of files to be placed in the working + | directory of each executor. | --properties-file FILE Path to a file from which to load extra properties. If not | specified, this will look for conf/spark-defaults.conf. | | --driver-memory MEM Memory for driver (e.g. 1000M, 2G) (Default: 512M). - | --driver-java-options Extra Java options to pass to the driver - | --driver-library-path Extra library path entries to pass to the driver + | --driver-java-options Extra Java options to pass to the driver. + | --driver-library-path Extra library path entries to pass to the driver. | --driver-class-path Extra class path entries to pass to the driver. Note that | jars added with --jars are automatically included in the | classpath. @@ -313,10 +351,10 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { | | YARN-only: | --executor-cores NUM Number of cores per executor (Default: 1). - | --queue QUEUE_NAME The YARN queue to submit to (Default: 'default'). - | --num-executors NUM Number of executors to (Default: 2). + | --queue QUEUE_NAME The YARN queue to submit to (Default: "default"). + | --num-executors NUM Number of executors to launch (Default: 2). | --archives ARCHIVES Comma separated list of archives to be extracted into the - | working dir of each executor.""".stripMargin + | working directory of each executor.""".stripMargin ) SparkSubmit.exitFn() } @@ -335,6 +373,6 @@ object SparkSubmitArguments { val message = s"Failed when loading Spark properties file ${file.getName}" throw new SparkException(message, e) } - properties.stringPropertyNames().toSeq.map(k => (k, properties(k))) + properties.stringPropertyNames().toSeq.map(k => (k, properties(k).trim)) } } diff --git a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala index 8901806de9262..896913d796604 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala @@ -60,6 +60,7 @@ private[spark] class AppClient( var master: ActorSelection = null var alreadyDisconnected = false // To avoid calling listener.disconnected() multiple times var alreadyDead = false // To avoid calling listener.dead() multiple times + var registrationRetryTimer: Option[Cancellable] = None override def preStart() { context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) @@ -83,22 +84,20 @@ private[spark] class AppClient( def registerWithMaster() { tryRegisterAllMasters() - import context.dispatcher var retries = 0 - lazy val retryTimer: Cancellable = + registrationRetryTimer = Some { context.system.scheduler.schedule(REGISTRATION_TIMEOUT, REGISTRATION_TIMEOUT) { retries += 1 if (registered) { - retryTimer.cancel() + registrationRetryTimer.foreach(_.cancel()) } else if (retries >= REGISTRATION_RETRIES) { - logError("All masters are unresponsive! Giving up.") - markDead() + markDead("All masters are unresponsive! Giving up.") } else { tryRegisterAllMasters() } } - retryTimer // start timer + } } def changeMaster(url: String) { @@ -126,8 +125,7 @@ private[spark] class AppClient( listener.connected(appId) case ApplicationRemoved(message) => - logError("Master removed our application: %s; stopping client".format(message)) - markDisconnected() + markDead("Master removed our application: %s".format(message)) context.stop(self) case ExecutorAdded(id: Int, workerId: String, hostPort: String, cores: Int, memory: Int) => @@ -158,7 +156,7 @@ private[spark] class AppClient( logWarning(s"Could not connect to $address: $cause") case StopAppClient => - markDead() + markDead("Application has been stopped.") sender ! true context.stop(self) } @@ -173,12 +171,17 @@ private[spark] class AppClient( } } - def markDead() { + def markDead(reason: String) { if (!alreadyDead) { - listener.dead() + listener.dead(reason) alreadyDead = true } } + + override def postStop() { + registrationRetryTimer.foreach(_.cancel()) + } + } def start() { diff --git a/core/src/main/scala/org/apache/spark/deploy/client/AppClientListener.scala b/core/src/main/scala/org/apache/spark/deploy/client/AppClientListener.scala index 1f20aa3dfa39b..e584952a9ad85 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/AppClientListener.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/AppClientListener.scala @@ -30,8 +30,8 @@ private[spark] trait AppClientListener { /** Disconnection may be a temporary state, as we fail over to a new Master. */ def disconnected(): Unit - /** Dead means that we couldn't find any Masters to connect to, and have given up. */ - def dead(): Unit + /** An application death is an unrecoverable failure condition. */ + def dead(reason: String): Unit def executorAdded(fullId: String, workerId: String, hostPort: String, cores: Int, memory: Int) diff --git a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala index 888dd45e93c6a..e15a87bd38fda 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala @@ -33,8 +33,8 @@ private[spark] object TestClient { System.exit(0) } - def dead() { - logInfo("Could not connect to master") + def dead(reason: String) { + logInfo("Application died with error: " + reason) System.exit(0) } 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 3e615e753b342..c7f0f244ea2fc 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala @@ -47,9 +47,16 @@ object CommandUtils extends Logging { */ def buildJavaOpts(command: Command, memory: Int, sparkHome: String): Seq[String] = { val memoryOpts = Seq(s"-Xms${memory}M", s"-Xmx${memory}M") - // Note, this will coalesce multiple options into a single command component val extraOpts = command.extraJavaOptions.map(Utils.splitCommandString).getOrElse(Seq()) + // Exists for backwards compatibility with older Spark versions + val workerLocalOpts = Option(getenv("SPARK_JAVA_OPTS")).map(Utils.splitCommandString) + .getOrElse(Nil) + if (workerLocalOpts.length > 0) { + logWarning("SPARK_JAVA_OPTS was set on the worker. It is deprecated in Spark 1.0.") + logWarning("Set SPARK_LOCAL_DIRS for node-specific storage locations.") + } + val libraryOpts = if (command.libraryPathEntries.size > 0) { val joined = command.libraryPathEntries.mkString(File.pathSeparator) @@ -66,7 +73,7 @@ object CommandUtils extends Logging { val userClassPath = command.classPathEntries ++ Seq(classPath) Seq("-cp", userClassPath.filterNot(_.isEmpty).mkString(File.pathSeparator)) ++ - libraryOpts ++ extraOpts ++ memoryOpts + libraryOpts ++ extraOpts ++ workerLocalOpts ++ memoryOpts } /** Spawn a thread that will redirect a given stream to a file */ diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index dd0a1360abe14..85d25dc7dbfa4 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -71,8 +71,6 @@ private[spark] class Worker( // TTL for app folders/data; after TTL expires it will be cleaned up val APP_DATA_RETENTION_SECS = conf.getLong("spark.worker.cleanup.appDataTtl", 7 * 24 * 3600) - // Index into masterUrls that we're currently trying to register with. - var masterIndex = 0 val masterLock: Object = new Object() var master: ActorSelection = null @@ -102,6 +100,8 @@ private[spark] class Worker( val metricsSystem = MetricsSystem.createMetricsSystem("worker", conf, securityMgr) val workerSource = new WorkerSource(this) + var registrationRetryTimer: Option[Cancellable] = None + def coresFree: Int = cores - coresUsed def memoryFree: Int = memory - memoryUsed @@ -163,13 +163,12 @@ private[spark] class Worker( def registerWithMaster() { tryRegisterAllMasters() - var retries = 0 - lazy val retryTimer: Cancellable = + registrationRetryTimer = Some { context.system.scheduler.schedule(REGISTRATION_TIMEOUT, REGISTRATION_TIMEOUT) { retries += 1 if (registered) { - retryTimer.cancel() + registrationRetryTimer.foreach(_.cancel()) } else if (retries >= REGISTRATION_RETRIES) { logError("All masters are unresponsive! Giving up.") System.exit(1) @@ -177,7 +176,7 @@ private[spark] class Worker( tryRegisterAllMasters() } } - retryTimer // start timer + } } override def receive = { @@ -244,7 +243,7 @@ private[spark] class Worker( } } catch { case e: Exception => { - logError("Failed to launch exector %s/%d for %s".format(appId, execId, appDesc.name)) + logError("Failed to launch executor %s/%d for %s".format(appId, execId, appDesc.name)) if (executors.contains(appId + "/" + execId)) { executors(appId + "/" + execId).kill() executors -= appId + "/" + execId @@ -346,6 +345,7 @@ private[spark] class Worker( } override def postStop() { + registrationRetryTimer.foreach(_.cancel()) executors.values.foreach(_.kill()) drivers.values.foreach(_.kill()) webUi.stop() diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala index 3836bf219ed3e..dc5158102054e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala @@ -22,7 +22,7 @@ import java.lang.management.ManagementFactory import org.apache.spark.util.{IntParam, MemoryParam, Utils} /** - * Command-line parser for the master. + * Command-line parser for the worker. */ private[spark] class WorkerArguments(args: Array[String]) { var host = Utils.localHostName() diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 9ac7365f47f9f..e912ae8a5d3c5 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -22,8 +22,9 @@ import java.nio.ByteBuffer import akka.actor._ import akka.remote._ -import org.apache.spark.{SecurityManager, SparkConf, Logging} +import org.apache.spark.{Logging, SecurityManager, SparkConf} import org.apache.spark.TaskState.TaskState +import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.worker.WorkerWatcher import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ import org.apache.spark.util.{AkkaUtils, Utils} @@ -94,25 +95,30 @@ private[spark] class CoarseGrainedExecutorBackend( private[spark] object CoarseGrainedExecutorBackend { def run(driverUrl: String, executorId: String, hostname: String, cores: Int, - workerUrl: Option[String]) { - // Debug code - Utils.checkHost(hostname) - - val conf = new SparkConf - // Create a new ActorSystem to run the backend, because we can't create a SparkEnv / Executor - // before getting started with all our system properties, etc - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("sparkExecutor", hostname, 0, - indestructible = true, conf = conf, new SecurityManager(conf)) - // set it - val sparkHostPort = hostname + ":" + boundPort - actorSystem.actorOf( - Props(classOf[CoarseGrainedExecutorBackend], driverUrl, executorId, - sparkHostPort, cores), - name = "Executor") - workerUrl.foreach{ url => - actorSystem.actorOf(Props(classOf[WorkerWatcher], url), name = "WorkerWatcher") + workerUrl: Option[String]) { + + SparkHadoopUtil.get.runAsSparkUser { () => + // Debug code + Utils.checkHost(hostname) + + val conf = new SparkConf + // Create a new ActorSystem to run the backend, because we can't create a + // SparkEnv / Executor before getting started with all our system properties, etc + val (actorSystem, boundPort) = AkkaUtils.createActorSystem("sparkExecutor", hostname, 0, + indestructible = true, conf = conf, new SecurityManager(conf)) + // set it + val sparkHostPort = hostname + ":" + boundPort + actorSystem.actorOf( + Props(classOf[CoarseGrainedExecutorBackend], driverUrl, executorId, + sparkHostPort, cores), + name = "Executor") + workerUrl.foreach { + url => + actorSystem.actorOf(Props(classOf[WorkerWatcher], url), name = "WorkerWatcher") + } + actorSystem.awaitTermination() + } - actorSystem.awaitTermination() } def main(args: Array[String]) { diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 272bcda5f8f2f..98e7e0be813be 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -128,8 +128,6 @@ private[spark] class Executor( // Maintains the list of running tasks. private val runningTasks = new ConcurrentHashMap[Long, TaskRunner] - val sparkUser = Option(System.getenv("SPARK_USER")).getOrElse(SparkContext.SPARK_UNKNOWN_USER) - def launchTask(context: ExecutorBackend, taskId: Long, serializedTask: ByteBuffer) { val tr = new TaskRunner(context, taskId, serializedTask) runningTasks.put(taskId, tr) @@ -172,7 +170,7 @@ private[spark] class Executor( } } - override def run(): Unit = SparkHadoopUtil.get.runAsUser(sparkUser) { () => + override def run() { val startTime = System.currentTimeMillis() SparkEnv.set(env) Thread.currentThread.setContextClassLoader(replClassLoader) diff --git a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala index 64e24506e8038..9b56f711e0e0b 100644 --- a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala @@ -23,10 +23,10 @@ import com.google.protobuf.ByteString import org.apache.mesos.{Executor => MesosExecutor, ExecutorDriver, MesosExecutorDriver, MesosNativeLibrary} import org.apache.mesos.Protos.{TaskStatus => MesosTaskStatus, _} -import org.apache.spark.Logging -import org.apache.spark.TaskState +import org.apache.spark.{Logging, TaskState} import org.apache.spark.TaskState.TaskState import org.apache.spark.util.Utils +import org.apache.spark.deploy.SparkHadoopUtil private[spark] class MesosExecutorBackend extends MesosExecutor @@ -95,9 +95,11 @@ private[spark] class MesosExecutorBackend */ private[spark] object MesosExecutorBackend { def main(args: Array[String]) { - MesosNativeLibrary.load() - // Create a new Executor and start it running - val runner = new MesosExecutorBackend() - new MesosExecutorDriver(runner).run() + SparkHadoopUtil.get.runAsSparkUser { () => + MesosNativeLibrary.load() + // Create a new Executor and start it running + val runner = new MesosExecutorBackend() + new MesosExecutorDriver(runner).run() + } } } diff --git a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala index e1a5ee316bb69..4b0fe1ab82999 100644 --- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala +++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala @@ -24,6 +24,7 @@ import org.xerial.snappy.{SnappyInputStream, SnappyOutputStream} import org.apache.spark.SparkConf import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.util.Utils /** * :: DeveloperApi :: @@ -49,7 +50,7 @@ private[spark] object CompressionCodec { } def createCodec(conf: SparkConf, codecName: String): CompressionCodec = { - val ctor = Class.forName(codecName, true, Thread.currentThread.getContextClassLoader) + val ctor = Class.forName(codecName, true, Utils.getContextOrSparkClassLoader) .getConstructor(classOf[SparkConf]) ctor.newInstance(conf).asInstanceOf[CompressionCodec] } diff --git a/core/src/main/scala/org/apache/spark/network/SecurityMessage.scala b/core/src/main/scala/org/apache/spark/network/SecurityMessage.scala index a1dfc4094cca7..9af9e2e8e9e59 100644 --- a/core/src/main/scala/org/apache/spark/network/SecurityMessage.scala +++ b/core/src/main/scala/org/apache/spark/network/SecurityMessage.scala @@ -106,7 +106,6 @@ private[spark] class SecurityMessage() extends Logging { * @return BufferMessage */ def toBufferMessage: BufferMessage = { - val startTime = System.currentTimeMillis val buffers = new ArrayBuffer[ByteBuffer]() // 4 bytes for the length of the connectionId diff --git a/core/src/main/scala/org/apache/spark/partial/GroupedMeanEvaluator.scala b/core/src/main/scala/org/apache/spark/partial/GroupedMeanEvaluator.scala index b5111891ed35b..af26c3d59ac02 100644 --- a/core/src/main/scala/org/apache/spark/partial/GroupedMeanEvaluator.scala +++ b/core/src/main/scala/org/apache/spark/partial/GroupedMeanEvaluator.scala @@ -61,7 +61,6 @@ private[spark] class GroupedMeanEvaluator[T](totalOutputs: Int, confidence: Doub } else if (outputsMerged == 0) { new HashMap[T, BoundedDouble] } else { - val p = outputsMerged.toDouble / totalOutputs val studentTCacher = new StudentTCacher(confidence) val result = new JHashMap[T, BoundedDouble](sums.size) val iter = sums.entrySet.iterator() diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 3b3524f33e811..a1ca612cc9a09 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -128,7 +128,7 @@ abstract class RDD[T: ClassTag]( @transient var name: String = null /** Assign a name to this RDD */ - def setName(_name: String): RDD[T] = { + def setName(_name: String): this.type = { name = _name this } @@ -138,7 +138,7 @@ abstract class RDD[T: ClassTag]( * it is computed. This can only be used to assign a new storage level if the RDD does not * have a storage level set yet.. */ - def persist(newLevel: StorageLevel): RDD[T] = { + def persist(newLevel: StorageLevel): this.type = { // TODO: Handle changes of StorageLevel if (storageLevel != StorageLevel.NONE && newLevel != storageLevel) { throw new UnsupportedOperationException( @@ -152,10 +152,10 @@ abstract class RDD[T: ClassTag]( } /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ - def persist(): RDD[T] = persist(StorageLevel.MEMORY_ONLY) + def persist(): this.type = persist(StorageLevel.MEMORY_ONLY) /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ - def cache(): RDD[T] = persist() + def cache(): this.type = persist() /** * Mark the RDD as non-persistent, and remove all blocks for it from memory and disk. @@ -163,7 +163,7 @@ abstract class RDD[T: ClassTag]( * @param blocking Whether to block until all blocks are deleted. * @return This RDD. */ - def unpersist(blocking: Boolean = true): RDD[T] = { + def unpersist(blocking: Boolean = true): this.type = { logInfo("Removing RDD " + id + " from persistence list") sc.unpersistRDD(id, blocking) storageLevel = StorageLevel.NONE diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index d822a8e55111a..7968a0691db10 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -18,13 +18,16 @@ package org.apache.spark.scheduler import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.fs.permission.FsPermission +import org.json4s.JsonAST.JValue import org.json4s.jackson.JsonMethods._ import org.apache.spark.{Logging, SparkConf, SparkContext} +import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.io.CompressionCodec import org.apache.spark.util.{FileLogger, JsonProtocol} @@ -40,31 +43,36 @@ import org.apache.spark.util.{FileLogger, JsonProtocol} */ private[spark] class EventLoggingListener( appName: String, - conf: SparkConf, - hadoopConfiguration: Configuration) + sparkConf: SparkConf, + hadoopConf: Configuration = SparkHadoopUtil.get.newConfiguration()) extends SparkListener with Logging { import EventLoggingListener._ - private val shouldCompress = conf.getBoolean("spark.eventLog.compress", false) - private val shouldOverwrite = conf.getBoolean("spark.eventLog.overwrite", false) - private val outputBufferSize = conf.getInt("spark.eventLog.buffer.kb", 100) * 1024 - private val logBaseDir = conf.get("spark.eventLog.dir", "/tmp/spark-events").stripSuffix("/") + private val shouldCompress = sparkConf.getBoolean("spark.eventLog.compress", false) + private val shouldOverwrite = sparkConf.getBoolean("spark.eventLog.overwrite", false) + private val testing = sparkConf.getBoolean("spark.eventLog.testing", false) + private val outputBufferSize = sparkConf.getInt("spark.eventLog.buffer.kb", 100) * 1024 + private val logBaseDir = sparkConf.get("spark.eventLog.dir", DEFAULT_LOG_DIR).stripSuffix("/") private val name = appName.replaceAll("[ :/]", "-").toLowerCase + "-" + System.currentTimeMillis val logDir = logBaseDir + "/" + name - private val logger = - new FileLogger(logDir, conf, hadoopConfiguration, outputBufferSize, shouldCompress, - shouldOverwrite, Some(LOG_FILE_PERMISSIONS)) + protected val logger = new FileLogger(logDir, sparkConf, hadoopConf, outputBufferSize, + shouldCompress, shouldOverwrite, Some(LOG_FILE_PERMISSIONS)) + + // For testing. Keep track of all JSON serialized events that have been logged. + private[scheduler] val loggedEvents = new ArrayBuffer[JValue] /** * Begin logging events. * If compression is used, log a file that indicates which compression library is used. */ def start() { + logger.start() logInfo("Logging events to %s".format(logDir)) if (shouldCompress) { - val codec = conf.get("spark.io.compression.codec", CompressionCodec.DEFAULT_COMPRESSION_CODEC) + val codec = + sparkConf.get("spark.io.compression.codec", CompressionCodec.DEFAULT_COMPRESSION_CODEC) logger.newFile(COMPRESSION_CODEC_PREFIX + codec) } logger.newFile(SPARK_VERSION_PREFIX + SparkContext.SPARK_VERSION) @@ -73,11 +81,14 @@ private[spark] class EventLoggingListener( /** Log the event as JSON. */ private def logEvent(event: SparkListenerEvent, flushLogger: Boolean = false) { - val eventJson = compact(render(JsonProtocol.sparkEventToJson(event))) - logger.logLine(eventJson) + val eventJson = JsonProtocol.sparkEventToJson(event) + logger.logLine(compact(render(eventJson))) if (flushLogger) { logger.flush() } + if (testing) { + loggedEvents += eventJson + } } // Events that do not trigger a flush @@ -121,13 +132,12 @@ private[spark] class EventLoggingListener( } private[spark] object EventLoggingListener extends Logging { + val DEFAULT_LOG_DIR = "/tmp/spark-events" val LOG_PREFIX = "EVENT_LOG_" val SPARK_VERSION_PREFIX = "SPARK_VERSION_" val COMPRESSION_CODEC_PREFIX = "COMPRESSION_CODEC_" val APPLICATION_COMPLETE = "APPLICATION_COMPLETE" - val LOG_FILE_PERMISSIONS: FsPermission = - FsPermission.createImmutable(Integer.parseInt("770", 8).toShort) - + val LOG_FILE_PERMISSIONS = FsPermission.createImmutable(Integer.parseInt("770", 8).toShort) // A cache for compression codecs to avoid creating the same codec many times private val codecMap = new mutable.HashMap[String, CompressionCodec] diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala index 02b62de7e36b6..2259df0b56bad 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala @@ -17,11 +17,13 @@ package org.apache.spark.scheduler +import scala.language.existentials + import java.io._ import java.util.zip.{GZIPInputStream, GZIPOutputStream} import scala.collection.mutable.HashMap -import scala.language.existentials +import scala.util.Try import org.apache.spark._ import org.apache.spark.executor.ShuffleWriteMetrics @@ -196,7 +198,11 @@ private[spark] class ShuffleMapTask( } finally { // Release the writers back to the shuffle block manager. if (shuffle != null && shuffle.writers != null) { - shuffle.releaseWriters(success) + try { + shuffle.releaseWriters(success) + } catch { + case e: Exception => logError("Failed to release shuffle writers", e) + } } // Execute the callbacks on task completion. context.executeOnCompleteCallbacks() diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala index d6df193d9bcf8..0286aac8769b2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala @@ -37,7 +37,7 @@ private[spark] trait SparkListenerBus { * Post an event to all attached listeners. This does nothing if the event is * SparkListenerShutdown. */ - protected def postToAll(event: SparkListenerEvent) { + def postToAll(event: SparkListenerEvent) { event match { case stageSubmitted: SparkListenerStageSubmitted => sparkListeners.foreach(_.onStageSubmitted(stageSubmitted)) 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 9544ca05dca70..cefa41729964a 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 @@ -83,10 +83,10 @@ private[spark] class SparkDeploySchedulerBackend( } } - override def dead() { + override def dead(reason: String) { if (!stopping) { - logError("Spark cluster looks dead, giving up.") - scheduler.error("Spark cluster looks down") + logError("Application has been killed. Reason: " + reason) + scheduler.error(reason) } } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala index a02dd9441d679..408a797088059 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala @@ -118,11 +118,9 @@ object BlockFetcherIterator { }) bytesInFlight += req.size val sizeMap = req.blocks.toMap // so we can look up the size of each blockID - val fetchStart = System.currentTimeMillis() val future = connectionManager.sendMessageReliably(cmId, blockMessageArray.toBufferMessage) future.onSuccess { case Some(message) => { - val fetchDone = System.currentTimeMillis() val bufferMessage = message.asInstanceOf[BufferMessage] val blockMessageArray = BlockMessageArray.fromBufferMessage(bufferMessage) for (blockMessage <- blockMessageArray) { diff --git a/core/src/main/scala/org/apache/spark/storage/BlockId.scala b/core/src/main/scala/org/apache/spark/storage/BlockId.scala index cffea28fbf794..42ec181b00bb3 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockId.scala @@ -19,14 +19,18 @@ package org.apache.spark.storage import java.util.UUID +import org.apache.spark.annotation.DeveloperApi + /** + * :: DeveloperApi :: * Identifies a particular Block of data, usually associated with a single file. * A Block can be uniquely identified by its filename, but each type of Block has a different * set of keys which produce its unique name. * * If your BlockId should be serializable, be sure to add it to the BlockId.apply() method. */ -private[spark] sealed abstract class BlockId { +@DeveloperApi +sealed abstract class BlockId { /** A globally unique identifier for this Block. Can be used for ser/de. */ def name: String @@ -44,24 +48,29 @@ private[spark] sealed abstract class BlockId { } } -private[spark] case class RDDBlockId(rddId: Int, splitIndex: Int) extends BlockId { +@DeveloperApi +case class RDDBlockId(rddId: Int, splitIndex: Int) extends BlockId { def name = "rdd_" + rddId + "_" + splitIndex } -private[spark] case class ShuffleBlockId(shuffleId: Int, mapId: Int, reduceId: Int) +@DeveloperApi +case class ShuffleBlockId(shuffleId: Int, mapId: Int, reduceId: Int) extends BlockId { def name = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId } -private[spark] case class BroadcastBlockId(broadcastId: Long, field: String = "") extends BlockId { +@DeveloperApi +case class BroadcastBlockId(broadcastId: Long, field: String = "") extends BlockId { def name = "broadcast_" + broadcastId + (if (field == "") "" else "_" + field) } -private[spark] case class TaskResultBlockId(taskId: Long) extends BlockId { +@DeveloperApi +case class TaskResultBlockId(taskId: Long) extends BlockId { def name = "taskresult_" + taskId } -private[spark] case class StreamBlockId(streamId: Int, uniqueId: Long) extends BlockId { +@DeveloperApi +case class StreamBlockId(streamId: Int, uniqueId: Long) extends BlockId { def name = "input-" + streamId + "-" + uniqueId } @@ -75,7 +84,8 @@ private[spark] case class TestBlockId(id: String) extends BlockId { def name = "test_" + id } -private[spark] object BlockId { +@DeveloperApi +object BlockId { val RDD = "rdd_([0-9]+)_([0-9]+)".r val SHUFFLE = "shuffle_([0-9]+)_([0-9]+)_([0-9]+)".r val BROADCAST = "broadcast_([0-9]+)([_A-Za-z0-9]*)".r diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala index be537d77309bc..b1585bd8199d1 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala @@ -20,17 +20,20 @@ package org.apache.spark.storage import java.io.{Externalizable, IOException, ObjectInput, ObjectOutput} import java.util.concurrent.ConcurrentHashMap +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.util.Utils /** + * :: DeveloperApi :: * This class represent an unique identifier for a BlockManager. - * The first 2 constructors of this class is made private to ensure that - * BlockManagerId objects can be created only using the apply method in - * the companion object. This allows de-duplication of ID objects. - * Also, constructor parameters are private to ensure that parameters cannot - * be modified from outside this class. + * + * The first 2 constructors of this class is made private to ensure that BlockManagerId objects + * can be created only using the apply method in the companion object. This allows de-duplication + * of ID objects. Also, constructor parameters are private to ensure that parameters cannot be + * modified from outside this class. */ -private[spark] class BlockManagerId private ( +@DeveloperApi +class BlockManagerId private ( private var executorId_ : String, private var host_ : String, private var port_ : Int, diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala index 63fa5d3eb6541..98fa0df6ec289 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -28,6 +28,7 @@ import akka.actor.{Actor, ActorRef, Cancellable} import akka.pattern.ask import org.apache.spark.{Logging, SparkConf, SparkException} +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.scheduler._ import org.apache.spark.storage.BlockManagerMessages._ import org.apache.spark.util.{AkkaUtils, Utils} @@ -411,7 +412,8 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus } } -private[spark] case class BlockStatus( +@DeveloperApi +case class BlockStatus( storageLevel: StorageLevel, memSize: Long, diskSize: Long, diff --git a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala index c9a52e0366d93..363de93e067b8 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala @@ -22,14 +22,17 @@ import java.io.{Externalizable, IOException, ObjectInput, ObjectOutput} import org.apache.spark.annotation.DeveloperApi /** + * :: DeveloperApi :: * Flags for controlling the storage of an RDD. Each StorageLevel records whether to use memory, * or Tachyon, whether to drop the RDD to disk if it falls out of memory or Tachyon , whether to * keep the data in memory in a serialized format, and whether to replicate the RDD partitions on * multiple nodes. + * * The [[org.apache.spark.storage.StorageLevel$]] singleton object contains some static constants * for commonly useful storage levels. To create your own storage level object, use the * factory method of the singleton object (`StorageLevel(...)`). */ +@DeveloperApi class StorageLevel private( private var useDisk_ : Boolean, private var useMemory_ : Boolean, @@ -54,9 +57,9 @@ class StorageLevel private( assert(replication < 40, "Replication restricted to be less than 40 for calculating hashcodes") if (useOffHeap) { - require(useDisk == false, "Off-heap storage level does not support using disk") - require(useMemory == false, "Off-heap storage level does not support using heap memory") - require(deserialized == false, "Off-heap storage level does not support deserialized storage") + require(!useDisk, "Off-heap storage level does not support using disk") + require(!useMemory, "Off-heap storage level does not support using heap memory") + require(!deserialized, "Off-heap storage level does not support deserialized storage") require(replication == 1, "Off-heap storage level does not support multiple replication") } @@ -146,7 +149,7 @@ object StorageLevel { /** * :: DeveloperApi :: - * Create a new StorageLevel object without setting useOffHeap + * Create a new StorageLevel object without setting useOffHeap. */ @DeveloperApi def apply(useDisk: Boolean, useMemory: Boolean, useOffHeap: Boolean, @@ -155,7 +158,7 @@ object StorageLevel { /** * :: DeveloperApi :: - * Create a new StorageLevel object + * Create a new StorageLevel object. */ @DeveloperApi def apply(useDisk: Boolean, useMemory: Boolean, @@ -164,7 +167,7 @@ object StorageLevel { /** * :: DeveloperApi :: - * Create a new StorageLevel object from its integer representation + * Create a new StorageLevel object from its integer representation. */ @DeveloperApi def apply(flags: Int, replication: Int): StorageLevel = @@ -172,7 +175,7 @@ object StorageLevel { /** * :: DeveloperApi :: - * Read StorageLevel object from ObjectInput stream + * Read StorageLevel object from ObjectInput stream. */ @DeveloperApi def apply(in: ObjectInput): StorageLevel = { diff --git a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala index 7a174959037be..a6e6627d54e01 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala @@ -19,12 +19,15 @@ package org.apache.spark.storage import scala.collection.mutable +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.scheduler._ /** - * A SparkListener that maintains executor storage status + * :: DeveloperApi :: + * A SparkListener that maintains executor storage status. */ -private[spark] class StorageStatusListener extends SparkListener { +@DeveloperApi +class StorageStatusListener extends SparkListener { private val executorIdToStorageStatus = mutable.Map[String, StorageStatus]() def storageStatusList = executorIdToStorageStatus.values.toSeq diff --git a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala index 1eddd1cdc483b..6f3252a2f6d31 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -21,9 +21,14 @@ import scala.collection.Map import scala.collection.mutable import org.apache.spark.SparkContext +import org.apache.spark.annotation.DeveloperApi -/** Storage information for each BlockManager. */ -private[spark] class StorageStatus( +/** + * :: DeveloperApi :: + * Storage information for each BlockManager. + */ +@DeveloperApi +class StorageStatus( val blockManagerId: BlockManagerId, val maxMem: Long, val blocks: mutable.Map[BlockId, BlockStatus] = mutable.Map.empty) { diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentTab.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentTab.scala index 03b46e1bd59af..bbbe55ecf44a1 100644 --- a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentTab.scala @@ -17,6 +17,7 @@ package org.apache.spark.ui.env +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.scheduler._ import org.apache.spark.ui._ @@ -30,9 +31,11 @@ private[ui] class EnvironmentTab(parent: SparkUI) extends WebUITab(parent, "envi } /** + * :: DeveloperApi :: * A SparkListener that prepares information to be displayed on the EnvironmentTab */ -private[ui] class EnvironmentListener extends SparkListener { +@DeveloperApi +class EnvironmentListener extends SparkListener { var jvmInformation = Seq[(String, String)]() var sparkProperties = Seq[(String, String)]() var systemProperties = Seq[(String, String)]() diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala index 5678bf34ac730..91d37b835b19d 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala @@ -20,6 +20,7 @@ package org.apache.spark.ui.exec import scala.collection.mutable.HashMap import org.apache.spark.ExceptionFailure +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.scheduler._ import org.apache.spark.storage.StorageStatusListener import org.apache.spark.ui.{SparkUI, WebUITab} @@ -34,9 +35,11 @@ private[ui] class ExecutorsTab(parent: SparkUI) extends WebUITab(parent, "execut } /** + * :: DeveloperApi :: * A SparkListener that prepares information to be displayed on the ExecutorsTab */ -private[ui] class ExecutorsListener(storageStatusListener: StorageStatusListener) +@DeveloperApi +class ExecutorsListener(storageStatusListener: StorageStatusListener) extends SparkListener { val executorToTasksActive = HashMap[String, Int]() diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorSummary.scala b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorSummary.scala index 1dfe1d4f1fa11..2aaf6329b792d 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorSummary.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorSummary.scala @@ -17,8 +17,14 @@ package org.apache.spark.ui.jobs -/** class for reporting aggregated metrics for each executors in stageUI */ -private[ui] class ExecutorSummary { +import org.apache.spark.annotation.DeveloperApi + +/** + * :: DeveloperApi :: + * Class for reporting aggregated metrics for each executor in stage UI. + */ +@DeveloperApi +class ExecutorSummary { var taskTime : Long = 0 var failedTasks : Int = 0 var succeededTasks : Int = 0 diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index 0db4afa701b41..396cbcbc8d268 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -20,19 +20,22 @@ package org.apache.spark.ui.jobs import scala.collection.mutable.{HashMap, ListBuffer} import org.apache.spark.{ExceptionFailure, SparkConf, SparkContext, Success} +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.TaskMetrics import org.apache.spark.scheduler._ import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.storage.BlockManagerId /** + * :: DeveloperApi :: * Tracks task-level information to be displayed in the UI. * * All access to the data structures in this class must be synchronized on the * class, since the UI thread and the EventBus loop may otherwise be reading and * updating the internal data structures concurrently. */ -private[ui] class JobProgressListener(conf: SparkConf) extends SparkListener { +@DeveloperApi +class JobProgressListener(conf: SparkConf) extends SparkListener { import JobProgressListener._ @@ -246,7 +249,8 @@ private[ui] class JobProgressListener(conf: SparkConf) extends SparkListener { } -private[ui] case class TaskUIData( +@DeveloperApi +case class TaskUIData( taskInfo: TaskInfo, taskMetrics: Option[TaskMetrics] = None, exception: Option[ExceptionFailure] = None) diff --git a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala index 07ec297841f20..c4bb7aab50393 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala @@ -19,6 +19,7 @@ package org.apache.spark.ui.storage import scala.collection.mutable +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.ui._ import org.apache.spark.scheduler._ import org.apache.spark.storage.{RDDInfo, StorageStatusListener, StorageUtils} @@ -35,9 +36,11 @@ private[ui] class StorageTab(parent: SparkUI) extends WebUITab(parent, "storage" } /** - * A SparkListener that prepares information to be displayed on the BlockManagerUI + * :: DeveloperApi :: + * A SparkListener that prepares information to be displayed on the BlockManagerUI. */ -private[ui] class StorageListener(storageStatusListener: StorageStatusListener) +@DeveloperApi +class StorageListener(storageStatusListener: StorageStatusListener) extends SparkListener { private val _rddInfoMap = mutable.Map[Int, RDDInfo]() diff --git a/core/src/main/scala/org/apache/spark/util/FileLogger.scala b/core/src/main/scala/org/apache/spark/util/FileLogger.scala index 0965e0f0f7828..0e6d21b22023a 100644 --- a/core/src/main/scala/org/apache/spark/util/FileLogger.scala +++ b/core/src/main/scala/org/apache/spark/util/FileLogger.scala @@ -17,7 +17,7 @@ package org.apache.spark.util -import java.io.{FileOutputStream, BufferedOutputStream, PrintWriter, IOException} +import java.io.{BufferedOutputStream, FileOutputStream, IOException, PrintWriter} import java.net.URI import java.text.SimpleDateFormat import java.util.Date @@ -27,6 +27,7 @@ import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path} import org.apache.hadoop.fs.permission.FsPermission import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.io.CompressionCodec /** @@ -39,8 +40,8 @@ import org.apache.spark.io.CompressionCodec */ private[spark] class FileLogger( logDir: String, - conf: SparkConf, - hadoopConfiguration: Configuration, + sparkConf: SparkConf, + hadoopConf: Configuration = SparkHadoopUtil.get.newConfiguration(), outputBufferSize: Int = 8 * 1024, // 8 KB compress: Boolean = false, overwrite: Boolean = true, @@ -55,14 +56,19 @@ private[spark] class FileLogger( var fileIndex = 0 // Only used if compression is enabled - private lazy val compressionCodec = CompressionCodec.createCodec(conf) + private lazy val compressionCodec = CompressionCodec.createCodec(sparkConf) // Only defined if the file system scheme is not local private var hadoopDataStream: Option[FSDataOutputStream] = None private var writer: Option[PrintWriter] = None - createLogDir() + /** + * Start this logger by creating the logging directory. + */ + def start() { + createLogDir() + } /** * Create a logging directory with the given path. @@ -83,7 +89,7 @@ private[spark] class FileLogger( } if (dirPermissions.isDefined) { val fsStatus = fileSystem.getFileStatus(path) - if (fsStatus.getPermission().toShort() != dirPermissions.get.toShort) { + if (fsStatus.getPermission.toShort != dirPermissions.get.toShort) { fileSystem.setPermission(path, dirPermissions.get) } } @@ -92,14 +98,14 @@ private[spark] class FileLogger( /** * Create a new writer for the file identified by the given path. * If the permissions are not passed in, it will default to use the permissions - * (dirpermissions) used when class was instantiated. + * (dirPermissions) used when class was instantiated. */ private def createWriter(fileName: String, perms: Option[FsPermission] = None): PrintWriter = { val logPath = logDir + "/" + fileName val uri = new URI(logPath) - val defaultFs = FileSystem.getDefaultUri(hadoopConfiguration).getScheme - val isDefaultLocal = (defaultFs == null || defaultFs == "file") val path = new Path(logPath) + val defaultFs = FileSystem.getDefaultUri(hadoopConf).getScheme + val isDefaultLocal = defaultFs == null || defaultFs == "file" /* The Hadoop LocalFileSystem (r1.0.4) has known issues with syncing (HADOOP-7844). * Therefore, for local files, use FileOutputStream instead. */ @@ -112,7 +118,7 @@ private[spark] class FileLogger( hadoopDataStream.get } - perms.orElse(dirPermissions).foreach {p => fileSystem.setPermission(path, p)} + perms.orElse(dirPermissions).foreach { p => fileSystem.setPermission(path, p) } val bstream = new BufferedOutputStream(dstream, outputBufferSize) val cstream = if (compress) compressionCodec.compressedOutputStream(bstream) else bstream new PrintWriter(cstream) @@ -127,7 +133,7 @@ private[spark] class FileLogger( val writeInfo = if (!withTime) { msg } else { - val date = new Date(System.currentTimeMillis()) + val date = new Date(System.currentTimeMillis) dateFormat.get.format(date) + ": " + msg } writer.foreach(_.print(writeInfo)) diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 9aed3e0985654..09825087bb048 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -646,11 +646,11 @@ private[spark] object JsonProtocol { } def propertiesFromJson(json: JValue): Properties = { - val properties = new Properties() - if (json != JNothing) { - mapFromJson(json).map { case (k, v) => properties.setProperty(k, v) } - } - properties + Utils.jsonOption(json).map { value => + val properties = new Properties + mapFromJson(json).foreach { case (k, v) => properties.setProperty(k, v) } + properties + }.getOrElse(null) } def UUIDFromJson(json: JValue): UUID = { diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 79f314c8dd36c..3f0ed61c5bbfb 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -31,8 +31,8 @@ import scala.reflect.ClassTag import scala.util.Try import com.google.common.io.Files -import org.apache.commons.lang.SystemUtils import com.google.common.util.concurrent.ThreadFactoryBuilder +import org.apache.commons.lang3.SystemUtils import org.apache.hadoop.fs.{FileSystem, FileUtil, Path} import org.json4s._ import tachyon.client.{TachyonFile,TachyonFS} @@ -46,11 +46,10 @@ import org.apache.spark.serializer.{DeserializationStream, SerializationStream, * Various utility methods used by Spark. */ private[spark] object Utils extends Logging { - val random = new Random() def sparkBin(sparkHome: String, which: String): File = { - val suffix = if (SystemUtils.IS_OS_WINDOWS) ".cmd" else "" + val suffix = if (isWindows) ".cmd" else "" new File(sparkHome + File.separator + "bin", which + suffix) } @@ -221,9 +220,9 @@ private[spark] object Utils extends Logging { def hasRootAsShutdownDeleteDir(file: File): Boolean = { val absolutePath = file.getAbsolutePath() val retval = shutdownDeletePaths.synchronized { - shutdownDeletePaths.find { path => + shutdownDeletePaths.exists { path => !absolutePath.equals(path) && absolutePath.startsWith(path) - }.isDefined + } } if (retval) { logInfo("path = " + file + ", already present as root for deletion.") @@ -236,10 +235,10 @@ private[spark] object Utils extends Logging { // paths - resulting in Exception and incomplete cleanup. def hasRootAsShutdownDeleteDir(file: TachyonFile): Boolean = { val absolutePath = file.getPath() - val retval = shutdownDeletePaths.synchronized { - shutdownDeletePaths.find { path => + val retval = shutdownDeleteTachyonPaths.synchronized { + shutdownDeleteTachyonPaths.exists { path => !absolutePath.equals(path) && absolutePath.startsWith(path) - }.isDefined + } } if (retval) { logInfo("path = " + file + ", already present as root for deletion.") @@ -614,7 +613,7 @@ private[spark] object Utils extends Logging { */ def isSymlink(file: File): Boolean = { if (file == null) throw new NullPointerException("File must not be null") - if (SystemUtils.IS_OS_WINDOWS) return false + if (isWindows) return false val fileInCanonicalDir = if (file.getParent() == null) { file } else { @@ -1018,7 +1017,7 @@ private[spark] object Utils extends Logging { throw new IOException("Destination must be relative") } var cmdSuffix = "" - val linkCmd = if (SystemUtils.IS_OS_WINDOWS) { + val linkCmd = if (isWindows) { // refer to http://technet.microsoft.com/en-us/library/cc771254.aspx cmdSuffix = " /s /e /k /h /y /i" "cmd /c xcopy " @@ -1062,10 +1061,68 @@ private[spark] object Utils extends Logging { getHadoopFileSystem(new URI(path)) } + /** + * Return the absolute path of a file in the given directory. + */ + def getFilePath(dir: File, fileName: String): Path = { + assert(dir.isDirectory) + val path = new File(dir, fileName).getAbsolutePath + new Path(path) + } + + /** + * Return true if this is Windows. + */ + def isWindows = SystemUtils.IS_OS_WINDOWS + /** * Indicates whether Spark is currently running unit tests. */ - private[spark] def isTesting = { + def isTesting = { sys.env.contains("SPARK_TESTING") || sys.props.contains("spark.testing") } + + /** + * Strip the directory from a path name + */ + def stripDirectory(path: String): String = { + path.split(File.separator).last + } + + /** + * Wait for a process to terminate for at most the specified duration. + * Return whether the process actually terminated after the given timeout. + */ + def waitForProcess(process: Process, timeoutMs: Long): Boolean = { + var terminated = false + val startTime = System.currentTimeMillis + while (!terminated) { + try { + process.exitValue + terminated = true + } catch { + case e: IllegalThreadStateException => + // Process not terminated yet + if (System.currentTimeMillis - startTime > timeoutMs) { + return false + } + Thread.sleep(100) + } + } + true + } + + /** + * Return the stderr of a process after waiting for the process to terminate. + * If the process does not terminate within the specified timeout, return None. + */ + def getStderr(process: Process, timeoutMs: Long): Option[String] = { + val terminated = Utils.waitForProcess(process, timeoutMs) + if (terminated) { + Some(Source.fromInputStream(process.getErrorStream).getLines().mkString("\n")) + } else { + None + } + } + } diff --git a/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala b/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala index 19af4f8cbe428..4e363b74f4bef 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala @@ -115,10 +115,10 @@ class OpenHashSet[@specialized(Long, Int) T: ClassTag]( * The caller is responsible for calling rehashIfNeeded. * * Use (retval & POSITION_MASK) to get the actual position, and - * (retval & EXISTENCE_MASK) != 0 for prior existence. + * (retval & NONEXISTENCE_MASK) == 0 for prior existence. * * @return The position where the key is placed, plus the highest order bit is set if the key - * exists previously. + * does not exists previously. */ def addWithoutResize(k: T): Int = { var pos = hashcode(hasher.hash(k)) & _mask diff --git a/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala b/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala index 7f220383f9f8b..55b5713706178 100644 --- a/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala +++ b/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala @@ -91,7 +91,7 @@ private[spark] object XORShiftRandom { val xorRand = new XORShiftRandom(seed) // this is just to warm up the JIT - we're not timing anything - timeIt(1e6.toInt) { + timeIt(million) { javaRand.nextInt() xorRand.nextInt() } diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index 76c6f5af82a5e..c3e03cea917b3 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -182,13 +182,30 @@ public void call(String s) { Assert.assertEquals(2, foreachCalls); } - @Test - public void toLocalIterator() { - List correct = Arrays.asList(1, 2, 3, 4); - JavaRDD rdd = sc.parallelize(correct); - List result = Lists.newArrayList(rdd.toLocalIterator()); - Assert.assertTrue(correct.equals(result)); - } + @Test + public void toLocalIterator() { + List correct = Arrays.asList(1, 2, 3, 4); + JavaRDD rdd = sc.parallelize(correct); + List result = Lists.newArrayList(rdd.toLocalIterator()); + Assert.assertTrue(correct.equals(result)); + } + + @Test + public void zipWithUniqueId() { + List dataArray = Arrays.asList(1, 2, 3, 4); + JavaPairRDD zip = sc.parallelize(dataArray).zipWithUniqueId(); + JavaRDD indexes = zip.values(); + Assert.assertTrue(new HashSet(indexes.collect()).size() == 4); + } + + @Test + public void zipWithIndex() { + List dataArray = Arrays.asList(1, 2, 3, 4); + JavaPairRDD zip = sc.parallelize(dataArray).zipWithIndex(); + JavaRDD indexes = zip.values(); + List correctIndexes = Arrays.asList(0L, 1L, 2L, 3L); + Assert.assertTrue(indexes.collect().equals(correctIndexes)); + } @SuppressWarnings("unchecked") @Test diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index b3541b4a40b79..d7e3b22ed476e 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -83,7 +83,7 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { } test("handle binary specified but not class") { - testPrematureExit(Array("foo.jar"), "Must specify a main class") + testPrematureExit(Array("foo.jar"), "No main class") } test("handles arguments with --key=val") { @@ -94,9 +94,9 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { } test("handles arguments to user program") { - val clArgs = Seq("--name", "myApp", "userjar.jar", "some", "--random", "args", "here") + val clArgs = Seq("--name", "myApp", "--class", "Foo", "userjar.jar", "some", "--weird", "args") val appArgs = new SparkSubmitArguments(clArgs) - appArgs.childArgs should be (Seq("some", "--random", "args", "here")) + appArgs.childArgs should be (Seq("some", "--weird", "args")) } test("handles YARN cluster mode") { diff --git a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala new file mode 100644 index 0000000000000..95f5bcd855665 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -0,0 +1,400 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.scheduler + +import scala.collection.mutable +import scala.io.Source +import scala.util.Try + +import com.google.common.io.Files +import org.apache.hadoop.fs.{FileStatus, Path} +import org.json4s.jackson.JsonMethods._ +import org.scalatest.{BeforeAndAfter, FunSuite} + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.io.CompressionCodec +import org.apache.spark.util.{JsonProtocol, Utils} + +/** + * Test whether EventLoggingListener logs events properly. + * + * This tests whether EventLoggingListener actually creates special files while logging events, + * whether the parsing of these special files is correct, and whether the logged events can be + * read and deserialized into actual SparkListenerEvents. + */ +class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter { + private val fileSystem = Utils.getHadoopFileSystem("/") + private val allCompressionCodecs = Seq[String]( + "org.apache.spark.io.LZFCompressionCodec", + "org.apache.spark.io.SnappyCompressionCodec" + ) + private val testDir = Files.createTempDir() + private val logDirPath = Utils.getFilePath(testDir, "spark-events") + + after { + Try { fileSystem.delete(logDirPath, true) } + } + + test("Parse names of special files") { + testParsingFileName() + } + + test("Verify special files exist") { + testSpecialFilesExist() + } + + test("Verify special files exist with compression") { + allCompressionCodecs.foreach { codec => + testSpecialFilesExist(compressionCodec = Some(codec)) + } + } + + test("Parse event logging info") { + testParsingLogInfo() + } + + test("Parse event logging info with compression") { + allCompressionCodecs.foreach { codec => + testParsingLogInfo(compressionCodec = Some(codec)) + } + } + + test("Basic event logging") { + testEventLogging() + } + + test("Basic event logging with compression") { + allCompressionCodecs.foreach { codec => + testEventLogging(compressionCodec = Some(codec)) + } + } + + test("End-to-end event logging") { + testApplicationEventLogging() + } + + test("End-to-end event logging with compression") { + allCompressionCodecs.foreach { codec => + testApplicationEventLogging(compressionCodec = Some(codec)) + } + } + + + /* ----------------- * + * Actual test logic * + * ----------------- */ + + import EventLoggingListenerSuite._ + + /** + * Test whether names of special files are correctly identified and parsed. + */ + private def testParsingFileName() { + val logPrefix = EventLoggingListener.LOG_PREFIX + val sparkVersionPrefix = EventLoggingListener.SPARK_VERSION_PREFIX + val compressionCodecPrefix = EventLoggingListener.COMPRESSION_CODEC_PREFIX + val applicationComplete = EventLoggingListener.APPLICATION_COMPLETE + assert(EventLoggingListener.isEventLogFile(logPrefix + "0")) + assert(EventLoggingListener.isEventLogFile(logPrefix + "100")) + assert(EventLoggingListener.isEventLogFile(logPrefix + "ANYTHING")) + assert(EventLoggingListener.isSparkVersionFile(sparkVersionPrefix + "0.9.1")) + assert(EventLoggingListener.isSparkVersionFile(sparkVersionPrefix + "1.0.0")) + assert(EventLoggingListener.isSparkVersionFile(sparkVersionPrefix + "ANYTHING")) + assert(EventLoggingListener.isApplicationCompleteFile(applicationComplete)) + allCompressionCodecs.foreach { codec => + assert(EventLoggingListener.isCompressionCodecFile(compressionCodecPrefix + codec)) + } + + // Negatives + assert(!EventLoggingListener.isEventLogFile("The greatest man of all mankind")) + assert(!EventLoggingListener.isSparkVersionFile("Will never falter in the face of death!")) + assert(!EventLoggingListener.isCompressionCodecFile("Unless he chooses to leave behind")) + assert(!EventLoggingListener.isApplicationCompleteFile("The very treasure he calls Macbeth")) + + // Verify that parsing is correct + assert(EventLoggingListener.parseSparkVersion(sparkVersionPrefix + "1.0.0") === "1.0.0") + allCompressionCodecs.foreach { codec => + assert(EventLoggingListener.parseCompressionCodec(compressionCodecPrefix + codec) === codec) + } + } + + /** + * Test whether the special files produced by EventLoggingListener exist. + * + * There should be exactly one event log and one spark version file throughout the entire + * execution. If a compression codec is specified, then the compression codec file should + * also exist. Only after the application has completed does the test expect the application + * completed file to be present. + */ + private def testSpecialFilesExist(compressionCodec: Option[String] = None) { + + def assertFilesExist(logFiles: Array[FileStatus], loggerStopped: Boolean) { + val numCompressionCodecFiles = if (compressionCodec.isDefined) 1 else 0 + val numApplicationCompleteFiles = if (loggerStopped) 1 else 0 + assert(logFiles.size === 2 + numCompressionCodecFiles + numApplicationCompleteFiles) + assert(eventLogsExist(logFiles)) + assert(sparkVersionExists(logFiles)) + assert(compressionCodecExists(logFiles) === compressionCodec.isDefined) + assert(applicationCompleteExists(logFiles) === loggerStopped) + assertSparkVersionIsValid(logFiles) + compressionCodec.foreach { codec => + assertCompressionCodecIsValid(logFiles, codec) + } + } + + // Verify logging directory exists + val conf = getLoggingConf(logDirPath, compressionCodec) + val eventLogger = new EventLoggingListener("test", conf) + eventLogger.start() + val logPath = new Path(eventLogger.logDir) + assert(fileSystem.exists(logPath)) + val logDir = fileSystem.getFileStatus(logPath) + assert(logDir.isDir) + + // Verify special files are as expected before stop() + var logFiles = fileSystem.listStatus(logPath) + assert(logFiles != null) + assertFilesExist(logFiles, loggerStopped = false) + + // Verify special files are as expected after stop() + eventLogger.stop() + logFiles = fileSystem.listStatus(logPath) + assertFilesExist(logFiles, loggerStopped = true) + } + + /** + * Test whether EventLoggingListener correctly parses the correct information from the logs. + * + * This includes whether it returns the correct Spark version, compression codec (if any), + * and the application's completion status. + */ + private def testParsingLogInfo(compressionCodec: Option[String] = None) { + + def assertInfoCorrect(info: EventLoggingInfo, loggerStopped: Boolean) { + assert(info.logPaths.size > 0) + assert(info.sparkVersion === SparkContext.SPARK_VERSION) + assert(info.compressionCodec.isDefined === compressionCodec.isDefined) + info.compressionCodec.foreach { codec => + assert(compressionCodec.isDefined) + val expectedCodec = compressionCodec.get.split('.').last + assert(codec.getClass.getSimpleName === expectedCodec) + } + assert(info.applicationComplete === loggerStopped) + } + + // Verify that all information is correctly parsed before stop() + val conf = getLoggingConf(logDirPath, compressionCodec) + val eventLogger = new EventLoggingListener("test", conf) + eventLogger.start() + var eventLoggingInfo = EventLoggingListener.parseLoggingInfo(eventLogger.logDir, fileSystem) + assertInfoCorrect(eventLoggingInfo, loggerStopped = false) + + // Verify that all information is correctly parsed after stop() + eventLogger.stop() + eventLoggingInfo = EventLoggingListener.parseLoggingInfo(eventLogger.logDir, fileSystem) + assertInfoCorrect(eventLoggingInfo, loggerStopped = true) + } + + /** + * Test basic event logging functionality. + * + * This creates two simple events, posts them to the EventLoggingListener, and verifies that + * exactly these two events are logged in the expected file. + */ + private def testEventLogging(compressionCodec: Option[String] = None) { + val conf = getLoggingConf(logDirPath, compressionCodec) + val eventLogger = new EventLoggingListener("test", conf) + val listenerBus = new LiveListenerBus + val applicationStart = SparkListenerApplicationStart("Greatest App (N)ever", 125L, "Mickey") + val applicationEnd = SparkListenerApplicationEnd(1000L) + + // A comprehensive test on JSON de/serialization of all events is in JsonProtocolSuite + eventLogger.start() + listenerBus.start() + listenerBus.addListener(eventLogger) + listenerBus.postToAll(applicationStart) + listenerBus.postToAll(applicationEnd) + + // Verify file contains exactly the two events logged + val eventLoggingInfo = EventLoggingListener.parseLoggingInfo(eventLogger.logDir, fileSystem) + assert(eventLoggingInfo.logPaths.size > 0) + val lines = readFileLines(eventLoggingInfo.logPaths.head, eventLoggingInfo.compressionCodec) + assert(lines.size === 2) + assert(lines(0).contains("SparkListenerApplicationStart")) + assert(lines(1).contains("SparkListenerApplicationEnd")) + assert(JsonProtocol.sparkEventFromJson(parse(lines(0))) === applicationStart) + assert(JsonProtocol.sparkEventFromJson(parse(lines(1))) === applicationEnd) + eventLogger.stop() + } + + /** + * Test end-to-end event logging functionality in an application. + * This runs a simple Spark job and asserts that the expected events are logged when expected. + */ + private def testApplicationEventLogging(compressionCodec: Option[String] = None) { + val conf = getLoggingConf(logDirPath, compressionCodec) + val sc = new SparkContext("local", "test", conf) + assert(sc.eventLogger.isDefined) + val eventLogger = sc.eventLogger.get + val expectedLogDir = logDirPath.toString + assert(eventLogger.logDir.startsWith(expectedLogDir)) + + // Begin listening for events that trigger asserts + val eventExistenceListener = new EventExistenceListener(eventLogger) + sc.addSparkListener(eventExistenceListener) + + // Trigger asserts for whether the expected events are actually logged + sc.parallelize(1 to 10000).count() + sc.stop() + + // Ensure all asserts have actually been triggered + eventExistenceListener.assertAllCallbacksInvoked() + } + + /** + * Assert that all of the specified events are logged by the given EventLoggingListener. + */ + private def assertEventsExist(eventLogger: EventLoggingListener, events: Seq[String]) { + val eventLoggingInfo = EventLoggingListener.parseLoggingInfo(eventLogger.logDir, fileSystem) + assert(eventLoggingInfo.logPaths.size > 0) + val lines = readFileLines(eventLoggingInfo.logPaths.head, eventLoggingInfo.compressionCodec) + val eventSet = mutable.Set(events: _*) + lines.foreach { line => + eventSet.foreach { event => + if (line.contains(event)) { + val parsedEvent = JsonProtocol.sparkEventFromJson(parse(line)) + val eventType = Utils.getFormattedClassName(parsedEvent) + if (eventType == event) { + eventSet.remove(event) + } + } + } + } + assert(eventSet.isEmpty, "The following events are missing: " + eventSet.toSeq) + } + + /** + * Read all lines from the file specified by the given path. + * If a compression codec is specified, use it to read the file. + */ + private def readFileLines( + filePath: Path, + compressionCodec: Option[CompressionCodec]): Seq[String] = { + val fstream = fileSystem.open(filePath) + val cstream = + compressionCodec.map { codec => + codec.compressedInputStream(fstream) + }.getOrElse(fstream) + Source.fromInputStream(cstream).getLines().toSeq + } + + /** + * A listener that asserts certain events are logged by the given EventLoggingListener. + * This is necessary because events are posted asynchronously in a different thread. + */ + private class EventExistenceListener(eventLogger: EventLoggingListener) extends SparkListener { + var jobStarted = false + var jobEnded = false + var appEnded = false + + override def onJobStart(jobStart: SparkListenerJobStart) { + assertEventsExist(eventLogger, Seq[String]( + Utils.getFormattedClassName(SparkListenerApplicationStart), + Utils.getFormattedClassName(SparkListenerBlockManagerAdded), + Utils.getFormattedClassName(SparkListenerEnvironmentUpdate) + )) + jobStarted = true + } + + override def onJobEnd(jobEnd: SparkListenerJobEnd) { + assertEventsExist(eventLogger, Seq[String]( + Utils.getFormattedClassName(SparkListenerJobStart), + Utils.getFormattedClassName(SparkListenerJobEnd), + Utils.getFormattedClassName(SparkListenerStageSubmitted), + Utils.getFormattedClassName(SparkListenerStageCompleted), + Utils.getFormattedClassName(SparkListenerTaskStart), + Utils.getFormattedClassName(SparkListenerTaskEnd) + )) + jobEnded = true + } + + override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd) { + assertEventsExist(eventLogger, Seq[String]( + Utils.getFormattedClassName(SparkListenerApplicationEnd) + )) + appEnded = true + } + + def assertAllCallbacksInvoked() { + assert(jobStarted, "JobStart callback not invoked!") + assert(jobEnded, "JobEnd callback not invoked!") + assert(appEnded, "ApplicationEnd callback not invoked!") + } + } + + + /* -------------------------------------------------------- * + * Helper methods for validating state of the special files * + * -------------------------------------------------------- */ + + private def eventLogsExist(logFiles: Array[FileStatus]): Boolean = { + logFiles.map(_.getPath.getName).exists(EventLoggingListener.isEventLogFile) + } + + private def sparkVersionExists(logFiles: Array[FileStatus]): Boolean = { + logFiles.map(_.getPath.getName).exists(EventLoggingListener.isSparkVersionFile) + } + + private def compressionCodecExists(logFiles: Array[FileStatus]): Boolean = { + logFiles.map(_.getPath.getName).exists(EventLoggingListener.isCompressionCodecFile) + } + + private def applicationCompleteExists(logFiles: Array[FileStatus]): Boolean = { + logFiles.map(_.getPath.getName).exists(EventLoggingListener.isApplicationCompleteFile) + } + + private def assertSparkVersionIsValid(logFiles: Array[FileStatus]) { + val file = logFiles.map(_.getPath.getName).find(EventLoggingListener.isSparkVersionFile) + assert(file.isDefined) + assert(EventLoggingListener.parseSparkVersion(file.get) === SparkContext.SPARK_VERSION) + } + + private def assertCompressionCodecIsValid(logFiles: Array[FileStatus], compressionCodec: String) { + val file = logFiles.map(_.getPath.getName).find(EventLoggingListener.isCompressionCodecFile) + assert(file.isDefined) + assert(EventLoggingListener.parseCompressionCodec(file.get) === compressionCodec) + } + +} + + +object EventLoggingListenerSuite { + + /** Get a SparkConf with event logging enabled. */ + def getLoggingConf(logDir: Path, compressionCodec: Option[String] = None) = { + val conf = new SparkConf + conf.set("spark.eventLog.enabled", "true") + conf.set("spark.eventLog.testing", "true") + conf.set("spark.eventLog.dir", logDir.toString) + compressionCodec.foreach { codec => + conf.set("spark.eventLog.compress", "true") + conf.set("spark.io.compression.codec", codec) + } + conf + } +} diff --git a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala new file mode 100644 index 0000000000000..d1fe1fc348961 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala @@ -0,0 +1,166 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.scheduler + +import java.io.PrintWriter + +import scala.util.Try + +import com.google.common.io.Files +import org.json4s.jackson.JsonMethods._ +import org.scalatest.{BeforeAndAfter, FunSuite} + +import org.apache.spark.SparkContext._ +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.io.CompressionCodec +import org.apache.spark.util.{JsonProtocol, Utils} + +/** + * Test whether ReplayListenerBus replays events from logs correctly. + */ +class ReplayListenerSuite extends FunSuite with BeforeAndAfter { + private val fileSystem = Utils.getHadoopFileSystem("/") + private val allCompressionCodecs = Seq[String]( + "org.apache.spark.io.LZFCompressionCodec", + "org.apache.spark.io.SnappyCompressionCodec" + ) + private val testDir = Files.createTempDir() + + after { + Try { fileSystem.delete(Utils.getFilePath(testDir, "events.txt"), true) } + Try { fileSystem.delete(Utils.getFilePath(testDir, "test-replay"), true) } + } + + test("Simple replay") { + testSimpleReplay() + } + + test("Simple replay with compression") { + allCompressionCodecs.foreach { codec => + testSimpleReplay(Some(codec)) + } + } + + // This assumes the correctness of EventLoggingListener + test("End-to-end replay") { + testApplicationReplay() + } + + // This assumes the correctness of EventLoggingListener + test("End-to-end replay with compression") { + allCompressionCodecs.foreach { codec => + testApplicationReplay(Some(codec)) + } + } + + + /* ----------------- * + * Actual test logic * + * ----------------- */ + + /** + * Test simple replaying of events. + */ + private def testSimpleReplay(codecName: Option[String] = None) { + val logFilePath = Utils.getFilePath(testDir, "events.txt") + val codec = codecName.map(getCompressionCodec) + val fstream = fileSystem.create(logFilePath) + val cstream = codec.map(_.compressedOutputStream(fstream)).getOrElse(fstream) + val writer = new PrintWriter(cstream) + val applicationStart = SparkListenerApplicationStart("Greatest App (N)ever", 125L, "Mickey") + val applicationEnd = SparkListenerApplicationEnd(1000L) + writer.println(compact(render(JsonProtocol.sparkEventToJson(applicationStart)))) + writer.println(compact(render(JsonProtocol.sparkEventToJson(applicationEnd)))) + writer.close() + val replayer = new ReplayListenerBus(Seq(logFilePath), fileSystem, codec) + val conf = EventLoggingListenerSuite.getLoggingConf(logFilePath, codecName) + val eventMonster = new EventMonster(conf) + replayer.addListener(eventMonster) + replayer.replay() + assert(eventMonster.loggedEvents.size === 2) + assert(eventMonster.loggedEvents(0) === JsonProtocol.sparkEventToJson(applicationStart)) + assert(eventMonster.loggedEvents(1) === JsonProtocol.sparkEventToJson(applicationEnd)) + } + + /** + * Test end-to-end replaying of events. + * + * This test runs a few simple jobs with event logging enabled, and compares each emitted + * event to the corresponding event replayed from the event logs. This test makes the + * assumption that the event logging behavior is correct (tested in a separate suite). + */ + private def testApplicationReplay(codecName: Option[String] = None) { + val logDirPath = Utils.getFilePath(testDir, "test-replay") + val conf = EventLoggingListenerSuite.getLoggingConf(logDirPath, codecName) + val sc = new SparkContext("local-cluster[2,1,512]", "Test replay", conf) + + // Run a few jobs + sc.parallelize(1 to 100, 1).count() + sc.parallelize(1 to 100, 2).map(i => (i, i)).count() + sc.parallelize(1 to 100, 3).map(i => (i, i)).groupByKey().count() + sc.parallelize(1 to 100, 4).map(i => (i, i)).groupByKey().persist().count() + sc.stop() + + // Prepare information needed for replay + val codec = codecName.map(getCompressionCodec) + val applications = fileSystem.listStatus(logDirPath) + assert(applications != null && applications.size > 0) + val eventLogDir = applications.sortBy(_.getAccessTime).last + assert(eventLogDir.isDir) + val logFiles = fileSystem.listStatus(eventLogDir.getPath) + assert(logFiles != null && logFiles.size > 0) + val logFile = logFiles.find(_.getPath.getName.startsWith("EVENT_LOG_")) + assert(logFile.isDefined) + val logFilePath = logFile.get.getPath + + // Replay events + val replayer = new ReplayListenerBus(Seq(logFilePath), fileSystem, codec) + val eventMonster = new EventMonster(conf) + replayer.addListener(eventMonster) + replayer.replay() + + // Verify the same events are replayed in the same order + assert(sc.eventLogger.isDefined) + val originalEvents = sc.eventLogger.get.loggedEvents + val replayedEvents = eventMonster.loggedEvents + originalEvents.zip(replayedEvents).foreach { case (e1, e2) => assert(e1 === e2) } + } + + /** + * A simple listener that buffers all the events it receives. + * + * The event buffering functionality must be implemented within EventLoggingListener itself. + * This is because of the following race condition: the event may be mutated between being + * processed by one listener and being processed by another. Thus, in order to establish + * a fair comparison between the original events and the replayed events, both functionalities + * must be implemented within one listener (i.e. the EventLoggingListener). + * + * This child listener inherits only the event buffering functionality, but does not actually + * log the events. + */ + private class EventMonster(conf: SparkConf) extends EventLoggingListener("test", conf) { + logger.close() + } + + private def getCompressionCodec(codecName: String) = { + val conf = new SparkConf + conf.set("spark.io.compression.codec", codecName) + CompressionCodec.createCodec(conf) + } + +} diff --git a/core/src/test/scala/org/apache/spark/util/FileLoggerSuite.scala b/core/src/test/scala/org/apache/spark/util/FileLoggerSuite.scala new file mode 100644 index 0000000000000..f675e1e5b4981 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/util/FileLoggerSuite.scala @@ -0,0 +1,163 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util + +import java.io.IOException + +import scala.io.Source +import scala.util.Try + +import com.google.common.io.Files +import org.apache.hadoop.fs.Path +import org.scalatest.{BeforeAndAfter, FunSuite} + +import org.apache.spark.SparkConf +import org.apache.spark.io.CompressionCodec + +/** + * Test writing files through the FileLogger. + */ +class FileLoggerSuite extends FunSuite with BeforeAndAfter { + private val fileSystem = Utils.getHadoopFileSystem("/") + private val allCompressionCodecs = Seq[String]( + "org.apache.spark.io.LZFCompressionCodec", + "org.apache.spark.io.SnappyCompressionCodec" + ) + private val testDir = Files.createTempDir() + private val logDirPath = Utils.getFilePath(testDir, "test-file-logger") + private val logDirPathString = logDirPath.toString + + after { + Try { fileSystem.delete(logDirPath, true) } + } + + test("Simple logging") { + testSingleFile() + } + + test ("Simple logging with compression") { + allCompressionCodecs.foreach { codec => + testSingleFile(Some(codec)) + } + } + + test("Logging multiple files") { + testMultipleFiles() + } + + test("Logging multiple files with compression") { + allCompressionCodecs.foreach { codec => + testMultipleFiles(Some(codec)) + } + } + + test("Logging when directory already exists") { + // Create the logging directory multiple times + new FileLogger(logDirPathString, new SparkConf, overwrite = true).start() + new FileLogger(logDirPathString, new SparkConf, overwrite = true).start() + new FileLogger(logDirPathString, new SparkConf, overwrite = true).start() + + // If overwrite is not enabled, an exception should be thrown + intercept[IOException] { + new FileLogger(logDirPathString, new SparkConf, overwrite = false).start() + } + } + + + /* ----------------- * + * Actual test logic * + * ----------------- */ + + /** + * Test logging to a single file. + */ + private def testSingleFile(codecName: Option[String] = None) { + val conf = getLoggingConf(codecName) + val codec = codecName.map { c => CompressionCodec.createCodec(conf) } + val logger = + if (codecName.isDefined) { + new FileLogger(logDirPathString, conf, compress = true) + } else { + new FileLogger(logDirPathString, conf) + } + logger.start() + assert(fileSystem.exists(logDirPath)) + assert(fileSystem.getFileStatus(logDirPath).isDir) + assert(fileSystem.listStatus(logDirPath).size === 0) + + logger.newFile() + val files = fileSystem.listStatus(logDirPath) + assert(files.size === 1) + val firstFile = files.head + val firstFilePath = firstFile.getPath + + logger.log("hello") + logger.flush() + assert(readFileContent(firstFilePath, codec) === "hello") + + logger.log(" world") + logger.close() + assert(readFileContent(firstFilePath, codec) === "hello world") + } + + /** + * Test logging to multiple files. + */ + private def testMultipleFiles(codecName: Option[String] = None) { + val conf = getLoggingConf(codecName) + val codec = codecName.map { c => CompressionCodec.createCodec(conf) } + val logger = + if (codecName.isDefined) { + new FileLogger(logDirPathString, conf, compress = true) + } else { + new FileLogger(logDirPathString, conf) + } + logger.start() + logger.newFile("Jean_Valjean") + logger.logLine("Who am I?") + logger.logLine("Destiny?") + logger.newFile("John_Valjohn") + logger.logLine("One") + logger.logLine("Two three...") + logger.newFile("Wolverine") + logger.logLine("There was a time") + logger.logLine("A time when our enemies knew honor.") + logger.close() + assert(readFileContent(new Path(logDirPath, "Jean_Valjean"), codec) === "Who am I?\nDestiny?") + assert(readFileContent(new Path(logDirPath, "John_Valjohn"), codec) === "One\nTwo three...") + assert(readFileContent(new Path(logDirPath, "Wolverine"), codec) === + "There was a time\nA time when our enemies knew honor.") + } + + /** + * Read the content of the file specified by the given path. + * If a compression codec is specified, use it to read the file. + */ + private def readFileContent(logPath: Path, codec: Option[CompressionCodec] = None): String = { + val fstream = fileSystem.open(logPath) + val cstream = codec.map(_.compressedInputStream(fstream)).getOrElse(fstream) + Source.fromInputStream(cstream).getLines().mkString("\n") + } + + private def getLoggingConf(codecName: Option[String]) = { + val conf = new SparkConf + codecName.foreach { c => conf.set("spark.io.compression.codec", c) } + conf + } + +} diff --git a/docs/README.md b/docs/README.md index 75b1811ba99af..f1eb644f93406 100644 --- a/docs/README.md +++ b/docs/README.md @@ -14,9 +14,10 @@ The markdown code can be compiled to HTML using the [Jekyll tool](http://jekyllrb.com). To use the `jekyll` command, you will need to have Jekyll installed. The easiest way to do this is via a Ruby Gem, see the -[jekyll installation instructions](http://jekyllrb.com/docs/installation). -Compiling the site with Jekyll will create a directory called -_site containing index.html as well as the rest of the compiled files. +[jekyll installation instructions](http://jekyllrb.com/docs/installation). +If not already installed, you need to install `kramdown` with `sudo gem install kramdown`. +Execute `jekyll` from the `docs/` directory. Compiling the site with Jekyll will create a directory called +`_site` containing index.html as well as the rest of the compiled files. You can modify the default Jekyll build as follows: @@ -44,6 +45,6 @@ You can build just the Spark scaladoc by running `sbt/sbt doc` from the SPARK_PR Similarly, you can build just the PySpark epydoc by running `epydoc --config epydoc.conf` from the SPARK_PROJECT_ROOT/pyspark directory. Documentation is only generated for classes that are listed as public in `__init__.py`. -When you run `jekyll` in the docs directory, it will also copy over the scaladoc for the various Spark subprojects into the docs directory (and then also into the _site directory). We use a jekyll plugin to run `sbt/sbt doc` before building the site so if you haven't run it (recently) it may take some time as it generates all of the scaladoc. The jekyll plugin also generates the PySpark docs using [epydoc](http://epydoc.sourceforge.net/). +When you run `jekyll` in the `docs` directory, it will also copy over the scaladoc for the various Spark subprojects into the `docs` directory (and then also into the `_site` directory). We use a jekyll plugin to run `sbt/sbt doc` before building the site so if you haven't run it (recently) it may take some time as it generates all of the scaladoc. The jekyll plugin also generates the PySpark docs using [epydoc](http://epydoc.sourceforge.net/). NOTE: To skip the step of building and copying over the Scala and Python API docs, run `SKIP_API=1 jekyll`. diff --git a/docs/_config.yml b/docs/_config.yml index d585b8c5ea763..d177e38f884ca 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -8,5 +8,5 @@ SPARK_VERSION_SHORT: 1.0.0 SCALA_BINARY_VERSION: "2.10" SCALA_VERSION: "2.10.4" MESOS_VERSION: 0.13.0 -SPARK_ISSUE_TRACKER_URL: https://spark-project.atlassian.net +SPARK_ISSUE_TRACKER_URL: https://issues.apache.org/jira/browse/SPARK SPARK_GITHUB_URL: https://github.com/apache/spark diff --git a/docs/bagel-programming-guide.md b/docs/bagel-programming-guide.md index da6d0c9dcd97b..14f43cb6d3946 100644 --- a/docs/bagel-programming-guide.md +++ b/docs/bagel-programming-guide.md @@ -46,7 +46,7 @@ import org.apache.spark.bagel.Bagel._ Next, we load a sample graph from a text file as a distributed dataset and package it into `PRVertex` objects. We also cache the distributed dataset because Bagel will use it multiple times and we'd like to avoid recomputing it. {% highlight scala %} -val input = sc.textFile("pagerank_data.txt") +val input = sc.textFile("data/pagerank_data.txt") val numVerts = input.count() diff --git a/docs/building-with-maven.md b/docs/building-with-maven.md index a5e5303467401..cac01ded60d94 100644 --- a/docs/building-with-maven.md +++ b/docs/building-with-maven.md @@ -29,9 +29,22 @@ You can fix this by setting the `MAVEN_OPTS` variable as discussed before. ## Specifying the Hadoop version ## -Because HDFS is not protocol-compatible across versions, if you want to read from HDFS, you'll need to build Spark against the specific HDFS version in your environment. You can do this through the "hadoop.version" property. If unset, Spark will build against Hadoop 1.0.4 by default. - -For Apache Hadoop versions 1.x, Cloudera CDH MRv1, and other Hadoop versions without YARN, use: +Because HDFS is not protocol-compatible across versions, if you want to read from HDFS, you'll need to build Spark against the specific HDFS version in your environment. You can do this through the "hadoop.version" property. If unset, Spark will build against Hadoop 1.0.4 by default. Note that certain build profiles are required for particular Hadoop versions: + + + + + + + + + + + + +
Hadoop versionProfile required
0.23.xhadoop-0.23
1.x to 2.1.x(none)
2.2.xhadoop-2.2
2.3.xhadoop-2.3
2.4.xhadoop-2.4
+ +For Apache Hadoop versions 1.x, Cloudera CDH "mr1" distributions, and other Hadoop versions without YARN, use: # Apache Hadoop 1.2.1 $ mvn -Dhadoop.version=1.2.1 -DskipTests clean package @@ -42,19 +55,40 @@ For Apache Hadoop versions 1.x, Cloudera CDH MRv1, and other Hadoop versions wit # Apache Hadoop 0.23.x $ mvn -Phadoop-0.23 -Dhadoop.version=0.23.7 -DskipTests clean package -For Apache Hadoop 2.x, 0.23.x, Cloudera CDH MRv2, and other Hadoop versions with YARN, you can enable the "yarn-alpha" or "yarn" profile and set the "hadoop.version", "yarn.version" property. Note that Hadoop 0.23.X requires a special `-Phadoop-0.23` profile: +For Apache Hadoop 2.x, 0.23.x, Cloudera CDH, and other Hadoop versions with YARN, you can enable the "yarn-alpha" or "yarn" profile and optionally set the "yarn.version" property if it is different from "hadoop.version". The additional build profile required depends on the YARN version: - # Apache Hadoop 2.0.5-alpha - $ mvn -Pyarn-alpha -Dhadoop.version=2.0.5-alpha -Dyarn.version=2.0.5-alpha -DskipTests clean package + + + + + + + + +
YARN versionProfile required
0.23.x to 2.1.xyarn-alpha
2.2.x and lateryarn
- # Cloudera CDH 4.2.0 with MapReduce v2 - $ mvn -Pyarn-alpha -Dhadoop.version=2.0.0-cdh4.2.0 -Dyarn.version=2.0.0-cdh4.2.0 -DskipTests clean package +Examples: + + # Apache Hadoop 2.0.5-alpha + $ mvn -Pyarn-alpha -Dhadoop.version=2.0.5-alpha -DskipTests clean package - # Apache Hadoop 2.2.X (e.g. 2.2.0 as below) and newer - $ mvn -Pyarn -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -DskipTests clean package + # Cloudera CDH 4.2.0 + $ mvn -Pyarn-alpha -Dhadoop.version=2.0.0-cdh4.2.0 -DskipTests clean package # Apache Hadoop 0.23.x - $ mvn -Pyarn-alpha -Phadoop-0.23 -Dhadoop.version=0.23.7 -Dyarn.version=0.23.7 -DskipTests clean package + $ mvn -Pyarn-alpha -Phadoop-0.23 -Dhadoop.version=0.23.7 -DskipTests clean package + + # Apache Hadoop 2.2.X + $ mvn -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 -DskipTests clean package + + # Apache Hadoop 2.3.X + $ mvn -Pyarn -Phadoop-2.3 -Dhadoop.version=2.3.0 -DskipTests clean package + + # Apache Hadoop 2.4.X + $ mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=2.4.0 -DskipTests clean package + + # Different versions of HDFS and YARN. + $ mvn -Pyarn-alpha -Phadoop-2.3 -Dhadoop.version=2.3.0 -Dyarn.version=0.23.7 -DskipTests clean package ## Spark Tests in Maven ## diff --git a/docs/cluster-overview.md b/docs/cluster-overview.md index b011679fede2d..162c415b5883e 100644 --- a/docs/cluster-overview.md +++ b/docs/cluster-overview.md @@ -86,7 +86,7 @@ the `--help` flag. Here are a few examples of common options: --master local[8] \ my-app.jar -# Run on a Spark cluster +# Run on a Spark standalone cluster ./bin/spark-submit \ --class my.main.ClassName --master spark://mycluster:7077 \ @@ -118,21 +118,22 @@ If you are ever unclear where configuration options are coming from. fine-graine information can be printed by adding the `--verbose` option to `./spark-submit`. ### Advanced Dependency Management -When using `./bin/spark-submit` jars will be automatically transferred to the cluster. For many -users this is sufficient. However, advanced users can add jars by calling `addFile` or `addJar` -on an existing SparkContext. This can be used to distribute JAR files (Java/Scala) or .egg and -.zip libraries (Python) to executors. Spark uses the following URL scheme to allow different +When using `./bin/spark-submit` the app jar along with any jars included with the `--jars` option +will be automatically transferred to the cluster. `--jars` can also be used to distribute .egg and .zip +libraries for Python to executors. Spark uses the following URL scheme to allow different strategies for disseminating jars: - **file:** - Absolute paths and `file:/` URIs are served by the driver's HTTP file server, and - every executor pulls the file from the driver HTTP server + every executor pulls the file from the driver HTTP server. - **hdfs:**, **http:**, **https:**, **ftp:** - these pull down files and JARs from the URI as expected - **local:** - a URI starting with local:/ is expected to exist as a local file on each worker node. This means that no network IO will be incurred, and works well for large files/JARs that are pushed to each worker, or shared via NFS, GlusterFS, etc. Note that JARs and files are copied to the working directory for each SparkContext on the executor nodes. -Over time this can use up a significant amount of space and will need to be cleaned up. +This can use up a significant amount of space over time and will need to be cleaned up. With YARN, cleanup +is handled automatically, and with Spark standalone, automatic cleanup can be configured with the +`spark.worker.cleanup.appDataTtl` property. # Monitoring @@ -180,7 +181,7 @@ The following table summarizes terms you'll see used to refer to cluster concept Distinguishes where the driver process runs. In "cluster" mode, the framework launches the driver inside of the cluster. In "client" mode, the submitter launches the driver outside of the cluster. - + Worker node Any node that can run application code in the cluster diff --git a/docs/configuration.md b/docs/configuration.md index b078c7c1112c2..5b034e3cb3d47 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -26,10 +26,10 @@ application name), as well as arbitrary key-value pairs through the `set()` meth initialize an application as follows: {% highlight scala %} -val conf = new SparkConf() - .setMaster("local") - .setAppName("My application") - .set("spark.executor.memory", "1g") +val conf = new SparkConf(). + setMaster("local"). + setAppName("My application"). + set("spark.executor.memory", "1g") val sc = new SparkContext(conf) {% endhighlight %} @@ -289,8 +289,7 @@ Apart from these, the following properties are also available, and may be useful spark.closure.serializer org.apache.spark.serializer.
JavaSerializer - Serializer class to use for closures. Generally Java is fine unless your distributed functions - (e.g. map functions) reference large objects in the driver program. + Serializer class to use for closures. Currently only the Java serializer is supported. @@ -319,7 +318,7 @@ Apart from these, the following properties are also available, and may be useful When serializing using org.apache.spark.serializer.JavaSerializer, the serializer caches objects to prevent writing redundant data, however that stops garbage collection of those objects. By calling 'reset' you flush that info from the serializer, and allow old - objects to be collected. To turn off this periodic reset set it to a value of <= 0. + objects to be collected. To turn off this periodic reset set it to a value <= 0. By default it will reset the serializer every 10,000 objects. @@ -463,7 +462,7 @@ Apart from these, the following properties are also available, and may be useful (infinite) Duration (seconds) of how long Spark will remember any metadata (stages generated, tasks generated, etc.). - Periodic cleanups will ensure that metadata older than this duration will be forgetten. This is + Periodic cleanups will ensure that metadata older than this duration will be forgotten. This is useful for running Spark for many hours / days (for example, running 24/7 in case of Spark Streaming applications). Note that any RDD that persists in memory for more than this duration will be cleared as well. @@ -472,8 +471,8 @@ Apart from these, the following properties are also available, and may be useful spark.streaming.blockInterval 200 - Duration (milliseconds) of how long to batch new objects coming from network receivers used - in Spark Streaming. + Interval (milliseconds) at which data received by Spark Streaming receivers is coalesced + into blocks of data before storing them in Spark. diff --git a/docs/index.md b/docs/index.md index 6fc9a4f03bb12..a2f1a84371ff4 100644 --- a/docs/index.md +++ b/docs/index.md @@ -9,25 +9,26 @@ It also supports a rich set of higher-level tools including [Shark](http://shark # Downloading -Get Spark by visiting the [downloads page](http://spark.apache.org/downloads.html) of the Apache Spark site. This documentation is for Spark version {{site.SPARK_VERSION}}. +Get Spark by visiting the [downloads page](http://spark.apache.org/downloads.html) of the Apache Spark site. This documentation is for Spark version {{site.SPARK_VERSION}}. The downloads page +contains Spark packages for many popular HDFS versions. If you'd like to build Spark from +scratch, visit the [building with Maven](building-with-maven.html) page. -Spark runs on both Windows and UNIX-like systems (e.g. Linux, Mac OS). All you need to run it is to have `java` to installed on your system `PATH`, or the `JAVA_HOME` environment variable pointing to a Java installation. +Spark runs on both Windows and UNIX-like systems (e.g. Linux, Mac OS). All you need to run it is +to have `java` to installed on your system `PATH`, or the `JAVA_HOME` environment variable +pointing to a Java installation. -# Building - -Spark uses [Simple Build Tool](http://www.scala-sbt.org), which is bundled with it. To compile the code, go into the top-level Spark directory and run - - sbt/sbt assembly - -For its Scala API, Spark {{site.SPARK_VERSION}} depends on Scala {{site.SCALA_BINARY_VERSION}}. If you write applications in Scala, you will need to use a compatible Scala version (e.g. {{site.SCALA_BINARY_VERSION}}.X) -- newer major versions may not work. You can get the right version of Scala from [scala-lang.org](http://www.scala-lang.org/download/). +For its Scala API, Spark {{site.SPARK_VERSION}} depends on Scala {{site.SCALA_BINARY_VERSION}}. +If you write applications in Scala, you will need to use a compatible Scala version +(e.g. {{site.SCALA_BINARY_VERSION}}.X) -- newer major versions may not work. You can get the +right version of Scala from [scala-lang.org](http://www.scala-lang.org/download/). # Running the Examples and Shell -Spark comes with several sample programs. Scala and Java examples are in the `examples` directory, and Python examples are in `python/examples`. +Spark comes with several sample programs. Scala, Java and Python examples are in the `examples/src/main` directory. To run one of the Java or Scala sample programs, use `./bin/run-example ` in the top-level Spark directory (the `bin/run-example` script sets up the appropriate paths and launches that program). For example, try `./bin/run-example org.apache.spark.examples.SparkPi local`. -To run a Python sample program, use `./bin/pyspark `. For example, try `./bin/pyspark ./python/examples/pi.py local`. +To run a Python sample program, use `./bin/pyspark `. For example, try `./bin/pyspark ./examples/src/main/python/pi.py local`. Each example prints usage help when run with no parameters. @@ -50,23 +51,6 @@ options for deployment: * [Apache Mesos](running-on-mesos.html) * [Hadoop YARN](running-on-yarn.html) -# A Note About Hadoop Versions - -Spark uses the Hadoop-client library to talk to HDFS and other Hadoop-supported -storage systems. Because the HDFS protocol has changed in different versions of -Hadoop, you must build Spark against the same version that your cluster uses. -By default, Spark links to Hadoop 1.0.4. You can change this by setting the -`SPARK_HADOOP_VERSION` variable when compiling: - - SPARK_HADOOP_VERSION=2.2.0 sbt/sbt assembly - -In addition, if you wish to run Spark on [YARN](running-on-yarn.html), set -`SPARK_YARN` to `true`: - - SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_YARN=true sbt/sbt assembly - -Note that on Windows, you need to set the environment variables on separate lines, e.g., `set SPARK_HADOOP_VERSION=1.2.1`. - # Where to Go from Here **Programming guides:** @@ -112,10 +96,10 @@ Note that on Windows, you need to set the environment variables on separate line * [Shark](http://shark.cs.berkeley.edu): Apache Hive over Spark * [Mailing Lists](http://spark.apache.org/mailing-lists.html): ask questions about Spark here * [AMP Camps](http://ampcamp.berkeley.edu/): a series of training camps at UC Berkeley that featured talks and - exercises about Spark, Shark, Mesos, and more. [Videos](http://ampcamp.berkeley.edu/agenda-2012), - [slides](http://ampcamp.berkeley.edu/agenda-2012) and [exercises](http://ampcamp.berkeley.edu/exercises-2012) are + exercises about Spark, Shark, Spark Streaming, Mesos, and more. [Videos](http://ampcamp.berkeley.edu/3/), + [slides](http://ampcamp.berkeley.edu/3/) and [exercises](http://ampcamp.berkeley.edu/3/exercises/) are available online for free. -* [Code Examples](http://spark.apache.org/examples.html): more are also available in the [examples subfolder](https://github.com/apache/spark/tree/master/examples/src/main/scala/) of Spark +* [Code Examples](http://spark.apache.org/examples.html): more are also available in the [examples subfolder](https://github.com/apache/spark/tree/master/examples/src/main/scala/org/apache/spark/) of Spark * [Paper Describing Spark](http://www.cs.berkeley.edu/~matei/papers/2012/nsdi_spark.pdf) * [Paper Describing Spark Streaming](http://www.eecs.berkeley.edu/Pubs/TechRpts/2012/EECS-2012-259.pdf) diff --git a/docs/java-programming-guide.md b/docs/java-programming-guide.md index 07c8512bf9294..c34eb28fc06a2 100644 --- a/docs/java-programming-guide.md +++ b/docs/java-programming-guide.md @@ -55,7 +55,7 @@ classes. RDD methods like `map` are overloaded by specialized `PairFunction` and `DoubleFunction` classes, allowing them to return RDDs of the appropriate types. Common methods like `filter` and `sample` are implemented by each specialized RDD class, so filtering a `PairRDD` returns a new `PairRDD`, -etc (this acheives the "same-result-type" principle used by the [Scala collections +etc (this achieves the "same-result-type" principle used by the [Scala collections framework](http://docs.scala-lang.org/overviews/core/architecture-of-scala-collections.html)). ## Function Interfaces @@ -102,7 +102,7 @@ the following changes: `Function` classes will need to use `implements` rather than `extends`. * Certain transformation functions now have multiple versions depending on the return type. In Spark core, the map functions (`map`, `flatMap`, and - `mapPartitons`) have type-specific versions, e.g. + `mapPartitions`) have type-specific versions, e.g. [`mapToPair`](api/java/org/apache/spark/api/java/JavaRDDLike.html#mapToPair(org.apache.spark.api.java.function.PairFunction)) and [`mapToDouble`](api/java/org/apache/spark/api/java/JavaRDDLike.html#mapToDouble(org.apache.spark.api.java.function.DoubleFunction)). Spark Streaming also uses the same approach, e.g. [`transformToPair`](api/java/org/apache/spark/streaming/api/java/JavaDStreamLike.html#transformToPair(org.apache.spark.api.java.function.Function)). @@ -115,11 +115,11 @@ As an example, we will implement word count using the Java API. import org.apache.spark.api.java.*; import org.apache.spark.api.java.function.*; -JavaSparkContext sc = new JavaSparkContext(...); -JavaRDD lines = ctx.textFile("hdfs://..."); +JavaSparkContext jsc = new JavaSparkContext(...); +JavaRDD lines = jsc.textFile("hdfs://..."); JavaRDD words = lines.flatMap( new FlatMapFunction() { - public Iterable call(String s) { + @Override public Iterable call(String s) { return Arrays.asList(s.split(" ")); } } @@ -140,10 +140,10 @@ Here, the `FlatMapFunction` was created inline; another option is to subclass {% highlight java %} class Split extends FlatMapFunction { - public Iterable call(String s) { + @Override public Iterable call(String s) { return Arrays.asList(s.split(" ")); } -); +} JavaRDD words = lines.flatMap(new Split()); {% endhighlight %} @@ -162,8 +162,8 @@ Continuing with the word count example, we map each word to a `(word, 1)` pair: import scala.Tuple2; JavaPairRDD ones = words.mapToPair( new PairFunction() { - public Tuple2 call(String s) { - return new Tuple2(s, 1); + @Override public Tuple2 call(String s) { + return new Tuple2(s, 1); } } ); @@ -178,7 +178,7 @@ occurrences of each word: {% highlight java %} JavaPairRDD counts = ones.reduceByKey( new Function2() { - public Integer call(Integer i1, Integer i2) { + @Override public Integer call(Integer i1, Integer i2) { return i1 + i2; } } diff --git a/docs/mllib-basics.md b/docs/mllib-basics.md index 710ce1721fe25..704308802d65b 100644 --- a/docs/mllib-basics.md +++ b/docs/mllib-basics.md @@ -9,7 +9,7 @@ title: MLlib - Basics MLlib supports local vectors and matrices stored on a single machine, as well as distributed matrices backed by one or more RDDs. In the current implementation, local vectors and matrices are simple data models -to serve public interfaces. The underly linear algebra operations are provided by +to serve public interfaces. The underlying linear algebra operations are provided by [Breeze](http://www.scalanlp.org/) and [jblas](http://jblas.org/). A training example used in supervised learning is called "labeled point" in MLlib. @@ -205,7 +205,7 @@ import org.apache.spark.mllib.regression.LabeledPoint; import org.apache.spark.mllib.util.MLUtils; import org.apache.spark.rdd.RDDimport; -RDD[LabeledPoint] training = MLUtils.loadLibSVMData(sc, "mllib/data/sample_libsvm_data.txt") +RDD training = MLUtils.loadLibSVMData(jsc, "mllib/data/sample_libsvm_data.txt"); {% endhighlight %} @@ -307,6 +307,7 @@ A [`RowMatrix`](api/mllib/index.html#org.apache.spark.mllib.linalg.distributed.R created from a `JavaRDD` instance. Then we can compute its column summary statistics. {% highlight java %} +import org.apache.spark.api.java.JavaRDD; import org.apache.spark.mllib.linalg.Vector; import org.apache.spark.mllib.linalg.distributed.RowMatrix; @@ -348,10 +349,10 @@ val mat: RowMatrix = ... // a RowMatrix val summary: MultivariateStatisticalSummary = mat.computeColumnSummaryStatistics() println(summary.mean) // a dense vector containing the mean value for each column println(summary.variance) // column-wise variance -println(summary.numNonzers) // number of nonzeros in each column +println(summary.numNonzeros) // number of nonzeros in each column // Compute the covariance matrix. -val Cov: Matrix = mat.computeCovariance() +val cov: Matrix = mat.computeCovariance() {% endhighlight %} @@ -397,11 +398,12 @@ wrapper over `(long, Vector)`. An `IndexedRowMatrix` can be converted to a `Row its row indices. {% highlight java %} +import org.apache.spark.api.java.JavaRDD; import org.apache.spark.mllib.linalg.distributed.IndexedRow; import org.apache.spark.mllib.linalg.distributed.IndexedRowMatrix; import org.apache.spark.mllib.linalg.distributed.RowMatrix; -JavaRDD[IndexedRow] rows = ... // a JavaRDD of indexed rows +JavaRDD rows = ... // a JavaRDD of indexed rows // Create an IndexedRowMatrix from a JavaRDD. IndexedRowMatrix mat = new IndexedRowMatrix(rows.rdd()); @@ -458,7 +460,9 @@ wrapper over `(long, long, double)`. A `CoordinateMatrix` can be converted to a with sparse rows by calling `toIndexedRowMatrix`. {% highlight scala %} +import org.apache.spark.api.java.JavaRDD; import org.apache.spark.mllib.linalg.distributed.CoordinateMatrix; +import org.apache.spark.mllib.linalg.distributed.IndexedRowMatrix; import org.apache.spark.mllib.linalg.distributed.MatrixEntry; JavaRDD entries = ... // a JavaRDD of matrix entries diff --git a/docs/mllib-clustering.md b/docs/mllib-clustering.md index b3293afe408d0..276868fa8490d 100644 --- a/docs/mllib-clustering.md +++ b/docs/mllib-clustering.md @@ -18,7 +18,7 @@ models are trained for each cluster). MLlib supports [k-means](http://en.wikipedia.org/wiki/K-means_clustering) clustering, one of the most commonly used clustering algorithms that clusters the data points into -predfined number of clusters. The MLlib implementation includes a parallelized +predefined number of clusters. The MLlib implementation includes a parallelized variant of the [k-means++](http://en.wikipedia.org/wiki/K-means%2B%2B) method called [kmeans||](http://theory.stanford.edu/~sergei/papers/vldb12-kmpar.pdf). The implementation in MLlib has the following parameters: @@ -30,7 +30,7 @@ initialization via k-means\|\|. * *runs* is the number of times to run the k-means algorithm (k-means is not guaranteed to find a globally optimal solution, and when run multiple times on a given dataset, the algorithm returns the best clustering result). -* *initializiationSteps* determines the number of steps in the k-means\|\| algorithm. +* *initializationSteps* determines the number of steps in the k-means\|\| algorithm. * *epsilon* determines the distance threshold within which we consider k-means to have converged. ## Examples diff --git a/docs/mllib-collaborative-filtering.md b/docs/mllib-collaborative-filtering.md index 79f5e3a7ca4fb..f486c56e55907 100644 --- a/docs/mllib-collaborative-filtering.md +++ b/docs/mllib-collaborative-filtering.md @@ -77,7 +77,7 @@ val ratesAndPreds = ratings.map{ }.join(predictions) val MSE = ratesAndPreds.map{ case ((user, product), (r1, r2)) => math.pow((r1- r2), 2) -}.reduce(_ + _)/ratesAndPreds.count +}.mean() println("Mean Squared Error = " + MSE) {% endhighlight %} diff --git a/docs/mllib-decision-tree.md b/docs/mllib-decision-tree.md index a2a2999a00e3a..acf0feff42a8d 100644 --- a/docs/mllib-decision-tree.md +++ b/docs/mllib-decision-tree.md @@ -83,14 +83,14 @@ Section 9.2.4 in [Elements of Statistical Machine Learning](http://statweb.stanford.edu/~tibs/ElemStatLearn/) for details). For example, for a binary classification problem with one categorical feature with three categories A, B and C with corresponding proportion of label 1 as 0.2, 0.6 and 0.4, the categorical -features are orded as A followed by C followed B or A, B, C. The two split candidates are A \| C, B +features are ordered as A followed by C followed B or A, B, C. The two split candidates are A \| C, B and A , B \| C where \| denotes the split. ### Stopping rule The recursive tree construction is stopped at a node when one of the two conditions is met: -1. The node depth is equal to the `maxDepth` training parammeter +1. The node depth is equal to the `maxDepth` training parameter 2. No split candidate leads to an information gain at the node. ### Max memory requirements @@ -99,8 +99,7 @@ For faster processing, the decision tree algorithm performs simultaneous histogr ### Practical limitations -1. The implemented algorithm reads both sparse and dense data. However, it is not optimized for - sparse input. +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. ## Examples @@ -176,7 +175,7 @@ val valuesAndPreds = parsedData.map { point => val prediction = model.predict(point.features) (point.label, prediction) } -val MSE = valuesAndPreds.map{ case(v, p) => math.pow((v - p), 2)}.reduce(_ + _)/valuesAndPreds.count +val MSE = valuesAndPreds.map{ case(v, p) => math.pow((v - p), 2)}.mean() println("training Mean Squared Error = " + MSE) {% endhighlight %} diff --git a/docs/mllib-dimensionality-reduction.md b/docs/mllib-dimensionality-reduction.md index 4e9ecf7c006fa..ab24663cfe258 100644 --- a/docs/mllib-dimensionality-reduction.md +++ b/docs/mllib-dimensionality-reduction.md @@ -44,6 +44,10 @@ say, less than $1000$, but many rows, which we call *tall-and-skinny*.
{% highlight scala %} +import org.apache.spark.mllib.linalg.Matrix +import org.apache.spark.mllib.linalg.distributed.RowMatrix +import org.apache.spark.mllib.linalg.SingularValueDecomposition + val mat: RowMatrix = ... // Compute the top 20 singular values and corresponding singular vectors. @@ -74,6 +78,9 @@ 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 +import org.apache.spark.mllib.linalg.distributed.RowMatrix + val mat: RowMatrix = ... // Compute the top 10 principal components. diff --git a/docs/mllib-guide.md b/docs/mllib-guide.md index c49f857d07557..842ca5c8c6d8a 100644 --- a/docs/mllib-guide.md +++ b/docs/mllib-guide.md @@ -94,7 +94,7 @@ import org.apache.spark.mllib.linalg.Vector; import org.apache.spark.mllib.linalg.Vectors; double[] array = ... // a double array -Vector vector = Vectors.dense(array) // a dense vector +Vector vector = Vectors.dense(array); // a dense vector {% endhighlight %} [`Vectors`](api/mllib/index.html#org.apache.spark.mllib.linalg.Vectors$) provides factory methods to diff --git a/docs/mllib-linear-methods.md b/docs/mllib-linear-methods.md index 8108440698495..40b7a7f80708c 100644 --- a/docs/mllib-linear-methods.md +++ b/docs/mllib-linear-methods.md @@ -63,7 +63,7 @@ methods MLlib supports: hinge loss$\max \{0, 1-y \wv^T \x \}, \quad y \in \{-1, +1\}$ - $\begin{cases}-y \cdot \x & \text{if $y \wv^T \x <1$}, \\ 0 & + $\begin{cases}-y \cdot \x & \text{if $y \wv^T \x <1$}, \\ 0 & \text{otherwise}.\end{cases}$ @@ -180,7 +180,7 @@ error. {% highlight scala %} import org.apache.spark.SparkContext import org.apache.spark.mllib.classification.SVMWithSGD -import org.apache.spark.mllib.evaluation.binary.BinaryClassificationMetrics +import org.apache.spark.mllib.evaluation.BinaryClassificationMetrics import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.util.MLUtils @@ -225,10 +225,11 @@ algorithm for 200 iterations. import org.apache.spark.mllib.optimization.L1Updater val svmAlg = new SVMWithSGD() -svmAlg.optimizer.setNumIterations(200) - .setRegParam(0.1) - .setUpdater(new L1Updater) -val modelL1 = svmAlg.run(parsedData) +svmAlg.optimizer. + setNumIterations(200). + setRegParam(0.1). + setUpdater(new L1Updater) +val modelL1 = svmAlg.run(training) {% endhighlight %} Similarly, you can use replace `SVMWithSGD` by @@ -322,7 +323,7 @@ val valuesAndPreds = parsedData.map { point => val prediction = model.predict(point.features) (point.label, prediction) } -val MSE = valuesAndPreds.map{case(v, p) => math.pow((v - p), 2)}.reduce(_ + _) / valuesAndPreds.count +val MSE = valuesAndPreds.map{case(v, p) => math.pow((v - p), 2)}.mean() println("training Mean Squared Error = " + MSE) {% endhighlight %} diff --git a/docs/mllib-naive-bayes.md b/docs/mllib-naive-bayes.md index 6160fe5b2fe8c..c47508b7daa2d 100644 --- a/docs/mllib-naive-bayes.md +++ b/docs/mllib-naive-bayes.md @@ -7,13 +7,13 @@ Naive Bayes is a simple multiclass classification algorithm with the assumption between every pair of features. Naive Bayes can be trained very efficiently. Within a single pass to the training data, it computes the conditional probability distribution of each feature given label, and then it applies Bayes' theorem to compute the conditional probability distribution of label -given an observation and use it for prediction. For more details, please visit the wikipedia page +given an observation and use it for prediction. For more details, please visit the Wikipedia page [Naive Bayes classifier](http://en.wikipedia.org/wiki/Naive_Bayes_classifier). In MLlib, we implemented multinomial naive Bayes, which is typically used for document classification. Within that context, each observation is a document, each feature represents a term, -whose value is the frequency of the term. For its formulation, please visit the wikipedia page -[Multinomial naive Bayes](http://en.wikipedia.org/wiki/Naive_Bayes_classifier#Multinomial_naive_Bayes) +whose value is the frequency of the term. For its formulation, please visit the Wikipedia page +[Multinomial Naive Bayes](http://en.wikipedia.org/wiki/Naive_Bayes_classifier#Multinomial_naive_Bayes) or the section [Naive Bayes text classification](http://nlp.stanford.edu/IR-book/html/htmledition/naive-bayes-text-classification-1.html) from the book Introduction to Information @@ -36,9 +36,18 @@ can be used for evaluation and prediction. {% highlight scala %} import org.apache.spark.mllib.classification.NaiveBayes - -val training: RDD[LabeledPoint] = ... // training set -val test: RDD[LabeledPoint] = ... // test set +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.regression.LabeledPoint + +val data = sc.textFile("mllib/data/sample_naive_bayes_data.txt") +val parsedData = data.map { line => + val parts = line.split(',') + LabeledPoint(parts(0).toDouble, Vectors.dense(parts(1).split(' ').map(_.toDouble))) +} +// Split data into training (60%) and test (40%). +val splits = parsedData.randomSplit(Array(0.6, 0.4), seed = 11L) +val training = splits(0) +val test = splits(1) val model = NaiveBayes.train(training, lambda = 1.0) val prediction = model.predict(test.map(_.features)) @@ -58,29 +67,36 @@ optionally smoothing parameter `lambda` as input, and output a can be used for evaluation and prediction. {% highlight java %} +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.function.Function; import org.apache.spark.mllib.classification.NaiveBayes; +import org.apache.spark.mllib.classification.NaiveBayesModel; +import org.apache.spark.mllib.regression.LabeledPoint; +import scala.Tuple2; JavaRDD training = ... // training set JavaRDD test = ... // test set -NaiveBayesModel model = NaiveBayes.train(training.rdd(), 1.0); +final NaiveBayesModel model = NaiveBayes.train(training.rdd(), 1.0); -JavaRDD prediction = model.predict(test.map(new Function() { - public Vector call(LabeledPoint p) { - return p.features(); +JavaRDD prediction = + test.map(new Function() { + @Override public Double call(LabeledPoint p) { + return model.predict(p.features()); } - }) + }); JavaPairRDD predictionAndLabel = prediction.zip(test.map(new Function() { - public Double call(LabeledPoint p) { + @Override public Double call(LabeledPoint p) { return p.label(); } - }) + })); double accuracy = 1.0 * predictionAndLabel.filter(new Function, Boolean>() { - public Boolean call(Tuple2 pl) { + @Override public Boolean call(Tuple2 pl) { return pl._1() == pl._2(); } - }).count() / test.count() + }).count() / test.count(); {% endhighlight %}
@@ -93,7 +109,7 @@ smoothing parameter `lambda` as input, and output a [NaiveBayesModel](api/pyspark/pyspark.mllib.classification.NaiveBayesModel-class.html), which can be used for evaluation and prediction. - + {% highlight python %} from pyspark.mllib.regression import LabeledPoint from pyspark.mllib.classification import NaiveBayes diff --git a/docs/python-programming-guide.md b/docs/python-programming-guide.md index 98233bf556b79..6813963bb080c 100644 --- a/docs/python-programming-guide.md +++ b/docs/python-programming-guide.md @@ -60,8 +60,8 @@ By default, PySpark requires `python` to be available on the system `PATH` and u All of PySpark's library dependencies, including [Py4J](http://py4j.sourceforge.net/), are bundled with PySpark and automatically imported. -Standalone PySpark applications should be run using the `bin/pyspark` script, which automatically configures the Java and Python environment using the settings in `conf/spark-env.sh` or `.cmd`. -The script automatically adds the `bin/pyspark` package to the `PYTHONPATH`. +Standalone PySpark applications should be run using the `bin/spark-submit` script, which automatically +configures the Java and Python environment for running Spark. # Interactive Use @@ -100,7 +100,7 @@ $ MASTER=local[4] ./bin/pyspark ## IPython -It is also possible to launch PySpark in [IPython](http://ipython.org), the +It is also possible to launch the PySpark shell in [IPython](http://ipython.org), the enhanced Python interpreter. PySpark works with IPython 1.0.0 and later. To use IPython, set the `IPYTHON` variable to `1` when running `bin/pyspark`: @@ -120,18 +120,17 @@ IPython also works on a cluster or on multiple cores if you set the `MASTER` env # Standalone Programs -PySpark can also be used from standalone Python scripts by creating a SparkContext in your script and running the script using `bin/pyspark`. +PySpark can also be used from standalone Python scripts by creating a SparkContext in your script and running the script using `bin/spark-submit`. The Quick Start guide includes a [complete example](quick-start.html#a-standalone-app-in-python) of a standalone Python application. -Code dependencies can be deployed by listing them in the `pyFiles` option in the SparkContext constructor: +Code dependencies can be deployed by passing .zip or .egg files in the `--py-files` option of `spark-submit`: -{% highlight python %} -from pyspark import SparkContext -sc = SparkContext("local", "App Name", pyFiles=['MyFile.py', 'lib.zip', 'app.egg']) +{% highlight bash %} +./bin/spark-submit --py-files lib1.zip,lib2.zip my_script.py {% endhighlight %} Files listed here will be added to the `PYTHONPATH` and shipped to remote worker machines. -Code dependencies can be added to an existing SparkContext using its `addPyFile()` method. +Code dependencies can also be added to an existing SparkContext at runtime using its `addPyFile()` method. You can set [configuration properties](configuration.html#spark-properties) by passing a [SparkConf](api/python/pyspark.conf.SparkConf-class.html) object to SparkContext: @@ -139,12 +138,16 @@ You can set [configuration properties](configuration.html#spark-properties) by p {% highlight python %} from pyspark import SparkConf, SparkContext conf = (SparkConf() - .setMaster("local") .setAppName("My app") .set("spark.executor.memory", "1g")) sc = SparkContext(conf = conf) {% endhighlight %} +`spark-submit` supports launching Python applications on standalone, Mesos or YARN clusters, through +its `--master` argument. However, it currently requires the Python driver program to run on the local +machine, not the cluster (i.e. the `--deploy-mode` parameter cannot be `cluster`). + + # API Docs [API documentation](api/python/index.html) for PySpark is available as Epydoc. @@ -158,9 +161,9 @@ some example applications. # Where to Go from Here -PySpark also includes several sample programs in the [`python/examples` folder](https://github.com/apache/spark/tree/master/python/examples). +PySpark also includes several sample programs in the [`examples/src/main/python` folder](https://github.com/apache/spark/tree/master/examples/src/main/python). You can run them by passing the files to `pyspark`; e.g.: - ./bin/pyspark python/examples/wordcount.py + ./bin/spark-submit examples/src/main/python/wordcount.py Each program prints usage help when run without arguments. diff --git a/docs/quick-start.md b/docs/quick-start.md index 64996b52e0404..478b790f92e17 100644 --- a/docs/quick-start.md +++ b/docs/quick-start.md @@ -9,11 +9,9 @@ title: Quick Start This tutorial provides a quick introduction to using Spark. We will first introduce the API through Spark's interactive Scala shell (don't worry if you don't know Scala -- you will not need much for this), then show how to write standalone applications in Scala, Java, and Python. See the [programming guide](scala-programming-guide.html) for a more complete reference. -To follow along with this guide, you only need to have successfully built Spark on one machine. Simply go into your Spark directory and run: - -{% highlight bash %} -$ sbt/sbt assembly -{% endhighlight %} +To follow along with this guide, first download a packaged release of Spark from the +[Spark website](http://spark.apache.org/downloads.html). Since we won't be using HDFS, +you can download a package for any version of Hadoop. # Interactive Analysis with the Spark Shell diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 9765062ec689a..68183ee8b4613 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -5,27 +5,13 @@ title: Launching Spark on YARN Support for running on [YARN (Hadoop NextGen)](http://hadoop.apache.org/docs/r2.0.2-alpha/hadoop-yarn/hadoop-yarn-site/YARN.html) -was added to Spark in version 0.6.0, and improved in 0.7.0 and 0.8.0. - -# Building a YARN-Enabled Assembly JAR - -We need a consolidated Spark JAR (which bundles all the required dependencies) to run Spark jobs on a YARN cluster. -This can be built by setting the Hadoop version and `SPARK_YARN` environment variable, as follows: - - SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_YARN=true sbt/sbt assembly - -The assembled JAR will be something like this: -`./assembly/target/scala-{{site.SCALA_BINARY_VERSION}}/spark-assembly_{{site.SPARK_VERSION}}-hadoop2.0.5.jar`. - -The build process now also supports new YARN versions (2.2.x). See below. +was added to Spark in version 0.6.0, and improved in subsequent releases. # Preparations -- Building a YARN-enabled assembly (see above). -- The assembled jar can be installed into HDFS or used locally. -- Your application code must be packaged into a separate JAR file. - -If you want to test out the YARN deployment mode, you can use the current Spark examples. A `spark-examples_{{site.SCALA_BINARY_VERSION}}-{{site.SPARK_VERSION}}` file can be generated by running `sbt/sbt assembly`. NOTE: since the documentation you're reading is for Spark version {{site.SPARK_VERSION}}, we are assuming here that you have downloaded Spark {{site.SPARK_VERSION}} or checked it out of source control. If you are using a different version of Spark, the version numbers in the jar generated by the sbt package command will obviously be different. +Running Spark-on-YARN requires a binary distribution of Spark which is built with YARN support. +Binary distributions can be downloaded from the Spark project website. +To build Spark yourself, refer to the [building with maven guide](building-with-maven.html). # Configuration @@ -44,86 +30,47 @@ System Properties: * `spark.yarn.max.executor.failures`, the maximum number of executor failures before failing the application. Default is the number of executors requested times 2 with minimum of 3. * `spark.yarn.historyServer.address`, the address of the Spark history server (i.e. host.com:18080). The address should not contain a scheme (http://). Defaults to not being set since the history server is an optional service. This address is given to the Yarn ResourceManager when the Spark application finishes to link the application from the ResourceManager UI to the Spark history server UI. +By default, Spark on YARN will use a Spark jar installed locally, but the Spark jar can also be in a world-readable location on HDFS. This allows YARN to cache it on nodes so that it doesn't need to be distributed each time an application runs. To point to a jar on HDFS, export SPARK_JAR=hdfs:///some/path. + # Launching Spark on YARN Ensure that HADOOP_CONF_DIR or YARN_CONF_DIR points to the directory which contains the (client side) configuration files for the Hadoop cluster. -These configs are used to connect to the cluster, write to the dfs, and connect to the YARN ResourceManager. +These configs are used to write to the dfs and connect to the YARN ResourceManager. There are two deploy modes that can be used to launch Spark applications on YARN. In yarn-cluster mode, the Spark driver runs inside an application master process which is managed by YARN on the cluster, and the client can go away after initiating the application. In yarn-client mode, the driver runs in the client process, and the application master is only used for requesting resources from YARN. Unlike in Spark standalone and Mesos mode, in which the master's address is specified in the "master" parameter, in YARN mode the ResourceManager's address is picked up from the Hadoop configuration. Thus, the master parameter is simply "yarn-client" or "yarn-cluster". -The spark-submit script described in the [cluster mode overview](cluster-overview.html) provides the most straightforward way to submit a compiled Spark application to YARN in either deploy mode. For info on the lower-level invocations it uses, read ahead. For running spark-shell against YARN, skip down to the yarn-client section. - -## Launching a Spark application with yarn-cluster mode. - -The command to launch the Spark application on the cluster is as follows: - - SPARK_JAR= ./bin/spark-class org.apache.spark.deploy.yarn.Client \ - --jar \ - --class \ - --arg \ - --num-executors \ - --driver-memory \ - --executor-memory \ - --executor-cores \ - --name \ - --queue \ - --addJars \ - --files \ - --archives - -To pass multiple arguments the "arg" option can be specified multiple times. For example: - - # Build the Spark assembly JAR and the Spark examples JAR - $ SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_YARN=true sbt/sbt assembly - - # Configure logging - $ cp conf/log4j.properties.template conf/log4j.properties - - # Submit Spark's ApplicationMaster to YARN's ResourceManager, and instruct Spark to run the SparkPi example - $ SPARK_JAR=./assembly/target/scala-{{site.SCALA_BINARY_VERSION}}/spark-assembly-{{site.SPARK_VERSION}}-hadoop2.0.5-alpha.jar \ - ./bin/spark-class org.apache.spark.deploy.yarn.Client \ - --jar examples/target/scala-{{site.SCALA_BINARY_VERSION}}/spark-examples-assembly-{{site.SPARK_VERSION}}.jar \ - --class org.apache.spark.examples.SparkPi \ - --arg yarn-cluster \ - --arg 5 \ - --num-executors 3 \ - --driver-memory 4g \ - --executor-memory 2g \ - --executor-cores 1 - -The above starts a YARN client program which starts the default Application Master. Then SparkPi will be run as a child thread of Application Master. The client will periodically poll the Application Master for status updates and display them in the console. The client will exit once your application has finished running. Refer to the "Viewing Logs" section below for how to see driver and executor logs. - -Because the application is run on a remote machine where the Application Master is running, applications that involve local interaction, such as spark-shell, will not work. - -## Launching a Spark application with yarn-client mode. - -With yarn-client mode, the application will be launched locally, just like running an application or spark-shell on Local / Mesos / Standalone client mode. The launch method is also the same, just make sure to specify the master URL as "yarn-client". You also need to export the env value for SPARK_JAR. +To launch a Spark application in yarn-cluster mode: -Configuration in yarn-client mode: + ./bin/spark-submit --class path.to.your.Class --master yarn-cluster [options] [app options] + +For example: -In order to tune executor cores/number/memory etc., you need to export environment variables or add them to the spark configuration file (./conf/spark_env.sh). The following are the list of options. + $ ./bin/spark-submit --class org.apache.spark.examples.SparkPi \ + --master yarn-cluster \ + --num-executors 3 \ + --driver-memory 4g \ + --executor-memory 2g \ + --executor-cores 1 + examples/target/scala-{{site.SCALA_BINARY_VERSION}}/spark-examples-assembly-{{site.SPARK_VERSION}}.jar \ + yarn-cluster 5 -* `SPARK_EXECUTOR_INSTANCES`, Number of executors to start (Default: 2) -* `SPARK_EXECUTOR_CORES`, Number of cores per executor (Default: 1). -* `SPARK_EXECUTOR_MEMORY`, Memory per executor (e.g. 1000M, 2G) (Default: 1G) -* `SPARK_DRIVER_MEMORY`, Memory for driver (e.g. 1000M, 2G) (Default: 512 Mb) -* `SPARK_YARN_APP_NAME`, The name of your application (Default: Spark) -* `SPARK_YARN_QUEUE`, The YARN queue to use for allocation requests (Default: 'default') -* `SPARK_YARN_DIST_FILES`, Comma separated list of files to be distributed with the job. -* `SPARK_YARN_DIST_ARCHIVES`, Comma separated list of archives to be distributed with the job. +The above starts a YARN client program which starts the default Application Master. Then SparkPi will be run as a child thread of Application Master. The client will periodically poll the Application Master for status updates and display them in the console. The client will exit once your application has finished running. Refer to the "Viewing Logs" section below for how to see driver and executor logs. -For example: +To launch a Spark application in yarn-client mode, do the same, but replace "yarn-cluster" with "yarn-client". To run spark-shell: - SPARK_JAR=./assembly/target/scala-{{site.SCALA_BINARY_VERSION}}/spark-assembly-{{site.SPARK_VERSION}}-hadoop2.0.5-alpha.jar \ - ./bin/run-example org.apache.spark.examples.SparkPi yarn-client + $ MASTER=yarn-client ./bin/spark-shell -or +## Adding additional jars - SPARK_JAR=./assembly/target/scala-{{site.SCALA_BINARY_VERSION}}/spark-assembly-{{site.SPARK_VERSION}}-hadoop2.0.5-alpha.jar \ - MASTER=yarn-client ./bin/spark-shell +In yarn-cluster mode, the driver runs on a different machine than the client, so SparkContext.addJar won't work out of the box with files that are local to the client. To make files on the client available to SparkContext.addJar, include them with the `--jars` option in the launch command. + $ ./bin/spark-submit --class my.main.Class \ + --master yarn-cluster \ + --jars my-other-jar.jar,my-other-other-jar.jar + my-main-jar.jar + yarn-cluster 5 # Viewing logs @@ -135,13 +82,9 @@ will print out the contents of all log files from all containers from the given When log aggregation isn't turned on, logs are retained locally on each machine under YARN_APP_LOGS_DIR, which is usually configured to /tmp/logs or $HADOOP_HOME/logs/userlogs depending on the Hadoop version and installation. Viewing logs for a container requires going to the host that contains them and looking in this directory. Subdirectories organize log files by application ID and container ID. -# Building Spark for Hadoop/YARN 2.2.x - -See [Building Spark with Maven](building-with-maven.html) for instructions on how to build Spark using Maven. - # Important notes - Before Hadoop 2.2, YARN does not support cores in container resource requests. Thus, when running against an earlier version, the numbers of cores given via command line arguments cannot be passed to YARN. Whether core requests are honored in scheduling decisions depends on which scheduler is in use and how it is configured. - The local directories used by Spark executors will be the local directories configured for YARN (Hadoop YARN config yarn.nodemanager.local-dirs). If the user specifies spark.local.dir, it will be ignored. - The --files and --archives options support specifying file names with the # similar to Hadoop. For example you can specify: --files localtest.txt#appSees.txt and this will upload the file you have locally named localtest.txt into HDFS but this will be linked to by the name appSees.txt, and your application should use the name as appSees.txt to reference it when running on YARN. -- The --addJars option allows the SparkContext.addJar function to work if you are using it with local files. It does not need to be used if you are using it with HDFS, HTTP, HTTPS, or FTP files. +- The --jars option allows the SparkContext.addJar function to work if you are using it with local files and running in yarn-cluster mode. It does not need to be used if you are using it with HDFS, HTTP, HTTPS, or FTP files. diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md index b8d89cf00ffbd..f25e9cca88524 100644 --- a/docs/scala-programming-guide.md +++ b/docs/scala-programming-guide.md @@ -48,12 +48,12 @@ how to access a cluster. To create a `SparkContext` you first need to build a `S that contains information about your application. {% highlight scala %} -val conf = new SparkConf().setAppName().setMaster() +val conf = new SparkConf().setAppName(appName).setMaster(master) new SparkContext(conf) {% endhighlight %} -The `` parameter is a string specifying a [Spark, Mesos or YARN cluster URL](#master-urls) -to connect to, or a special "local" string to run in local mode, as described below. `` is +The `master` parameter is a string specifying a [Spark, Mesos or YARN cluster URL](#master-urls) +to connect to, or a special "local" string to run in local mode, as described below. `appName` is a name for your application, which will be shown in the cluster web UI. It's also possible to set these variables [using a configuration file](cluster-overview.html#loading-configurations-from-a-file) which avoids hard-coding the master name in your application. @@ -81,9 +81,8 @@ The master URL passed to Spark can be in one of the following formats: - - @@ -146,7 +145,7 @@ RDDs support two types of operations: *transformations*, which create a new data All transformations in Spark are lazy, in that they do not compute their results right away. Instead, they just remember the transformations applied to some base dataset (e.g. a file). The transformations are only computed when an action requires a result to be returned to the driver program. This design enables Spark to run more efficiently -- for example, we can realize that a dataset created through `map` will be used in a `reduce` and return only the result of the `reduce` to the driver, rather than the larger mapped dataset. -By default, each transformed RDD is recomputed each time you run an action on it. However, you may also *persist* an RDD in memory using the `persist` (or `cache`) method, in which case Spark will keep the elements around on the cluster for much faster access the next time you query it. There is also support for persisting datasets on disk, or replicated across the cluster. The next section in this document describes these options. +By default, each transformed RDD may be recomputed each time you run an action on it. However, you may also *persist* an RDD in memory using the `persist` (or `cache`) method, in which case Spark will keep the elements around on the cluster for much faster access the next time you query it. There is also support for persisting datasets on disk, or replicated across the cluster. The next section in this document describes these options. The following tables list the transformations and actions currently supported (see also the [RDD API doc](api/scala/index.html#org.apache.spark.rdd.RDD) for details): @@ -280,8 +279,8 @@ it is computed in an action, it will be kept in memory on the nodes. The cache i if any partition of an RDD is lost, it will automatically be recomputed using the transformations that originally created it. -In addition, each RDD can be stored using a different *storage level*, allowing you, for example, to -persist the dataset on disk, or persist it in memory but as serialized Java objects (to save space), +In addition, each persisted RDD can be stored using a different *storage level*, allowing you, for example, +to persist the dataset on disk, or persist it in memory but as serialized Java objects (to save space), or replicate it across nodes, or store the data in off-heap memory in [Tachyon](http://tachyon-project.org/). These levels are chosen by passing a [`org.apache.spark.storage.StorageLevel`](api/scala/index.html#org.apache.spark.storage.StorageLevel) @@ -331,6 +330,8 @@ available storage levels is:
Master URLMeaning
local Run Spark locally with one worker thread (i.e. no parallelism at all).
local[K] Run Spark locally with K worker threads (ideally, set this to the number of cores on your machine). +
local[K] Run Spark locally with K worker threads (ideally, set this to the number of cores on your machine).
local[*] Run Spark locally with as many worker threads as logical cores on your machine.
spark://HOST:PORT Connect to the given Spark standalone cluster master. The port must be whichever one your master is configured to use, which is 7077 by default.
+Spark sometimes automatically persists intermediate state from RDD operations, even without users calling persist() or cache(). In particular, if a shuffle happens when computing an RDD, Spark will keep the outputs from the map side of the shuffle on disk to avoid re-computing the entire dependency graph if an RDD is re-used. We still recommend users call persist() if they plan to re-use an RDD iteratively. + ### Which Storage Level to Choose? Spark's storage levels are meant to provide different trade-offs between memory usage and CPU diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 0c743c9d6010d..8a785450adfde 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -416,3 +416,4 @@ results = hiveCtx.hql("FROM src SELECT key, value").collect() {% endhighlight %}
+ diff --git a/docs/streaming-custom-receivers.md b/docs/streaming-custom-receivers.md index 3cfa4516cc12d..a2dc3a8961dfc 100644 --- a/docs/streaming-custom-receivers.md +++ b/docs/streaming-custom-receivers.md @@ -3,126 +3,219 @@ layout: global title: Spark Streaming Custom Receivers --- -A "Spark Streaming" receiver can be a simple network stream, streams of messages from a message queue, files etc. A receiver can also assume roles more than just receiving data like filtering, preprocessing, to name a few of the possibilities. The api to plug-in any user defined custom receiver is thus provided to encourage development of receivers which may be well suited to ones specific need. +Spark Streaming can receive streaming data from any arbitrary data source beyond +the one's for which it has in-built support (that is, beyond Flume, Kafka, files, sockets, etc.). +This requires the developer to implement a *receiver* that is customized for receiving data from +the concerned data source. This guide walks through the process of implementing a custom receiver +and using it in a Spark Streaming application. + +### Implementing a Custom Receiver + +This starts with implementing a [Receiver](api/scala/index.html#org.apache.spark.streaming.receiver.Receiver). +A custom receiver must extend this abstract class by implementing two methods +- `onStart()`: Things to do to start receiving data. +- `onStop()`: Things to do to stop receiving data. + +Note that `onStart()` and `onStop()` must not block indefinitely. Typically, onStart() would start the threads +that responsible for receiving the data and `onStop()` would ensure that the receiving by those threads +are stopped. The receiving threads can also use `isStopped()`, a `Receiver` method, to check whether they +should stop receiving data. + +Once the data is received, that data can be stored inside Spark +by calling `store(data)`, which is a method provided by the +[Receiver](api/scala/index.html#org.apache.spark.streaming.receiver.Receiver) class. +There are number of flavours of `store()` which allow you store the received data +record-at-a-time or as whole collection of objects / serialized bytes. + +Any exception in the receiving threads should be caught and handled properly to avoid silent +failures of the receiver. `restart()` will restart the receiver by +asynchronously calling `onStop()` and then calling `onStart()` after a delay. +`stop()` will call `onStop()` and terminate the receiver. Also, `reportError()` +reports a error message to the driver (visible in the logs and UI) without stopping / restarting +the receiver. + +The following is a custom receiver that receives a stream of text over a socket. It treats +'\n' delimited lines in the text stream as records and stores them with Spark. If the receiving thread +has any error connecting or receiving, the receiver is restarted to make another attempt to connect. + +
+
-This guide shows the programming model and features by walking through a simple sample receiver and corresponding Spark Streaming application. +{% highlight scala %} -### Writing a Simple Receiver +class CustomReceiver(host: String, port: Int) + extends Receiver[String](StorageLevel.MEMORY_AND_DISK_2) with Logging { + + def onStart() { + // Start the thread that receives data over a connection + new Thread("Socket Receiver") { + override def run() { receive() } + }.start() + } + + def onStop() { + // There is nothing much to do as the thread calling receive() + // is designed to stop by itself isStopped() returns false + } + + /** Create a socket connection and receive data until receiver is stopped */ + private def receive() { + var socket: Socket = null + var userInput: String = null + try { + // Connect to host:port + socket = new Socket(host, port) + + // Until stopped or connection broken continue reading + val reader = new BufferedReader(new InputStreamReader(socket.getInputStream(), "UTF-8")) + userInput = reader.readLine() + while(!isStopped && userInput != null) { + store(userInput) + userInput = reader.readLine() + } + reader.close() + socket.close() + + // Restart in an attempt to connect again when server is active again + restart("Trying to connect again") + } catch { + case e: java.net.ConnectException => + // restart if could not connect to server + restart("Error connecting to " + host + ":" + port, e) + case t: Throwable => + // restart if there is any other error + restart("Error receiving data", t) + } + } +} -This starts with implementing [NetworkReceiver](api/scala/index.html#org.apache.spark.streaming.dstream.NetworkReceiver). +{% endhighlight %} -The following is a simple socket text-stream receiver. +
+
+ +{% highlight java %} + +public class JavaCustomReceiver extends Receiver { + + String host = null; + int port = -1; + + public JavaCustomReceiver(String host_ , int port_) { + super(StorageLevel.MEMORY_AND_DISK_2()); + host = host_; + port = port_; + } + + public void onStart() { + // Start the thread that receives data over a connection + new Thread() { + @Override public void run() { + receive(); + } + }.start(); + } + + public void onStop() { + // There is nothing much to do as the thread calling receive() + // is designed to stop by itself isStopped() returns false + } + + /** Create a socket connection and receive data until receiver is stopped */ + private void receive() { + Socket socket = null; + String userInput = null; + + try { + // connect to the server + socket = new Socket(host, port); + + BufferedReader reader = new BufferedReader(new InputStreamReader(socket.getInputStream())); + + // Until stopped or connection broken continue reading + while (!isStopped() && (userInput = reader.readLine()) != null) { + System.out.println("Received data '" + userInput + "'"); + store(userInput); + } + reader.close(); + socket.close(); + + // Restart in an attempt to connect again when server is active again + restart("Trying to connect again"); + } catch(ConnectException ce) { + // restart if could not connect to server + restart("Could not connect", ce); + } catch(Throwable t) { + // restart if there is any other error + restart("Error receiving data", t); + } + } +} -{% highlight scala %} - class SocketTextStreamReceiver(host: String, port: Int) - extends NetworkReceiver[String] - { - protected lazy val blocksGenerator: BlockGenerator = - new BlockGenerator(StorageLevel.MEMORY_ONLY_SER_2) - - protected def onStart() = { - blocksGenerator.start() - val socket = new Socket(host, port) - val dataInputStream = new BufferedReader(new InputStreamReader(socket.getInputStream(), "UTF-8")) - var data: String = dataInputStream.readLine() - while (data != null) { - blocksGenerator += data - data = dataInputStream.readLine() - } - } - - protected def onStop() { - blocksGenerator.stop() - } - } {% endhighlight %} +
+
-All we did here is extended NetworkReceiver and called blockGenerator's API method (i.e. +=) to push our blocks of data. Please refer to scala-docs of NetworkReceiver for more details. +### Using the custom receiver in a Spark Streaming application -### An Actor as Receiver +The custom receiver can be used in a Spark Streaming application by using +`streamingContext.receiverStream()`. This will create +input DStream using data received by the instance of custom receiver, as shown below -This starts with implementing [Actor](#References) - -Following is a simple socket text-stream receiver, which is appearently overly simplified using Akka's socket.io api. +
+
{% highlight scala %} - class SocketTextStreamReceiver (host:String, - port:Int, - bytesToString: ByteString => String) extends Actor with Receiver { - - override def preStart = IOManager(context.system).connect(host, port) - - def receive = { - case IO.Read(socket, bytes) => pushBlock(bytesToString(bytes)) - } - - } +// Assuming ssc is the StreamingContext +val customReceiverStream = ssc.receiverStream(new CustomReceiver(host, port)) +val words = lines.flatMap(_.split(" ")) +... {% endhighlight %} -All we did here is mixed in trait Receiver and called pushBlock api method to push our blocks of data. Please refer to scala-docs of Receiver for more details. - -### A Sample Spark Application +The full source code is in the example [CustomReceiver.scala](https://github.com/apache/spark/blob/master/examples/src/main/scala/org/apache/spark/streaming/examples/CustomReceiver.scala). -* First create a Spark streaming context with master url and batchduration. +
+
-{% highlight scala %} - val ssc = new StreamingContext(master, "WordCountCustomStreamSource", - Seconds(batchDuration)) +{% highlight java %} +// Assuming ssc is the JavaStreamingContext +JavaDStream customReceiverStream = ssc.receiverStream(new JavaCustomReceiver(host, port)); +JavaDStream words = lines.flatMap(new FlatMapFunction() { ... }); +... {% endhighlight %} -* Plug-in the custom receiver into the spark streaming context and create a DStream. +The full source code is in the example [JavaCustomReceiver.java](https://github.com/apache/spark/blob/master/examples/src/main/java/org/apache/spark/streaming/examples/JavaCustomReceiver.java). -{% highlight scala %} - val lines = ssc.networkStream[String](new SocketTextStreamReceiver( - "localhost", 8445)) -{% endhighlight %} +
+
-* OR Plug-in the actor as receiver into the spark streaming context and create a DStream. -{% highlight scala %} - val lines = ssc.actorStream[String](Props(new SocketTextStreamReceiver( - "localhost",8445, z => z.utf8String)),"SocketReceiver") -{% endhighlight %} -* Process it. +### Implementing and Using a Custom Actor-based Receiver -{% highlight scala %} - val words = lines.flatMap(_.split(" ")) - val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _) +Custom [Akka Actors](http://doc.akka.io/docs/akka/2.2.4/scala/actors.html) can also be used to +receive data. The [`ActorHelper`](api/scala/index.html#org.apache.spark.streaming.receiver.ActorHelper) +trait can be applied on any Akka actor, which allows received data to be stored in Spark using + `store(...)` methods. The supervisor strategy of this actor can be configured to handle failures, etc. - wordCounts.print() - ssc.start() +{% highlight scala %} +class CustomActor extends Actor with ActorHelper { + def receive = { + case data: String => store(data) + } +} {% endhighlight %} -* After processing it, stream can be tested using the netcat utility. - - $ nc -l localhost 8445 - hello world - hello hello - - -## Multiple Homogeneous/Heterogeneous Receivers. - -A DStream union operation is provided for taking union on multiple input streams. +And a new input stream can be created with this custom actor as {% highlight scala %} - val lines = ssc.actorStream[String](Props(new SocketTextStreamReceiver( - "localhost",8445, z => z.utf8String)),"SocketReceiver") - - // Another socket stream receiver - val lines2 = ssc.actorStream[String](Props(new SocketTextStreamReceiver( - "localhost",8446, z => z.utf8String)),"SocketReceiver") - - val union = lines.union(lines2) +// Assuming ssc is the StreamingContext +val lines = ssc.actorStream[String](Props(new CustomActor()), "CustomReceiver") {% endhighlight %} -Above stream can be easily process as described earlier. - -_A more comprehensive example is provided in the spark streaming examples_ +See [ActorWordCount.scala](https://github.com/apache/spark/blob/master/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala) +for an end-to-end example. -## References -1.[Akka Actor documentation](http://doc.akka.io/docs/akka/2.0.5/scala/actors.html) -2.[NetworkReceiver](api/scala/index.html#org.apache.spark.streaming.dstream.NetworkReceiver) diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 7ad06427cac5c..939599aa6855b 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -76,16 +76,19 @@ Besides Spark's configuration, we specify that any DStream will be processed in 1 second batches. {% highlight scala %} -// Create a StreamingContext with a SparkConf configuration -val ssc = new StreamingContext(sparkConf, Seconds(1)) +import org.apache.spark.api.java.function._ +import org.apache.spark.streaming._ +import org.apache.spark.streaming.api._ +// Create a StreamingContext with a local master +val ssc = new StreamingContext("local", "NetworkWordCount", Seconds(1)) {% endhighlight %} Using this context, we then create a new DStream by specifying the IP address and port of the data server. {% highlight scala %} -// Create a DStream that will connect to serverIP:serverPort -val lines = ssc.socketTextStream(serverIP, serverPort) +// Create a DStream that will connect to serverIP:serverPort, like localhost:9999 +val lines = ssc.socketTextStream("localhost", 9999) {% endhighlight %} This `lines` DStream represents the stream of data that will be received from the data @@ -103,6 +106,7 @@ each line will be split into multiple words and the stream of words is represent `words` DStream. Next, we want to count these words. {% highlight scala %} +import org.apache.spark.streaming.StreamingContext._ // Count each word in each batch val pairs = words.map(word => (word, 1)) val wordCounts = pairs.reduceByKey(_ + _) @@ -125,29 +129,33 @@ ssc.awaitTermination() // Wait for the computation to terminate {% endhighlight %} The complete code can be found in the Spark Streaming example -[NetworkWordCount]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala). +[NetworkWordCount]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/scala/org/apache/spark/examples/streaming/NetworkWordCount.scala).
First, we create a -[JavaStreamingContext](api/scala/index.html#org.apache.spark.streaming.api.java.JavaStreamingContext) object, +[JavaStreamingContext](api/java/org/apache/spark/streaming/api/java/JavaStreamingContext.html) object, which is the main entry point for all streaming functionality. Besides Spark's configuration, we specify that any DStream would be processed in 1 second batches. {% highlight java %} -// Create a StreamingContext with a SparkConf configuration -JavaStreamingContext jssc = StreamingContext(sparkConf, new Duration(1000)) +import org.apache.spark.api.java.function.*; +import org.apache.spark.streaming.*; +import org.apache.spark.streaming.api.java.*; +import scala.Tuple2; +// Create a StreamingContext with a local master +JavaStreamingContext jssc = new JavaStreamingContext("local", "JavaNetworkWordCount", new Duration(1000)) {% endhighlight %} Using this context, we then create a new DStream by specifying the IP address and port of the data server. {% highlight java %} -// Create a DStream that will connect to serverIP:serverPort -JavaDStream lines = jssc.socketTextStream(serverIP, serverPort); +// Create a DStream that will connect to serverIP:serverPort, like localhost:9999 +JavaReceiverInputDStream lines = jssc.socketTextStream("localhost", 9999); {% endhighlight %} This `lines` DStream represents the stream of data that will be received from the data @@ -159,7 +167,7 @@ space into words. JavaDStream words = lines.flatMap( new FlatMapFunction() { @Override public Iterable call(String x) { - return Lists.newArrayList(x.split(" ")); + return Arrays.asList(x.split(" ")); } }); {% endhighlight %} @@ -207,7 +215,7 @@ jssc.awaitTermination(); // Wait for the computation to terminate {% endhighlight %} The complete code can be found in the Spark Streaming example -[JavaNetworkWordCount]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java). +[JavaNetworkWordCount]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/java/org/apache/spark/examples/streaming/JavaNetworkWordCount.java).
@@ -226,12 +234,12 @@ Then, in a different terminal, you can start the example by using
{% highlight bash %} -$ ./bin/run-example org.apache.spark.streaming.examples.NetworkWordCount local[2] localhost 9999 +$ ./bin/run-example org.apache.spark.examples.streaming.NetworkWordCount local[2] localhost 9999 {% endhighlight %}
{% highlight bash %} -$ ./bin/run-example org.apache.spark.streaming.examples.JavaNetworkWordCount local[2] localhost 9999 +$ ./bin/run-example org.apache.spark.examples.streaming.JavaNetworkWordCount local[2] localhost 9999 {% endhighlight %}
@@ -260,7 +268,7 @@ hello world {% highlight bash %} # TERMINAL 2: RUNNING NetworkWordCount or JavaNetworkWordCount -$ ./bin/run-example org.apache.spark.streaming.examples.NetworkWordCount local[2] localhost 9999 +$ ./bin/run-example org.apache.spark.examples.streaming.NetworkWordCount local[2] localhost 9999 ... ------------------------------------------- Time: 1357008430000 ms @@ -359,7 +367,7 @@ as explained earlier. Finally, the last two parameters are needed to deploy your if running in distributed mode, as described in the [Spark programming guide](scala-programming-guide.html#deploying-code-on-a-cluster). Additionally, the underlying SparkContext can be accessed as -`streamingContext.sparkContext`. +`ssc.sparkContext`. The batch interval must be set based on the latency requirements of your application and available cluster resources. See the [Performance Tuning](#setting-the-right-batch-size) @@ -399,7 +407,7 @@ These operations are discussed in detail in later sections. ## Input Sources -We have already taken a look at the `streamingContext.socketTextStream(...)` in the [quick +We have already taken a look at the `ssc.socketTextStream(...)` in the [quick example](#a-quick-example) which creates a DStream from text data received over a TCP socket connection. Besides sockets, the core Spark Streaming API provides methods for creating DStreams from files and Akka actors as input sources. @@ -409,12 +417,12 @@ Specifically, for files, the DStream can be created as
{% highlight scala %} -streamingContext.fileStream(dataDirectory) +ssc.fileStream(dataDirectory) {% endhighlight %}
{% highlight java %} -javaStreamingContext.fileStream(dataDirectory); +jssc.fileStream(dataDirectory); {% endhighlight %}
@@ -443,13 +451,13 @@ project dependencies, you can create a DStream from Kafka as
{% highlight scala %} import org.apache.spark.streaming.kafka._ -KafkaUtils.createStream(streamingContext, kafkaParams, ...) +KafkaUtils.createStream(ssc, kafkaParams, ...) {% endhighlight %}
{% highlight java %} -import org.apache.spark.streaming.kafka.* -KafkaUtils.createStream(javaStreamingContext, kafkaParams, ...); +import org.apache.spark.streaming.kafka.*; +KafkaUtils.createStream(jssc, kafkaParams, ...); {% endhighlight %}
@@ -578,13 +586,14 @@ val runningCounts = pairs.updateStateByKey[Int](updateFunction _)
{% highlight java %} +import com.google.common.base.Optional; Function2, Optional, Optional> updateFunction = new Function2, Optional, Optional>() { @Override public Optional call(List values, Optional state) { Integer newSum = ... // add the new values with the previous running count to get the new count - return Optional.of(newSum) + return Optional.of(newSum); } - } + }; {% endhighlight %} This is applied on a DStream containing words (say, the `pairs` DStream containing `(word, @@ -600,7 +609,7 @@ JavaPairDStream runningCounts = pairs.updateStateByKey(updateFu The update function will be called for each word, with `newValues` having a sequence of 1's (from the `(word, 1)` pairs) and the `runningCount` having the previous count. For the complete Scala code, take a look at the example -[StatefulNetworkWordCount]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala). +[StatefulNetworkWordCount]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala).

Transform Operation

@@ -617,9 +626,9 @@ spam information (maybe generated with Spark as well) and then filtering based o
{% highlight scala %} -val spamInfoRDD = sparkContext.hadoopFile(...) // RDD containing spam information +val spamInfoRDD = ssc.sparkContext.newAPIHadoopRDD(...) // RDD containing spam information -val cleanedDStream = inputDStream.transform(rdd => { +val cleanedDStream = wordCounts.transform(rdd => { rdd.join(spamInfoRDD).filter(...) // join data stream with spam information to do data cleaning ... }) @@ -629,13 +638,14 @@ val cleanedDStream = inputDStream.transform(rdd => {
{% highlight java %} +import org.apache.spark.streaming.api.java.*; // RDD containing spam information -JavaPairRDD spamInfoRDD = javaSparkContext.hadoopFile(...); +final JavaPairRDD spamInfoRDD = jssc.sparkContext().newAPIHadoopRDD(...); -JavaPairDStream cleanedDStream = inputDStream.transform( +JavaPairDStream cleanedDStream = wordCounts.transform( new Function, JavaPairRDD>() { @Override public JavaPairRDD call(JavaPairRDD rdd) throws Exception { - rdd.join(spamInfoRDD).filter(...) // join data stream with spam information to do data cleaning + rdd.join(spamInfoRDD).filter(...); // join data stream with spam information to do data cleaning ... } }); @@ -684,7 +694,7 @@ operation `reduceByKeyAndWindow`. {% highlight scala %} // Reduce last 30 seconds of data, every 10 seconds -val windowedWordCounts = pairs.reduceByKeyAndWindow(_ + _, Seconds(30), Seconds(10)) +val windowedWordCounts = pairs.reduceByKeyAndWindow((a:Int,b:Int) => (a + b), Seconds(30), Seconds(10)) {% endhighlight %}
@@ -699,7 +709,7 @@ Function2 reduceFunc = new Function2 windowedWordCounts = pair.reduceByKeyAndWindow(reduceFunc, new Duration(30000), new Duration(10000)); +JavaPairDStream windowedWordCounts = pairs.reduceByKeyAndWindow(reduceFunc, new Duration(30000), new Duration(10000)); {% endhighlight %}
@@ -853,6 +863,51 @@ For DStreams that must be checkpointed (that is, DStreams created by `updateStat `reduceByKeyAndWindow` with inverse function), the checkpoint interval of the DStream is by default set to a multiple of the DStream's sliding interval such that its at least 10 seconds. +## Deployment +A Spark Streaming application is deployed on a cluster in the same way as any other Spark application. +Please refer to the [deployment guide](cluster-overview.html) for more details. + +If a running Spark Streaming application needs to be upgraded (with new application code), then +there are two possible mechanism. + +- The upgraded Spark Streaming application is started and run in parallel to the existing application. +Once the new one (receiving the same data as the old one) has been warmed up and ready +for prime time, the old one be can be brought down. Note that this can be done for data sources that support +sending the data to two destinations (i.e., the earlier and upgraded applications). + +- The existing application is shutdown gracefully (see +[`StreamingContext.stop(...)`](api/scala/index.html#org.apache.spark.streaming.StreamingContext) +or [`JavaStreamingContext.stop(...)`](api/java/org/apache/spark/streaming/api/java/JavaStreamingContext.html) +for graceful shutdown options) which ensure data that have been received is completely +processed before shutdown. Then the +upgraded application can be started, which will start processing from the same point where the earlier +application left off. Note that this can be done only with input sources that support source-side buffering +(like Kafka, and Flume) as data needs to be buffered while the previous application down and +the upgraded application is not yet up. + +## Monitoring +Beyond Spark's [monitoring capabilities](monitoring.html), there are additional capabilities +specific to Spark Streaming. When a StreamingContext is used, the +[Spark web UI](monitoring.html#web-interfaces) shows +an additional `Streaming` tab which shows statistics about running receivers (whether +receivers are active, number of records received, receiver error, etc.) +and completed batches (batch processing times, queueing delays, etc.). This can be used to +monitor the progress of the streaming application. + +The following two metrics in web UI is particularly important - +*Processing Time* and *Scheduling Delay* (under *Batch Processing Statistics*). The first is the +time to process each batch of data, and the second is the time a batch waits in a queue +for the processing of previous batches to finish. If the batch processing time is consistently more +than the batch interval and/or the queueing delay keeps increasing, then it indicates the system is +not able to process the batches as fast they are being generated and falling behind. +In that case, consider +[reducing](#reducing-the-processing-time-of-each-batch) the batch processing time. + +The progress of a Spark Streaming program can also be monitored using the +[StreamingListener](api/scala/index.html#org.apache.spark.scheduler.StreamingListener) interface, +which allows you to get receiver status and processing times. Note that this is a developer API +and it is likely to be improved upon (i.e., more information reported) in the future. + *************************************************************************************************** # Performance Tuning @@ -865,7 +920,8 @@ improve the performance of you application. At a high level, you need to conside Reducing the processing time of each batch of data by efficiently using cluster resources.
  • - Setting the right batch size such that the data processing can keep up with the data ingestion. + Setting the right batch size such that the batches of data can be processed as fast as they + are received (that is, data processing keeps up with the data ingestion).
  • @@ -874,7 +930,30 @@ There are a number of optimizations that can be done in Spark to minimize the pr each batch. These have been discussed in detail in [Tuning Guide](tuning.html). This section highlights some of the most important ones. -### Level of Parallelism +### Level of Parallelism in Data Receiving +Receiving data over the network (like Kafka, Flume, socket, etc.) requires the data to deserialized +and stored in Spark. If the data receiving becomes a bottleneck in the system, then consider +parallelizing the data receiving. Note that each input DStream +creates a single receiver (running on a worker machine) that receives a single stream of data. +Receiving multiple data streams can therefore be achieved by creating multiple input DStreams +and configuring them to receive different partitions of the data stream from the source(s). +For example, a single Kafka input stream receiving two topics of data can be split into two +Kafka input streams, each receiving only one topic. This would run two receivers on two workers, +thus allowing data to received in parallel, and increasing overall throughput. + +Another parameter that should be considered is the receiver's blocking interval. For most receivers, +the received data is coalesced together into large blocks of data before storing inside Spark's memory. +The number of blocks in each batch determines the number of tasks that will be used to process those +the received data in a map-like transformation. This blocking interval is determined by the +[configuration parameter](configuration.html) `spark.streaming.blockInterval` and the default value +is 200 milliseconds. + +An alternative to receiving data with multiple input streams / receivers is to explicitly repartition +the input data stream (using `inputStream.repartition()`). +This distributes the received batches of data across all the machines in the cluster +before further processing. + +### Level of Parallelism in Data Processing Cluster resources maybe under-utilized if the number of parallel tasks used in any stage of the computation is not high enough. For example, for distributed reduce operations like `reduceByKey` and `reduceByKeyAndWindow`, the default number of parallel tasks is 8. You can pass the level of @@ -911,16 +990,22 @@ These changes may reduce batch processing time by 100s of milliseconds, thus allowing sub-second batch size to be viable. ## Setting the Right Batch Size -For a Spark Streaming application running on a cluster to be stable, the processing of the data -streams must keep up with the rate of ingestion of the data streams. Depending on the type of -computation, the batch size used may have significant impact on the rate of ingestion that can be -sustained by the Spark Streaming application on a fixed cluster resources. For example, let us +For a Spark Streaming application running on a cluster to be stable, the system should be able to +process data as fast as it is being received. In other words, batches of data should be processed +as fast as they are being generated. Whether this is true for an application can be found by +[monitoring](#monitoring) the processing times in the streaming web UI, where the batch +processing time should be less than the batch interval. + +Depending on the nature of the streaming +computation, the batch interval used may have significant impact on the data rates that can be +sustained by the application on a fixed set of cluster resources. For example, let us consider the earlier WordCountNetwork example. For a particular data rate, the system may be able -to keep up with reporting word counts every 2 seconds (i.e., batch size of 2 seconds), but not -every 500 milliseconds. +to keep up with reporting word counts every 2 seconds (i.e., batch interval of 2 seconds), but not +every 500 milliseconds. So the batch interval needs to be set such that the expected data rate in +production can be sustained. A good approach to figure out the right batch size for your application is to test it with a -conservative batch size (say, 5-10 seconds) and a low data rate. To verify whether the system +conservative batch interval (say, 5-10 seconds) and a low data rate. To verify whether the system is able to keep up with data rate, you can check the value of the end-to-end delay experienced by each processed batch (either look for "Total delay" in Spark driver log4j logs, or use the [StreamingListener](api/scala/index.html#org.apache.spark.streaming.scheduler.StreamingListener) @@ -932,29 +1017,6 @@ data rate and/or reducing the batch size. Note that momentary increase in the de temporary data rate increases maybe fine as long as the delay reduces back to a low value (i.e., less than batch size). -## 24/7 Operation -By default, Spark does not forget any of the metadata (RDDs generated, stages processed, etc.). -But for a Spark Streaming application to operate 24/7, it is necessary for Spark to do periodic -cleanup of it metadata. This can be enabled by setting the -[configuration property](configuration.html#spark-properties) `spark.cleaner.ttl` to the number of -seconds you want any metadata to persist. For example, setting `spark.cleaner.ttl` to 600 would -cause Spark periodically cleanup all metadata and persisted RDDs that are older than 10 minutes. -Note, that this property needs to be set before the SparkContext is created. - -This value is closely tied with any window operation that is being used. Any window operation -would require the input data to be persisted in memory for at least the duration of the window. -Hence it is necessary to set the delay to at least the value of the largest window operation used -in the Spark Streaming application. If this delay is set too low, the application will throw an -exception saying so. - -## Monitoring -Besides Spark's in-built [monitoring capabilities](monitoring.html), -the progress of a Spark Streaming program can also be monitored using the [StreamingListener] -(api/scala/index.html#org.apache.spark.scheduler.StreamingListener) interface, -which allows you to get statistics of batch processing times, queueing delays, -and total end-to-end delays. Note that this is still an experimental API and it is likely to be -improved upon (i.e., more information reported) in the future. - ## Memory Tuning Tuning the memory usage and GC behavior of Spark applications have been discussed in great detail in the [Tuning Guide](tuning.html). It is recommended that you read that. In this section, @@ -1073,7 +1135,7 @@ If the `checkpointDirectory` exists, then the context will be recreated from the If the directory does not exist (i.e., running for the first time), then the function `functionToCreateContext` will be called to create a new context and set up the DStreams. See the Scala example -[RecoverableNetworkWordCount]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/scala/org/apache/spark/streaming/examples/RecoverableNetworkWordCount.scala). +[RecoverableNetworkWordCount]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala). This example appends the word counts of network data into a file. You can also explicitly create a `StreamingContext` from the checkpoint data and start the @@ -1087,7 +1149,7 @@ This behavior is made simple by using `JavaStreamingContext.getOrCreate`. This i {% highlight java %} // Create a factory object that can create a and setup a new JavaStreamingContext JavaStreamingContextFactory contextFactory = new JavaStreamingContextFactory() { - JavaStreamingContextFactory create() { + @Override public JavaStreamingContext create() { JavaStreamingContext jssc = new JavaStreamingContext(...); // new context JavaDStream lines = jssc.socketTextStream(...); // create DStreams ... @@ -1112,7 +1174,7 @@ If the `checkpointDirectory` exists, then the context will be recreated from the If the directory does not exist (i.e., running for the first time), then the function `contextFactory` will be called to create a new context and set up the DStreams. See the Scala example -[JavaRecoverableWordCount]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/scala/org/apache/spark/streaming/examples/JavaRecoverableWordCount.scala) +[JavaRecoverableWordCount]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/scala/org/apache/spark/examples/streaming/JavaRecoverableWordCount.scala) (note that this example is missing in the 0.9 release, so you can test it using the master branch). This example appends the word counts of network data into a file. @@ -1239,18 +1301,79 @@ in the file. This is what the sequence of outputs would be with and without a dr If the driver had crashed in the middle of the processing of time 3, then it will process time 3 and output 30 after recovery. +*************************************************************************************************** + +# Migration Guide from 0.9.1 or below to 1.x +Between Spark 0.9.1 and Spark 1.0, there were a few API changes made to ensure future API stability. +This section elaborates the steps required to migrate your existing code to 1.0. + +**Input DStreams**: All operations that create an input stream (e.g., `StreamingContext.socketStream`, +`FlumeUtils.createStream`, etc.) now returns +[InputDStream](api/scala/index.html#org.apache.spark.streaming.dstream.InputDStream) / +[ReceiverInputDStream](api/scala/index.html#org.apache.spark.streaming.dstream.ReceiverInputDStream) +(instead of DStream) for Scala, and [JavaInputDStream](api/java/org/apache/spark/streaming/api/java/JavaInputDStream.html) / +[JavaPairInputDStream](api/java/org/apache/spark/streaming/api/java/JavaPairInputDStream.html) / +[JavaReceiverInputDStream](api/java/org/apache/spark/streaming/api/java/JavaReceiverInputDStream.html) / +[JavaPairReceiverInputDStream](api/java/org/apache/spark/streaming/api/java/JavaPairReceiverInputDStream.html) +(instead of JavaDStream) for Java. This ensures that functionality specific to input streams can +be added to these classes in the future without breaking binary compatibility. +Note that your existing Spark Streaming applications should not require any change +(as these new classes are subclasses of DStream/JavaDStream) but may require recompilation with Spark 1.0. + +**Custom Network Receivers**: Since the release to Spark Streaming, custom network receivers could be defined +in Scala using the class NetworkReceiver. However, the API was limited in terms of error handling +and reporting, and could not be used from Java. Starting Spark 1.0, this class has been +replaced by [Receiver](api/scala/index.html#org.apache.spark.streaming.receiver.Receiver) which has +the following advantages. + +* Methods like `stop` and `restart` have been added to for better control of the lifecycle of a receiver. See +the [custom receiver guide](streaming-custom-receiver.html) for more details. +* Custom receivers can be implemented using both Scala and Java. + +To migrate your existing custom receivers from the earlier NetworkReceiver to the new Receiver, you have +to do the following. + +* Make your custom receiver class extend +[`org.apache.spark.streaming.receiver.Receiver`](api/scala/index.html#org.apache.spark.streaming.receiver.Receiver) +instead of `org.apache.spark.streaming.dstream.NetworkReceiver`. +* Earlier, a BlockGenerator object had to be created by the custom receiver, to which received data was +added for being stored in Spark. It had to be explicitly started and stopped from `onStart()` and `onStop()` +methods. The new Receiver class makes this unnecessary as it adds a set of methods named `store()` +that can be called to store the data in Spark. So, to migrate your custom network receiver, remove any +BlockGenerator object (does not exist any more in Spark 1.0 anyway), and use `store(...)` methods on +received data. + +**Actor-based Receivers**: Data could have been received using any Akka Actors by extending the actor class with +`org.apache.spark.streaming.receivers.Receiver` trait. This has been renamed to +[`org.apache.spark.streaming.receiver.ActorHelper`](api/scala/index.html#org.apache.spark.streaming.receiver.ActorHelper) +and the `pushBlock(...)` methods to store received data has been renamed to `store(...)`. Other helper classes in +the `org.apache.spark.streaming.receivers` package were also moved +to [`org.apache.spark.streaming.receiver`](api/scala/index.html#org.apache.spark.streaming.receiver.package) +package and renamed for better clarity. + +*************************************************************************************************** + # Where to Go from Here * API documentation - - Main docs of StreamingContext and DStreams in [Scala](api/scala/index.html#org.apache.spark.streaming.package) - and [Java](api/scala/index.html#org.apache.spark.streaming.api.java.package) - - Additional docs for - [Kafka](api/scala/index.html#org.apache.spark.streaming.kafka.KafkaUtils$), - [Flume](api/scala/index.html#org.apache.spark.streaming.flume.FlumeUtils$), - [Twitter](api/scala/index.html#org.apache.spark.streaming.twitter.TwitterUtils$), - [ZeroMQ](api/scala/index.html#org.apache.spark.streaming.zeromq.ZeroMQUtils$), and - [MQTT](api/scala/index.html#org.apache.spark.streaming.mqtt.MQTTUtils$) - -* More examples in [Scala]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/scala/org/apache/spark/streaming/examples) - and [Java]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/java/org/apache/spark/streaming/examples) -* [Paper](http://www.eecs.berkeley.edu/Pubs/TechRpts/2012/EECS-2012-259.pdf) describing Spark Streaming. + - Scala docs + * [StreamingContext](api/scala/index.html#org.apache.spark.streaming.StreamingContext) and + [DStream](api/scala/index.html#org.apache.spark.streaming.dstream.DStream) + * [KafkaUtils](api/scala/index.html#org.apache.spark.streaming.kafka.KafkaUtils$), + [FlumeUtils](api/scala/index.html#org.apache.spark.streaming.flume.FlumeUtils$), + [TwitterUtils](api/scala/index.html#org.apache.spark.streaming.twitter.TwitterUtils$), + [ZeroMQUtils](api/scala/index.html#org.apache.spark.streaming.zeromq.ZeroMQUtils$), and + [MQTTUtils](api/scala/index.html#org.apache.spark.streaming.mqtt.MQTTUtils$) + - Java docs + * [JavaStreamingContext](api/java/org/apache/spark/streaming/api/java/JavaStreamingContext.html), + [JavaDStream](api/java/org/apache/spark/streaming/api/java/JavaDStream.html) and + [PairJavaDStream](api/java/org/apache/spark/streaming/api/java/PairJavaDStream.html) + * [KafkaUtils](api/java/org/apache/spark/streaming/kafka/KafkaUtils.html), + [FlumeUtils](api/java/org/apache/spark/streaming/flume/FlumeUtils.html), + [TwitterUtils](api/java/org/apache/spark/streaming/twitter/TwitterUtils.html), + [ZeroMQUtils](api/java/org/apache/spark/streaming/zeromq/ZeroMQUtils.html), and + [MQTTUtils](api/java/org/apache/spark/streaming/mqtt/MQTTUtils.html) + +* More examples in [Scala]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/scala/org/apache/spark/examples/streaming) + and [Java]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/java/org/apache/spark/examples/streaming) +* [Paper](http://www.eecs.berkeley.edu/Pubs/TechRpts/2012/EECS-2012-259.pdf) and [video](http://youtu.be/g171ndOHgJ0) describing Spark Streaming. diff --git a/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh b/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh index 42e8faa26ed09..3570891be804e 100644 --- a/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh +++ b/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh @@ -28,3 +28,5 @@ export SPARK_VERSION="{{spark_version}}" export SHARK_VERSION="{{shark_version}}" export HADOOP_MAJOR_VERSION="{{hadoop_major_version}}" export SWAP_MB="{{swap}}" +export SPARK_WORKER_INSTANCES="{{spark_worker_instances}}" +export SPARK_MASTER_OPTS="{{spark_master_opts}}" diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 31209a662bbe1..0a8d6ca5d7519 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -103,6 +103,12 @@ def parse_args(): help="When destroying a cluster, delete the security groups that were created") parser.add_option("--use-existing-master", action="store_true", default=False, help="Launch fresh slaves, but use an existing stopped master if possible") + parser.add_option("--worker-instances", type="int", default=1, + help="Number of instances per worker: variable SPARK_WORKER_INSTANCES (default: 1)") + parser.add_option("--master-opts", type="string", default="", + help="Extra options to give to master through SPARK_MASTER_OPTS variable (e.g -Dspark.worker.timeout=180)") + + (opts, args) = parser.parse_args() if len(args) != 2: @@ -223,7 +229,7 @@ def launch_cluster(conn, opts, cluster_name): sys.exit(1) if opts.key_pair is None: print >> stderr, "ERROR: Must provide a key pair name (-k) to use on instances." - sys.exit(1) + sys.exit(1) print "Setting up security groups..." master_group = get_or_make_group(conn, cluster_name + "-master") slave_group = get_or_make_group(conn, cluster_name + "-slaves") @@ -551,7 +557,9 @@ def deploy_files(conn, root_dir, opts, master_nodes, slave_nodes, modules): "modules": '\n'.join(modules), "spark_version": spark_v, "shark_version": shark_v, - "hadoop_major_version": opts.hadoop_major_version + "hadoop_major_version": opts.hadoop_major_version, + "spark_worker_instances": "%d" % opts.worker_instances, + "spark_master_opts": opts.master_opts } # Create a temp directory in which we will place all the files to be @@ -604,7 +612,7 @@ def ssh_command(opts): return ['ssh'] + ssh_args(opts) -# Run a command on a host through ssh, retrying up to two times +# Run a command on a host through ssh, retrying up to five times # and then throwing an exception if ssh continues to fail. def ssh(host, opts, command): tries = 0 @@ -613,7 +621,7 @@ def ssh(host, opts, command): return subprocess.check_call( ssh_command(opts) + ['-t', '-t', '%s@%s' % (opts.user, host), stringify_command(command)]) except subprocess.CalledProcessError as e: - if (tries > 2): + if (tries > 5): # If this was an ssh failure, provide the user with hints. if e.returncode == 255: raise UsageError("Failed to SSH to remote host {0}.\nPlease check that you have provided the correct --identity-file and --key-pair parameters and try again.".format(host)) @@ -640,7 +648,7 @@ def ssh_write(host, opts, command, input): status = proc.wait() if status == 0: break - elif (tries > 2): + elif (tries > 5): raise RuntimeError("ssh_write failed with error %s" % proc.returncode) else: print >> stderr, "Error {0} while executing remote command, retrying after 30 seconds".format(status) @@ -806,6 +814,7 @@ def main(): real_main() except UsageError, e: print >> stderr, "\nError:\n", e + sys.exit(1) if __name__ == "__main__": diff --git a/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java b/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java index 2a4278d3c30e5..3f7a879538016 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java @@ -75,7 +75,6 @@ public String toString() { public static Tuple3 extractKey(String line) { Matcher m = apacheLogRegex.matcher(line); - List key = Collections.emptyList(); if (m.find()) { String ip = m.group(1); String user = m.group(3); diff --git a/examples/src/main/java/org/apache/spark/examples/JavaTC.java b/examples/src/main/java/org/apache/spark/examples/JavaTC.java index 1d776940f06c6..d66b9ba265fe8 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaTC.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaTC.java @@ -85,7 +85,7 @@ public Tuple2 call(Tuple2 e) { } }); - long oldCount = 0; + long oldCount; long nextCount = tc.count(); do { oldCount = nextCount; diff --git a/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaALS.java similarity index 98% rename from examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java rename to examples/src/main/java/org/apache/spark/examples/mllib/JavaALS.java index c516199d61c72..4533c4c5f241a 100644 --- a/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaALS.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.mllib.examples; +package org.apache.spark.examples.mllib; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; diff --git a/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaKMeans.java similarity index 98% rename from examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java rename to examples/src/main/java/org/apache/spark/examples/mllib/JavaKMeans.java index 7461609ab9e8f..0cfb8e69ed28f 100644 --- a/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaKMeans.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.mllib.examples; +package org.apache.spark.examples.mllib; import java.util.regex.Pattern; diff --git a/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaLR.java similarity index 98% rename from examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java rename to examples/src/main/java/org/apache/spark/examples/mllib/JavaLR.java index e3ab87cc722f3..f6e48b498727b 100644 --- a/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaLR.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.mllib.examples; +package org.apache.spark.examples.mllib; import java.util.regex.Pattern; diff --git a/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java b/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java index b5b438e9753a7..d62a72f53443c 100644 --- a/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java +++ b/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java @@ -23,7 +23,6 @@ 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.VoidFunction; import org.apache.spark.sql.api.java.JavaSQLContext; import org.apache.spark.sql.api.java.JavaSchemaRDD; diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaCustomReceiver.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaCustomReceiver.java similarity index 95% rename from examples/src/main/java/org/apache/spark/streaming/examples/JavaCustomReceiver.java rename to examples/src/main/java/org/apache/spark/examples/streaming/JavaCustomReceiver.java index a94fa621dc328..7f558f3ee713a 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaCustomReceiver.java +++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaCustomReceiver.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.streaming.examples; +package org.apache.spark.examples.streaming; import com.google.common.collect.Lists; @@ -26,6 +26,7 @@ import org.apache.spark.streaming.Duration; import org.apache.spark.streaming.api.java.JavaDStream; import org.apache.spark.streaming.api.java.JavaPairDStream; +import org.apache.spark.streaming.api.java.JavaReceiverInputDStream; import org.apache.spark.streaming.api.java.JavaStreamingContext; import org.apache.spark.streaming.receiver.Receiver; import scala.Tuple2; @@ -47,7 +48,7 @@ * To run this on your local machine, you need to first run a Netcat server * `$ nc -lk 9999` * and then run the example - * `$ ./run org.apache.spark.streaming.examples.JavaCustomReceiver local[2] localhost 9999` + * `$ ./run org.apache.spark.examples.streaming.JavaCustomReceiver local[2] localhost 9999` */ public class JavaCustomReceiver extends Receiver { @@ -69,7 +70,7 @@ public static void main(String[] args) { // Create a input stream with the custom receiver on target ip:port and count the // words in input stream of \n delimited text (eg. generated by 'nc') - JavaDStream lines = ssc.receiverStream( + JavaReceiverInputDStream lines = ssc.receiverStream( new JavaCustomReceiver(args[1], Integer.parseInt(args[2]))); JavaDStream words = lines.flatMap(new FlatMapFunction() { @Override @@ -148,5 +149,3 @@ private void receive() { } } } - - diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaFlumeEventCount.java similarity index 92% rename from examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java rename to examples/src/main/java/org/apache/spark/examples/streaming/JavaFlumeEventCount.java index f061001dd264d..a5ece68cef870 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java +++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaFlumeEventCount.java @@ -15,9 +15,10 @@ * limitations under the License. */ -package org.apache.spark.streaming.examples; +package org.apache.spark.examples.streaming; import org.apache.spark.api.java.function.Function; +import org.apache.spark.examples.streaming.StreamingExamples; import org.apache.spark.streaming.*; import org.apache.spark.streaming.api.java.*; import org.apache.spark.streaming.flume.FlumeUtils; @@ -58,7 +59,7 @@ public static void main(String[] args) { JavaStreamingContext ssc = new JavaStreamingContext(master, "FlumeEventCount", batchInterval, System.getenv("SPARK_HOME"), JavaStreamingContext.jarOfClass(JavaFlumeEventCount.class)); - JavaDStream flumeStream = FlumeUtils.createStream(ssc, "localhost", port); + JavaReceiverInputDStream flumeStream = FlumeUtils.createStream(ssc, "localhost", port); flumeStream.count(); diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaKafkaWordCount.java similarity index 91% rename from examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java rename to examples/src/main/java/org/apache/spark/examples/streaming/JavaKafkaWordCount.java index d704be08d6945..da51eb189a649 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java +++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaKafkaWordCount.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.streaming.examples; +package org.apache.spark.examples.streaming; import java.util.Map; import java.util.HashMap; @@ -26,9 +26,11 @@ import org.apache.spark.api.java.function.Function; import org.apache.spark.api.java.function.Function2; import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.examples.streaming.StreamingExamples; import org.apache.spark.streaming.Duration; import org.apache.spark.streaming.api.java.JavaDStream; import org.apache.spark.streaming.api.java.JavaPairDStream; +import org.apache.spark.streaming.api.java.JavaPairReceiverInputDStream; import org.apache.spark.streaming.api.java.JavaStreamingContext; import org.apache.spark.streaming.kafka.KafkaUtils; import scala.Tuple2; @@ -43,7 +45,7 @@ * is the number of threads the kafka consumer should use * * Example: - * `./bin/run-example org.apache.spark.streaming.examples.JavaKafkaWordCount local[2] zoo01,zoo02, + * `./bin/run-example org.apache.spark.examples.streaming.JavaKafkaWordCount local[2] zoo01,zoo02, * zoo03 my-consumer-group topic1,topic2 1` */ @@ -73,7 +75,8 @@ public static void main(String[] args) { topicMap.put(topic, numThreads); } - JavaPairDStream messages = KafkaUtils.createStream(jssc, args[1], args[2], topicMap); + JavaPairReceiverInputDStream messages = + KafkaUtils.createStream(jssc, args[1], args[2], topicMap); JavaDStream lines = messages.map(new Function, String>() { @Override diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaNetworkWordCount.java similarity index 88% rename from examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java rename to examples/src/main/java/org/apache/spark/examples/streaming/JavaNetworkWordCount.java index 0cc9d0ae1a08e..ac84991d87b8b 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java +++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaNetworkWordCount.java @@ -15,13 +15,15 @@ * limitations under the License. */ -package org.apache.spark.streaming.examples; +package org.apache.spark.examples.streaming; import com.google.common.collect.Lists; +import org.apache.spark.streaming.api.java.JavaReceiverInputDStream; import scala.Tuple2; import org.apache.spark.api.java.function.FlatMapFunction; import org.apache.spark.api.java.function.Function2; import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.examples.streaming.StreamingExamples; import org.apache.spark.streaming.Duration; import org.apache.spark.streaming.api.java.JavaDStream; import org.apache.spark.streaming.api.java.JavaPairDStream; @@ -38,7 +40,7 @@ * To run this on your local machine, you need to first run a Netcat server * `$ nc -lk 9999` * and then run the example - * `$ ./run org.apache.spark.streaming.examples.JavaNetworkWordCount local[2] localhost 9999` + * `$ ./run org.apache.spark.examples.streaming.JavaNetworkWordCount local[2] localhost 9999` */ public final class JavaNetworkWordCount { private static final Pattern SPACE = Pattern.compile(" "); @@ -57,9 +59,9 @@ public static void main(String[] args) { new Duration(1000), System.getenv("SPARK_HOME"), JavaStreamingContext.jarOfClass(JavaNetworkWordCount.class)); - // Create a NetworkInputDStream on target ip:port and count the + // Create a JavaReceiverInputDStream on target ip:port and count the // words in input stream of \n delimited text (eg. generated by 'nc') - JavaDStream lines = ssc.socketTextStream(args[1], Integer.parseInt(args[2])); + JavaReceiverInputDStream lines = ssc.socketTextStream(args[1], Integer.parseInt(args[2])); JavaDStream words = lines.flatMap(new FlatMapFunction() { @Override public Iterable call(String x) { diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaQueueStream.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaQueueStream.java similarity index 96% rename from examples/src/main/java/org/apache/spark/streaming/examples/JavaQueueStream.java rename to examples/src/main/java/org/apache/spark/examples/streaming/JavaQueueStream.java index 88ad341641e0a..819311968fac5 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaQueueStream.java +++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaQueueStream.java @@ -15,13 +15,14 @@ * limitations under the License. */ -package org.apache.spark.streaming.examples; +package org.apache.spark.examples.streaming; import com.google.common.collect.Lists; import scala.Tuple2; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.function.Function2; import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.examples.streaming.StreamingExamples; import org.apache.spark.streaming.Duration; import org.apache.spark.streaming.api.java.JavaDStream; import org.apache.spark.streaming.api.java.JavaPairDStream; diff --git a/python/examples/als.py b/examples/src/main/python/als.py similarity index 100% rename from python/examples/als.py rename to examples/src/main/python/als.py diff --git a/python/examples/kmeans.py b/examples/src/main/python/kmeans.py similarity index 98% rename from python/examples/kmeans.py rename to examples/src/main/python/kmeans.py index d8387b0b183e6..e3596488faf9e 100755 --- a/python/examples/kmeans.py +++ b/examples/src/main/python/kmeans.py @@ -18,7 +18,7 @@ """ The K-means algorithm written from scratch against PySpark. In practice, one may prefer to use the KMeans algorithm in MLlib, as shown in -python/examples/mllib/kmeans.py. +examples/src/main/python/mllib/kmeans.py. This example requires NumPy (http://www.numpy.org/). """ diff --git a/python/examples/logistic_regression.py b/examples/src/main/python/logistic_regression.py similarity index 97% rename from python/examples/logistic_regression.py rename to examples/src/main/python/logistic_regression.py index 28d52e6a40b45..fe5373cf799b1 100755 --- a/python/examples/logistic_regression.py +++ b/examples/src/main/python/logistic_regression.py @@ -20,7 +20,7 @@ to act on batches of input data using efficient matrix operations. In practice, one may prefer to use the LogisticRegression algorithm in -MLlib, as shown in python/examples/mllib/logistic_regression.py. +MLlib, as shown in examples/src/main/python/mllib/logistic_regression.py. """ from collections import namedtuple diff --git a/python/examples/mllib/kmeans.py b/examples/src/main/python/mllib/kmeans.py similarity index 100% rename from python/examples/mllib/kmeans.py rename to examples/src/main/python/mllib/kmeans.py diff --git a/python/examples/mllib/logistic_regression.py b/examples/src/main/python/mllib/logistic_regression.py similarity index 100% rename from python/examples/mllib/logistic_regression.py rename to examples/src/main/python/mllib/logistic_regression.py diff --git a/python/examples/pagerank.py b/examples/src/main/python/pagerank.py similarity index 100% rename from python/examples/pagerank.py rename to examples/src/main/python/pagerank.py diff --git a/python/examples/pi.py b/examples/src/main/python/pi.py similarity index 100% rename from python/examples/pi.py rename to examples/src/main/python/pi.py diff --git a/python/examples/sort.py b/examples/src/main/python/sort.py similarity index 100% rename from python/examples/sort.py rename to examples/src/main/python/sort.py diff --git a/python/examples/transitive_closure.py b/examples/src/main/python/transitive_closure.py similarity index 100% rename from python/examples/transitive_closure.py rename to examples/src/main/python/transitive_closure.py diff --git a/python/examples/wordcount.py b/examples/src/main/python/wordcount.py similarity index 100% rename from python/examples/wordcount.py rename to examples/src/main/python/wordcount.py diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassification.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassification.scala index ec9de022c1d47..4001908c98015 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassification.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassification.scala @@ -22,7 +22,7 @@ import scopt.OptionParser import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.mllib.classification.{LogisticRegressionWithSGD, SVMWithSGD} -import org.apache.spark.mllib.evaluation.binary.BinaryClassificationMetrics +import org.apache.spark.mllib.evaluation.BinaryClassificationMetrics import org.apache.spark.mllib.util.MLUtils import org.apache.spark.mllib.optimization.{SquaredL2Updater, L1Updater} @@ -96,7 +96,7 @@ object BinaryClassification { Logger.getRootLogger.setLevel(Level.WARN) - val examples = MLUtils.loadLibSVMData(sc, params.input).cache() + val examples = MLUtils.loadLibSVMFile(sc, params.input).cache() val splits = examples.randomSplit(Array(0.8, 0.2)) val training = splits(0).cache() diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala index 1723ca6931021..658d370f8656e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala @@ -22,7 +22,7 @@ import scopt.OptionParser import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.mllib.regression.LinearRegressionWithSGD -import org.apache.spark.mllib.util.{MulticlassLabelParser, MLUtils} +import org.apache.spark.mllib.util.MLUtils import org.apache.spark.mllib.optimization.{SimpleUpdater, SquaredL2Updater, L1Updater} /** @@ -82,7 +82,7 @@ object LinearRegression extends App { Logger.getRootLogger.setLevel(Level.WARN) - val examples = MLUtils.loadLibSVMData(sc, params.input, MulticlassLabelParser).cache() + val examples = MLUtils.loadLibSVMFile(sc, params.input, multiclass = true).cache() val splits = examples.randomSplit(Array(0.8, 0.2)) val training = splits(0).cache() diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/SparseNaiveBayes.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/SparseNaiveBayes.scala index 25b6768b8d72b..537e68a0991aa 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/SparseNaiveBayes.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/SparseNaiveBayes.scala @@ -75,8 +75,8 @@ object SparseNaiveBayes { val minPartitions = if (params.minPartitions > 0) params.minPartitions else sc.defaultMinPartitions - val examples = MLUtils.loadLibSVMData(sc, params.input, MulticlassLabelParser, - params.numFeatures, minPartitions) + val examples = + MLUtils.loadLibSVMFile(sc, params.input, multiclass = true, params.numFeatures, minPartitions) // Cache examples because it will be used in both training and evaluation. examples.cache() diff --git a/examples/src/main/scala/org/apache/spark/sql/examples/RDDRelation.scala b/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala similarity index 98% rename from examples/src/main/scala/org/apache/spark/sql/examples/RDDRelation.scala rename to examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala index 8210ad977f066..ff9254b044c24 100644 --- a/examples/src/main/scala/org/apache/spark/sql/examples/RDDRelation.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.examples +package org.apache.spark.examples.sql import org.apache.spark.SparkContext import org.apache.spark.sql.SQLContext diff --git a/examples/src/main/scala/org/apache/spark/sql/examples/HiveFromSpark.scala b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala similarity index 98% rename from examples/src/main/scala/org/apache/spark/sql/examples/HiveFromSpark.scala rename to examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala index 62329bde84481..66ce93a26ef42 100644 --- a/examples/src/main/scala/org/apache/spark/sql/examples/HiveFromSpark.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.hive.examples +package org.apache.spark.examples.sql.hive import org.apache.spark.SparkContext import org.apache.spark.sql._ diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/ActorWordCount.scala similarity index 95% rename from examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala rename to examples/src/main/scala/org/apache/spark/examples/streaming/ActorWordCount.scala index c845dd8904c90..84cf43df0f96c 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/ActorWordCount.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.streaming.examples +package org.apache.spark.examples.streaming import scala.collection.mutable.LinkedList import scala.reflect.ClassTag @@ -78,7 +78,7 @@ class FeederActor extends Actor { * goes and subscribe to a typical publisher/feeder actor and receives * data. * - * @see [[org.apache.spark.streaming.examples.FeederActor]] + * @see [[org.apache.spark.examples.streaming.FeederActor]] */ class SampleActorReceiver[T: ClassTag](urlOfPublisher: String) extends Actor with ActorHelper { @@ -131,9 +131,9 @@ object FeederActor { * and describe the AkkaSystem that Spark Sample feeder is running on. * * To run this example locally, you may run Feeder Actor as - * `$ ./bin/run-example org.apache.spark.streaming.examples.FeederActor 127.0.1.1 9999` + * `$ ./bin/run-example org.apache.spark.examples.streaming.FeederActor 127.0.1.1 9999` * and then run the example - * `./bin/run-example org.apache.spark.streaming.examples.ActorWordCount local[2] 127.0.1.1 9999` + * `./bin/run-example org.apache.spark.examples.streaming.ActorWordCount local[2] 127.0.1.1 9999` */ object ActorWordCount { def main(args: Array[String]) { diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/CustomReceiver.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala similarity index 97% rename from examples/src/main/scala/org/apache/spark/streaming/examples/CustomReceiver.scala rename to examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala index eebffd824983f..e317e2d36ae43 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/CustomReceiver.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.streaming.examples +package org.apache.spark.examples.streaming import java.io.{InputStreamReader, BufferedReader, InputStream} import java.net.Socket @@ -37,7 +37,7 @@ import org.apache.spark.streaming.receiver.Receiver * To run this on your local machine, you need to first run a Netcat server * `$ nc -lk 9999` * and then run the example - * `$ ./run org.apache.spark.streaming.examples.CustomReceiver local[2] localhost 9999` + * `$ ./run org.apache.spark.examples.streaming.CustomReceiver local[2] localhost 9999` */ object CustomReceiver { def main(args: Array[String]) { diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/FlumeEventCount.scala similarity index 98% rename from examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala rename to examples/src/main/scala/org/apache/spark/examples/streaming/FlumeEventCount.scala index 26b6024534124..5b2a1035fc779 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/FlumeEventCount.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.streaming.examples +package org.apache.spark.examples.streaming import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming._ diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/HdfsWordCount.scala similarity index 95% rename from examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala rename to examples/src/main/scala/org/apache/spark/examples/streaming/HdfsWordCount.scala index 7f86fc792aacf..b440956ba3137 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/HdfsWordCount.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.streaming.examples +package org.apache.spark.examples.streaming import org.apache.spark.streaming.{Seconds, StreamingContext} import org.apache.spark.streaming.StreamingContext._ @@ -27,7 +27,7 @@ import org.apache.spark.streaming.StreamingContext._ * is the directory that Spark Streaming will use to find and read new text files. * * To run this on your local machine on directory `localdir`, run this example - * `$ ./bin/run-example org.apache.spark.streaming.examples.HdfsWordCount local[2] localdir` + * `$ ./bin/run-example org.apache.spark.examples.streaming.HdfsWordCount local[2] localdir` * Then create a text file in `localdir` and the words in the file will get counted. */ object HdfsWordCount { diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala similarity index 93% rename from examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala rename to examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala index 2aa4f1474a59e..c3aae5af05b1c 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.streaming.examples +package org.apache.spark.examples.streaming import java.util.Properties @@ -24,7 +24,6 @@ import kafka.producer._ import org.apache.spark.streaming._ import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.streaming.kafka._ -import org.apache.spark.streaming.util.RawTextHelper._ // scalastyle:off /** @@ -37,7 +36,7 @@ import org.apache.spark.streaming.util.RawTextHelper._ * is the number of threads the kafka consumer should use * * Example: - * `./bin/run-example org.apache.spark.streaming.examples.KafkaWordCount local[2] zoo01,zoo02,zoo03 my-consumer-group topic1,topic2 1` + * `./bin/run-example org.apache.spark.examples.streaming.KafkaWordCount local[2] zoo01,zoo02,zoo03 my-consumer-group topic1,topic2 1` */ // scalastyle:on object KafkaWordCount { @@ -59,7 +58,7 @@ object KafkaWordCount { val lines = KafkaUtils.createStream(ssc, zkQuorum, group, topicpMap).map(_._2) val words = lines.flatMap(_.split(" ")) val wordCounts = words.map(x => (x, 1L)) - .reduceByKeyAndWindow(add _, subtract _, Minutes(10), Seconds(2), 2) + .reduceByKeyAndWindow(_ + _, _ - _, Minutes(10), Seconds(2), 2) wordCounts.print() ssc.start() diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/MQTTWordCount.scala similarity index 95% rename from examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala rename to examples/src/main/scala/org/apache/spark/examples/streaming/MQTTWordCount.scala index 62aef0fb47107..47bf1e5a06439 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/MQTTWordCount.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.streaming.examples +package org.apache.spark.examples.streaming import org.eclipse.paho.client.mqttv3.{MqttClient, MqttClientPersistence, MqttException, MqttMessage, MqttTopic} import org.eclipse.paho.client.mqttv3.persist.MqttDefaultFilePersistence @@ -79,9 +79,9 @@ object MQTTPublisher { * and describe where Mqtt publisher is running. * * To run this example locally, you may run publisher as - * `$ ./bin/run-example org.apache.spark.streaming.examples.MQTTPublisher tcp://localhost:1883 foo` + * `$ ./bin/run-example org.apache.spark.examples.streaming.MQTTPublisher tcp://localhost:1883 foo` * and run the example as - * `$ ./bin/run-example org.apache.spark.streaming.examples.MQTTWordCount local[2] tcp://localhost:1883 foo` + * `$ ./bin/run-example org.apache.spark.examples.streaming.MQTTWordCount local[2] tcp://localhost:1883 foo` */ // scalastyle:on object MQTTWordCount { diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/NetworkWordCount.scala similarity index 95% rename from examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala rename to examples/src/main/scala/org/apache/spark/examples/streaming/NetworkWordCount.scala index 272ab11212451..acfe9a4da3596 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/NetworkWordCount.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.streaming.examples +package org.apache.spark.examples.streaming import org.apache.spark.streaming.{Seconds, StreamingContext} import org.apache.spark.streaming.StreamingContext._ @@ -32,7 +32,7 @@ import org.apache.spark.storage.StorageLevel * To run this on your local machine, you need to first run a Netcat server * `$ nc -lk 9999` * and then run the example - * `$ ./bin/run-example org.apache.spark.streaming.examples.NetworkWordCount local[2] localhost 9999` + * `$ ./bin/run-example org.apache.spark.examples.streaming.NetworkWordCount local[2] localhost 9999` */ // scalastyle:on object NetworkWordCount { diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/QueueStream.scala similarity index 97% rename from examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala rename to examples/src/main/scala/org/apache/spark/examples/streaming/QueueStream.scala index ff2a205ec1c15..f92f72f2de876 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/QueueStream.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.streaming.examples +package org.apache.spark.examples.streaming import scala.collection.mutable.SynchronizedQueue diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/RawNetworkGrep.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/RawNetworkGrep.scala similarity index 92% rename from examples/src/main/scala/org/apache/spark/streaming/examples/RawNetworkGrep.scala rename to examples/src/main/scala/org/apache/spark/examples/streaming/RawNetworkGrep.scala index d915c0c39b334..1b0319a046433 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/RawNetworkGrep.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/RawNetworkGrep.scala @@ -15,11 +15,10 @@ * limitations under the License. */ -package org.apache.spark.streaming.examples +package org.apache.spark.examples.streaming import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming._ -import org.apache.spark.streaming.util.RawTextHelper import org.apache.spark.util.IntParam /** @@ -52,9 +51,6 @@ object RawNetworkGrep { val ssc = new StreamingContext(master, "RawNetworkGrep", Milliseconds(batchMillis), System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) - // Warm up the JVMs on master and slave for JIT compilation to kick in - RawTextHelper.warmUp(ssc.sparkContext) - val rawStreams = (1 to numStreams).map(_ => ssc.rawSocketStream[String](host, port, StorageLevel.MEMORY_ONLY_SER_2)).toArray val union = ssc.union(rawStreams) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/RecoverableNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala similarity index 96% rename from examples/src/main/scala/org/apache/spark/streaming/examples/RecoverableNetworkWordCount.scala rename to examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala index 4aacbb1991418..b0bc31cc66ab5 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/RecoverableNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.streaming.examples +package org.apache.spark.examples.streaming import org.apache.spark.streaming.{Time, Seconds, StreamingContext} import org.apache.spark.streaming.StreamingContext._ @@ -44,7 +44,7 @@ import java.nio.charset.Charset * * and run the example as * - * `$ ./run-example org.apache.spark.streaming.examples.RecoverableNetworkWordCount \ + * `$ ./run-example org.apache.spark.examples.streaming.RecoverableNetworkWordCount \ * local[2] localhost 9999 ~/checkpoint/ ~/out` * * If the directory ~/checkpoint/ does not exist (e.g. running for the first time), it will create @@ -56,7 +56,7 @@ import java.nio.charset.Charset * * `$ ./spark-class org.apache.spark.deploy.Client -s launch \ * \ - * org.apache.spark.streaming.examples.RecoverableNetworkWordCount \ + * org.apache.spark.examples.streaming.RecoverableNetworkWordCount \ * localhost 9999 ~/checkpoint ~/out` * * would typically be diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala similarity index 96% rename from examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala rename to examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala index ef94c9298dd93..8001d56c98d86 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.streaming.examples +package org.apache.spark.examples.streaming import org.apache.spark.streaming._ import org.apache.spark.streaming.StreamingContext._ @@ -31,7 +31,7 @@ import org.apache.spark.streaming.StreamingContext._ * To run this on your local machine, you need to first run a Netcat server * `$ nc -lk 9999` * and then run the example - * `$ ./bin/run-example org.apache.spark.streaming.examples.StatefulNetworkWordCount local[2] localhost 9999` + * `$ ./bin/run-example org.apache.spark.examples.streaming.StatefulNetworkWordCount local[2] localhost 9999` */ // scalastyle:on object StatefulNetworkWordCount { diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/StreamingExamples.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/StreamingExamples.scala similarity index 97% rename from examples/src/main/scala/org/apache/spark/streaming/examples/StreamingExamples.scala rename to examples/src/main/scala/org/apache/spark/examples/streaming/StreamingExamples.scala index 99f1502046f53..8396e65d0d588 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/StreamingExamples.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/StreamingExamples.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.streaming.examples +package org.apache.spark.examples.streaming import org.apache.spark.Logging diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala similarity index 99% rename from examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala rename to examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala index c38905e8f3663..b12617d881787 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.streaming.examples +package org.apache.spark.examples.streaming import com.twitter.algebird._ diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdHLL.scala similarity index 98% rename from examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala rename to examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdHLL.scala index c067046f9020e..22f232c72545c 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdHLL.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.streaming.examples +package org.apache.spark.examples.streaming import com.twitter.algebird.HyperLogLogMonoid import com.twitter.algebird.HyperLogLog._ diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterPopularTags.scala similarity index 98% rename from examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala rename to examples/src/main/scala/org/apache/spark/examples/streaming/TwitterPopularTags.scala index 2597c8178862a..5b58e94600a16 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterPopularTags.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.streaming.examples +package org.apache.spark.examples.streaming import org.apache.spark.streaming.{Seconds, StreamingContext} import StreamingContext._ diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/ZeroMQWordCount.scala similarity index 95% rename from examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala rename to examples/src/main/scala/org/apache/spark/examples/streaming/ZeroMQWordCount.scala index 109ff855b5228..de46e5f5b10b6 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/ZeroMQWordCount.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.streaming.examples +package org.apache.spark.examples.streaming import akka.actor.ActorSystem import akka.actor.actorRef2Scala @@ -68,9 +68,9 @@ object SimpleZeroMQPublisher { * and describe where zeroMq publisher is running. * * To run this example locally, you may run publisher as - * `$ ./bin/run-example org.apache.spark.streaming.examples.SimpleZeroMQPublisher tcp://127.0.1.1:1234 foo.bar` + * `$ ./bin/run-example org.apache.spark.examples.streaming.SimpleZeroMQPublisher tcp://127.0.1.1:1234 foo.bar` * and run the example as - * `$ ./bin/run-example org.apache.spark.streaming.examples.ZeroMQWordCount local[2] tcp://127.0.1.1:1234 foo` + * `$ ./bin/run-example org.apache.spark.examples.streaming.ZeroMQWordCount local[2] tcp://127.0.1.1:1234 foo` */ // scalastyle:on object ZeroMQWordCount { diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewGenerator.scala similarity index 95% rename from examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala rename to examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewGenerator.scala index 251f65fe4df9c..97e0cb92078dc 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewGenerator.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.streaming.examples.clickstream +package org.apache.spark.examples.streaming.clickstream import java.net.ServerSocket import java.io.PrintWriter @@ -40,8 +40,8 @@ object PageView extends Serializable { /** Generates streaming events to simulate page views on a website. * * This should be used in tandem with PageViewStream.scala. Example: - * $ ./bin/run-example org.apache.spark.streaming.examples.clickstream.PageViewGenerator 44444 10 - * $ ./bin/run-example org.apache.spark.streaming.examples.clickstream.PageViewStream errorRatePerZipCode localhost 44444 + * $ ./bin/run-example org.apache.spark.examples.streaming.clickstream.PageViewGenerator 44444 10 + * $ ./bin/run-example org.apache.spark.examples.streaming.clickstream.PageViewStream errorRatePerZipCode localhost 44444 * * When running this, you may want to set the root logging level to ERROR in * conf/log4j.properties to reduce the verbosity of the output. diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewStream.scala similarity index 94% rename from examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala rename to examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewStream.scala index 673013f7cf948..d30ceffbe29cb 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewStream.scala @@ -15,19 +15,19 @@ * limitations under the License. */ -package org.apache.spark.streaming.examples.clickstream +package org.apache.spark.examples.streaming.clickstream import org.apache.spark.SparkContext._ import org.apache.spark.streaming.{Seconds, StreamingContext} import org.apache.spark.streaming.StreamingContext._ -import org.apache.spark.streaming.examples.StreamingExamples +import org.apache.spark.examples.streaming.StreamingExamples // scalastyle:off /** Analyses a streaming dataset of web page views. This class demonstrates several types of * operators available in Spark streaming. * * This should be used in tandem with PageViewStream.scala. Example: - * $ ./bin/run-example org.apache.spark.streaming.examples.clickstream.PageViewGenerator 44444 10 - * $ ./bin/run-example org.apache.spark.streaming.examples.clickstream.PageViewStream errorRatePerZipCode localhost 44444 + * $ ./bin/run-example org.apache.spark.examples.streaming.clickstream.PageViewGenerator 44444 10 + * $ ./bin/run-example org.apache.spark.examples.streaming.clickstream.PageViewStream errorRatePerZipCode localhost 44444 */ // scalastyle:on object PageViewStream { diff --git a/extras/java8-tests/src/test/java/org/apache/spark/Java8APISuite.java b/extras/java8-tests/src/test/java/org/apache/spark/Java8APISuite.java index 7eb8b45fc3cf0..feabca6733484 100644 --- a/extras/java8-tests/src/test/java/org/apache/spark/Java8APISuite.java +++ b/extras/java8-tests/src/test/java/org/apache/spark/Java8APISuite.java @@ -23,7 +23,7 @@ import scala.Tuple2; -import com.google.common.collections.Iterables; +import com.google.common.collect.Iterables; import com.google.common.base.Optional; import com.google.common.io.Files; import org.apache.hadoop.io.IntWritable; diff --git a/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala b/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala index 6d04bf790e3a5..fa78ca99b8891 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala @@ -51,18 +51,12 @@ class EdgeRDD[@specialized ED: ClassTag]( override def collect(): Array[Edge[ED]] = this.map(_.copy()).collect() - override def persist(newLevel: StorageLevel): EdgeRDD[ED] = { + override def persist(newLevel: StorageLevel): this.type = { partitionsRDD.persist(newLevel) this } - /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ - override def persist(): EdgeRDD[ED] = persist(StorageLevel.MEMORY_ONLY) - - /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ - override def cache(): EdgeRDD[ED] = persist() - - override def unpersist(blocking: Boolean = true): EdgeRDD[ED] = { + override def unpersist(blocking: Boolean = true): this.type = { partitionsRDD.unpersist(blocking) this } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala index d6788d4d4b9fd..f0fc605c88575 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala @@ -71,18 +71,12 @@ class VertexRDD[@specialized VD: ClassTag]( override protected def getPreferredLocations(s: Partition): Seq[String] = partitionsRDD.preferredLocations(s) - override def persist(newLevel: StorageLevel): VertexRDD[VD] = { + override def persist(newLevel: StorageLevel): this.type = { partitionsRDD.persist(newLevel) this } - /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ - override def persist(): VertexRDD[VD] = persist(StorageLevel.MEMORY_ONLY) - - /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ - override def cache(): VertexRDD[VD] = persist() - - override def unpersist(blocking: Boolean = true): VertexRDD[VD] = { + override def unpersist(blocking: Boolean = true): this.type = { partitionsRDD.unpersist(blocking) this } diff --git a/make-distribution.sh b/make-distribution.sh index c05dcd89d90a7..759e555b4b69a 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -39,14 +39,29 @@ # 5) ./bin/spark-shell --master spark://my-master-ip:7077 # +set -o pipefail # Figure out where the Spark framework is installed FWDIR="$(cd `dirname $0`; pwd)" DISTDIR="$FWDIR/dist" -VERSION=$(mvn help:evaluate -Dexpression=project.version | grep -v "INFO" | tail -n 1) -if [ $? == -1 ] ;then +if [ -z "$JAVA_HOME" ]; then + echo "Error: JAVA_HOME is not set, cannot proceed." + exit -1 +fi + +JAVA_CMD="$JAVA_HOME"/bin/java +JAVA_VERSION=$("$JAVA_CMD" -version 2>&1) +if ! [[ "$JAVA_VERSION" =~ "1.6" ]]; then + echo "***NOTE***: JAVA_HOME is not set to a JDK 6 installation. The resulting" + echo " distribution will not support Java 6. See SPARK-1703." + echo "Output from 'java -version' was:" + echo "$JAVA_VERSION" +fi + +VERSION=$(mvn help:evaluate -Dexpression=project.version 2>/dev/null | grep -v "INFO" | tail -n 1) +if [ $? != 0 ]; then echo -e "You need Maven installed to build Spark." - echo -e "Download Maven from https://maven.apache.org." + echo -e "Download Maven from https://maven.apache.org/" exit -1; fi @@ -116,27 +131,34 @@ cd $FWDIR export MAVEN_OPTS="-Xmx2g -XX:MaxPermSize=512M -XX:ReservedCodeCacheSize=512m" -if [ "$SPARK_HIVE" == "true" ]; then - MAYBE_HIVE="-Phive" -else - MAYBE_HIVE="" -fi - -if [ "$SPARK_YARN" == "true" ]; then - if [[ "$SPARK_HADOOP_VERSION" =~ "0.23." ]]; then - mvn clean package -DskipTests -Pyarn-alpha -Dhadoop.version=$SPARK_HADOOP_VERSION \ - -Dyarn.version=$SPARK_HADOOP_VERSION $MAYBE_HIVE -Phadoop-0.23 - else - mvn clean package -DskipTests -Pyarn -Dhadoop.version=$SPARK_HADOOP_VERSION \ - -Dyarn.version=$SPARK_HADOOP_VERSION $MAYBE_HIVE - fi -else - if [[ "$SPARK_HADOOP_VERSION" =~ "0.23." ]]; then - mvn clean package -Phadoop-0.23 -DskipTests -Dhadoop.version=$SPARK_HADOOP_VERSION $MAYBE_HIVE - else - mvn clean package -DskipTests -Dhadoop.version=$SPARK_HADOOP_VERSION $MAYBE_HIVE +BUILD_COMMAND="mvn clean package" + +# Use special profiles for hadoop versions 0.23.x, 2.2.x, 2.3.x, 2.4.x +if [[ "$SPARK_HADOOP_VERSION" =~ ^0\.23\. ]]; then BUILD_COMMAND="$BUILD_COMMAND -Phadoop-0.23"; fi +if [[ "$SPARK_HADOOP_VERSION" =~ ^2\.2\. ]]; then BUILD_COMMAND="$BUILD_COMMAND -Phadoop-2.2"; fi +if [[ "$SPARK_HADOOP_VERSION" =~ ^2\.3\. ]]; then BUILD_COMMAND="$BUILD_COMMAND -Phadoop-2.3"; fi +if [[ "$SPARK_HADOOP_VERSION" =~ ^2\.4\. ]]; then BUILD_COMMAND="$BUILD_COMMAND -Phadoop-2.4"; fi +if [[ "$SPARK_HIVE" == "true" ]]; then BUILD_COMMAND="$BUILD_COMMAND -Phive"; fi +if [[ "$SPARK_YARN" == "true" ]]; then + # For hadoop versions 0.23.x to 2.1.x, use the yarn-alpha profile + if [[ "$SPARK_HADOOP_VERSION" =~ ^0\.2[3-9]\. ]] || + [[ "$SPARK_HADOOP_VERSION" =~ ^0\.[3-9][0-9]\. ]] || + [[ "$SPARK_HADOOP_VERSION" =~ ^1\.[0-9]\. ]] || + [[ "$SPARK_HADOOP_VERSION" =~ ^2\.[0-1]\. ]]; then + BUILD_COMMAND="$BUILD_COMMAND -Pyarn-alpha" + # For hadoop versions 2.2+, use the yarn profile + elif [[ "$SPARK_HADOOP_VERSION" =~ ^2.[2-9]. ]]; then + BUILD_COMMAND="$BUILD_COMMAND -Pyarn" fi + BUILD_COMMAND="$BUILD_COMMAND -Dyarn.version=$SPARK_HADOOP_VERSION" fi +BUILD_COMMAND="$BUILD_COMMAND -Dhadoop.version=$SPARK_HADOOP_VERSION" +BUILD_COMMAND="$BUILD_COMMAND -DskipTests" + +# Actually build the jar +echo -e "\nBuilding with..." +echo -e "\$ $BUILD_COMMAND\n" +${BUILD_COMMAND} # Make directories rm -rf "$DISTDIR" @@ -147,6 +169,10 @@ echo "Spark $VERSION built for Hadoop $SPARK_HADOOP_VERSION" > "$DISTDIR/RELEASE cp $FWDIR/assembly/target/scala*/*assembly*hadoop*.jar "$DISTDIR/lib/" cp $FWDIR/examples/target/scala*/spark-examples*.jar "$DISTDIR/lib/" +if [ "$SPARK_HIVE" == "true" ]; then + cp $FWDIR/lib_managed/jars/datanucleus*.jar "$DISTDIR/lib/" +fi + # Copy other things mkdir "$DISTDIR"/conf cp "$FWDIR"/conf/*.template "$DISTDIR"/conf @@ -155,7 +181,6 @@ cp -r "$FWDIR/bin" "$DISTDIR" cp -r "$FWDIR/python" "$DISTDIR" cp -r "$FWDIR/sbin" "$DISTDIR" - # Download and copy in tachyon, if requested if [ "$SPARK_TACHYON" == "true" ]; then TACHYON_VERSION="0.4.1" @@ -164,7 +189,7 @@ if [ "$SPARK_TACHYON" == "true" ]; then TMPD=`mktemp -d 2>/dev/null || mktemp -d -t 'disttmp'` pushd $TMPD > /dev/null - echo "Fetchting tachyon tgz" + echo "Fetching tachyon tgz" wget "$TACHYON_URL" tar xf "tachyon-${TACHYON_VERSION}-bin.tar.gz" diff --git a/mllib/data/sample_naive_bayes_data.txt b/mllib/data/sample_naive_bayes_data.txt index f874adbaf4665..981da382d6ac8 100644 --- a/mllib/data/sample_naive_bayes_data.txt +++ b/mllib/data/sample_naive_bayes_data.txt @@ -1,6 +1,6 @@ -0, 1 0 0 -0, 2 0 0 -1, 0 1 0 -1, 0 2 0 -2, 0 0 1 -2, 0 0 2 +0,1 0 0 +0,2 0 0 +1,0 1 0 +1,0 2 0 +2,0 0 1 +2,0 0 2 diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala index 780e8bae42b84..90aa8ac998ba9 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala @@ -17,6 +17,7 @@ package org.apache.spark.mllib.classification +import org.apache.spark.annotation.Experimental import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.regression._ @@ -29,7 +30,7 @@ import org.apache.spark.rdd.RDD * @param weights Weights computed for every feature. * @param intercept Intercept computed for this model. */ -class LogisticRegressionModel( +class LogisticRegressionModel private[mllib] ( override val weights: Vector, override val intercept: Double) extends GeneralizedLinearModel(weights, intercept) with ClassificationModel with Serializable { @@ -37,18 +38,22 @@ class LogisticRegressionModel( private var threshold: Option[Double] = Some(0.5) /** + * :: Experimental :: * Sets the threshold that separates positive predictions from negative predictions. An example * with prediction score greater than or equal to this threshold is identified as an positive, * and negative otherwise. The default value is 0.5. */ + @Experimental def setThreshold(threshold: Double): this.type = { this.threshold = Some(threshold) this } /** + * :: Experimental :: * Clears the threshold so that `predict` will output raw prediction scores. */ + @Experimental def clearThreshold(): this.type = { threshold = None this diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala index f6f62ce2de04e..b6e0c4a80e27b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala @@ -19,7 +19,6 @@ package org.apache.spark.mllib.classification import breeze.linalg.{DenseMatrix => BDM, DenseVector => BDV, argmax => brzArgmax, sum => brzSum} -import org.apache.spark.annotation.Experimental import org.apache.spark.Logging import org.apache.spark.SparkContext._ import org.apache.spark.mllib.linalg.Vector @@ -27,7 +26,6 @@ import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.rdd.RDD /** - * :: Experimental :: * Model for Naive Bayes Classifiers. * * @param labels list of labels @@ -35,8 +33,7 @@ import org.apache.spark.rdd.RDD * @param theta log of class conditional probabilities, whose dimension is C-by-D, * where D is number of features */ -@Experimental -class NaiveBayesModel( +class NaiveBayesModel private[mllib] ( val labels: Array[Double], val pi: Array[Double], val theta: Array[Array[Double]]) extends ClassificationModel with Serializable { @@ -124,6 +121,9 @@ class NaiveBayes private (private var lambda: Double) extends Serializable with } } +/** + * Top-level methods for calling naive Bayes. + */ object NaiveBayes { /** * Trains a Naive Bayes model given an RDD of `(label, features)` pairs. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala index 81b126717e9a4..e05213536e64a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala @@ -17,6 +17,7 @@ package org.apache.spark.mllib.classification +import org.apache.spark.annotation.Experimental import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.regression._ @@ -29,7 +30,7 @@ import org.apache.spark.rdd.RDD * @param weights Weights computed for every feature. * @param intercept Intercept computed for this model. */ -class SVMModel( +class SVMModel private[mllib] ( override val weights: Vector, override val intercept: Double) extends GeneralizedLinearModel(weights, intercept) with ClassificationModel with Serializable { @@ -37,18 +38,22 @@ class SVMModel( private var threshold: Option[Double] = Some(0.0) /** + * :: Experimental :: * Sets the threshold that separates positive predictions from negative predictions. An example * with prediction score greater than or equal to this threshold is identified as an positive, * and negative otherwise. The default value is 0.0. */ + @Experimental def setThreshold(threshold: Double): this.type = { this.threshold = Some(threshold) this } /** + * :: Experimental :: * Clears the threshold so that `predict` will output raw prediction scores. */ + @Experimental def clearThreshold(): this.type = { threshold = None this diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala index a64c5d44be406..de22fbb6ffc10 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala @@ -21,6 +21,7 @@ import scala.collection.mutable.ArrayBuffer import breeze.linalg.{DenseVector => BDV, Vector => BV, norm => breezeNorm} +import org.apache.spark.annotation.Experimental import org.apache.spark.Logging import org.apache.spark.SparkContext._ import org.apache.spark.mllib.linalg.{Vector, Vectors} @@ -81,6 +82,7 @@ class KMeans private ( * this many times with random starting conditions (configured by the initialization mode), then * return the best clustering found over any run. Default: 1. */ + @Experimental def setRuns(runs: Int): KMeans = { if (runs <= 0) { throw new IllegalArgumentException("Number of runs must be positive") 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 18abbf2758b86..ce14b06241932 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 @@ -24,7 +24,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(val clusterCenters: Array[Vector]) extends Serializable { +class KMeansModel private[mllib] (val clusterCenters: Array[Vector]) extends Serializable { /** Total number of clusters. */ def k: Int = clusterCenters.length diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryClassificationMetrics.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetrics.scala similarity index 64% rename from mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryClassificationMetrics.scala rename to mllib/src/main/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetrics.scala index ed7b0fc943367..079743742d86d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryClassificationMetrics.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetrics.scala @@ -15,83 +15,22 @@ * limitations under the License. */ -package org.apache.spark.mllib.evaluation.binary +package org.apache.spark.mllib.evaluation -import org.apache.spark.rdd.{UnionRDD, RDD} -import org.apache.spark.SparkContext._ -import org.apache.spark.mllib.evaluation.AreaUnderCurve +import org.apache.spark.annotation.Experimental import org.apache.spark.Logging +import org.apache.spark.SparkContext._ +import org.apache.spark.mllib.evaluation.binary._ +import org.apache.spark.rdd.{RDD, UnionRDD} /** - * Implementation of [[org.apache.spark.mllib.evaluation.binary.BinaryConfusionMatrix]]. - * - * @param count label counter for labels with scores greater than or equal to the current score - * @param totalCount label counter for all labels - */ -private case class BinaryConfusionMatrixImpl( - count: LabelCounter, - totalCount: LabelCounter) extends BinaryConfusionMatrix with Serializable { - - /** number of true positives */ - override def numTruePositives: Long = count.numPositives - - /** number of false positives */ - override def numFalsePositives: Long = count.numNegatives - - /** number of false negatives */ - override def numFalseNegatives: Long = totalCount.numPositives - count.numPositives - - /** number of true negatives */ - override def numTrueNegatives: Long = totalCount.numNegatives - count.numNegatives - - /** number of positives */ - override def numPositives: Long = totalCount.numPositives - - /** number of negatives */ - override def numNegatives: Long = totalCount.numNegatives -} - -/** + * :: Experimental :: * Evaluator for binary classification. * * @param scoreAndLabels an RDD of (score, label) pairs. */ -class BinaryClassificationMetrics(scoreAndLabels: RDD[(Double, Double)]) - extends Serializable with Logging { - - private lazy val ( - cumulativeCounts: RDD[(Double, LabelCounter)], - confusions: RDD[(Double, BinaryConfusionMatrix)]) = { - // Create a bin for each distinct score value, count positives and negatives within each bin, - // and then sort by score values in descending order. - val counts = scoreAndLabels.combineByKey( - createCombiner = (label: Double) => new LabelCounter(0L, 0L) += label, - mergeValue = (c: LabelCounter, label: Double) => c += label, - mergeCombiners = (c1: LabelCounter, c2: LabelCounter) => c1 += c2 - ).sortByKey(ascending = false) - val agg = counts.values.mapPartitions({ iter => - val agg = new LabelCounter() - iter.foreach(agg += _) - Iterator(agg) - }, preservesPartitioning = true).collect() - val partitionwiseCumulativeCounts = - agg.scanLeft(new LabelCounter())((agg: LabelCounter, c: LabelCounter) => agg.clone() += c) - val totalCount = partitionwiseCumulativeCounts.last - logInfo(s"Total counts: $totalCount") - val cumulativeCounts = counts.mapPartitionsWithIndex( - (index: Int, iter: Iterator[(Double, LabelCounter)]) => { - val cumCount = partitionwiseCumulativeCounts(index) - iter.map { case (score, c) => - cumCount += c - (score, cumCount.clone()) - } - }, preservesPartitioning = true) - cumulativeCounts.persist() - val confusions = cumulativeCounts.map { case (score, cumCount) => - (score, BinaryConfusionMatrixImpl(cumCount, totalCount).asInstanceOf[BinaryConfusionMatrix]) - } - (cumulativeCounts, confusions) - } +@Experimental +class BinaryClassificationMetrics(scoreAndLabels: RDD[(Double, Double)]) extends Logging { /** Unpersist intermediate RDDs used in the computation. */ def unpersist() { @@ -154,6 +93,41 @@ class BinaryClassificationMetrics(scoreAndLabels: RDD[(Double, Double)]) /** Returns the (threshold, recall) curve. */ def recallByThreshold(): RDD[(Double, Double)] = createCurve(Recall) + private lazy val ( + cumulativeCounts: RDD[(Double, BinaryLabelCounter)], + confusions: RDD[(Double, BinaryConfusionMatrix)]) = { + // Create a bin for each distinct score value, count positives and negatives within each bin, + // and then sort by score values in descending order. + val counts = scoreAndLabels.combineByKey( + createCombiner = (label: Double) => new BinaryLabelCounter(0L, 0L) += label, + mergeValue = (c: BinaryLabelCounter, label: Double) => c += label, + mergeCombiners = (c1: BinaryLabelCounter, c2: BinaryLabelCounter) => c1 += c2 + ).sortByKey(ascending = false) + val agg = counts.values.mapPartitions({ iter => + val agg = new BinaryLabelCounter() + iter.foreach(agg += _) + Iterator(agg) + }, preservesPartitioning = true).collect() + val partitionwiseCumulativeCounts = + agg.scanLeft(new BinaryLabelCounter())( + (agg: BinaryLabelCounter, c: BinaryLabelCounter) => agg.clone() += c) + val totalCount = partitionwiseCumulativeCounts.last + logInfo(s"Total counts: $totalCount") + val cumulativeCounts = counts.mapPartitionsWithIndex( + (index: Int, iter: Iterator[(Double, BinaryLabelCounter)]) => { + val cumCount = partitionwiseCumulativeCounts(index) + iter.map { case (score, c) => + cumCount += c + (score, cumCount.clone()) + } + }, preservesPartitioning = true) + cumulativeCounts.persist() + val confusions = cumulativeCounts.map { case (score, cumCount) => + (score, BinaryConfusionMatrixImpl(cumCount, totalCount).asInstanceOf[BinaryConfusionMatrix]) + } + (cumulativeCounts, confusions) + } + /** Creates a curve of (threshold, metric). */ private def createCurve(y: BinaryClassificationMetricComputer): RDD[(Double, Double)] = { confusions.map { case (s, c) => @@ -170,35 +144,3 @@ class BinaryClassificationMetrics(scoreAndLabels: RDD[(Double, Double)]) } } } - -/** - * A counter for positives and negatives. - * - * @param numPositives number of positive labels - * @param numNegatives number of negative labels - */ -private class LabelCounter( - var numPositives: Long = 0L, - var numNegatives: Long = 0L) extends Serializable { - - /** Processes a label. */ - def +=(label: Double): LabelCounter = { - // Though we assume 1.0 for positive and 0.0 for negative, the following check will handle - // -1.0 for negative as well. - if (label > 0.5) numPositives += 1L else numNegatives += 1L - this - } - - /** Merges another counter. */ - def +=(other: LabelCounter): LabelCounter = { - numPositives += other.numPositives - numNegatives += other.numNegatives - this - } - - override def clone: LabelCounter = { - new LabelCounter(numPositives, numNegatives) - } - - override def toString: String = s"{numPos: $numPositives, numNeg: $numNegatives}" -} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryConfusionMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryConfusionMatrix.scala index 75a75b216002a..559c6ef7e7251 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryConfusionMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryConfusionMatrix.scala @@ -39,3 +39,32 @@ private[evaluation] trait BinaryConfusionMatrix { /** number of negatives */ def numNegatives: Long = numFalsePositives + numTrueNegatives } + +/** + * Implementation of [[org.apache.spark.mllib.evaluation.binary.BinaryConfusionMatrix]]. + * + * @param count label counter for labels with scores greater than or equal to the current score + * @param totalCount label counter for all labels + */ +private[evaluation] case class BinaryConfusionMatrixImpl( + count: BinaryLabelCounter, + totalCount: BinaryLabelCounter) extends BinaryConfusionMatrix { + + /** number of true positives */ + override def numTruePositives: Long = count.numPositives + + /** number of false positives */ + override def numFalsePositives: Long = count.numNegatives + + /** number of false negatives */ + override def numFalseNegatives: Long = totalCount.numPositives - count.numPositives + + /** number of true negatives */ + override def numTrueNegatives: Long = totalCount.numNegatives - count.numNegatives + + /** number of positives */ + override def numPositives: Long = totalCount.numPositives + + /** number of negatives */ + override def numNegatives: Long = totalCount.numNegatives +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryLabelCounter.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryLabelCounter.scala new file mode 100644 index 0000000000000..1e610c20092a7 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryLabelCounter.scala @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.evaluation.binary + +/** + * A counter for positives and negatives. + * + * @param numPositives number of positive labels + * @param numNegatives number of negative labels + */ +private[evaluation] class BinaryLabelCounter( + var numPositives: Long = 0L, + var numNegatives: Long = 0L) extends Serializable { + + /** Processes a label. */ + def +=(label: Double): BinaryLabelCounter = { + // Though we assume 1.0 for positive and 0.0 for negative, the following check will handle + // -1.0 for negative as well. + if (label > 0.5) numPositives += 1L else numNegatives += 1L + this + } + + /** Merges another counter. */ + def +=(other: BinaryLabelCounter): BinaryLabelCounter = { + numPositives += other.numPositives + numNegatives += other.numNegatives + this + } + + override def clone: BinaryLabelCounter = { + new BinaryLabelCounter(numPositives, numNegatives) + } + + override def toString: String = s"{numPos: $numPositives, numNeg: $numNegatives}" +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/SingularValueDecomposition.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/SingularValueDecomposition.scala index 46b105457430c..9669c364bad8f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/SingularValueDecomposition.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/SingularValueDecomposition.scala @@ -17,5 +17,11 @@ package org.apache.spark.mllib.linalg -/** Represents singular value decomposition (SVD) factors. */ +import org.apache.spark.annotation.Experimental + +/** + * :: Experimental :: + * Represents singular value decomposition (SVD) factors. + */ +@Experimental case class SingularValueDecomposition[UType, VType](U: UType, s: Vector, V: VType) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala index 56b8fdcda66eb..06d8915f3bfa1 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala @@ -25,11 +25,13 @@ import org.apache.spark.SparkContext._ import org.apache.spark.mllib.linalg.Vectors /** + * :: Experimental :: * Represents an entry in an distributed matrix. * @param i row index * @param j column index * @param value value of the entry */ +@Experimental case class MatrixEntry(i: Long, j: Long, value: Double) /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala index 0c0afcd9ec0d7..b10857fe7c8a1 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala @@ -427,6 +427,7 @@ class RowMatrix( } } +@Experimental object RowMatrix { /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala index c75909bac9248..7030eeabe400a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala @@ -21,19 +21,17 @@ import scala.collection.mutable.ArrayBuffer import breeze.linalg.{DenseVector => BDV} -import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.annotation.{Experimental, DeveloperApi} import org.apache.spark.Logging import org.apache.spark.rdd.RDD import org.apache.spark.mllib.linalg.{Vectors, Vector} /** - * :: DeveloperApi :: * Class used to solve an optimization problem using Gradient Descent. * @param gradient Gradient function to be used. * @param updater Updater to be used to update weights after every iteration. */ -@DeveloperApi -class GradientDescent(private var gradient: Gradient, private var updater: Updater) +class GradientDescent private[mllib] (private var gradient: Gradient, private var updater: Updater) extends Optimizer with Logging { private var stepSize: Double = 1.0 @@ -51,9 +49,11 @@ class GradientDescent(private var gradient: Gradient, private var updater: Updat } /** + * :: Experimental :: * Set fraction of data to be used for each SGD iteration. * Default 1.0 (corresponding to deterministic/classical gradient descent) */ + @Experimental def setMiniBatchFraction(fraction: Double): this.type = { this.miniBatchFraction = fraction this @@ -95,6 +95,14 @@ class GradientDescent(private var gradient: Gradient, private var updater: Updat this } + /** + * :: DeveloperApi :: + * Runs gradient descent on the given training data. + * @param data training data + * @param initialWeights initial weights + * @return solution vector + */ + @DeveloperApi def optimize(data: RDD[(Double, Vector)], initialWeights: Vector): Vector = { val (weights, _) = GradientDescent.runMiniBatchSGD( data, diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala index 2a77e1a9efb2e..0cf9a7f909081 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala @@ -472,13 +472,15 @@ class ALS private ( // Compute the XtX and Xy values for each user by adding products it rated in each product // block for (productBlock <- 0 until numBlocks) { - for (p <- 0 until blockFactors(productBlock).length) { + var p = 0 + while (p < blockFactors(productBlock).length) { val x = wrapDoubleArray(blockFactors(productBlock)(p)) tempXtX.fill(0.0) dspr(1.0, x, tempXtX) val (us, rs) = inLinkBlock.ratingsForBlock(productBlock)(p) - for (i <- 0 until us.length) { - if (implicitPrefs) { + if (implicitPrefs) { + var i = 0 + while (i < us.length) { // Extension to the original paper to handle rs(i) < 0. confidence is a function // of |rs(i)| instead so that it is never negative: val confidence = 1 + alpha * abs(rs(i)) @@ -489,11 +491,17 @@ class ALS private ( if (rs(i) > 0) { SimpleBlas.axpy(confidence, x, userXy(us(i))) } - } else { + i += 1 + } + } else { + var i = 0 + while (i < us.length) { userXtX(us(i)).addi(tempXtX) SimpleBlas.axpy(rs(i), x, userXy(us(i))) + i += 1 } } + p += 1 } } @@ -502,7 +510,11 @@ class ALS private ( // Compute the full XtX matrix from the lower-triangular part we got above fillFullMatrix(userXtX(index), fullXtX) // Add regularization - (0 until rank).foreach(i => fullXtX.data(i*rank + i) += lambda) + var i = 0 + while (i < rank) { + fullXtX.data(i * rank + i) += lambda + i += 1 + } // Solve the resulting matrix, which is symmetric and positive-definite if (implicitPrefs) { Solve.solvePositive(fullXtX.addi(YtY.get.value), userXy(index)).data diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala index 471546cd82c7d..899286d235a9d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala @@ -17,7 +17,7 @@ package org.apache.spark.mllib.recommendation -import org.jblas._ +import org.jblas.DoubleMatrix import org.apache.spark.annotation.DeveloperApi import org.apache.spark.api.java.JavaRDD @@ -25,7 +25,6 @@ import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext._ import org.apache.spark.mllib.api.python.PythonMLLibAPI - /** * Model representing the result of matrix factorization. * @@ -35,12 +34,10 @@ import org.apache.spark.mllib.api.python.PythonMLLibAPI * @param productFeatures RDD of tuples where each tuple represents the productId * and the features computed for this product. */ -class MatrixFactorizationModel( +class MatrixFactorizationModel private[mllib] ( val rank: Int, val userFeatures: RDD[(Int, Array[Double])], - val productFeatures: RDD[(Int, Array[Double])]) - extends Serializable -{ + val productFeatures: RDD[(Int, Array[Double])]) extends Serializable { /** Predict the rating of one user for one product. */ def predict(user: Int, product: Int): Double = { val userVector = new DoubleMatrix(userFeatures.lookup(user).head) @@ -76,6 +73,7 @@ class MatrixFactorizationModel( * @param usersProductsJRDD A JavaRDD with serialized tuples (user, product) * @return JavaRDD of serialized Rating objects. */ + @DeveloperApi def predict(usersProductsJRDD: JavaRDD[Array[Byte]]): JavaRDD[Array[Byte]] = { val pythonAPI = new PythonMLLibAPI() val usersProducts = usersProductsJRDD.rdd.map(xBytes => pythonAPI.unpackTuple(xBytes)) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala index d969e7aa60061..8cca926f1c92e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala @@ -19,13 +19,14 @@ package org.apache.spark.mllib.regression import breeze.linalg.{DenseVector => BDV, SparseVector => BSV} -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.{Logging, SparkException} import org.apache.spark.rdd.RDD import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.linalg.{Vectors, Vector} /** + * :: DeveloperApi :: * GeneralizedLinearModel (GLM) represents a model trained using * GeneralizedLinearAlgorithm. GLMs consist of a weight vector and * an intercept. @@ -33,6 +34,7 @@ import org.apache.spark.mllib.linalg.{Vectors, Vector} * @param weights Weights computed for every feature. * @param intercept Intercept computed for this model. */ +@DeveloperApi abstract class GeneralizedLinearModel(val weights: Vector, val intercept: Double) extends Serializable { @@ -72,9 +74,11 @@ abstract class GeneralizedLinearModel(val weights: Vector, val intercept: Double } /** + * :: DeveloperApi :: * GeneralizedLinearAlgorithm implements methods to train a Generalized Linear Model (GLM). * This class should be extended with an Optimizer to create a new GLM. */ +@DeveloperApi abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] extends Logging with Serializable { @@ -83,8 +87,8 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] /** The optimizer to solve the problem. */ def optimizer: Optimizer - /** Whether to add intercept (default: true). */ - protected var addIntercept: Boolean = true + /** Whether to add intercept (default: false). */ + protected var addIntercept: Boolean = false protected var validateData: Boolean = true @@ -94,7 +98,8 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] protected def createModel(weights: Vector, intercept: Double): M /** - * Set if the algorithm should add an intercept. Default true. + * Set if the algorithm should add an intercept. Default false. + * We set the default to false because adding the intercept will cause memory allocation. */ def setIntercept(addIntercept: Boolean): this.type = { this.addIntercept = addIntercept @@ -102,10 +107,8 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] } /** - * :: Experimental :: * Set if the algorithm should validate data before training. Default true. */ - @Experimental def setValidateData(validateData: Boolean): this.type = { this.validateData = validateData this diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala index 0e6fb1b1caa41..a05dfc045fb8e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala @@ -17,6 +17,7 @@ package org.apache.spark.mllib.regression +import org.apache.spark.annotation.Experimental import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.optimization._ import org.apache.spark.rdd.RDD @@ -27,7 +28,7 @@ import org.apache.spark.rdd.RDD * @param weights Weights computed for every feature. * @param intercept Intercept computed for this model. */ -class LassoModel( +class LassoModel private[mllib] ( override val weights: Vector, override val intercept: Double) extends GeneralizedLinearModel(weights, intercept) @@ -64,21 +65,12 @@ class LassoWithSGD private ( .setRegParam(regParam) .setMiniBatchFraction(miniBatchFraction) - // We don't want to penalize the intercept, so set this to false. - super.setIntercept(false) - /** * Construct a Lasso object with default parameters: {stepSize: 1.0, numIterations: 100, * regParam: 1.0, miniBatchFraction: 1.0}. */ def this() = this(1.0, 100, 1.0, 1.0) - override def setIntercept(addIntercept: Boolean): this.type = { - // TODO: Support adding intercept. - if (addIntercept) throw new UnsupportedOperationException("Adding intercept is not supported.") - this - } - override protected def createModel(weights: Vector, intercept: Double) = { new LassoModel(weights, intercept) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala index 1532ff90d846d..0ebad4eb58d88 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala @@ -27,7 +27,7 @@ import org.apache.spark.mllib.optimization._ * @param weights Weights computed for every feature. * @param intercept Intercept computed for this model. */ -class LinearRegressionModel( +class LinearRegressionModel private[mllib] ( override val weights: Vector, override val intercept: Double) extends GeneralizedLinearModel(weights, intercept) with RegressionModel with Serializable { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala index 5f7e25a9b8be1..bd983bac001a0 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala @@ -17,6 +17,7 @@ package org.apache.spark.mllib.regression +import org.apache.spark.annotation.Experimental import org.apache.spark.rdd.RDD import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.linalg.Vector @@ -27,7 +28,7 @@ import org.apache.spark.mllib.linalg.Vector * @param weights Weights computed for every feature. * @param intercept Intercept computed for this model. */ -class RidgeRegressionModel( +class RidgeRegressionModel private[mllib] ( override val weights: Vector, override val intercept: Double) extends GeneralizedLinearModel(weights, intercept) @@ -65,21 +66,12 @@ class RidgeRegressionWithSGD private ( .setRegParam(regParam) .setMiniBatchFraction(miniBatchFraction) - // We don't want to penalize the intercept in RidgeRegression, so set this to false. - super.setIntercept(false) - /** * Construct a RidgeRegression object with default parameters: {stepSize: 1.0, numIterations: 100, * regParam: 1.0, miniBatchFraction: 1.0}. */ def this() = this(1.0, 100, 1.0, 1.0) - override def setIntercept(addIntercept: Boolean): this.type = { - // TODO: Support adding intercept. - if (addIntercept) throw new UnsupportedOperationException("Adding intercept is not supported.") - this - } - override protected def createModel(weights: Vector, intercept: Double) = { new RidgeRegressionModel(weights, intercept) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/README.md b/mllib/src/main/scala/org/apache/spark/mllib/tree/README.md deleted file mode 100644 index 0fd71aa9735bc..0000000000000 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/README.md +++ /dev/null @@ -1,17 +0,0 @@ -This package contains the default implementation of the decision tree algorithm. - -The decision tree algorithm supports: -+ Binary classification -+ Regression -+ Information loss calculation with entropy and gini for classification and variance for regression -+ Both continuous and categorical features - -# Tree improvements -+ Node model pruning -+ Printing to dot files - -# Future Ensemble Extensions - -+ Random forests -+ Boosting -+ Extremely randomized trees diff --git a/mllib/src/main/scala/org/apache/spark/mllib/rdd/VectorRDDs.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/package.scala similarity index 69% rename from mllib/src/main/scala/org/apache/spark/mllib/rdd/VectorRDDs.scala rename to mllib/src/main/scala/org/apache/spark/mllib/tree/package.scala index 9096d6a1a16d6..bcaacc1b1f191 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/rdd/VectorRDDs.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/package.scala @@ -15,18 +15,15 @@ * limitations under the License. */ -package org.apache.spark.mllib.rdd - -import org.apache.spark.rdd.RDD -import org.apache.spark.mllib.linalg.{Vectors, Vector} +package org.apache.spark.mllib /** - * Factory methods for `RDD[Vector]`. + * This package contains the default implementation of the decision tree algorithm, which supports: + * - binary classification, + * - regression, + * - information loss calculation with entropy and Gini for classification and + * variance for regression, + * - both continuous and categorical features. */ -object VectorRDDs { - - /** - * Converts an `RDD[Array[Double]]` to `RDD[Vector]`. - */ - def fromArrayRDD(rdd: RDD[Array[Double]]): RDD[Vector] = rdd.map(v => Vectors.dense(v)) +package object tree { } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/LabelParsers.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LabelParsers.scala index f7966d3ebb613..e25bf18b780bf 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/LabelParsers.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/LabelParsers.scala @@ -18,16 +18,23 @@ package org.apache.spark.mllib.util /** Trait for label parsers. */ -trait LabelParser extends Serializable { +private trait LabelParser extends Serializable { /** Parses a string label into a double label. */ def parse(labelString: String): Double } +/** Factory methods for label parsers. */ +private object LabelParser { + def getInstance(multiclass: Boolean): LabelParser = { + if (multiclass) MulticlassLabelParser else BinaryLabelParser + } +} + /** * Label parser for binary labels, which outputs 1.0 (positive) if the value is greater than 0.5, * or 0.0 (negative) otherwise. So it works with +1/-1 labeling and +1/0 labeling. */ -object BinaryLabelParser extends LabelParser { +private object BinaryLabelParser extends LabelParser { /** Gets the default instance of BinaryLabelParser. */ def getInstance(): LabelParser = this @@ -41,7 +48,7 @@ object BinaryLabelParser extends LabelParser { /** * Label parser for multiclass labels, which converts the input label to double. */ -object MulticlassLabelParser extends LabelParser { +private object MulticlassLabelParser extends LabelParser { /** Gets the default instance of MulticlassLabelParser. */ def getInstance(): LabelParser = this diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala index 3d6e7e0d5c953..e598b6cb171a8 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala @@ -19,16 +19,17 @@ package org.apache.spark.mllib.util import scala.reflect.ClassTag -import breeze.linalg.{Vector => BV, SparseVector => BSV, squaredDistance => breezeSquaredDistance} +import breeze.linalg.{Vector => BV, DenseVector => BDV, SparseVector => BSV, + squaredDistance => breezeSquaredDistance} import org.apache.spark.annotation.Experimental import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD import org.apache.spark.rdd.PartitionwiseSampledRDD -import org.apache.spark.SparkContext._ import org.apache.spark.util.random.BernoulliSampler import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.storage.StorageLevel /** * Helper methods to load, save and pre-process data used in ML Lib. @@ -54,13 +55,16 @@ object MLUtils { * * @param sc Spark context * @param path file or directory path in any Hadoop-supported file system URI - * @param labelParser parser for labels, default: 1.0 if label > 0.5 or 0.0 otherwise + * @param labelParser parser for labels * @param numFeatures number of features, which will be determined from the input data if a - * negative value is given. The default value is -1. - * @param minPartitions min number of partitions, default: sc.defaultMinPartitions + * nonpositive value is given. This is useful when the dataset is already split + * into multiple files and you want to load them separately, because some + * features may not present in certain files, which leads to inconsistent + * feature dimensions. + * @param minPartitions min number of partitions * @return labeled data stored as an RDD[LabeledPoint] */ - def loadLibSVMData( + private def loadLibSVMFile( sc: SparkContext, path: String, labelParser: LabelParser, @@ -68,63 +72,112 @@ object MLUtils { minPartitions: Int): RDD[LabeledPoint] = { val parsed = sc.textFile(path, minPartitions) .map(_.trim) - .filter(!_.isEmpty) - .map(_.split(' ')) + .filter(line => !(line.isEmpty || line.startsWith("#"))) + .map { line => + val items = line.split(' ') + val label = labelParser.parse(items.head) + val (indices, values) = items.tail.map { item => + val indexAndValue = item.split(':') + val index = indexAndValue(0).toInt - 1 // Convert 1-based indices to 0-based. + val value = indexAndValue(1).toDouble + (index, value) + }.unzip + (label, indices.toArray, values.toArray) + } + // Determine number of features. - val d = if (numFeatures >= 0) { + val d = if (numFeatures > 0) { numFeatures } else { - parsed.map { items => - if (items.length > 1) { - items.last.split(':')(0).toInt - } else { - 0 - } - }.reduce(math.max) + parsed.persist(StorageLevel.MEMORY_ONLY) + parsed.map { case (label, indices, values) => + indices.lastOption.getOrElse(0) + }.reduce(math.max) + 1 } - parsed.map { items => - val label = labelParser.parse(items.head) - val (indices, values) = items.tail.map { item => - val indexAndValue = item.split(':') - val index = indexAndValue(0).toInt - 1 - val value = indexAndValue(1).toDouble - (index, value) - }.unzip - LabeledPoint(label, Vectors.sparse(d, indices.toArray, values.toArray)) + + parsed.map { case (label, indices, values) => + LabeledPoint(label, Vectors.sparse(d, indices, values)) } } - // Convenient methods for calling from Java. + // Convenient methods for `loadLibSVMFile`. /** - * Loads binary labeled data in the LIBSVM format into an RDD[LabeledPoint], - * with number of features determined automatically and the default number of partitions. + * Loads labeled data in the LIBSVM format into an RDD[LabeledPoint]. + * The LIBSVM format is a text-based format used by LIBSVM and LIBLINEAR. + * Each line represents a labeled sparse feature vector using the following format: + * {{{label index1:value1 index2:value2 ...}}} + * where the indices are one-based and in ascending order. + * This method parses each line into a [[org.apache.spark.mllib.regression.LabeledPoint]], + * where the feature indices are converted to zero-based. + * + * @param sc Spark context + * @param path file or directory path in any Hadoop-supported file system URI + * @param multiclass whether the input labels contain more than two classes. If false, any label + * with value greater than 0.5 will be mapped to 1.0, or 0.0 otherwise. So it + * works for both +1/-1 and 1/0 cases. If true, the double value parsed directly + * from the label string will be used as the label value. + * @param numFeatures number of features, which will be determined from the input data if a + * nonpositive value is given. This is useful when the dataset is already split + * into multiple files and you want to load them separately, because some + * features may not present in certain files, which leads to inconsistent + * feature dimensions. + * @param minPartitions min number of partitions + * @return labeled data stored as an RDD[LabeledPoint] */ - def loadLibSVMData(sc: SparkContext, path: String): RDD[LabeledPoint] = - loadLibSVMData(sc, path, BinaryLabelParser, -1, sc.defaultMinPartitions) + def loadLibSVMFile( + sc: SparkContext, + path: String, + multiclass: Boolean, + numFeatures: Int, + minPartitions: Int): RDD[LabeledPoint] = + loadLibSVMFile(sc, path, LabelParser.getInstance(multiclass), numFeatures, minPartitions) /** - * Loads labeled data in the LIBSVM format into an RDD[LabeledPoint], - * with the given label parser, number of features determined automatically, - * and the default number of partitions. + * Loads labeled data in the LIBSVM format into an RDD[LabeledPoint], with the default number of + * partitions. */ - def loadLibSVMData( + def loadLibSVMFile( sc: SparkContext, path: String, - labelParser: LabelParser): RDD[LabeledPoint] = - loadLibSVMData(sc, path, labelParser, -1, sc.defaultMinPartitions) + multiclass: Boolean, + numFeatures: Int): RDD[LabeledPoint] = + loadLibSVMFile(sc, path, multiclass, numFeatures, sc.defaultMinPartitions) /** - * Loads labeled data in the LIBSVM format into an RDD[LabeledPoint], - * with the given label parser, number of features specified explicitly, - * and the default number of partitions. + * Loads labeled data in the LIBSVM format into an RDD[LabeledPoint], with the number of features + * determined automatically and the default number of partitions. */ - def loadLibSVMData( + def loadLibSVMFile( sc: SparkContext, path: String, - labelParser: LabelParser, - numFeatures: Int): RDD[LabeledPoint] = - loadLibSVMData(sc, path, labelParser, numFeatures, sc.defaultMinPartitions) + multiclass: Boolean): RDD[LabeledPoint] = + loadLibSVMFile(sc, path, multiclass, -1, sc.defaultMinPartitions) + + /** + * Loads binary labeled data in the LIBSVM format into an RDD[LabeledPoint], with number of + * features determined automatically and the default number of partitions. + */ + def loadLibSVMFile(sc: SparkContext, path: String): RDD[LabeledPoint] = + loadLibSVMFile(sc, path, multiclass = false, -1, sc.defaultMinPartitions) + + /** + * Save labeled data in LIBSVM format. + * @param data an RDD of LabeledPoint to be saved + * @param dir directory to save the data + * + * @see [[org.apache.spark.mllib.util.MLUtils#loadLibSVMFile]] + */ + def saveAsLibSVMFile(data: RDD[LabeledPoint], dir: String) { + // TODO: allow to specify label precision and feature precision. + val dataStr = data.map { case LabeledPoint(label, features) => + val featureStrings = features.toBreeze.activeIterator.map { case (i, v) => + s"${i + 1}:$v" + } + (Iterator(label) ++ featureStrings).mkString(" ") + } + dataStr.saveAsTextFile(dir) + } /** * :: Experimental :: @@ -163,10 +216,12 @@ object MLUtils { } /** + * :: Experimental :: * Return a k element array of pairs of RDDs with the first element of each pair * containing the training data, a complement of the validation data and the second * element, the validation data, containing a unique 1/kth of the data. Where k=numFolds. */ + @Experimental def kFold[T: ClassTag](rdd: RDD[T], numFolds: Int, seed: Int): Array[(RDD[T], RDD[T])] = { val numFoldsF = numFolds.toFloat (1 to numFolds).map { fold => @@ -178,6 +233,18 @@ object MLUtils { }.toArray } + /** + * Returns a new vector with `1.0` (bias) appended to the input vector. + */ + def appendBias(vector: Vector): Vector = { + val vector1 = vector.toBreeze match { + case dv: BDV[Double] => BDV.vertcat(dv, new BDV[Double](Array(1.0))) + case sv: BSV[Double] => BSV.vertcat(sv, new BSV[Double](Array(0), Array(1.0), 1)) + case v: Any => throw new IllegalArgumentException("Do not support vector type " + v.getClass) + } + Vectors.fromBreeze(vector1) + } + /** * Returns the squared Euclidean distance between two vectors. The following formula will be used * if it does not introduce too much numerical error: diff --git a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaLogisticRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaLogisticRegressionSuite.java index e18e3bc6a86be..d75d3a6b26730 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaLogisticRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaLogisticRegressionSuite.java @@ -68,6 +68,7 @@ public void runLRUsingConstructor() { LogisticRegressionSuite.generateLogisticInputAsList(A, B, nPoints, 17); LogisticRegressionWithSGD lrImpl = new LogisticRegressionWithSGD(); + lrImpl.setIntercept(true); lrImpl.optimizer().setStepSize(1.0) .setRegParam(1.0) .setNumIterations(100); @@ -80,8 +81,8 @@ public void runLRUsingConstructor() { @Test public void runLRUsingStaticMethods() { int nPoints = 10000; - double A = 2.0; - double B = -1.5; + double A = 0.0; + double B = -2.5; JavaRDD testRDD = sc.parallelize( LogisticRegressionSuite.generateLogisticInputAsList(A, B, nPoints, 42), 2).cache(); @@ -92,6 +93,7 @@ public void runLRUsingStaticMethods() { testRDD.rdd(), 100, 1.0, 1.0); int numAccurate = validatePrediction(validationData, model); + System.out.println(numAccurate); Assert.assertTrue(numAccurate > nPoints * 4.0 / 5.0); } diff --git a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaSVMSuite.java b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaSVMSuite.java index 4701a5e545020..667f76a1bd55f 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaSVMSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaSVMSuite.java @@ -67,6 +67,7 @@ public void runSVMUsingConstructor() { SVMSuite.generateSVMInputAsList(A, weights, nPoints, 17); SVMWithSGD svmSGDImpl = new SVMWithSGD(); + svmSGDImpl.setIntercept(true); svmSGDImpl.optimizer().setStepSize(1.0) .setRegParam(1.0) .setNumIterations(100); @@ -79,7 +80,7 @@ public void runSVMUsingConstructor() { @Test public void runSVMUsingStaticMethods() { int nPoints = 10000; - double A = 2.0; + double A = 0.0; double[] weights = {-1.5, 1.0}; JavaRDD testRDD = sc.parallelize(SVMSuite.generateSVMInputAsList(A, diff --git a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLinearRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLinearRegressionSuite.java index 5a4410a632649..7151e553512b3 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLinearRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLinearRegressionSuite.java @@ -68,6 +68,7 @@ public void runLinearRegressionUsingConstructor() { LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 17, 0.1); LinearRegressionWithSGD linSGDImpl = new LinearRegressionWithSGD(); + linSGDImpl.setIntercept(true); LinearRegressionModel model = linSGDImpl.run(testRDD.rdd()); int numAccurate = validatePrediction(validationData, model); @@ -77,7 +78,7 @@ public void runLinearRegressionUsingConstructor() { @Test public void runLinearRegressionUsingStaticMethods() { int nPoints = 100; - double A = 3.0; + double A = 0.0; double[] weights = {10, 10}; JavaRDD testRDD = sc.parallelize( diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala index 1e03c9df820b0..4d7b984e3ec29 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala @@ -46,24 +46,14 @@ object LogisticRegressionSuite { val rnd = new Random(seed) val x1 = Array.fill[Double](nPoints)(rnd.nextGaussian()) - // NOTE: if U is uniform[0, 1] then ln(u) - ln(1-u) is Logistic(0,1) - val unifRand = new scala.util.Random(45) - val rLogis = (0 until nPoints).map { i => - val u = unifRand.nextDouble() - math.log(u) - math.log(1.0-u) - } - - // y <- A + B*x + rLogis() - // y <- as.numeric(y > 0) - val y: Seq[Int] = (0 until nPoints).map { i => - val yVal = offset + scale * x1(i) + rLogis(i) - if (yVal > 0) 1 else 0 + val y = (0 until nPoints).map { i => + val p = 1.0 / (1.0 + math.exp(-(offset + scale * x1(i)))) + if (rnd.nextDouble() < p) 1.0 else 0.0 } val testData = (0 until nPoints).map(i => LabeledPoint(y(i), Vectors.dense(Array(x1(i))))) testData } - } class LogisticRegressionSuite extends FunSuite with LocalSparkContext with ShouldMatchers { @@ -85,7 +75,7 @@ class LogisticRegressionSuite extends FunSuite with LocalSparkContext with Shoul val testRDD = sc.parallelize(testData, 2) testRDD.cache() - val lr = new LogisticRegressionWithSGD() + val lr = new LogisticRegressionWithSGD().setIntercept(true) lr.optimizer.setStepSize(10.0).setNumIterations(20) val model = lr.run(testRDD) @@ -118,7 +108,7 @@ class LogisticRegressionSuite extends FunSuite with LocalSparkContext with Shoul testRDD.cache() // Use half as many iterations as the previous test. - val lr = new LogisticRegressionWithSGD() + val lr = new LogisticRegressionWithSGD().setIntercept(true) lr.optimizer.setStepSize(10.0).setNumIterations(10) val model = lr.run(testRDD, initialWeights) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala index dfacbfeee6fb4..77d6f04b32320 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala @@ -69,7 +69,6 @@ class SVMSuite extends FunSuite with LocalSparkContext { assert(numOffPredictions < input.length / 5) } - test("SVM using local random SGD") { val nPoints = 10000 @@ -83,7 +82,7 @@ class SVMSuite extends FunSuite with LocalSparkContext { val testRDD = sc.parallelize(testData, 2) testRDD.cache() - val svm = new SVMWithSGD() + val svm = new SVMWithSGD().setIntercept(true) svm.optimizer.setStepSize(1.0).setRegParam(1.0).setNumIterations(100) val model = svm.run(testRDD) @@ -115,7 +114,7 @@ class SVMSuite extends FunSuite with LocalSparkContext { val testRDD = sc.parallelize(testData, 2) testRDD.cache() - val svm = new SVMWithSGD() + val svm = new SVMWithSGD().setIntercept(true) svm.optimizer.setStepSize(1.0).setRegParam(1.0).setNumIterations(100) val model = svm.run(testRDD, initialWeights) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/binary/BinaryClassificationMetricsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetricsSuite.scala similarity index 96% rename from mllib/src/test/scala/org/apache/spark/mllib/evaluation/binary/BinaryClassificationMetricsSuite.scala rename to mllib/src/test/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetricsSuite.scala index 173fdaefab3da..9d16182f9d8c4 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/binary/BinaryClassificationMetricsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetricsSuite.scala @@ -15,12 +15,11 @@ * limitations under the License. */ -package org.apache.spark.mllib.evaluation.binary +package org.apache.spark.mllib.evaluation import org.scalatest.FunSuite import org.apache.spark.mllib.util.LocalSparkContext -import org.apache.spark.mllib.evaluation.AreaUnderCurve class BinaryClassificationMetricsSuite extends FunSuite with LocalSparkContext { test("binary evaluation metrics") { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala index 4dfcd4b52ec66..2d944f3eb7ff9 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala @@ -27,7 +27,6 @@ import org.jblas.DoubleMatrix import org.apache.spark.mllib.util.LocalSparkContext import org.apache.spark.SparkContext._ -import org.apache.spark.Partitioner object ALSSuite { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala index 6aad9eb84e13c..bfa42959c8ead 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala @@ -112,10 +112,4 @@ class LassoSuite extends FunSuite with LocalSparkContext { // Test prediction on Array. validatePrediction(validationData.map(row => model.predict(row.features)), validationData) } - - test("do not support intercept") { - intercept[UnsupportedOperationException] { - new LassoWithSGD().setIntercept(true) - } - } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala index 2f7d30708ce17..7aaad7d7a3e39 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala @@ -37,7 +37,7 @@ class LinearRegressionSuite extends FunSuite with LocalSparkContext { test("linear regression") { val testRDD = sc.parallelize(LinearDataGenerator.generateLinearInput( 3.0, Array(10.0, 10.0), 100, 42), 2).cache() - val linReg = new LinearRegressionWithSGD() + val linReg = new LinearRegressionWithSGD().setIntercept(true) linReg.optimizer.setNumIterations(1000).setStepSize(1.0) val model = linReg.run(testRDD) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala index f66fc6ea6c1ec..67768e17fbe6d 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala @@ -72,10 +72,4 @@ class RidgeRegressionSuite extends FunSuite with LocalSparkContext { assert(ridgeErr < linearErr, "ridgeError (" + ridgeErr + ") was not less than linearError(" + linearErr + ")") } - - test("do not support intercept") { - intercept[UnsupportedOperationException] { - new RidgeRegressionWithSGD().setIntercept(true) - } - } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala index bc3b1a3fbe95c..35e92d71dc63f 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala @@ -17,10 +17,8 @@ package org.apache.spark.mllib.tree -import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite -import org.apache.spark.SparkContext import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.impurity.{Entropy, Gini, Variance} import org.apache.spark.mllib.tree.model.Filter @@ -29,19 +27,9 @@ import org.apache.spark.mllib.tree.configuration.{FeatureType, Strategy} import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.configuration.FeatureType._ import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.util.LocalSparkContext -class DecisionTreeSuite extends FunSuite with BeforeAndAfterAll { - - @transient private var sc: SparkContext = _ - - override def beforeAll() { - sc = new SparkContext("local", "test") - } - - override def afterAll() { - sc.stop() - System.clearProperty("spark.driver.port") - } +class DecisionTreeSuite extends FunSuite with LocalSparkContext { test("split and bin calculation") { val arr = DecisionTreeSuite.generateOrderedLabeledPointsWithLabel1() diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala index 674378a34ce34..3f64baf6fe41f 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala @@ -19,8 +19,8 @@ package org.apache.spark.mllib.util import java.io.File +import scala.io.Source import scala.math -import scala.util.Random import org.scalatest.FunSuite @@ -29,7 +29,8 @@ import breeze.linalg.{DenseVector => BDV, SparseVector => BSV, norm => breezeNor import com.google.common.base.Charsets import com.google.common.io.Files -import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vectors} +import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.util.MLUtils._ class MLUtilsSuite extends FunSuite with LocalSparkContext { @@ -58,7 +59,7 @@ class MLUtilsSuite extends FunSuite with LocalSparkContext { } } - test("loadLibSVMData") { + test("loadLibSVMFile") { val lines = """ |+1 1:1.0 3:2.0 5:3.0 @@ -70,8 +71,8 @@ class MLUtilsSuite extends FunSuite with LocalSparkContext { Files.write(lines, file, Charsets.US_ASCII) val path = tempDir.toURI.toString - val pointsWithNumFeatures = MLUtils.loadLibSVMData(sc, path, BinaryLabelParser, 6).collect() - val pointsWithoutNumFeatures = MLUtils.loadLibSVMData(sc, path).collect() + val pointsWithNumFeatures = loadLibSVMFile(sc, path, multiclass = false, 6).collect() + val pointsWithoutNumFeatures = loadLibSVMFile(sc, path).collect() for (points <- Seq(pointsWithNumFeatures, pointsWithoutNumFeatures)) { assert(points.length === 3) @@ -83,29 +84,54 @@ class MLUtilsSuite extends FunSuite with LocalSparkContext { assert(points(2).features === Vectors.sparse(6, Seq((1, 4.0), (3, 5.0), (5, 6.0)))) } - val multiclassPoints = MLUtils.loadLibSVMData(sc, path, MulticlassLabelParser).collect() + val multiclassPoints = loadLibSVMFile(sc, path, multiclass = true).collect() assert(multiclassPoints.length === 3) assert(multiclassPoints(0).label === 1.0) assert(multiclassPoints(1).label === -1.0) assert(multiclassPoints(2).label === -1.0) - try { - file.delete() - tempDir.delete() - } catch { - case t: Throwable => - } + deleteQuietly(tempDir) + } + + test("saveAsLibSVMFile") { + val examples = sc.parallelize(Seq( + LabeledPoint(1.1, Vectors.sparse(3, Seq((0, 1.23), (2, 4.56)))), + LabeledPoint(0.0, Vectors.dense(1.01, 2.02, 3.03)) + ), 2) + val tempDir = Files.createTempDir() + val outputDir = new File(tempDir, "output") + MLUtils.saveAsLibSVMFile(examples, outputDir.toURI.toString) + val lines = outputDir.listFiles() + .filter(_.getName.startsWith("part-")) + .flatMap(Source.fromFile(_).getLines()) + .toSet + val expected = Set("1.1 1:1.23 3:4.56", "0.0 1:1.01 2:2.02 3:3.03") + assert(lines === expected) + deleteQuietly(tempDir) + } + + test("appendBias") { + val sv = Vectors.sparse(3, Seq((0, 1.0), (2, 3.0))) + val sv1 = appendBias(sv).asInstanceOf[SparseVector] + assert(sv1.size === 4) + assert(sv1.indices === Array(0, 2, 3)) + assert(sv1.values === Array(1.0, 3.0, 1.0)) + + val dv = Vectors.dense(1.0, 0.0, 3.0) + val dv1 = appendBias(dv).asInstanceOf[DenseVector] + assert(dv1.size === 4) + assert(dv1.values === Array(1.0, 0.0, 3.0, 1.0)) } test("kFold") { val data = sc.parallelize(1 to 100, 2) val collectedData = data.collect().sorted - val twoFoldedRdd = MLUtils.kFold(data, 2, 1) + val twoFoldedRdd = kFold(data, 2, 1) assert(twoFoldedRdd(0)._1.collect().sorted === twoFoldedRdd(1)._2.collect().sorted) assert(twoFoldedRdd(0)._2.collect().sorted === twoFoldedRdd(1)._1.collect().sorted) for (folds <- 2 to 10) { for (seed <- 1 to 5) { - val foldedRdds = MLUtils.kFold(data, folds, seed) + val foldedRdds = kFold(data, folds, seed) assert(foldedRdds.size === folds) foldedRdds.map { case (training, validation) => val result = validation.union(training).collect().sorted @@ -132,4 +158,16 @@ class MLUtilsSuite extends FunSuite with LocalSparkContext { } } + /** Delete a file/directory quietly. */ + def deleteQuietly(f: File) { + if (f.isDirectory) { + f.listFiles().foreach(deleteQuietly) + } + try { + f.delete() + } catch { + case _: Throwable => + } + } } + diff --git a/pom.xml b/pom.xml index 646753fe30301..fb904e868cfaf 100644 --- a/pom.xml +++ b/pom.xml @@ -16,7 +16,8 @@ ~ limitations under the License. --> - + 4.0.0 org.apache @@ -119,7 +120,7 @@ 1.2.17 1.0.4 2.4.1 - 0.23.7 + ${hadoop.version} 0.94.6 0.12.0 1.3.2 @@ -128,6 +129,7 @@ 0.3.6 3.0.0 1.7.4 + 0.7.1 64m 512m @@ -135,7 +137,8 @@ - maven-repo + maven-repo + Maven Repository http://repo.maven.apache.org/maven2 @@ -219,6 +222,11 @@ guava 14.0.1 + + org.apache.commons + commons-lang3 + 3.3.2 + com.google.code.findbugs jsr305 @@ -553,10 +561,18 @@ + net.java.dev.jets3t jets3t - 0.7.1 + ${jets3t.version} + runtime + + + commons-logging + commons-logging + + org.apache.hadoop @@ -581,6 +597,29 @@ org.apache.hadoop hadoop-yarn-common ${yarn.version} + + + asm + asm + + + org.ow2.asm + asm + + + org.jboss.netty + netty + + + javax.servlet + servlet-api + + + + + org.apache.hadoop + hadoop-yarn-server-web-proxy + ${yarn.version} asm @@ -596,7 +635,6 @@ - org.apache.hadoop hadoop-yarn-client @@ -614,6 +652,10 @@ org.jboss.netty netty + + javax.servlet + servlet-api + @@ -828,35 +870,6 @@ - - - hadoop-0.23 - - - org.apache.avro - avro - - - - - - yarn-alpha - - 2 - - 0.23.7 - - - - yarn - - - - org.apache.avro - avro - - - @@ -891,17 +904,54 @@ + + - yarn + hadoop-0.23 + + + + org.apache.avro + avro + + + + + + hadoop-2.2 - 2 - 2.2.0 2.5.0 + + + + hadoop-2.3 + + 2.5.0 + 0.9.0 + + + + + hadoop-2.4 + + 2.5.0 + 0.9.0 + + + + + yarn-alpha yarn + + + yarn + + yarn + @@ -926,6 +976,11 @@ hadoop-yarn-common provided + + org.apache.hadoop + hadoop-yarn-server-web-proxy + provided + org.apache.hadoop hadoop-yarn-client diff --git a/project/MimaBuild.scala b/project/MimaBuild.scala index d540dc0a986e9..efdb38e907d14 100644 --- a/project/MimaBuild.scala +++ b/project/MimaBuild.scala @@ -74,6 +74,8 @@ object MimaBuild { ) ++ excludeSparkClass("rdd.ClassTags") ++ excludeSparkClass("util.XORShiftRandom") ++ + excludeSparkClass("graphx.EdgeRDD") ++ + excludeSparkClass("graphx.VertexRDD") ++ excludeSparkClass("mllib.recommendation.MFDataGenerator") ++ excludeSparkClass("mllib.optimization.SquaredGradient") ++ excludeSparkClass("mllib.regression.RidgeRegressionWithSGD") ++ diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 51f7335111166..7f9746ec4acc0 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -95,7 +95,7 @@ object SparkBuild extends Build { lazy val hadoopVersion = Properties.envOrElse("SPARK_HADOOP_VERSION", DEFAULT_HADOOP_VERSION) lazy val isNewHadoop = Properties.envOrNone("SPARK_IS_NEW_HADOOP") match { case None => { - val isNewHadoopVersion = "2.[2-9]+".r.findFirstIn(hadoopVersion).isDefined + val isNewHadoopVersion = "^2\\.[2-9]+".r.findFirstIn(hadoopVersion).isDefined (isNewHadoopVersion|| DEFAULT_IS_NEW_HADOOP) } case Some(v) => v.toBoolean @@ -297,6 +297,7 @@ object SparkBuild extends Build { val chillVersion = "0.3.6" val codahaleMetricsVersion = "3.0.0" val jblasVersion = "1.2.3" + val jets3tVersion = if ("^2\\.[3-9]+".r.findFirstIn(hadoopVersion).isDefined) "0.9.0" else "0.7.1" val jettyVersion = "8.1.14.v20131031" val hiveVersion = "0.12.0" val parquetVersion = "1.3.2" @@ -326,6 +327,7 @@ object SparkBuild extends Build { name := "spark-core", libraryDependencies ++= Seq( "com.google.guava" % "guava" % "14.0.1", + "org.apache.commons" % "commons-lang3" % "3.3.2", "com.google.code.findbugs" % "jsr305" % "1.3.9", "log4j" % "log4j" % "1.2.17", "org.slf4j" % "slf4j-api" % slf4jVersion, @@ -342,7 +344,7 @@ object SparkBuild extends Build { "colt" % "colt" % "1.2.0", "org.apache.mesos" % "mesos" % "0.13.0", "commons-net" % "commons-net" % "2.2", - "net.java.dev.jets3t" % "jets3t" % "0.7.1" excludeAll(excludeCommonsLogging), + "net.java.dev.jets3t" % "jets3t" % jets3tVersion excludeAll(excludeCommonsLogging), "org.apache.derby" % "derby" % "10.4.2.0" % "test", "org.apache.hadoop" % hadoopClient % hadoopVersion excludeAll(excludeNetty, excludeAsm, excludeCommonsLogging, excludeSLF4J, excludeOldAsm), "org.apache.curator" % "curator-recipes" % "2.4.0" excludeAll(excludeNetty), @@ -354,7 +356,8 @@ object SparkBuild extends Build { "com.twitter" % "chill-java" % chillVersion excludeAll(excludeAsm), "org.tachyonproject" % "tachyon" % "0.4.1-thrift" excludeAll(excludeHadoop, excludeCurator, excludeEclipseJetty, excludePowermock), "com.clearspring.analytics" % "stream" % "2.5.1" excludeAll(excludeFastutil), - "org.spark-project" % "pyrolite" % "2.0.1" + "org.spark-project" % "pyrolite" % "2.0.1", + "net.sf.py4j" % "py4j" % "0.8.1" ), libraryDependencies ++= maybeAvro ) @@ -561,12 +564,12 @@ object SparkBuild extends Build { "org.apache.hadoop" % hadoopClient % hadoopVersion excludeAll(excludeNetty, excludeAsm, excludeOldAsm), "org.apache.hadoop" % "hadoop-yarn-api" % hadoopVersion excludeAll(excludeNetty, excludeAsm, excludeOldAsm), "org.apache.hadoop" % "hadoop-yarn-common" % hadoopVersion excludeAll(excludeNetty, excludeAsm, excludeOldAsm), - "org.apache.hadoop" % "hadoop-yarn-client" % hadoopVersion excludeAll(excludeNetty, excludeAsm, excludeOldAsm) + "org.apache.hadoop" % "hadoop-yarn-client" % hadoopVersion excludeAll(excludeNetty, excludeAsm, excludeOldAsm), + "org.apache.hadoop" % "hadoop-yarn-server-web-proxy" % hadoopVersion excludeAll(excludeNetty, excludeAsm, excludeOldAsm) ) ) def assemblyProjSettings = sharedSettings ++ Seq( - libraryDependencies += "net.sf.py4j" % "py4j" % "0.8.1", name := "spark-assembly", assembleDeps in Compile <<= (packageProjects.map(packageBin in Compile in _) ++ Seq(packageDependency in Compile)).dependOn, jarName in assembly <<= version map { v => "spark-assembly-" + v + "-hadoop" + hadoopVersion + ".jar" }, diff --git a/python/.gitignore b/python/.gitignore index 5c56e638f923a..80b361ffbd51c 100644 --- a/python/.gitignore +++ b/python/.gitignore @@ -1,2 +1,5 @@ *.pyc docs/ +pyspark.egg-info +build/ +dist/ diff --git a/python/lib/PY4J_VERSION.txt b/python/lib/PY4J_VERSION.txt deleted file mode 100644 index 04a0cd52a8d9c..0000000000000 --- a/python/lib/PY4J_VERSION.txt +++ /dev/null @@ -1 +0,0 @@ -b7924aabe9c5e63f0a4d8bbd17019534c7ec014e diff --git a/python/pyspark/__init__.py b/python/pyspark/__init__.py index 73fe7378ffa63..07df8697bd1a8 100644 --- a/python/pyspark/__init__.py +++ b/python/pyspark/__init__.py @@ -49,13 +49,6 @@ Main entry point for accessing data stored in Apache Hive.. """ - - -import sys -import os -sys.path.insert(0, os.path.join(os.environ["SPARK_HOME"], "python/lib/py4j-0.8.1-src.zip")) - - from pyspark.conf import SparkConf from pyspark.context import SparkContext from pyspark.sql import SQLContext diff --git a/python/pyspark/context.py b/python/pyspark/context.py index c74dc5fd4f854..cac133d0fcf6c 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -158,6 +158,12 @@ def __init__(self, master=None, appName=None, sparkHome=None, pyFiles=None, for path in (pyFiles or []): self.addPyFile(path) + # Deploy code dependencies set by spark-submit; these will already have been added + # with SparkContext.addFile, so we just need to add them + for path in self._conf.get("spark.submit.pyFiles", "").split(","): + if path != "": + self._python_includes.append(os.path.basename(path)) + # Create a temporary directory inside spark.local.dir: local_dir = self._jvm.org.apache.spark.util.Utils.getLocalDir(self._jsc.sc().conf()) self._temp_dir = \ @@ -447,7 +453,7 @@ def setJobGroup(self, groupId, description, interruptOnCancel=False): >>> lock = threading.Lock() >>> def map_func(x): ... sleep(100) - ... return x * x + ... raise Exception("Task should have been cancelled") >>> def start_job(x): ... global result ... try: diff --git a/python/pyspark/daemon.py b/python/pyspark/daemon.py index eb18ec08c9139..b2f226a55ec13 100644 --- a/python/pyspark/daemon.py +++ b/python/pyspark/daemon.py @@ -74,6 +74,17 @@ def handle_sigchld(*args): raise signal.signal(SIGCHLD, handle_sigchld) + # Blocks until the socket is closed by draining the input stream + # until it raises an exception or returns EOF. + def waitSocketClose(sock): + try: + while True: + # Empty string is returned upon EOF (and only then). + if sock.recv(4096) == '': + return + except: + pass + # Handle clients while not should_exit(): # Wait until a client arrives or we have to exit @@ -105,7 +116,8 @@ def handle_sigchld(*args): exit_code = exc.code finally: outfile.flush() - sock.close() + # The Scala side will close the socket upon task completion. + waitSocketClose(sock) os._exit(compute_real_exit_code(exit_code)) else: sock.close() diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py index 6bb6c877c942d..3d0936fdca911 100644 --- a/python/pyspark/java_gateway.py +++ b/python/pyspark/java_gateway.py @@ -24,41 +24,46 @@ from py4j.java_gateway import java_import, JavaGateway, GatewayClient -SPARK_HOME = os.environ["SPARK_HOME"] - - def launch_gateway(): - # Launch the Py4j gateway using Spark's run command so that we pick up the - # proper classpath and settings from spark-env.sh - on_windows = platform.system() == "Windows" - script = "./bin/spark-class.cmd" if on_windows else "./bin/spark-class" - command = [os.path.join(SPARK_HOME, script), "py4j.GatewayServer", - "--die-on-broken-pipe", "0"] - if not on_windows: - # Don't send ctrl-c / SIGINT to the Java gateway: - def preexec_func(): - signal.signal(signal.SIGINT, signal.SIG_IGN) - proc = Popen(command, stdout=PIPE, stdin=PIPE, preexec_fn=preexec_func) + SPARK_HOME = os.environ["SPARK_HOME"] + + gateway_port = -1 + if "PYSPARK_GATEWAY_PORT" in os.environ: + gateway_port = int(os.environ["PYSPARK_GATEWAY_PORT"]) else: - # preexec_fn not supported on Windows - proc = Popen(command, stdout=PIPE, stdin=PIPE) - # Determine which ephemeral port the server started on: - port = int(proc.stdout.readline()) - # Create a thread to echo output from the GatewayServer, which is required - # for Java log output to show up: - class EchoOutputThread(Thread): - def __init__(self, stream): - Thread.__init__(self) - self.daemon = True - self.stream = stream + # Launch the Py4j gateway using Spark's run command so that we pick up the + # proper classpath and settings from spark-env.sh + on_windows = platform.system() == "Windows" + script = "./bin/spark-class.cmd" if on_windows else "./bin/spark-class" + command = [os.path.join(SPARK_HOME, script), "py4j.GatewayServer", + "--die-on-broken-pipe", "0"] + if not on_windows: + # Don't send ctrl-c / SIGINT to the Java gateway: + def preexec_func(): + signal.signal(signal.SIGINT, signal.SIG_IGN) + proc = Popen(command, stdout=PIPE, stdin=PIPE, preexec_fn=preexec_func) + else: + # preexec_fn not supported on Windows + proc = Popen(command, stdout=PIPE, stdin=PIPE) + # Determine which ephemeral port the server started on: + gateway_port = int(proc.stdout.readline()) + # Create a thread to echo output from the GatewayServer, which is required + # for Java log output to show up: + class EchoOutputThread(Thread): + def __init__(self, stream): + Thread.__init__(self) + self.daemon = True + self.stream = stream + + def run(self): + while True: + line = self.stream.readline() + sys.stderr.write(line) + EchoOutputThread(proc.stdout).start() - def run(self): - while True: - line = self.stream.readline() - sys.stderr.write(line) - EchoOutputThread(proc.stdout).start() # Connect to the gateway - gateway = JavaGateway(GatewayClient(port=port), auto_convert=False) + gateway = JavaGateway(GatewayClient(port=gateway_port), auto_convert=False) + # Import the classes used by PySpark java_import(gateway.jvm, "org.apache.spark.SparkConf") java_import(gateway.jvm, "org.apache.spark.api.java.*") @@ -69,4 +74,5 @@ def run(self): java_import(gateway.jvm, "org.apache.spark.sql.hive.LocalHiveContext") java_import(gateway.jvm, "org.apache.spark.sql.hive.TestHiveContext") java_import(gateway.jvm, "scala.Tuple2") + return gateway diff --git a/python/pyspark/mllib/classification.py b/python/pyspark/mllib/classification.py index c5844597c95f2..6772e4337ef39 100644 --- a/python/pyspark/mllib/classification.py +++ b/python/pyspark/mllib/classification.py @@ -90,7 +90,7 @@ class SVMModel(LinearModel): >>> svm.predict(array([1.0])) > 0 True >>> sparse_data = [ - ... LabeledPoint(0.0, SparseVector(2, {0: 0.0})), + ... LabeledPoint(0.0, SparseVector(2, {0: -1.0})), ... LabeledPoint(1.0, SparseVector(2, {1: 1.0})), ... LabeledPoint(0.0, SparseVector(2, {0: 0.0})), ... LabeledPoint(1.0, SparseVector(2, {1: 2.0})) @@ -98,7 +98,7 @@ class SVMModel(LinearModel): >>> svm = SVMWithSGD.train(sc.parallelize(sparse_data)) >>> svm.predict(SparseVector(2, {1: 1.0})) > 0 True - >>> svm.predict(SparseVector(2, {1: 0.0})) <= 0 + >>> svm.predict(SparseVector(2, {0: -1.0})) <= 0 True """ def predict(self, x): diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index a59778c72130e..3a1c56af5b221 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -537,8 +537,8 @@ def pipe(self, command, env={}): """ Return an RDD created by piping elements to a forked external process. - >>> sc.parallelize([1, 2, 3]).pipe('cat').collect() - ['1', '2', '3'] + >>> sc.parallelize(['1', '2', '', '3']).pipe('cat').collect() + ['1', '2', '', '3'] """ def func(iterator): pipe = Popen(shlex.split(command), env=env, stdin=PIPE, stdout=PIPE) @@ -547,7 +547,7 @@ def pipe_objs(out): out.write(str(obj).rstrip('\n') + '\n') out.close() Thread(target=pipe_objs, args=[pipe.stdin]).start() - return (x.rstrip('\n') for x in pipe.stdout) + return (x.rstrip('\n') for x in iter(pipe.stdout.readline, '')) return self.mapPartitions(func) def foreach(self, f): diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 1a62031db5c41..6789d7002b3b7 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -360,6 +360,35 @@ def getCheckpointFile(self): else: return None + def coalesce(self, numPartitions, shuffle=False): + rdd = self._jschema_rdd.coalesce(numPartitions, shuffle) + return SchemaRDD(rdd, self.sql_ctx) + + def distinct(self): + rdd = self._jschema_rdd.distinct() + return SchemaRDD(rdd, self.sql_ctx) + + def intersection(self, other): + if (other.__class__ is SchemaRDD): + rdd = self._jschema_rdd.intersection(other._jschema_rdd) + return SchemaRDD(rdd, self.sql_ctx) + else: + raise ValueError("Can only intersect with another SchemaRDD") + + def repartition(self, numPartitions): + rdd = self._jschema_rdd.repartition(numPartitions) + return SchemaRDD(rdd, self.sql_ctx) + + def subtract(self, other, numPartitions=None): + if (other.__class__ is SchemaRDD): + if numPartitions is None: + rdd = self._jschema_rdd.subtract(other._jschema_rdd) + else: + rdd = self._jschema_rdd.subtract(other._jschema_rdd, numPartitions) + return SchemaRDD(rdd, self.sql_ctx) + else: + raise ValueError("Can only subtract another SchemaRDD") + def _test(): import doctest from pyspark.context import SparkContext diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 527104587fd31..64f2eeb12b4fc 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -22,18 +22,23 @@ from fileinput import input from glob import glob import os +import re import shutil +import subprocess import sys -from tempfile import NamedTemporaryFile +import tempfile import time import unittest +import zipfile from pyspark.context import SparkContext from pyspark.files import SparkFiles -from pyspark.java_gateway import SPARK_HOME from pyspark.serializers import read_int +SPARK_HOME = os.environ["SPARK_HOME"] + + class PySparkTestCase(unittest.TestCase): def setUp(self): @@ -53,7 +58,7 @@ class TestCheckpoint(PySparkTestCase): def setUp(self): PySparkTestCase.setUp(self) - self.checkpointDir = NamedTemporaryFile(delete=False) + self.checkpointDir = tempfile.NamedTemporaryFile(delete=False) os.unlink(self.checkpointDir.name) self.sc.setCheckpointDir(self.checkpointDir.name) @@ -146,7 +151,7 @@ def test_save_as_textfile_with_unicode(self): # Regression test for SPARK-970 x = u"\u00A1Hola, mundo!" data = self.sc.parallelize([x]) - tempFile = NamedTemporaryFile(delete=True) + tempFile = tempfile.NamedTemporaryFile(delete=True) tempFile.close() data.saveAsTextFile(tempFile.name) raw_contents = ''.join(input(glob(tempFile.name + "/part-0000*"))) @@ -170,7 +175,7 @@ def test_cartesian_on_textfile(self): def test_deleting_input_files(self): # Regression test for SPARK-1025 - tempFile = NamedTemporaryFile(delete=False) + tempFile = tempfile.NamedTemporaryFile(delete=False) tempFile.write("Hello World!") tempFile.close() data = self.sc.textFile(tempFile.name) @@ -234,5 +239,125 @@ def test_termination_sigterm(self): from signal import SIGTERM self.do_termination_test(lambda daemon: os.kill(daemon.pid, SIGTERM)) + +class TestSparkSubmit(unittest.TestCase): + def setUp(self): + self.programDir = tempfile.mkdtemp() + self.sparkSubmit = os.path.join(os.environ.get("SPARK_HOME"), "bin", "spark-submit") + + def tearDown(self): + shutil.rmtree(self.programDir) + + def createTempFile(self, name, content): + """ + Create a temp file with the given name and content and return its path. + Strips leading spaces from content up to the first '|' in each line. + """ + pattern = re.compile(r'^ *\|', re.MULTILINE) + content = re.sub(pattern, '', content.strip()) + path = os.path.join(self.programDir, name) + with open(path, "w") as f: + f.write(content) + return path + + def createFileInZip(self, name, content): + """ + Create a zip archive containing a file with the given content and return its path. + Strips leading spaces from content up to the first '|' in each line. + """ + pattern = re.compile(r'^ *\|', re.MULTILINE) + content = re.sub(pattern, '', content.strip()) + path = os.path.join(self.programDir, name + ".zip") + with zipfile.ZipFile(path, 'w') as zip: + zip.writestr(name, content) + return path + + def test_single_script(self): + """Submit and test a single script file""" + script = self.createTempFile("test.py", """ + |from pyspark import SparkContext + | + |sc = SparkContext() + |print sc.parallelize([1, 2, 3]).map(lambda x: x * 2).collect() + """) + proc = subprocess.Popen([self.sparkSubmit, script], stdout=subprocess.PIPE) + out, err = proc.communicate() + self.assertEqual(0, proc.returncode) + self.assertIn("[2, 4, 6]", out) + + def test_script_with_local_functions(self): + """Submit and test a single script file calling a global function""" + script = self.createTempFile("test.py", """ + |from pyspark import SparkContext + | + |def foo(x): + | return x * 3 + | + |sc = SparkContext() + |print sc.parallelize([1, 2, 3]).map(foo).collect() + """) + proc = subprocess.Popen([self.sparkSubmit, script], stdout=subprocess.PIPE) + out, err = proc.communicate() + self.assertEqual(0, proc.returncode) + self.assertIn("[3, 6, 9]", out) + + def test_module_dependency(self): + """Submit and test a script with a dependency on another module""" + script = self.createTempFile("test.py", """ + |from pyspark import SparkContext + |from mylib import myfunc + | + |sc = SparkContext() + |print sc.parallelize([1, 2, 3]).map(myfunc).collect() + """) + zip = self.createFileInZip("mylib.py", """ + |def myfunc(x): + | return x + 1 + """) + proc = subprocess.Popen([self.sparkSubmit, "--py-files", zip, script], + stdout=subprocess.PIPE) + out, err = proc.communicate() + self.assertEqual(0, proc.returncode) + self.assertIn("[2, 3, 4]", out) + + def test_module_dependency_on_cluster(self): + """Submit and test a script with a dependency on another module on a cluster""" + script = self.createTempFile("test.py", """ + |from pyspark import SparkContext + |from mylib import myfunc + | + |sc = SparkContext() + |print sc.parallelize([1, 2, 3]).map(myfunc).collect() + """) + zip = self.createFileInZip("mylib.py", """ + |def myfunc(x): + | return x + 1 + """) + proc = subprocess.Popen( + [self.sparkSubmit, "--py-files", zip, "--master", "local-cluster[1,1,512]", script], + stdout=subprocess.PIPE) + out, err = proc.communicate() + self.assertEqual(0, proc.returncode) + self.assertIn("[2, 3, 4]", out) + + def test_single_script_on_cluster(self): + """Submit and test a single script on a cluster""" + script = self.createTempFile("test.py", """ + |from pyspark import SparkContext + | + |def foo(x): + | return x * 2 + | + |sc = SparkContext() + |print sc.parallelize([1, 2, 3]).map(foo).collect() + """) + proc = subprocess.Popen( + [self.sparkSubmit, "--master", "local-cluster[1,1,512]", script], + stdout=subprocess.PIPE) + out, err = proc.communicate() + self.assertEqual(0, proc.returncode) + self.assertIn("[2, 4, 6]", out) + + if __name__ == "__main__": unittest.main() diff --git a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala index e33f4f9803054..566d96e16ed83 100644 --- a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala +++ b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala @@ -45,8 +45,7 @@ class ReplSuite extends FunSuite { } val interp = new SparkILoop(in, new PrintWriter(out), master) org.apache.spark.repl.Main.interp = interp - val separator = System.getProperty("path.separator") - interp.process(Array("-classpath", paths.mkString(separator))) + interp.process(Array("-classpath", paths.mkString(File.pathSeparator))) org.apache.spark.repl.Main.interp = null if (interp.sparkContext != null) { interp.sparkContext.stop() diff --git a/sbin/spark-config.sh b/sbin/spark-config.sh index cd2c7b7b0d496..147b506dd5ca3 100755 --- a/sbin/spark-config.sh +++ b/sbin/spark-config.sh @@ -34,3 +34,6 @@ this="$config_bin/$script" export SPARK_PREFIX=`dirname "$this"`/.. export SPARK_HOME=${SPARK_PREFIX} export SPARK_CONF_DIR="$SPARK_HOME/conf" +# Add the PySpark classes to the PYTHONPATH: +export PYTHONPATH=$SPARK_HOME/python:$PYTHONPATH +export PYTHONPATH=$SPARK_HOME/python/lib/py4j-0.8.1-src.zip:$PYTHONPATH 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 446d0e0bd7f54..792ef6cee6f5d 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 @@ -44,7 +44,8 @@ object ScalaReflection { case t if t <:< typeOf[Product] => val params = t.member("": TermName).asMethod.paramss StructType( - params.head.map(p => StructField(p.name.toString, schemaFor(p.typeSignature), true))) + params.head.map(p => + StructField(p.name.toString, schemaFor(p.typeSignature), nullable = true))) // Need to decide if we actually need a special type here. case t if t <:< typeOf[Array[Byte]] => BinaryType case t if t <:< typeOf[Array[_]] => @@ -58,6 +59,17 @@ object ScalaReflection { case t if t <:< typeOf[String] => StringType case t if t <:< typeOf[Timestamp] => TimestampType case t if t <:< typeOf[BigDecimal] => DecimalType + case t if t <:< typeOf[Option[_]] => + val TypeRef(_, _, Seq(optType)) = t + schemaFor(optType) + case t if t <:< typeOf[java.lang.Integer] => IntegerType + case t if t <:< typeOf[java.lang.Long] => LongType + case t if t <:< typeOf[java.lang.Double] => DoubleType + case t if t <:< typeOf[java.lang.Float] => FloatType + case t if t <:< typeOf[java.lang.Short] => ShortType + case t if t <:< typeOf[java.lang.Byte] => ByteType + case t if t <:< typeOf[java.lang.Boolean] => BooleanType + // TODO: The following datatypes could be marked as non-nullable. case t if t <:< definitions.IntTpe => IntegerType case t if t <:< definitions.LongTpe => LongType case t if t <:< definitions.DoubleTpe => DoubleType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala index 987befe8e22ee..dc83485df195c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -114,37 +114,37 @@ package object dsl { def attr = analysis.UnresolvedAttribute(s) /** Creates a new AttributeReference of type boolean */ - def boolean = AttributeReference(s, BooleanType, nullable = false)() + def boolean = AttributeReference(s, BooleanType, nullable = true)() /** Creates a new AttributeReference of type byte */ - def byte = AttributeReference(s, ByteType, nullable = false)() + def byte = AttributeReference(s, ByteType, nullable = true)() /** Creates a new AttributeReference of type short */ - def short = AttributeReference(s, ShortType, nullable = false)() + def short = AttributeReference(s, ShortType, nullable = true)() /** Creates a new AttributeReference of type int */ - def int = AttributeReference(s, IntegerType, nullable = false)() + def int = AttributeReference(s, IntegerType, nullable = true)() /** Creates a new AttributeReference of type long */ - def long = AttributeReference(s, LongType, nullable = false)() + def long = AttributeReference(s, LongType, nullable = true)() /** Creates a new AttributeReference of type float */ - def float = AttributeReference(s, FloatType, nullable = false)() + def float = AttributeReference(s, FloatType, nullable = true)() /** Creates a new AttributeReference of type double */ - def double = AttributeReference(s, DoubleType, nullable = false)() + def double = AttributeReference(s, DoubleType, nullable = true)() /** Creates a new AttributeReference of type string */ - def string = AttributeReference(s, StringType, nullable = false)() + def string = AttributeReference(s, StringType, nullable = true)() /** Creates a new AttributeReference of type decimal */ - def decimal = AttributeReference(s, DecimalType, nullable = false)() + def decimal = AttributeReference(s, DecimalType, nullable = true)() /** Creates a new AttributeReference of type timestamp */ - def timestamp = AttributeReference(s, TimestampType, nullable = false)() + def timestamp = AttributeReference(s, TimestampType, nullable = true)() /** Creates a new AttributeReference of type binary */ - def binary = AttributeReference(s, BinaryType, nullable = false)() + def binary = AttributeReference(s, BinaryType, nullable = true)() } implicit class DslAttribute(a: AttributeReference) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index 40d2b42a0cda3..0b3a4e728ec54 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -182,7 +182,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { case x: NumericType => b => x.numeric.asInstanceOf[Numeric[Any]].toFloat(b) } - def cast: Any => Any = dataType match { + private lazy val cast: Any => Any = dataType match { case StringType => castToString case BinaryType => castToBinary case DecimalType => castToDecimal 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 dd9332ada80dd..41398ff956edd 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 @@ -44,7 +44,6 @@ abstract class Expression extends TreeNode[Expression] { * - A [[expressions.Cast Cast]] or [[expressions.UnaryMinus UnaryMinus]] is foldable if its * child is foldable. */ - // TODO: Supporting more foldable expressions. For example, deterministic Hive UDFs. def foldable: Boolean = false def nullable: Boolean def references: Set[Attribute] 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 08b2f11d20f5e..d2b7685e73065 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 @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.errors.TreeNodeException +import org.apache.spark.sql.catalyst.trees abstract sealed class SortDirection case object Ascending extends SortDirection @@ -27,7 +28,10 @@ case object Descending extends SortDirection * An expression that can be used to sort a tuple. This class extends expression primarily so that * transformations over expression will descend into its child. */ -case class SortOrder(child: Expression, direction: SortDirection) extends UnaryExpression { +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/complexTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala index c947155cb701c..195ca2eb3d589 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 @@ -28,6 +28,7 @@ case class GetItem(child: Expression, ordinal: Expression) extends Expression { val children = child :: ordinal :: Nil /** `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 @@ -40,23 +41,27 @@ case class GetItem(child: Expression, ordinal: Expression) extends Expression { override def toString = s"$child[$ordinal]" override def eval(input: Row): Any = { - if (child.dataType.isInstanceOf[ArrayType]) { - val baseValue = child.eval(input).asInstanceOf[Seq[_]] - val o = ordinal.eval(input).asInstanceOf[Int] - if (baseValue == null) { - null - } else if (o >= baseValue.size || o < 0) { - null - } else { - baseValue(o) - } + val value = child.eval(input) + if (value == null) { + null } else { - val baseValue = child.eval(input).asInstanceOf[Map[Any, _]] val key = ordinal.eval(input) - if (baseValue == null) { + if (key == null) { null } else { - baseValue.get(key).orNull + if (child.dataType.isInstanceOf[ArrayType]) { + val baseValue = value.asInstanceOf[Seq[_]] + val o = key.asInstanceOf[Int] + if (o >= baseValue.size || o < 0) { + null + } else { + baseValue(o) + } + } else { + val baseValue = value.asInstanceOf[Map[Any, _]] + val key = ordinal.eval(input) + baseValue.get(key).orNull + } } } } @@ -69,7 +74,8 @@ case class GetField(child: Expression, fieldName: String) extends UnaryExpressio type EvaluatedType = Any def dataType = field.dataType - def nullable = field.nullable + override def nullable = field.nullable + override def foldable = child.foldable protected def structType = child.dataType match { case s: StructType => s 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 82c7af684459f..6ee479939d25c 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 @@ -65,8 +65,7 @@ abstract class BinaryPredicate extends BinaryExpression with Predicate { def nullable = left.nullable || right.nullable } -case class Not(child: Expression) extends Predicate with trees.UnaryNode[Expression] { - def references = child.references +case class Not(child: Expression) extends UnaryExpression with Predicate { override def foldable = child.foldable def nullable = child.nullable override def toString = s"NOT $child" 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 c0a09a16ac98d..3037d45cc6e35 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 @@ -26,6 +26,7 @@ import org.apache.spark.sql.catalyst.types._ object Optimizer extends RuleExecutor[LogicalPlan] { val batches = Batch("ConstantFolding", Once, + NullPropagation, ConstantFolding, BooleanSimplification, SimplifyFilters, @@ -85,6 +86,72 @@ object ColumnPruning extends Rule[LogicalPlan] { } } +/** + * Replaces [[catalyst.expressions.Expression Expressions]] that can be statically evaluated with + * equivalent [[catalyst.expressions.Literal Literal]] values. This rule is more specific with + * Null value propagation from bottom to top of the expression tree. + */ +object NullPropagation extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case q: LogicalPlan => q transformExpressionsUp { + case e @ Count(Literal(null, _)) => Literal(0, e.dataType) + case e @ Sum(Literal(c, _)) if c == 0 => Literal(0, e.dataType) + case e @ Average(Literal(c, _)) if c == 0 => Literal(0.0, e.dataType) + case e @ IsNull(c) if c.nullable == false => Literal(false, BooleanType) + case e @ IsNotNull(c) if c.nullable == false => Literal(true, BooleanType) + case e @ GetItem(Literal(null, _), _) => Literal(null, e.dataType) + case e @ GetItem(_, Literal(null, _)) => Literal(null, e.dataType) + case e @ GetField(Literal(null, _), _) => Literal(null, e.dataType) + case e @ Coalesce(children) => { + val newChildren = children.filter(c => c match { + case Literal(null, _) => false + case _ => true + }) + if (newChildren.length == 0) { + Literal(null, e.dataType) + } else if (newChildren.length == 1) { + newChildren(0) + } else { + Coalesce(newChildren) + } + } + case e @ If(Literal(v, _), trueValue, falseValue) => if (v == true) trueValue else falseValue + case e @ In(Literal(v, _), list) if (list.exists(c => c match { + case Literal(candidate, _) if candidate == v => true + case _ => false + })) => Literal(true, BooleanType) + case e: UnaryMinus => e.child match { + case Literal(null, _) => Literal(null, e.dataType) + case _ => e + } + case e: Cast => e.child match { + case Literal(null, _) => Literal(null, e.dataType) + case _ => e + } + case e: Not => e.child match { + case Literal(null, _) => Literal(null, e.dataType) + case _ => e + } + // Put exceptional cases above if any + case e: BinaryArithmetic => e.children match { + case Literal(null, _) :: right :: Nil => Literal(null, e.dataType) + case left :: Literal(null, _) :: Nil => Literal(null, e.dataType) + case _ => e + } + case e: BinaryComparison => e.children match { + case Literal(null, _) :: right :: Nil => Literal(null, e.dataType) + case left :: Literal(null, _) :: Nil => Literal(null, e.dataType) + case _ => e + } + case e: StringRegexExpression => e.children match { + case Literal(null, _) :: right :: Nil => Literal(null, e.dataType) + case left :: Literal(null, _) :: Nil => Literal(null, e.dataType) + case _ => e + } + } + } +} + /** * Replaces [[catalyst.expressions.Expression Expressions]] that can be statically evaluated with * equivalent [[catalyst.expressions.Literal Literal]] values. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala index 9db96f89dd03c..e32adb76fe146 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala @@ -50,18 +50,18 @@ abstract class RuleExecutor[TreeType <: TreeNode[_]] extends Logging { var curPlan = plan batches.foreach { batch => + val batchStartPlan = curPlan var iteration = 1 var lastPlan = curPlan - curPlan = batch.rules.foldLeft(curPlan) { case (plan, rule) => rule(plan) } + var continue = true // Run until fix point (or the max number of iterations as specified in the strategy. - while (iteration < batch.strategy.maxIterations && !curPlan.fastEquals(lastPlan)) { - lastPlan = curPlan + while (continue) { curPlan = batch.rules.foldLeft(curPlan) { case (plan, rule) => val result = rule(plan) if (!result.fastEquals(plan)) { - logger.debug( + logger.trace( s""" |=== Applying Rule ${rule.ruleName} === |${sideBySide(plan.treeString, result.treeString).mkString("\n")} @@ -71,6 +71,26 @@ abstract class RuleExecutor[TreeType <: TreeNode[_]] extends Logging { result } iteration += 1 + if (iteration > batch.strategy.maxIterations) { + logger.info(s"Max iterations ($iteration) reached for batch ${batch.name}") + continue = false + } + + if (curPlan.fastEquals(lastPlan)) { + logger.trace(s"Fixed point reached for batch ${batch.name} after $iteration iterations.") + continue = false + } + lastPlan = curPlan + } + + if (!batchStartPlan.fastEquals(curPlan)) { + logger.debug( + s""" + |=== Result of Batch ${batch.name} === + |${sideBySide(plan.treeString, curPlan.treeString).mkString("\n")} + """.stripMargin) + } else { + logger.trace(s"Batch ${batch.name} has no effect.") } } 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 d287ad73b9e9f..91605d0a260e5 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 @@ -108,9 +108,7 @@ class ExpressionEvaluationSuite extends FunSuite { truthTable.foreach { case (l,r,answer) => val expr = op(Literal(l, BooleanType), Literal(r, BooleanType)) - val result = expr.eval(null) - if (result != answer) - fail(s"$expr should not evaluate to $result, expected: $answer") + checkEvaluation(expr, answer) } } } @@ -131,6 +129,7 @@ class ExpressionEvaluationSuite extends FunSuite { test("LIKE literal Regular Expression") { checkEvaluation(Literal(null, StringType).like("a"), null) + checkEvaluation(Literal("a", StringType).like(Literal(null, StringType)), null) checkEvaluation(Literal(null, StringType).like(Literal(null, StringType)), null) checkEvaluation("abdef" like "abdef", true) checkEvaluation("a_%b" like "a\\__b", true) @@ -159,9 +158,14 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation("abc" like regEx, true, new GenericRow(Array[Any]("a%"))) checkEvaluation("abc" like regEx, false, new GenericRow(Array[Any]("b%"))) checkEvaluation("abc" like regEx, false, new GenericRow(Array[Any]("bc%"))) + + checkEvaluation(Literal(null, StringType) like regEx, null, new GenericRow(Array[Any]("bc%"))) } test("RLIKE literal Regular Expression") { + checkEvaluation(Literal(null, StringType) rlike "abdef", null) + checkEvaluation("abdef" rlike Literal(null, StringType), null) + checkEvaluation(Literal(null, StringType) rlike Literal(null, StringType), null) checkEvaluation("abdef" rlike "abdef", true) checkEvaluation("abbbbc" rlike "a.*c", true) @@ -257,6 +261,8 @@ class ExpressionEvaluationSuite extends FunSuite { assert(("abcdef" cast DecimalType).nullable === true) assert(("abcdef" cast DoubleType).nullable === true) assert(("abcdef" cast FloatType).nullable === true) + + checkEvaluation(Cast(Literal(null, IntegerType), ShortType), null) } test("timestamp") { @@ -287,5 +293,108 @@ class ExpressionEvaluationSuite extends FunSuite { // A test for higher precision than millis checkEvaluation(Cast(Cast(0.00000001, TimestampType), DoubleType), 0.00000001) } + + test("null checking") { + val row = new GenericRow(Array[Any]("^Ba*n", null, true, null)) + val c1 = 'a.string.at(0) + val c2 = 'a.string.at(1) + val c3 = 'a.boolean.at(2) + val c4 = 'a.boolean.at(3) + + checkEvaluation(IsNull(c1), false, row) + checkEvaluation(IsNotNull(c1), true, row) + + checkEvaluation(IsNull(c2), true, row) + checkEvaluation(IsNotNull(c2), false, row) + + checkEvaluation(IsNull(Literal(1, ShortType)), false) + checkEvaluation(IsNotNull(Literal(1, ShortType)), true) + + checkEvaluation(IsNull(Literal(null, ShortType)), true) + checkEvaluation(IsNotNull(Literal(null, ShortType)), false) + + checkEvaluation(Coalesce(c1 :: c2 :: Nil), "^Ba*n", row) + checkEvaluation(Coalesce(Literal(null, StringType) :: Nil), null, row) + checkEvaluation(Coalesce(Literal(null, StringType) :: c1 :: c2 :: Nil), "^Ba*n", row) + + checkEvaluation(If(c3, Literal("a", StringType), Literal("b", StringType)), "a", row) + checkEvaluation(If(c3, c1, c2), "^Ba*n", row) + checkEvaluation(If(c4, c2, c1), "^Ba*n", row) + checkEvaluation(If(Literal(null, BooleanType), c2, c1), "^Ba*n", row) + checkEvaluation(If(Literal(true, BooleanType), c1, c2), "^Ba*n", row) + checkEvaluation(If(Literal(false, BooleanType), c2, c1), "^Ba*n", row) + checkEvaluation(If(Literal(false, BooleanType), + Literal("a", StringType), Literal("b", StringType)), "b", row) + + checkEvaluation(In(c1, c1 :: c2 :: Nil), true, row) + checkEvaluation(In(Literal("^Ba*n", StringType), + Literal("^Ba*n", StringType) :: Nil), true, row) + checkEvaluation(In(Literal("^Ba*n", StringType), + Literal("^Ba*n", StringType) :: c2 :: Nil), true, row) + } + + test("complex type") { + val row = new GenericRow(Array[Any]( + "^Ba*n", // 0 + null.asInstanceOf[String], // 1 + new GenericRow(Array[Any]("aa", "bb")), // 2 + Map("aa"->"bb"), // 3 + Seq("aa", "bb") // 4 + )) + + val typeS = StructType( + StructField("a", StringType, true) :: StructField("b", StringType, true) :: Nil + ) + val typeMap = MapType(StringType, StringType) + val typeArray = ArrayType(StringType) + + checkEvaluation(GetItem(BoundReference(3, AttributeReference("c", typeMap)()), + Literal("aa")), "bb", row) + checkEvaluation(GetItem(Literal(null, typeMap), Literal("aa")), null, row) + checkEvaluation(GetItem(Literal(null, typeMap), Literal(null, StringType)), null, row) + checkEvaluation(GetItem(BoundReference(3, AttributeReference("c", typeMap)()), + Literal(null, StringType)), null, row) + + checkEvaluation(GetItem(BoundReference(4, AttributeReference("c", typeArray)()), + Literal(1)), "bb", row) + checkEvaluation(GetItem(Literal(null, typeArray), Literal(1)), null, row) + checkEvaluation(GetItem(Literal(null, typeArray), Literal(null, IntegerType)), null, row) + checkEvaluation(GetItem(BoundReference(4, AttributeReference("c", typeArray)()), + Literal(null, IntegerType)), null, row) + + checkEvaluation(GetField(BoundReference(2, AttributeReference("c", typeS)()), "a"), "aa", row) + checkEvaluation(GetField(Literal(null, typeS), "a"), null, row) + } + + test("arithmetic") { + val row = new GenericRow(Array[Any](1, 2, 3, null)) + val c1 = 'a.int.at(0) + val c2 = 'a.int.at(1) + val c3 = 'a.int.at(2) + val c4 = 'a.int.at(3) + + checkEvaluation(UnaryMinus(c1), -1, row) + checkEvaluation(UnaryMinus(Literal(100, IntegerType)), -100) + + checkEvaluation(Add(c1, c4), null, row) + checkEvaluation(Add(c1, c2), 3, row) + checkEvaluation(Add(c1, Literal(null, IntegerType)), null, row) + checkEvaluation(Add(Literal(null, IntegerType), c2), null, row) + checkEvaluation(Add(Literal(null, IntegerType), Literal(null, IntegerType)), null, row) + } + + test("BinaryComparison") { + val row = new GenericRow(Array[Any](1, 2, 3, null)) + val c1 = 'a.int.at(0) + val c2 = 'a.int.at(1) + val c3 = 'a.int.at(2) + val c4 = 'a.int.at(3) + + checkEvaluation(LessThan(c1, c4), null, row) + checkEvaluation(LessThan(c1, c2), true, row) + checkEvaluation(LessThan(c1, Literal(null, IntegerType)), null, row) + checkEvaluation(LessThan(Literal(null, IntegerType), c2), null, row) + checkEvaluation(LessThan(Literal(null, IntegerType), Literal(null, IntegerType)), null, row) + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/rdd/VectorRDDsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ExpressionOptimizationSuite.scala similarity index 51% rename from mllib/src/test/scala/org/apache/spark/mllib/rdd/VectorRDDsSuite.scala rename to sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ExpressionOptimizationSuite.scala index 692f025e959ae..890d6289b9dfb 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/rdd/VectorRDDsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ExpressionOptimizationSuite.scala @@ -15,19 +15,22 @@ * limitations under the License. */ -package org.apache.spark.mllib.rdd +package org.apache.spark.sql.catalyst.optimizer -import org.scalatest.FunSuite +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.util.LocalSparkContext - -class VectorRDDsSuite extends FunSuite with LocalSparkContext { - - test("from array rdd") { - val data = Seq(Array(1.0, 2.0), Array(3.0, 4.0)) - val arrayRdd = sc.parallelize(data, 2) - val vectorRdd = VectorRDDs.fromArrayRDD(arrayRdd) - assert(arrayRdd.collect().map(v => Vectors.dense(v)) === vectorRdd.collect()) +/** + * Overrides our expression evaluation tests and reruns them after optimization has occured. This + * is to ensure that constant folding and other optimizations do not break anything. + */ +class ExpressionOptimizationSuite extends ExpressionEvaluationSuite { + override def checkEvaluation( + expression: Expression, + expected: Any, + inputRow: Row = EmptyRow): Unit = { + val plan = Project(Alias(expression, s"Optimized($expression)")() :: Nil, NoRelation) + val optimizedPlan = Optimizer(plan) + super.checkEvaluation(optimizedPlan.expressions.head, expected, inputRow) } -} +} \ No newline at end of file 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 e25201a6c1775..bfebfa0c28c52 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 @@ -162,8 +162,11 @@ class SQLContext(@transient val sparkContext: SparkContext) /** Caches the specified table in-memory. */ def cacheTable(tableName: String): Unit = { val currentTable = catalog.lookupRelation(None, tableName) + val useCompression = + sparkContext.conf.getBoolean("spark.sql.inMemoryColumnarStorage.compressed", false) val asInMemoryRelation = - InMemoryColumnarTableScan(currentTable.output, executePlan(currentTable).executedPlan) + InMemoryColumnarTableScan( + currentTable.output, executePlan(currentTable).executedPlan, useCompression) catalog.registerTable(None, tableName, SparkLogicalPlan(asInMemoryRelation)) } @@ -173,7 +176,7 @@ class SQLContext(@transient val sparkContext: SparkContext) EliminateAnalysisOperators(catalog.lookupRelation(None, tableName)) match { // This is kind of a hack to make sure that if this was just an RDD registered as a table, // we reregister the RDD as a table. - case SparkLogicalPlan(inMem @ InMemoryColumnarTableScan(_, e: ExistingRdd)) => + case SparkLogicalPlan(inMem @ InMemoryColumnarTableScan(_, e: ExistingRdd, _)) => inMem.cachedColumnBuffers.unpersist() catalog.unregisterTable(None, tableName) catalog.registerTable(None, tableName, SparkLogicalPlan(e)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index ca6e0a696405a..34200be3ac955 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala @@ -19,14 +19,16 @@ package org.apache.spark.sql import net.razorvine.pickle.Pickler -import org.apache.spark.{Dependency, OneToOneDependency, Partition, TaskContext} +import org.apache.spark.{Dependency, OneToOneDependency, Partition, Partitioner, TaskContext} import org.apache.spark.annotation.{AlphaComponent, Experimental} import org.apache.spark.rdd.RDD +import org.apache.spark.sql.api.java.JavaSchemaRDD import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.{Inner, JoinType} import org.apache.spark.sql.catalyst.types.BooleanType +import org.apache.spark.sql.execution.{ExistingRdd, SparkLogicalPlan} import org.apache.spark.api.java.JavaRDD import java.util.{Map => JMap} @@ -133,7 +135,7 @@ class SchemaRDD( new SchemaRDD(sqlContext, Project(exprs, logicalPlan)) /** - * Filters the ouput, only returning those rows where `condition` evaluates to true. + * Filters the output, only returning those rows where `condition` evaluates to true. * * {{{ * schemaRDD.where('a === 'b) @@ -151,9 +153,9 @@ class SchemaRDD( * * @param otherPlan the [[SchemaRDD]] that should be joined with this one. * @param joinType One of `Inner`, `LeftOuter`, `RightOuter`, or `FullOuter`. Defaults to `Inner.` - * @param on An optional condition for the join operation. This is equivilent to the `ON` + * @param on An optional condition for the join operation. This is equivalent to the `ON` * clause in standard SQL. In the case of `Inner` joins, specifying a - * `condition` is equivilent to adding `where` clauses after the `join`. + * `condition` is equivalent to adding `where` clauses after the `join`. * * @group Query */ @@ -195,7 +197,7 @@ class SchemaRDD( /** * Applies a qualifier to the attributes of this relation. Can be used to disambiguate attributes - * with the same name, for example, when peforming self-joins. + * with the same name, for example, when performing self-joins. * * {{{ * val x = schemaRDD.where('a === 1).as('x) @@ -296,6 +298,13 @@ class SchemaRDD( */ def toSchemaRDD = this + /** + * Returns this RDD as a JavaSchemaRDD. + * + * @group schema + */ + def toJavaSchemaRDD: JavaSchemaRDD = new JavaSchemaRDD(sqlContext, logicalPlan) + private[sql] def javaToPython: JavaRDD[Array[Byte]] = { val fieldNames: Seq[String] = this.queryExecution.analyzed.output.map(_.name) this.mapPartitions { iter => @@ -314,4 +323,60 @@ class SchemaRDD( } } } + + /** + * Creates SchemaRDD by applying own schema to derived RDD. Typically used to wrap return value + * of base RDD functions that do not change schema. + * + * @param rdd RDD derived from this one and has same schema + * + * @group schema + */ + private def applySchema(rdd: RDD[Row]): SchemaRDD = { + new SchemaRDD(sqlContext, SparkLogicalPlan(ExistingRdd(logicalPlan.output, rdd))) + } + + // ======================================================================= + // Base RDD functions that do NOT change schema + // ======================================================================= + + // Transformations (return a new RDD) + + override def coalesce(numPartitions: Int, shuffle: Boolean = false) + (implicit ord: Ordering[Row] = null): SchemaRDD = + applySchema(super.coalesce(numPartitions, shuffle)(ord)) + + override def distinct(): SchemaRDD = + applySchema(super.distinct()) + + override def distinct(numPartitions: Int) + (implicit ord: Ordering[Row] = null): SchemaRDD = + applySchema(super.distinct(numPartitions)(ord)) + + override def filter(f: Row => Boolean): SchemaRDD = + applySchema(super.filter(f)) + + override def intersection(other: RDD[Row]): SchemaRDD = + applySchema(super.intersection(other)) + + override def intersection(other: RDD[Row], partitioner: Partitioner) + (implicit ord: Ordering[Row] = null): SchemaRDD = + applySchema(super.intersection(other, partitioner)(ord)) + + override def intersection(other: RDD[Row], numPartitions: Int): SchemaRDD = + applySchema(super.intersection(other, numPartitions)) + + override def repartition(numPartitions: Int) + (implicit ord: Ordering[Row] = null): SchemaRDD = + applySchema(super.repartition(numPartitions)(ord)) + + override def subtract(other: RDD[Row]): SchemaRDD = + applySchema(super.subtract(other)) + + override def subtract(other: RDD[Row], numPartitions: Int): SchemaRDD = + applySchema(super.subtract(other, numPartitions)) + + override def subtract(other: RDD[Row], p: Partitioner) + (implicit ord: Ordering[Row] = null): SchemaRDD = + applySchema(super.subtract(other, p)(ord)) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala index a7347088794a8..57facbe10fc96 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala @@ -132,6 +132,14 @@ class JavaSQLContext(sparkContext: JavaSparkContext) { case c: Class[_] if c == java.lang.Byte.TYPE => ByteType case c: Class[_] if c == java.lang.Float.TYPE => FloatType case c: Class[_] if c == java.lang.Boolean.TYPE => BooleanType + + case c: Class[_] if c == classOf[java.lang.Short] => ShortType + case c: Class[_] if c == classOf[java.lang.Integer] => IntegerType + case c: Class[_] if c == classOf[java.lang.Long] => LongType + case c: Class[_] if c == classOf[java.lang.Double] => DoubleType + case c: Class[_] if c == classOf[java.lang.Byte] => ByteType + case c: Class[_] if c == classOf[java.lang.Float] => FloatType + case c: Class[_] if c == classOf[java.lang.Boolean] => BooleanType } // TODO: Nullability could be stricter. AttributeReference(property.getName, dataType, nullable = true)() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala index d43d672938f51..22f57b758dd02 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala @@ -17,10 +17,13 @@ package org.apache.spark.sql.api.java +import org.apache.spark.Partitioner import org.apache.spark.api.java.{JavaRDDLike, JavaRDD} +import org.apache.spark.api.java.function.{Function => JFunction} import org.apache.spark.sql.{SQLContext, SchemaRDD, SchemaRDDLike} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel /** * An RDD of [[Row]] objects that is returned as the result of a Spark SQL query. In addition to @@ -45,4 +48,141 @@ class JavaSchemaRDD( override def wrapRDD(rdd: RDD[Row]): JavaRDD[Row] = JavaRDD.fromRDD(rdd) val rdd = baseSchemaRDD.map(new Row(_)) + + override def toString: String = baseSchemaRDD.toString + + // ======================================================================= + // Base RDD functions that do NOT change schema + // ======================================================================= + + // Common RDD functions + + /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ + def cache(): JavaSchemaRDD = { + baseSchemaRDD.cache() + this + } + + /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */ + def persist(): JavaSchemaRDD = { + baseSchemaRDD.persist() + this + } + + /** + * Set this RDD's storage level to persist its values across operations after the first time + * it is computed. This can only be used to assign a new storage level if the RDD does not + * have a storage level set yet.. + */ + def persist(newLevel: StorageLevel): JavaSchemaRDD = { + baseSchemaRDD.persist(newLevel) + this + } + + /** + * Mark the RDD as non-persistent, and remove all blocks for it from memory and disk. + * + * @param blocking Whether to block until all blocks are deleted. + * @return This RDD. + */ + def unpersist(blocking: Boolean = true): JavaSchemaRDD = { + baseSchemaRDD.unpersist(blocking) + this + } + + /** Assign a name to this RDD */ + def setName(name: String): JavaSchemaRDD = { + baseSchemaRDD.setName(name) + this + } + + // Transformations (return a new RDD) + + /** + * Return a new RDD that is reduced into `numPartitions` partitions. + */ + def coalesce(numPartitions: Int, shuffle: Boolean = false): JavaSchemaRDD = + baseSchemaRDD.coalesce(numPartitions, shuffle).toJavaSchemaRDD + + /** + * Return a new RDD containing the distinct elements in this RDD. + */ + def distinct(): JavaSchemaRDD = + baseSchemaRDD.distinct().toJavaSchemaRDD + + /** + * Return a new RDD containing the distinct elements in this RDD. + */ + def distinct(numPartitions: Int): JavaSchemaRDD = + baseSchemaRDD.distinct(numPartitions).toJavaSchemaRDD + + /** + * Return a new RDD containing only the elements that satisfy a predicate. + */ + def filter(f: JFunction[Row, java.lang.Boolean]): JavaSchemaRDD = + baseSchemaRDD.filter(x => f.call(new Row(x)).booleanValue()).toJavaSchemaRDD + + /** + * Return the intersection of this RDD and another one. The output will not contain any + * duplicate elements, even if the input RDDs did. + * + * Note that this method performs a shuffle internally. + */ + def intersection(other: JavaSchemaRDD): JavaSchemaRDD = + this.baseSchemaRDD.intersection(other.baseSchemaRDD).toJavaSchemaRDD + + /** + * Return the intersection of this RDD and another one. The output will not contain any + * duplicate elements, even if the input RDDs did. + * + * Note that this method performs a shuffle internally. + * + * @param partitioner Partitioner to use for the resulting RDD + */ + def intersection(other: JavaSchemaRDD, partitioner: Partitioner): JavaSchemaRDD = + this.baseSchemaRDD.intersection(other.baseSchemaRDD, partitioner).toJavaSchemaRDD + + /** + * Return the intersection of this RDD and another one. The output will not contain any + * duplicate elements, even if the input RDDs did. Performs a hash partition across the cluster + * + * Note that this method performs a shuffle internally. + * + * @param numPartitions How many partitions to use in the resulting RDD + */ + def intersection(other: JavaSchemaRDD, numPartitions: Int): JavaSchemaRDD = + this.baseSchemaRDD.intersection(other.baseSchemaRDD, numPartitions).toJavaSchemaRDD + + /** + * Return a new RDD that has exactly `numPartitions` partitions. + * + * Can increase or decrease the level of parallelism in this RDD. Internally, this uses + * a shuffle to redistribute data. + * + * If you are decreasing the number of partitions in this RDD, consider using `coalesce`, + * which can avoid performing a shuffle. + */ + def repartition(numPartitions: Int): JavaSchemaRDD = + baseSchemaRDD.repartition(numPartitions).toJavaSchemaRDD + + /** + * Return an RDD with the elements from `this` that are not in `other`. + * + * Uses `this` partitioner/partition size, because even if `other` is huge, the resulting + * RDD will be <= us. + */ + def subtract(other: JavaSchemaRDD): JavaSchemaRDD = + this.baseSchemaRDD.subtract(other.baseSchemaRDD).toJavaSchemaRDD + + /** + * Return an RDD with the elements from `this` that are not in `other`. + */ + def subtract(other: JavaSchemaRDD, numPartitions: Int): JavaSchemaRDD = + this.baseSchemaRDD.subtract(other.baseSchemaRDD, numPartitions).toJavaSchemaRDD + + /** + * Return an RDD with the elements from `this` that are not in `other`. + */ + def subtract(other: JavaSchemaRDD, p: Partitioner): JavaSchemaRDD = + this.baseSchemaRDD.subtract(other.baseSchemaRDD, p).toJavaSchemaRDD } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/Row.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/Row.scala index 362fe769581d7..9b0dd2176149b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/Row.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/Row.scala @@ -22,7 +22,7 @@ import org.apache.spark.sql.catalyst.expressions.{Row => ScalaRow} /** * A result row from a SparkSQL query. */ -class Row(row: ScalaRow) extends Serializable { +class Row(private[spark] val row: ScalaRow) extends Serializable { /** Returns the number of columns present in this Row. */ def length: Int = row.length diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala index 048ee66bff44b..4be048cd742d6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala @@ -28,7 +28,7 @@ private[sql] trait ColumnBuilder { /** * Initializes with an approximate lower bound on the expected number of elements in this column. */ - def initialize(initialSize: Int, columnName: String = "") + def initialize(initialSize: Int, columnName: String = "", useCompression: Boolean = false) /** * Appends `row(ordinal)` to the column builder. @@ -55,7 +55,11 @@ private[sql] class BasicColumnBuilder[T <: DataType, JvmType]( protected var buffer: ByteBuffer = _ - override def initialize(initialSize: Int, columnName: String = "") = { + override def initialize( + initialSize: Int, + columnName: String = "", + useCompression: Boolean = false) = { + val size = if (initialSize == 0) DEFAULT_INITIAL_BUFFER_SIZE else initialSize this.columnName = columnName @@ -130,7 +134,12 @@ private[sql] object ColumnBuilder { } } - def apply(typeId: Int, initialSize: Int = 0, columnName: String = ""): ColumnBuilder = { + def apply( + typeId: Int, + initialSize: Int = 0, + columnName: String = "", + useCompression: Boolean = false): ColumnBuilder = { + val builder = (typeId match { case INT.typeId => new IntColumnBuilder case LONG.typeId => new LongColumnBuilder @@ -144,7 +153,7 @@ private[sql] object ColumnBuilder { case GENERIC.typeId => new GenericColumnBuilder }).asInstanceOf[ColumnBuilder] - builder.initialize(initialSize, columnName) + builder.initialize(initialSize, columnName, useCompression) builder } } 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 8a24733047423..fdf28e1bb1261 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 @@ -20,8 +20,12 @@ package org.apache.spark.sql.columnar import org.apache.spark.sql.catalyst.expressions.{GenericMutableRow, Attribute} import org.apache.spark.sql.execution.{SparkPlan, LeafNode} import org.apache.spark.sql.Row +import org.apache.spark.SparkConf -private[sql] case class InMemoryColumnarTableScan(attributes: Seq[Attribute], child: SparkPlan) +private[sql] case class InMemoryColumnarTableScan( + attributes: Seq[Attribute], + child: SparkPlan, + useCompression: Boolean) extends LeafNode { override def output: Seq[Attribute] = attributes @@ -30,7 +34,7 @@ private[sql] case class InMemoryColumnarTableScan(attributes: Seq[Attribute], ch val output = child.output val cached = child.execute().mapPartitions { iterator => val columnBuilders = output.map { attribute => - ColumnBuilder(ColumnType(attribute.dataType).typeId, 0, attribute.name) + ColumnBuilder(ColumnType(attribute.dataType).typeId, 0, attribute.name, useCompression) }.toArray var row: Row = null diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala index 2a3b6fc1e46d3..d008806eedbe1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala @@ -40,12 +40,12 @@ private[sql] trait NullableColumnBuilder extends ColumnBuilder { private var pos: Int = _ private var nullCount: Int = _ - abstract override def initialize(initialSize: Int, columnName: String) { + abstract override def initialize(initialSize: Int, columnName: String, useCompression: Boolean) { nulls = ByteBuffer.allocate(1024) nulls.order(ByteOrder.nativeOrder()) pos = 0 nullCount = 0 - super.initialize(initialSize, columnName) + super.initialize(initialSize, columnName, useCompression) } abstract override def appendFrom(row: Row, ordinal: Int) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnAccessor.scala index 878cb84de106f..b4120a3d4368b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnAccessor.scala @@ -32,5 +32,7 @@ private[sql] trait CompressibleColumnAccessor[T <: NativeType] extends ColumnAcc decoder = CompressionScheme(underlyingBuffer.getInt()).decoder(buffer, columnType) } - abstract override def extractSingle(buffer: ByteBuffer): T#JvmType = decoder.next() + abstract override def hasNext = super.hasNext || decoder.hasNext + + override def extractSingle(buffer: ByteBuffer): T#JvmType = decoder.next() } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala index 0f808f68f2eec..4c6675c3c87bf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala @@ -47,7 +47,17 @@ private[sql] trait CompressibleColumnBuilder[T <: NativeType] import CompressionScheme._ - val compressionEncoders = schemes.filter(_.supports(columnType)).map(_.encoder[T]) + var compressionEncoders: Seq[Encoder[T]] = _ + + abstract override def initialize(initialSize: Int, columnName: String, useCompression: Boolean) { + compressionEncoders = + if (useCompression) { + schemes.filter(_.supports(columnType)).map(_.encoder[T]) + } else { + Seq(PassThrough.encoder) + } + super.initialize(initialSize, columnName, useCompression) + } protected def isWorthCompressing(encoder: Encoder[T]) = { encoder.compressionRatio < 0.8 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala index 800009d3195e1..8cf9ec74ca2de 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala @@ -157,7 +157,7 @@ private[sql] case object RunLengthEncoding extends CompressionScheme { currentValue } - override def hasNext = buffer.hasRemaining + override def hasNext = valueCount < run || buffer.hasRemaining } } 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 50124dd407447..235a9b1692460 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 @@ -77,7 +77,7 @@ case class SparkLogicalPlan(alreadyPlanned: SparkPlan) SparkLogicalPlan( alreadyPlanned match { case ExistingRdd(output, rdd) => ExistingRdd(output.map(_.newInstance), rdd) - case scan @ InMemoryColumnarTableScan(output, child) => + case scan @ InMemoryColumnarTableScan(output, _, _) => scan.copy(attributes = output.map(_.newInstance)) case _ => sys.error("Multiple instance of the same relation detected.") }).asInstanceOf[this.type] 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 d807187a5ffb8..8969794c69933 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 @@ -164,6 +164,7 @@ case class Sort( @DeveloperApi object ExistingRdd { def convertToCatalyst(a: Any): Any = a match { + case o: Option[_] => o.orNull case s: Seq[Any] => s.map(convertToCatalyst) case p: Product => new GenericRow(p.productIterator.map(convertToCatalyst).toArray) case other => other @@ -180,7 +181,7 @@ object ExistingRdd { bufferedIterator.map { r => var i = 0 while (i < mutableRow.length) { - mutableRow(i) = r.productElement(i) + mutableRow(i) = convertToCatalyst(r.productElement(i)) i += 1 } 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 1cbf973c34917..f2934da9a031d 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 @@ -36,6 +36,24 @@ case class ReflectData( timestampField: Timestamp, seqInt: Seq[Int]) +case class NullReflectData( + intField: java.lang.Integer, + longField: java.lang.Long, + floatField: java.lang.Float, + doubleField: java.lang.Double, + shortField: java.lang.Short, + byteField: java.lang.Byte, + booleanField: java.lang.Boolean) + +case class OptionalReflectData( + intField: Option[Int], + longField: Option[Long], + floatField: Option[Float], + doubleField: Option[Double], + shortField: Option[Short], + byteField: Option[Byte], + booleanField: Option[Boolean]) + case class ReflectBinary(data: Array[Byte]) class ScalaReflectionRelationSuite extends FunSuite { @@ -48,6 +66,22 @@ class ScalaReflectionRelationSuite extends FunSuite { assert(sql("SELECT * FROM reflectData").collect().head === data.productIterator.toSeq) } + test("query case class RDD with nulls") { + val data = NullReflectData(null, null, null, null, null, null, null) + val rdd = sparkContext.parallelize(data :: Nil) + rdd.registerAsTable("reflectNullData") + + assert(sql("SELECT * FROM reflectNullData").collect().head === Seq.fill(7)(null)) + } + + test("query case class RDD with Nones") { + val data = OptionalReflectData(None, None, None, None, None, None, None) + val rdd = sparkContext.parallelize(data :: Nil) + rdd.registerAsTable("reflectOptionalData") + + assert(sql("SELECT * FROM reflectOptionalData").collect().head === Seq.fill(7)(null)) + } + // Equality is broken for Arrays, so we test that separately. test("query binary data") { val rdd = sparkContext.parallelize(ReflectBinary(Array[Byte](1)) :: Nil) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala index 002b7f0adafab..b5973c0f51be8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala @@ -73,4 +73,15 @@ object TestData { ArrayData(Seq(1,2,3), Seq(Seq(1,2,3))) :: ArrayData(Seq(2,3,4), Seq(Seq(2,3,4))) :: Nil) arrayData.registerAsTable("arrayData") + + case class StringData(s: String) + val repeatedData = + TestSQLContext.sparkContext.parallelize(List.fill(2)(StringData("test"))) + repeatedData.registerAsTable("repeatedData") + + val nullableRepeatedData = + TestSQLContext.sparkContext.parallelize( + List.fill(2)(StringData(null)) ++ + List.fill(2)(StringData("test"))) + nullableRepeatedData.registerAsTable("nullableRepeatedData") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/api/java/JavaSQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/api/java/JavaSQLSuite.scala index def0e046a3831..9fff7222fe840 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/api/java/JavaSQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/api/java/JavaSQLSuite.scala @@ -35,6 +35,17 @@ class PersonBean extends Serializable { var age: Int = _ } +class AllTypesBean extends Serializable { + @BeanProperty var stringField: String = _ + @BeanProperty var intField: java.lang.Integer = _ + @BeanProperty var longField: java.lang.Long = _ + @BeanProperty var floatField: java.lang.Float = _ + @BeanProperty var doubleField: java.lang.Double = _ + @BeanProperty var shortField: java.lang.Short = _ + @BeanProperty var byteField: java.lang.Byte = _ + @BeanProperty var booleanField: java.lang.Boolean = _ +} + class JavaSQLSuite extends FunSuite { val javaCtx = new JavaSparkContext(TestSQLContext.sparkContext) val javaSqlCtx = new JavaSQLContext(javaCtx) @@ -50,4 +61,54 @@ class JavaSQLSuite extends FunSuite { schemaRDD.registerAsTable("people") javaSqlCtx.sql("SELECT * FROM people").collect() } + + test("all types in JavaBeans") { + val bean = new AllTypesBean + bean.setStringField("") + bean.setIntField(0) + bean.setLongField(0) + bean.setFloatField(0.0F) + bean.setDoubleField(0.0) + bean.setShortField(0.toShort) + bean.setByteField(0.toByte) + bean.setBooleanField(false) + + val rdd = javaCtx.parallelize(bean :: Nil) + val schemaRDD = javaSqlCtx.applySchema(rdd, classOf[AllTypesBean]) + schemaRDD.registerAsTable("allTypes") + + assert( + javaSqlCtx.sql( + """ + |SELECT stringField, intField, longField, floatField, doubleField, shortField, byteField, + | booleanField + |FROM allTypes + """.stripMargin).collect.head.row === + Seq("", 0, 0L, 0F, 0.0, 0.toShort, 0.toByte, false)) + } + + test("all types null in JavaBeans") { + val bean = new AllTypesBean + bean.setStringField(null) + bean.setIntField(null) + bean.setLongField(null) + bean.setFloatField(null) + bean.setDoubleField(null) + bean.setShortField(null) + bean.setByteField(null) + bean.setBooleanField(null) + + val rdd = javaCtx.parallelize(bean :: Nil) + val schemaRDD = javaSqlCtx.applySchema(rdd, classOf[AllTypesBean]) + schemaRDD.registerAsTable("allTypes") + + assert( + javaSqlCtx.sql( + """ + |SELECT stringField, intField, longField, floatField, doubleField, shortField, byteField, + | booleanField + |FROM allTypes + """.stripMargin).collect.head.row === + Seq.fill(8)(null)) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala index 325173cf95fdf..71be41056768f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala @@ -21,11 +21,12 @@ import java.nio.ByteBuffer import org.scalatest.FunSuite +import org.apache.spark.sql.Logging import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.columnar.ColumnarTestUtils._ import org.apache.spark.sql.execution.SparkSqlSerializer -class ColumnTypeSuite extends FunSuite { +class ColumnTypeSuite extends FunSuite with Logging { val DEFAULT_BUFFER_SIZE = 512 test("defaultSize") { @@ -163,7 +164,7 @@ class ColumnTypeSuite extends FunSuite { buffer.rewind() seq.foreach { expected => - println("buffer = " + buffer + ", expected = " + expected) + logger.info("buffer = " + buffer + ", expected = " + expected) val extracted = columnType.extract(buffer) assert( expected === extracted, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala index 16a13b8a74960..31c5dfba92954 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala @@ -28,14 +28,14 @@ class InMemoryColumnarQuerySuite extends QueryTest { test("simple columnar query") { val plan = TestSQLContext.executePlan(testData.logicalPlan).executedPlan - val scan = SparkLogicalPlan(InMemoryColumnarTableScan(plan.output, plan)) + val scan = SparkLogicalPlan(InMemoryColumnarTableScan(plan.output, plan, true)) checkAnswer(scan, testData.collect().toSeq) } test("projection") { val plan = TestSQLContext.executePlan(testData.select('value, 'key).logicalPlan).executedPlan - val scan = SparkLogicalPlan(InMemoryColumnarTableScan(plan.output, plan)) + val scan = SparkLogicalPlan(InMemoryColumnarTableScan(plan.output, plan, true)) checkAnswer(scan, testData.collect().map { case Row(key: Int, value: String) => value -> key @@ -44,9 +44,33 @@ class InMemoryColumnarQuerySuite extends QueryTest { test("SPARK-1436 regression: in-memory columns must be able to be accessed multiple times") { val plan = TestSQLContext.executePlan(testData.logicalPlan).executedPlan - val scan = SparkLogicalPlan(InMemoryColumnarTableScan(plan.output, plan)) + val scan = SparkLogicalPlan(InMemoryColumnarTableScan(plan.output, plan, true)) checkAnswer(scan, testData.collect().toSeq) checkAnswer(scan, testData.collect().toSeq) } + + test("SPARK-1678 regression: compression must not lose repeated values") { + checkAnswer( + sql("SELECT * FROM repeatedData"), + repeatedData.collect().toSeq) + + TestSQLContext.cacheTable("repeatedData") + + checkAnswer( + sql("SELECT * FROM repeatedData"), + repeatedData.collect().toSeq) + } + + test("with null values") { + checkAnswer( + sql("SELECT * FROM nullableRepeatedData"), + nullableRepeatedData.collect().toSeq) + + TestSQLContext.cacheTable("nullableRepeatedData") + + checkAnswer( + sql("SELECT * FROM nullableRepeatedData"), + nullableRepeatedData.collect().toSeq) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/BooleanBitSetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/BooleanBitSetSuite.scala index a754f98f7fbf1..93259a19b9fe7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/BooleanBitSetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/BooleanBitSetSuite.scala @@ -72,7 +72,12 @@ class BooleanBitSetSuite extends FunSuite { buffer.rewind().position(headerSize + 4) val decoder = BooleanBitSet.decoder(buffer, BOOLEAN) - values.foreach(expectResult(_, "Wrong decoded value")(decoder.next())) + if (values.nonEmpty) { + values.foreach { + assert(decoder.hasNext) + expectResult(_, "Wrong decoded value")(decoder.next()) + } + } assert(!decoder.hasNext) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala index eab27987e08ea..198dcd8819341 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala @@ -98,8 +98,11 @@ class DictionaryEncodingSuite extends FunSuite { val decoder = DictionaryEncoding.decoder(buffer, columnType) - inputSeq.foreach { i => - expectResult(values(i), "Wrong decoded value")(decoder.next()) + if (inputSeq.nonEmpty) { + inputSeq.foreach { i => + assert(decoder.hasNext) + expectResult(values(i), "Wrong decoded value")(decoder.next()) + } } assert(!decoder.hasNext) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala index ce419ca7269ba..46af6e001c633 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala @@ -96,7 +96,12 @@ class IntegralDeltaSuite extends FunSuite { buffer.rewind().position(headerSize + 4) val decoder = scheme.decoder(buffer, columnType) - input.foreach(expectResult(_, "Wrong decoded value")(decoder.next())) + if (input.nonEmpty) { + input.foreach{ + assert(decoder.hasNext) + expectResult(_, "Wrong decoded value")(decoder.next()) + } + } assert(!decoder.hasNext) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala index 89f9b60a4397b..d3b73ba19d476 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala @@ -81,8 +81,11 @@ class RunLengthEncodingSuite extends FunSuite { val decoder = RunLengthEncoding.decoder(buffer, columnType) - inputSeq.foreach { i => - expectResult(values(i), "Wrong decoded value")(decoder.next()) + if (inputSeq.nonEmpty) { + inputSeq.foreach { i => + assert(decoder.hasNext) + expectResult(values(i), "Wrong decoded value")(decoder.next()) + } } assert(!decoder.hasNext) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala index 81bf5e99d19b9..6d688ea95cfc0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala @@ -38,7 +38,7 @@ object TestCompressibleColumnBuilder { scheme: CompressionScheme) = { val builder = new TestCompressibleColumnBuilder(columnStats, columnType, Seq(scheme)) - builder.initialize(0) + builder.initialize(0, "", useCompression = true) builder } } 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 6c907887db79e..ba837a274c51c 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 @@ -130,7 +130,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with castChildOutput(p, table, child) case p @ logical.InsertIntoTable(SparkLogicalPlan(InMemoryColumnarTableScan( - _, HiveTableScan(_, table, _))), _, child, _) => + _, HiveTableScan(_, table, _), _)), _, child, _) => castChildOutput(p, table, child) } 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 1777e96b6713d..1f688fe1117fe 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 @@ -347,7 +347,11 @@ private[hive] object HiveQl { protected def nodeToPlan(node: Node): LogicalPlan = node match { // Just fake explain for any of the native commands. case Token("TOK_EXPLAIN", explainArgs) if nativeCommands contains explainArgs.head.getText => - NoRelation + ExplainCommand(NoRelation) + // Create tables aren't native commands due to CTAS queries, but we still don't need to + // explain them. + case Token("TOK_EXPLAIN", explainArgs) if explainArgs.head.getText == "TOK_CREATETABLE" => + ExplainCommand(NoRelation) case Token("TOK_EXPLAIN", explainArgs) => // Ignore FORMATTED if present. val Some(query) :: _ :: _ :: Nil = 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 d9a6e0e88932e..b2157074a41bf 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -45,7 +45,7 @@ private[hive] trait HiveStrategies { case logical.InsertIntoTable(table: MetastoreRelation, partition, child, overwrite) => InsertIntoHiveTable(table, partition, planLater(child), overwrite)(hiveContext) :: Nil case logical.InsertIntoTable(SparkLogicalPlan(InMemoryColumnarTableScan( - _, HiveTableScan(_, table, _))), partition, child, overwrite) => + _, HiveTableScan(_, table, _), _)), partition, child, overwrite) => InsertIntoHiveTable(table, partition, planLater(child), overwrite)(hiveContext) :: Nil case _ => Nil } 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 c7de4ab6d3955..d50e2c65b7b36 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 @@ -22,6 +22,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.hadoop.hive.common.`type`.HiveDecimal import org.apache.hadoop.hive.ql.exec.UDF import org.apache.hadoop.hive.ql.exec.{FunctionInfo, FunctionRegistry} +import org.apache.hadoop.hive.ql.udf.{UDFType => HiveUDFType} import org.apache.hadoop.hive.ql.udf.generic._ import org.apache.hadoop.hive.serde2.objectinspector._ import org.apache.hadoop.hive.serde2.objectinspector.primitive._ @@ -237,6 +238,16 @@ private[hive] case class HiveGenericUdf(name: String, children: Seq[Expression]) @transient protected lazy val returnInspector = function.initialize(argumentInspectors.toArray) + @transient + protected lazy val isUDFDeterministic = { + val udfType = function.getClass().getAnnotation(classOf[HiveUDFType]) + (udfType != null && udfType.deterministic()) + } + + override def foldable = { + isUDFDeterministic && children.foldLeft(true)((prev, n) => prev && n.foldable) + } + val dataType: DataType = inspectorToDataType(returnInspector) override def eval(input: Row): Any = { diff --git a/sql/hive/src/test/resources/golden/Constant Folding Optimization for AVG_SUM_COUNT-0-45f5619d9e4510195fe67f7c8d14a5c0 b/sql/hive/src/test/resources/golden/Constant Folding Optimization for AVG_SUM_COUNT-0-45f5619d9e4510195fe67f7c8d14a5c0 new file mode 100644 index 0000000000000..7643569a2c234 --- /dev/null +++ b/sql/hive/src/test/resources/golden/Constant Folding Optimization for AVG_SUM_COUNT-0-45f5619d9e4510195fe67f7c8d14a5c0 @@ -0,0 +1,309 @@ +0.0 0 0 0 3 +2.0 0 2 0 1 +4.0 0 4 0 1 +5.0 0 15 0 3 +8.0 0 8 0 1 +9.0 0 9 0 1 +10.0 0 10 0 1 +11.0 0 11 0 1 +12.0 0 24 0 2 +15.0 0 30 0 2 +17.0 0 17 0 1 +18.0 0 36 0 2 +19.0 0 19 0 1 +20.0 0 20 0 1 +24.0 0 48 0 2 +26.0 0 52 0 2 +27.0 0 27 0 1 +28.0 0 28 0 1 +30.0 0 30 0 1 +33.0 0 33 0 1 +34.0 0 34 0 1 +35.0 0 105 0 3 +37.0 0 74 0 2 +41.0 0 41 0 1 +42.0 0 84 0 2 +43.0 0 43 0 1 +44.0 0 44 0 1 +47.0 0 47 0 1 +51.0 0 102 0 2 +53.0 0 53 0 1 +54.0 0 54 0 1 +57.0 0 57 0 1 +58.0 0 116 0 2 +64.0 0 64 0 1 +65.0 0 65 0 1 +66.0 0 66 0 1 +67.0 0 134 0 2 +69.0 0 69 0 1 +70.0 0 210 0 3 +72.0 0 144 0 2 +74.0 0 74 0 1 +76.0 0 152 0 2 +77.0 0 77 0 1 +78.0 0 78 0 1 +80.0 0 80 0 1 +82.0 0 82 0 1 +83.0 0 166 0 2 +84.0 0 168 0 2 +85.0 0 85 0 1 +86.0 0 86 0 1 +87.0 0 87 0 1 +90.0 0 270 0 3 +92.0 0 92 0 1 +95.0 0 190 0 2 +96.0 0 96 0 1 +97.0 0 194 0 2 +98.0 0 196 0 2 +100.0 0 200 0 2 +103.0 0 206 0 2 +104.0 0 208 0 2 +105.0 0 105 0 1 +111.0 0 111 0 1 +113.0 0 226 0 2 +114.0 0 114 0 1 +116.0 0 116 0 1 +118.0 0 236 0 2 +119.0 0 357 0 3 +120.0 0 240 0 2 +125.0 0 250 0 2 +126.0 0 126 0 1 +128.0 0 384 0 3 +129.0 0 258 0 2 +131.0 0 131 0 1 +133.0 0 133 0 1 +134.0 0 268 0 2 +136.0 0 136 0 1 +137.0 0 274 0 2 +138.0 0 552 0 4 +143.0 0 143 0 1 +145.0 0 145 0 1 +146.0 0 292 0 2 +149.0 0 298 0 2 +150.0 0 150 0 1 +152.0 0 304 0 2 +153.0 0 153 0 1 +155.0 0 155 0 1 +156.0 0 156 0 1 +157.0 0 157 0 1 +158.0 0 158 0 1 +160.0 0 160 0 1 +162.0 0 162 0 1 +163.0 0 163 0 1 +164.0 0 328 0 2 +165.0 0 330 0 2 +166.0 0 166 0 1 +167.0 0 501 0 3 +168.0 0 168 0 1 +169.0 0 676 0 4 +170.0 0 170 0 1 +172.0 0 344 0 2 +174.0 0 348 0 2 +175.0 0 350 0 2 +176.0 0 352 0 2 +177.0 0 177 0 1 +178.0 0 178 0 1 +179.0 0 358 0 2 +180.0 0 180 0 1 +181.0 0 181 0 1 +183.0 0 183 0 1 +186.0 0 186 0 1 +187.0 0 561 0 3 +189.0 0 189 0 1 +190.0 0 190 0 1 +191.0 0 382 0 2 +192.0 0 192 0 1 +193.0 0 579 0 3 +194.0 0 194 0 1 +195.0 0 390 0 2 +196.0 0 196 0 1 +197.0 0 394 0 2 +199.0 0 597 0 3 +200.0 0 400 0 2 +201.0 0 201 0 1 +202.0 0 202 0 1 +203.0 0 406 0 2 +205.0 0 410 0 2 +207.0 0 414 0 2 +208.0 0 624 0 3 +209.0 0 418 0 2 +213.0 0 426 0 2 +214.0 0 214 0 1 +216.0 0 432 0 2 +217.0 0 434 0 2 +218.0 0 218 0 1 +219.0 0 438 0 2 +221.0 0 442 0 2 +222.0 0 222 0 1 +223.0 0 446 0 2 +224.0 0 448 0 2 +226.0 0 226 0 1 +228.0 0 228 0 1 +229.0 0 458 0 2 +230.0 0 1150 0 5 +233.0 0 466 0 2 +235.0 0 235 0 1 +237.0 0 474 0 2 +238.0 0 476 0 2 +239.0 0 478 0 2 +241.0 0 241 0 1 +242.0 0 484 0 2 +244.0 0 244 0 1 +247.0 0 247 0 1 +248.0 0 248 0 1 +249.0 0 249 0 1 +252.0 0 252 0 1 +255.0 0 510 0 2 +256.0 0 512 0 2 +257.0 0 257 0 1 +258.0 0 258 0 1 +260.0 0 260 0 1 +262.0 0 262 0 1 +263.0 0 263 0 1 +265.0 0 530 0 2 +266.0 0 266 0 1 +272.0 0 544 0 2 +273.0 0 819 0 3 +274.0 0 274 0 1 +275.0 0 275 0 1 +277.0 0 1108 0 4 +278.0 0 556 0 2 +280.0 0 560 0 2 +281.0 0 562 0 2 +282.0 0 564 0 2 +283.0 0 283 0 1 +284.0 0 284 0 1 +285.0 0 285 0 1 +286.0 0 286 0 1 +287.0 0 287 0 1 +288.0 0 576 0 2 +289.0 0 289 0 1 +291.0 0 291 0 1 +292.0 0 292 0 1 +296.0 0 296 0 1 +298.0 0 894 0 3 +302.0 0 302 0 1 +305.0 0 305 0 1 +306.0 0 306 0 1 +307.0 0 614 0 2 +308.0 0 308 0 1 +309.0 0 618 0 2 +310.0 0 310 0 1 +311.0 0 933 0 3 +315.0 0 315 0 1 +316.0 0 948 0 3 +317.0 0 634 0 2 +318.0 0 954 0 3 +321.0 0 642 0 2 +322.0 0 644 0 2 +323.0 0 323 0 1 +325.0 0 650 0 2 +327.0 0 981 0 3 +331.0 0 662 0 2 +332.0 0 332 0 1 +333.0 0 666 0 2 +335.0 0 335 0 1 +336.0 0 336 0 1 +338.0 0 338 0 1 +339.0 0 339 0 1 +341.0 0 341 0 1 +342.0 0 684 0 2 +344.0 0 688 0 2 +345.0 0 345 0 1 +348.0 0 1740 0 5 +351.0 0 351 0 1 +353.0 0 706 0 2 +356.0 0 356 0 1 +360.0 0 360 0 1 +362.0 0 362 0 1 +364.0 0 364 0 1 +365.0 0 365 0 1 +366.0 0 366 0 1 +367.0 0 734 0 2 +368.0 0 368 0 1 +369.0 0 1107 0 3 +373.0 0 373 0 1 +374.0 0 374 0 1 +375.0 0 375 0 1 +377.0 0 377 0 1 +378.0 0 378 0 1 +379.0 0 379 0 1 +382.0 0 764 0 2 +384.0 0 1152 0 3 +386.0 0 386 0 1 +389.0 0 389 0 1 +392.0 0 392 0 1 +393.0 0 393 0 1 +394.0 0 394 0 1 +395.0 0 790 0 2 +396.0 0 1188 0 3 +397.0 0 794 0 2 +399.0 0 798 0 2 +400.0 0 400 0 1 +401.0 0 2005 0 5 +402.0 0 402 0 1 +403.0 0 1209 0 3 +404.0 0 808 0 2 +406.0 0 1624 0 4 +407.0 0 407 0 1 +409.0 0 1227 0 3 +411.0 0 411 0 1 +413.0 0 826 0 2 +414.0 0 828 0 2 +417.0 0 1251 0 3 +418.0 0 418 0 1 +419.0 0 419 0 1 +421.0 0 421 0 1 +424.0 0 848 0 2 +427.0 0 427 0 1 +429.0 0 858 0 2 +430.0 0 1290 0 3 +431.0 0 1293 0 3 +432.0 0 432 0 1 +435.0 0 435 0 1 +436.0 0 436 0 1 +437.0 0 437 0 1 +438.0 0 1314 0 3 +439.0 0 878 0 2 +443.0 0 443 0 1 +444.0 0 444 0 1 +446.0 0 446 0 1 +448.0 0 448 0 1 +449.0 0 449 0 1 +452.0 0 452 0 1 +453.0 0 453 0 1 +454.0 0 1362 0 3 +455.0 0 455 0 1 +457.0 0 457 0 1 +458.0 0 916 0 2 +459.0 0 918 0 2 +460.0 0 460 0 1 +462.0 0 924 0 2 +463.0 0 926 0 2 +466.0 0 1398 0 3 +467.0 0 467 0 1 +468.0 0 1872 0 4 +469.0 0 2345 0 5 +470.0 0 470 0 1 +472.0 0 472 0 1 +475.0 0 475 0 1 +477.0 0 477 0 1 +478.0 0 956 0 2 +479.0 0 479 0 1 +480.0 0 1440 0 3 +481.0 0 481 0 1 +482.0 0 482 0 1 +483.0 0 483 0 1 +484.0 0 484 0 1 +485.0 0 485 0 1 +487.0 0 487 0 1 +489.0 0 1956 0 4 +490.0 0 490 0 1 +491.0 0 491 0 1 +492.0 0 984 0 2 +493.0 0 493 0 1 +494.0 0 494 0 1 +495.0 0 495 0 1 +496.0 0 496 0 1 +497.0 0 497 0 1 +498.0 0 1494 0 3 diff --git a/sql/hive/src/test/resources/golden/Constant Folding Optimization for AVG_SUM_COUNT-0-a393cfc24ad74f930f3284743254c10c b/sql/hive/src/test/resources/golden/Constant Folding Optimization for AVG_SUM_COUNT-0-a393cfc24ad74f930f3284743254c10c new file mode 100644 index 0000000000000..f23b45c32ecca --- /dev/null +++ b/sql/hive/src/test/resources/golden/Constant Folding Optimization for AVG_SUM_COUNT-0-a393cfc24ad74f930f3284743254c10c @@ -0,0 +1,309 @@ +0.0 0.0 0 0 0 3 +0.0 2.0 0 2 0 1 +0.0 4.0 0 4 0 1 +0.0 5.0 0 15 0 3 +0.0 8.0 0 8 0 1 +0.0 9.0 0 9 0 1 +0.0 10.0 0 10 0 1 +0.0 11.0 0 11 0 1 +0.0 12.0 0 24 0 2 +0.0 15.0 0 30 0 2 +0.0 17.0 0 17 0 1 +0.0 18.0 0 36 0 2 +0.0 19.0 0 19 0 1 +0.0 20.0 0 20 0 1 +0.0 24.0 0 48 0 2 +0.0 26.0 0 52 0 2 +0.0 27.0 0 27 0 1 +0.0 28.0 0 28 0 1 +0.0 30.0 0 30 0 1 +0.0 33.0 0 33 0 1 +0.0 34.0 0 34 0 1 +0.0 35.0 0 105 0 3 +0.0 37.0 0 74 0 2 +0.0 41.0 0 41 0 1 +0.0 42.0 0 84 0 2 +0.0 43.0 0 43 0 1 +0.0 44.0 0 44 0 1 +0.0 47.0 0 47 0 1 +0.0 51.0 0 102 0 2 +0.0 53.0 0 53 0 1 +0.0 54.0 0 54 0 1 +0.0 57.0 0 57 0 1 +0.0 58.0 0 116 0 2 +0.0 64.0 0 64 0 1 +0.0 65.0 0 65 0 1 +0.0 66.0 0 66 0 1 +0.0 67.0 0 134 0 2 +0.0 69.0 0 69 0 1 +0.0 70.0 0 210 0 3 +0.0 72.0 0 144 0 2 +0.0 74.0 0 74 0 1 +0.0 76.0 0 152 0 2 +0.0 77.0 0 77 0 1 +0.0 78.0 0 78 0 1 +0.0 80.0 0 80 0 1 +0.0 82.0 0 82 0 1 +0.0 83.0 0 166 0 2 +0.0 84.0 0 168 0 2 +0.0 85.0 0 85 0 1 +0.0 86.0 0 86 0 1 +0.0 87.0 0 87 0 1 +0.0 90.0 0 270 0 3 +0.0 92.0 0 92 0 1 +0.0 95.0 0 190 0 2 +0.0 96.0 0 96 0 1 +0.0 97.0 0 194 0 2 +0.0 98.0 0 196 0 2 +0.0 100.0 0 200 0 2 +0.0 103.0 0 206 0 2 +0.0 104.0 0 208 0 2 +0.0 105.0 0 105 0 1 +0.0 111.0 0 111 0 1 +0.0 113.0 0 226 0 2 +0.0 114.0 0 114 0 1 +0.0 116.0 0 116 0 1 +0.0 118.0 0 236 0 2 +0.0 119.0 0 357 0 3 +0.0 120.0 0 240 0 2 +0.0 125.0 0 250 0 2 +0.0 126.0 0 126 0 1 +0.0 128.0 0 384 0 3 +0.0 129.0 0 258 0 2 +0.0 131.0 0 131 0 1 +0.0 133.0 0 133 0 1 +0.0 134.0 0 268 0 2 +0.0 136.0 0 136 0 1 +0.0 137.0 0 274 0 2 +0.0 138.0 0 552 0 4 +0.0 143.0 0 143 0 1 +0.0 145.0 0 145 0 1 +0.0 146.0 0 292 0 2 +0.0 149.0 0 298 0 2 +0.0 150.0 0 150 0 1 +0.0 152.0 0 304 0 2 +0.0 153.0 0 153 0 1 +0.0 155.0 0 155 0 1 +0.0 156.0 0 156 0 1 +0.0 157.0 0 157 0 1 +0.0 158.0 0 158 0 1 +0.0 160.0 0 160 0 1 +0.0 162.0 0 162 0 1 +0.0 163.0 0 163 0 1 +0.0 164.0 0 328 0 2 +0.0 165.0 0 330 0 2 +0.0 166.0 0 166 0 1 +0.0 167.0 0 501 0 3 +0.0 168.0 0 168 0 1 +0.0 169.0 0 676 0 4 +0.0 170.0 0 170 0 1 +0.0 172.0 0 344 0 2 +0.0 174.0 0 348 0 2 +0.0 175.0 0 350 0 2 +0.0 176.0 0 352 0 2 +0.0 177.0 0 177 0 1 +0.0 178.0 0 178 0 1 +0.0 179.0 0 358 0 2 +0.0 180.0 0 180 0 1 +0.0 181.0 0 181 0 1 +0.0 183.0 0 183 0 1 +0.0 186.0 0 186 0 1 +0.0 187.0 0 561 0 3 +0.0 189.0 0 189 0 1 +0.0 190.0 0 190 0 1 +0.0 191.0 0 382 0 2 +0.0 192.0 0 192 0 1 +0.0 193.0 0 579 0 3 +0.0 194.0 0 194 0 1 +0.0 195.0 0 390 0 2 +0.0 196.0 0 196 0 1 +0.0 197.0 0 394 0 2 +0.0 199.0 0 597 0 3 +0.0 200.0 0 400 0 2 +0.0 201.0 0 201 0 1 +0.0 202.0 0 202 0 1 +0.0 203.0 0 406 0 2 +0.0 205.0 0 410 0 2 +0.0 207.0 0 414 0 2 +0.0 208.0 0 624 0 3 +0.0 209.0 0 418 0 2 +0.0 213.0 0 426 0 2 +0.0 214.0 0 214 0 1 +0.0 216.0 0 432 0 2 +0.0 217.0 0 434 0 2 +0.0 218.0 0 218 0 1 +0.0 219.0 0 438 0 2 +0.0 221.0 0 442 0 2 +0.0 222.0 0 222 0 1 +0.0 223.0 0 446 0 2 +0.0 224.0 0 448 0 2 +0.0 226.0 0 226 0 1 +0.0 228.0 0 228 0 1 +0.0 229.0 0 458 0 2 +0.0 230.0 0 1150 0 5 +0.0 233.0 0 466 0 2 +0.0 235.0 0 235 0 1 +0.0 237.0 0 474 0 2 +0.0 238.0 0 476 0 2 +0.0 239.0 0 478 0 2 +0.0 241.0 0 241 0 1 +0.0 242.0 0 484 0 2 +0.0 244.0 0 244 0 1 +0.0 247.0 0 247 0 1 +0.0 248.0 0 248 0 1 +0.0 249.0 0 249 0 1 +0.0 252.0 0 252 0 1 +0.0 255.0 0 510 0 2 +0.0 256.0 0 512 0 2 +0.0 257.0 0 257 0 1 +0.0 258.0 0 258 0 1 +0.0 260.0 0 260 0 1 +0.0 262.0 0 262 0 1 +0.0 263.0 0 263 0 1 +0.0 265.0 0 530 0 2 +0.0 266.0 0 266 0 1 +0.0 272.0 0 544 0 2 +0.0 273.0 0 819 0 3 +0.0 274.0 0 274 0 1 +0.0 275.0 0 275 0 1 +0.0 277.0 0 1108 0 4 +0.0 278.0 0 556 0 2 +0.0 280.0 0 560 0 2 +0.0 281.0 0 562 0 2 +0.0 282.0 0 564 0 2 +0.0 283.0 0 283 0 1 +0.0 284.0 0 284 0 1 +0.0 285.0 0 285 0 1 +0.0 286.0 0 286 0 1 +0.0 287.0 0 287 0 1 +0.0 288.0 0 576 0 2 +0.0 289.0 0 289 0 1 +0.0 291.0 0 291 0 1 +0.0 292.0 0 292 0 1 +0.0 296.0 0 296 0 1 +0.0 298.0 0 894 0 3 +0.0 302.0 0 302 0 1 +0.0 305.0 0 305 0 1 +0.0 306.0 0 306 0 1 +0.0 307.0 0 614 0 2 +0.0 308.0 0 308 0 1 +0.0 309.0 0 618 0 2 +0.0 310.0 0 310 0 1 +0.0 311.0 0 933 0 3 +0.0 315.0 0 315 0 1 +0.0 316.0 0 948 0 3 +0.0 317.0 0 634 0 2 +0.0 318.0 0 954 0 3 +0.0 321.0 0 642 0 2 +0.0 322.0 0 644 0 2 +0.0 323.0 0 323 0 1 +0.0 325.0 0 650 0 2 +0.0 327.0 0 981 0 3 +0.0 331.0 0 662 0 2 +0.0 332.0 0 332 0 1 +0.0 333.0 0 666 0 2 +0.0 335.0 0 335 0 1 +0.0 336.0 0 336 0 1 +0.0 338.0 0 338 0 1 +0.0 339.0 0 339 0 1 +0.0 341.0 0 341 0 1 +0.0 342.0 0 684 0 2 +0.0 344.0 0 688 0 2 +0.0 345.0 0 345 0 1 +0.0 348.0 0 1740 0 5 +0.0 351.0 0 351 0 1 +0.0 353.0 0 706 0 2 +0.0 356.0 0 356 0 1 +0.0 360.0 0 360 0 1 +0.0 362.0 0 362 0 1 +0.0 364.0 0 364 0 1 +0.0 365.0 0 365 0 1 +0.0 366.0 0 366 0 1 +0.0 367.0 0 734 0 2 +0.0 368.0 0 368 0 1 +0.0 369.0 0 1107 0 3 +0.0 373.0 0 373 0 1 +0.0 374.0 0 374 0 1 +0.0 375.0 0 375 0 1 +0.0 377.0 0 377 0 1 +0.0 378.0 0 378 0 1 +0.0 379.0 0 379 0 1 +0.0 382.0 0 764 0 2 +0.0 384.0 0 1152 0 3 +0.0 386.0 0 386 0 1 +0.0 389.0 0 389 0 1 +0.0 392.0 0 392 0 1 +0.0 393.0 0 393 0 1 +0.0 394.0 0 394 0 1 +0.0 395.0 0 790 0 2 +0.0 396.0 0 1188 0 3 +0.0 397.0 0 794 0 2 +0.0 399.0 0 798 0 2 +0.0 400.0 0 400 0 1 +0.0 401.0 0 2005 0 5 +0.0 402.0 0 402 0 1 +0.0 403.0 0 1209 0 3 +0.0 404.0 0 808 0 2 +0.0 406.0 0 1624 0 4 +0.0 407.0 0 407 0 1 +0.0 409.0 0 1227 0 3 +0.0 411.0 0 411 0 1 +0.0 413.0 0 826 0 2 +0.0 414.0 0 828 0 2 +0.0 417.0 0 1251 0 3 +0.0 418.0 0 418 0 1 +0.0 419.0 0 419 0 1 +0.0 421.0 0 421 0 1 +0.0 424.0 0 848 0 2 +0.0 427.0 0 427 0 1 +0.0 429.0 0 858 0 2 +0.0 430.0 0 1290 0 3 +0.0 431.0 0 1293 0 3 +0.0 432.0 0 432 0 1 +0.0 435.0 0 435 0 1 +0.0 436.0 0 436 0 1 +0.0 437.0 0 437 0 1 +0.0 438.0 0 1314 0 3 +0.0 439.0 0 878 0 2 +0.0 443.0 0 443 0 1 +0.0 444.0 0 444 0 1 +0.0 446.0 0 446 0 1 +0.0 448.0 0 448 0 1 +0.0 449.0 0 449 0 1 +0.0 452.0 0 452 0 1 +0.0 453.0 0 453 0 1 +0.0 454.0 0 1362 0 3 +0.0 455.0 0 455 0 1 +0.0 457.0 0 457 0 1 +0.0 458.0 0 916 0 2 +0.0 459.0 0 918 0 2 +0.0 460.0 0 460 0 1 +0.0 462.0 0 924 0 2 +0.0 463.0 0 926 0 2 +0.0 466.0 0 1398 0 3 +0.0 467.0 0 467 0 1 +0.0 468.0 0 1872 0 4 +0.0 469.0 0 2345 0 5 +0.0 470.0 0 470 0 1 +0.0 472.0 0 472 0 1 +0.0 475.0 0 475 0 1 +0.0 477.0 0 477 0 1 +0.0 478.0 0 956 0 2 +0.0 479.0 0 479 0 1 +0.0 480.0 0 1440 0 3 +0.0 481.0 0 481 0 1 +0.0 482.0 0 482 0 1 +0.0 483.0 0 483 0 1 +0.0 484.0 0 484 0 1 +0.0 485.0 0 485 0 1 +0.0 487.0 0 487 0 1 +0.0 489.0 0 1956 0 4 +0.0 490.0 0 490 0 1 +0.0 491.0 0 491 0 1 +0.0 492.0 0 984 0 2 +0.0 493.0 0 493 0 1 +0.0 494.0 0 494 0 1 +0.0 495.0 0 495 0 1 +0.0 496.0 0 496 0 1 +0.0 497.0 0 497 0 1 +0.0 498.0 0 1494 0 3 diff --git a/sql/hive/src/test/resources/golden/Constant Folding Optimization for AVG_SUM_COUNT-0-ae497f1556f548c1e2da9244397a985d b/sql/hive/src/test/resources/golden/Constant Folding Optimization for AVG_SUM_COUNT-0-ae497f1556f548c1e2da9244397a985d new file mode 100644 index 0000000000000..7839d714c25d2 --- /dev/null +++ b/sql/hive/src/test/resources/golden/Constant Folding Optimization for AVG_SUM_COUNT-0-ae497f1556f548c1e2da9244397a985d @@ -0,0 +1,309 @@ +0 3 +0 1 +0 1 +0 3 +0 1 +0 1 +0 1 +0 1 +0 2 +0 2 +0 1 +0 2 +0 1 +0 1 +0 2 +0 2 +0 1 +0 1 +0 1 +0 1 +0 1 +0 3 +0 2 +0 1 +0 2 +0 1 +0 1 +0 1 +0 2 +0 1 +0 1 +0 1 +0 2 +0 1 +0 1 +0 1 +0 2 +0 1 +0 3 +0 2 +0 1 +0 2 +0 1 +0 1 +0 1 +0 1 +0 2 +0 2 +0 1 +0 1 +0 1 +0 3 +0 1 +0 2 +0 1 +0 2 +0 2 +0 2 +0 2 +0 2 +0 1 +0 1 +0 2 +0 1 +0 1 +0 2 +0 3 +0 2 +0 2 +0 1 +0 3 +0 2 +0 1 +0 1 +0 2 +0 1 +0 2 +0 4 +0 1 +0 1 +0 2 +0 2 +0 1 +0 2 +0 1 +0 1 +0 1 +0 1 +0 1 +0 1 +0 1 +0 1 +0 2 +0 2 +0 1 +0 3 +0 1 +0 4 +0 1 +0 2 +0 2 +0 2 +0 2 +0 1 +0 1 +0 2 +0 1 +0 1 +0 1 +0 1 +0 3 +0 1 +0 1 +0 2 +0 1 +0 3 +0 1 +0 2 +0 1 +0 2 +0 3 +0 2 +0 1 +0 1 +0 2 +0 2 +0 2 +0 3 +0 2 +0 2 +0 1 +0 2 +0 2 +0 1 +0 2 +0 2 +0 1 +0 2 +0 2 +0 1 +0 1 +0 2 +0 5 +0 2 +0 1 +0 2 +0 2 +0 2 +0 1 +0 2 +0 1 +0 1 +0 1 +0 1 +0 1 +0 2 +0 2 +0 1 +0 1 +0 1 +0 1 +0 1 +0 2 +0 1 +0 2 +0 3 +0 1 +0 1 +0 4 +0 2 +0 2 +0 2 +0 2 +0 1 +0 1 +0 1 +0 1 +0 1 +0 2 +0 1 +0 1 +0 1 +0 1 +0 3 +0 1 +0 1 +0 1 +0 2 +0 1 +0 2 +0 1 +0 3 +0 1 +0 3 +0 2 +0 3 +0 2 +0 2 +0 1 +0 2 +0 3 +0 2 +0 1 +0 2 +0 1 +0 1 +0 1 +0 1 +0 1 +0 2 +0 2 +0 1 +0 5 +0 1 +0 2 +0 1 +0 1 +0 1 +0 1 +0 1 +0 1 +0 2 +0 1 +0 3 +0 1 +0 1 +0 1 +0 1 +0 1 +0 1 +0 2 +0 3 +0 1 +0 1 +0 1 +0 1 +0 1 +0 2 +0 3 +0 2 +0 2 +0 1 +0 5 +0 1 +0 3 +0 2 +0 4 +0 1 +0 3 +0 1 +0 2 +0 2 +0 3 +0 1 +0 1 +0 1 +0 2 +0 1 +0 2 +0 3 +0 3 +0 1 +0 1 +0 1 +0 1 +0 3 +0 2 +0 1 +0 1 +0 1 +0 1 +0 1 +0 1 +0 1 +0 3 +0 1 +0 1 +0 2 +0 2 +0 1 +0 2 +0 2 +0 3 +0 1 +0 4 +0 5 +0 1 +0 1 +0 1 +0 1 +0 2 +0 1 +0 3 +0 1 +0 1 +0 1 +0 1 +0 1 +0 1 +0 4 +0 1 +0 1 +0 2 +0 1 +0 1 +0 1 +0 1 +0 1 +0 3 diff --git a/sql/hive/src/test/resources/golden/Constant Folding Optimization for AVG_SUM_COUNT-0-c60b6075da793b826db5eb4b08d7bab9 b/sql/hive/src/test/resources/golden/Constant Folding Optimization for AVG_SUM_COUNT-0-c60b6075da793b826db5eb4b08d7bab9 new file mode 100644 index 0000000000000..cd6b14ac2a425 --- /dev/null +++ b/sql/hive/src/test/resources/golden/Constant Folding Optimization for AVG_SUM_COUNT-0-c60b6075da793b826db5eb4b08d7bab9 @@ -0,0 +1,309 @@ +0.0 0 0 3 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 3 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 3 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 3 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 3 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 2 +0.0 0 0 2 +0.0 0 0 2 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 3 +0.0 0 0 2 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 3 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 4 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 3 +0.0 0 0 1 +0.0 0 0 4 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 2 +0.0 0 0 2 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 3 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 3 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 3 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 2 +0.0 0 0 2 +0.0 0 0 3 +0.0 0 0 2 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 5 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 2 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 3 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 4 +0.0 0 0 2 +0.0 0 0 2 +0.0 0 0 2 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 3 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 3 +0.0 0 0 1 +0.0 0 0 3 +0.0 0 0 2 +0.0 0 0 3 +0.0 0 0 2 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 3 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 5 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 3 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 3 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 3 +0.0 0 0 2 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 5 +0.0 0 0 1 +0.0 0 0 3 +0.0 0 0 2 +0.0 0 0 4 +0.0 0 0 1 +0.0 0 0 3 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 2 +0.0 0 0 3 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 3 +0.0 0 0 3 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 3 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 3 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 2 +0.0 0 0 3 +0.0 0 0 1 +0.0 0 0 4 +0.0 0 0 5 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 3 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 4 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 2 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 1 +0.0 0 0 3 diff --git a/sql/hive/src/test/resources/golden/create_union_table-0-8e765b54f15b948fc88392da69da283 b/sql/hive/src/test/resources/golden/create_union_table-0-8e765b54f15b948fc88392da69da283 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_union_table-1-aee4ce62fc2631423af0f569f4448353 b/sql/hive/src/test/resources/golden/create_union_table-1-aee4ce62fc2631423af0f569f4448353 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_union_table-2-b1feb4a197caf28d5223e72e10a91e78 b/sql/hive/src/test/resources/golden/create_union_table-2-b1feb4a197caf28d5223e72e10a91e78 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ctas_uses_database_location-0-1c3ace37d0bbb5c8033c48cde7680d21 b/sql/hive/src/test/resources/golden/ctas_uses_database_location-0-1c3ace37d0bbb5c8033c48cde7680d21 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/ctas_uses_database_location-0-1c3ace37d0bbb5c8033c48cde7680d21 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/fileformat_sequencefile-0-7df8fcbfff1c967de72295b90c530776 b/sql/hive/src/test/resources/golden/fileformat_sequencefile-0-7df8fcbfff1c967de72295b90c530776 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/fileformat_sequencefile-1-3114fdebb1e9b0830de9e9c3fff8a67e b/sql/hive/src/test/resources/golden/fileformat_sequencefile-1-3114fdebb1e9b0830de9e9c3fff8a67e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/fileformat_sequencefile-2-80ec34a069bc561aa6dc87314391b131 b/sql/hive/src/test/resources/golden/fileformat_sequencefile-2-80ec34a069bc561aa6dc87314391b131 new file mode 100644 index 0000000000000..6280b32facd66 --- /dev/null +++ b/sql/hive/src/test/resources/golden/fileformat_sequencefile-2-80ec34a069bc561aa6dc87314391b131 @@ -0,0 +1,4 @@ +key int None +value string None + +Detailed Table Information Table(tableName:dest1, dbName:default, owner:marmbrus, createTime:1398823397, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse6323689881248298063/dest1, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1398823397}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/fileformat_sequencefile-3-699522b1ff3f38cf6a75b06d83820b87 b/sql/hive/src/test/resources/golden/fileformat_sequencefile-3-699522b1ff3f38cf6a75b06d83820b87 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/fileformat_sequencefile-4-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/fileformat_sequencefile-4-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..e614f37ecc8bf --- /dev/null +++ b/sql/hive/src/test/resources/golden/fileformat_sequencefile-4-adc1ec67836b26b60d8547c4996bfd8f @@ -0,0 +1,10 @@ +0 val_0 +4 val_4 +8 val_8 +0 val_0 +0 val_0 +5 val_5 +5 val_5 +2 val_2 +5 val_5 +9 val_9 diff --git a/sql/hive/src/test/resources/golden/fileformat_text-0-c84258297070d93009fabc59c1f40a1e b/sql/hive/src/test/resources/golden/fileformat_text-0-c84258297070d93009fabc59c1f40a1e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/fileformat_text-1-6375f4ba7af19f94c6afc366dc75429e b/sql/hive/src/test/resources/golden/fileformat_text-1-6375f4ba7af19f94c6afc366dc75429e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/fileformat_text-2-80ec34a069bc561aa6dc87314391b131 b/sql/hive/src/test/resources/golden/fileformat_text-2-80ec34a069bc561aa6dc87314391b131 new file mode 100644 index 0000000000000..e793ec2f946e5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/fileformat_text-2-80ec34a069bc561aa6dc87314391b131 @@ -0,0 +1,4 @@ +key int None +value string None + +Detailed Table Information Table(tableName:dest1, dbName:default, owner:marmbrus, createTime:1398823407, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse6323689881248298063/dest1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1398823407}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/fileformat_text-3-699522b1ff3f38cf6a75b06d83820b87 b/sql/hive/src/test/resources/golden/fileformat_text-3-699522b1ff3f38cf6a75b06d83820b87 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/fileformat_text-4-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/fileformat_text-4-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..e614f37ecc8bf --- /dev/null +++ b/sql/hive/src/test/resources/golden/fileformat_text-4-adc1ec67836b26b60d8547c4996bfd8f @@ -0,0 +1,10 @@ +0 val_0 +4 val_4 +8 val_8 +0 val_0 +0 val_0 +5 val_5 +5 val_5 +2 val_2 +5 val_5 +9 val_9 diff --git a/sql/hive/src/test/resources/golden/input15-0-1570712216ce86428ee39974242ae961 b/sql/hive/src/test/resources/golden/input15-0-1570712216ce86428ee39974242ae961 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input15-1-86edc12357bf278d5e601a654358c32f b/sql/hive/src/test/resources/golden/input15-1-86edc12357bf278d5e601a654358c32f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input15-2-ae5e0fbdf88ecca2c7d67df1ad141919 b/sql/hive/src/test/resources/golden/input15-2-ae5e0fbdf88ecca2c7d67df1ad141919 new file mode 100644 index 0000000000000..ded361eb294f0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/input15-2-ae5e0fbdf88ecca2c7d67df1ad141919 @@ -0,0 +1,2 @@ +key int None +value string None diff --git a/sql/hive/src/test/resources/golden/inputddl1-0-52f161c97a2a02494d26ee0737b28364 b/sql/hive/src/test/resources/golden/inputddl1-0-52f161c97a2a02494d26ee0737b28364 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/inputddl1-1-dfe05b7247284b326f39481b3d66a0c3 b/sql/hive/src/test/resources/golden/inputddl1-1-dfe05b7247284b326f39481b3d66a0c3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/inputddl1-2-b516db5218781d89aebd2e4813ea3660 b/sql/hive/src/test/resources/golden/inputddl1-2-b516db5218781d89aebd2e4813ea3660 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/inputddl2-0-4519aea06db13ec6780f60b4d101d262 b/sql/hive/src/test/resources/golden/inputddl2-0-4519aea06db13ec6780f60b4d101d262 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/inputddl2-1-f526507cae3cad09c69d20c7f0e36abe b/sql/hive/src/test/resources/golden/inputddl2-1-f526507cae3cad09c69d20c7f0e36abe new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/inputddl2-2-7c1c0606b5ea8a1c90d46fe221993b57 b/sql/hive/src/test/resources/golden/inputddl2-2-7c1c0606b5ea8a1c90d46fe221993b57 new file mode 100644 index 0000000000000..679d54cb5cb5e --- /dev/null +++ b/sql/hive/src/test/resources/golden/inputddl2-2-7c1c0606b5ea8a1c90d46fe221993b57 @@ -0,0 +1,10 @@ +key int None +value string None +ds string None +country string None + +# Partition Information +# col_name data_type comment + +ds string None +country string None diff --git a/sql/hive/src/test/resources/golden/inputddl3-0-a14253f6c752c9f5e9f56875152f8353 b/sql/hive/src/test/resources/golden/inputddl3-0-a14253f6c752c9f5e9f56875152f8353 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/inputddl3-1-f066827dd440213f306ab7d680bc8526 b/sql/hive/src/test/resources/golden/inputddl3-1-f066827dd440213f306ab7d680bc8526 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/inputddl3-2-73f945a673d2e388847c317f683f160c b/sql/hive/src/test/resources/golden/inputddl3-2-73f945a673d2e388847c317f683f160c new file mode 100644 index 0000000000000..ded361eb294f0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/inputddl3-2-73f945a673d2e388847c317f683f160c @@ -0,0 +1,2 @@ +key int None +value string None diff --git a/sql/hive/src/test/resources/golden/load_binary_data-4-3ebc340f7f63740f8534706d42dd37ca b/sql/hive/src/test/resources/golden/load_binary_data-4-3ebc340f7f63740f8534706d42dd37ca new file mode 100644 index 0000000000000..de5212a3c320f Binary files /dev/null and b/sql/hive/src/test/resources/golden/load_binary_data-4-3ebc340f7f63740f8534706d42dd37ca differ diff --git a/sql/hive/src/test/resources/golden/mapjoin_filter_on_outerjoin-1-8c68ce10095a8924c68c8ee4b3c0071a b/sql/hive/src/test/resources/golden/mapjoin_filter_on_outerjoin-1-8c68ce10095a8924c68c8ee4b3c0071a new file mode 100644 index 0000000000000..f81251e1360f2 --- /dev/null +++ b/sql/hive/src/test/resources/golden/mapjoin_filter_on_outerjoin-1-8c68ce10095a8924c68c8ee4b3c0071a @@ -0,0 +1,22 @@ +NULL NULL 66 val_66 66 val_66 +NULL NULL 98 val_98 98 val_98 +NULL NULL 98 val_98 98 val_98 +NULL NULL 128 128 val_128 +NULL NULL 128 128 val_128 +NULL NULL 128 128 val_128 +NULL NULL 146 val_146 146 val_146 +NULL NULL 146 val_146 146 val_146 +NULL NULL 150 val_150 150 val_150 +NULL NULL 213 val_213 213 val_213 +NULL NULL 213 val_213 213 val_213 +NULL NULL 224 224 val_224 +NULL NULL 224 224 val_224 +NULL NULL 238 val_238 238 val_238 +NULL NULL 238 val_238 238 val_238 +NULL NULL 255 val_255 255 val_255 +NULL NULL 255 val_255 255 val_255 +NULL NULL 273 val_273 273 val_273 +NULL NULL 273 val_273 273 val_273 +NULL NULL 273 val_273 273 val_273 +NULL NULL 278 val_278 278 val_278 +NULL NULL 278 val_278 278 val_278 diff --git a/sql/hive/src/test/resources/golden/mapjoin_filter_on_outerjoin-2-8e5d5472f2e214c091c879f6830a0c b/sql/hive/src/test/resources/golden/mapjoin_filter_on_outerjoin-2-8e5d5472f2e214c091c879f6830a0c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mapjoin_filter_on_outerjoin-3-381cc29f131d8caba744dd9fe6c121e7 b/sql/hive/src/test/resources/golden/mapjoin_filter_on_outerjoin-3-381cc29f131d8caba744dd9fe6c121e7 new file mode 100644 index 0000000000000..f81251e1360f2 --- /dev/null +++ b/sql/hive/src/test/resources/golden/mapjoin_filter_on_outerjoin-3-381cc29f131d8caba744dd9fe6c121e7 @@ -0,0 +1,22 @@ +NULL NULL 66 val_66 66 val_66 +NULL NULL 98 val_98 98 val_98 +NULL NULL 98 val_98 98 val_98 +NULL NULL 128 128 val_128 +NULL NULL 128 128 val_128 +NULL NULL 128 128 val_128 +NULL NULL 146 val_146 146 val_146 +NULL NULL 146 val_146 146 val_146 +NULL NULL 150 val_150 150 val_150 +NULL NULL 213 val_213 213 val_213 +NULL NULL 213 val_213 213 val_213 +NULL NULL 224 224 val_224 +NULL NULL 224 224 val_224 +NULL NULL 238 val_238 238 val_238 +NULL NULL 238 val_238 238 val_238 +NULL NULL 255 val_255 255 val_255 +NULL NULL 255 val_255 255 val_255 +NULL NULL 273 val_273 273 val_273 +NULL NULL 273 val_273 273 val_273 +NULL NULL 273 val_273 273 val_273 +NULL NULL 278 val_278 278 val_278 +NULL NULL 278 val_278 278 val_278 diff --git a/sql/hive/src/test/resources/golden/mapjoin_filter_on_outerjoin-4-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/mapjoin_filter_on_outerjoin-4-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/mapjoin_filter_on_outerjoin-4-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/mapjoin_filter_on_outerjoin-5-82cbc3186de23f3a2411e9ab87c0008c b/sql/hive/src/test/resources/golden/mapjoin_filter_on_outerjoin-5-82cbc3186de23f3a2411e9ab87c0008c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mapjoin_filter_on_outerjoin-6-c6844de37be0926316f5fbf36a905306 b/sql/hive/src/test/resources/golden/mapjoin_filter_on_outerjoin-6-c6844de37be0926316f5fbf36a905306 new file mode 100644 index 0000000000000..f81251e1360f2 --- /dev/null +++ b/sql/hive/src/test/resources/golden/mapjoin_filter_on_outerjoin-6-c6844de37be0926316f5fbf36a905306 @@ -0,0 +1,22 @@ +NULL NULL 66 val_66 66 val_66 +NULL NULL 98 val_98 98 val_98 +NULL NULL 98 val_98 98 val_98 +NULL NULL 128 128 val_128 +NULL NULL 128 128 val_128 +NULL NULL 128 128 val_128 +NULL NULL 146 val_146 146 val_146 +NULL NULL 146 val_146 146 val_146 +NULL NULL 150 val_150 150 val_150 +NULL NULL 213 val_213 213 val_213 +NULL NULL 213 val_213 213 val_213 +NULL NULL 224 224 val_224 +NULL NULL 224 224 val_224 +NULL NULL 238 val_238 238 val_238 +NULL NULL 238 val_238 238 val_238 +NULL NULL 255 val_255 255 val_255 +NULL NULL 255 val_255 255 val_255 +NULL NULL 273 val_273 273 val_273 +NULL NULL 273 val_273 273 val_273 +NULL NULL 273 val_273 273 val_273 +NULL NULL 278 val_278 278 val_278 +NULL NULL 278 val_278 278 val_278 diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-0-3ca1c197c5209d2fc9423fe84ad66e7d b/sql/hive/src/test/resources/golden/parallel_orderby-0-3ca1c197c5209d2fc9423fe84ad66e7d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-1-be7423a5e77b7289153f74bc3fd97f44 b/sql/hive/src/test/resources/golden/parallel_orderby-1-be7423a5e77b7289153f74bc3fd97f44 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-10-767848077fc2929ccedfd28e50564b19 b/sql/hive/src/test/resources/golden/parallel_orderby-10-767848077fc2929ccedfd28e50564b19 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/parallel_orderby-10-767848077fc2929ccedfd28e50564b19 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-11-39767502cbda75590c0c4b8fd089b793 b/sql/hive/src/test/resources/golden/parallel_orderby-11-39767502cbda75590c0c4b8fd089b793 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-12-7943485bcc72b2040c45c62e45ac8853 b/sql/hive/src/test/resources/golden/parallel_orderby-12-7943485bcc72b2040c45c62e45ac8853 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-13-988591bf9ab008fdc4d71560aa57a736 b/sql/hive/src/test/resources/golden/parallel_orderby-13-988591bf9ab008fdc4d71560aa57a736 new file mode 100644 index 0000000000000..b431d3fc6dcf6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/parallel_orderby-13-988591bf9ab008fdc4d71560aa57a736 @@ -0,0 +1,32 @@ +# col_name data_type comment + +key string None +value string None + +# Detailed Table Information +Database: default +Owner: marmbrus +CreateTime: Tue Apr 29 20:55:07 PDT 2014 +LastAccessTime: UNKNOWN +Protect Mode: None +Retention: 0 +Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse6323689881248298063/total_ordered +Table Type: MANAGED_TABLE +Table Parameters: + numFiles 1 + numPartitions 0 + numRows 48 + rawDataSize 512 + totalSize 560 + transient_lastDdlTime 1398830107 + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-14-d93951df2ffc18dc09ab0ba2d46f1823 b/sql/hive/src/test/resources/golden/parallel_orderby-14-d93951df2ffc18dc09ab0ba2d46f1823 new file mode 100644 index 0000000000000..3b733e2d6c451 --- /dev/null +++ b/sql/hive/src/test/resources/golden/parallel_orderby-14-d93951df2ffc18dc09ab0ba2d46f1823 @@ -0,0 +1,48 @@ +128 val_128 +128 val_128 +150 val_150 +150 val_150 +165 val_165 +165 val_165 +193 val_193 +193 val_193 +213 val_213 +213 val_213 +213 val_213 +213 val_213 +213 val_214 +213 val_214 +224 val_224 +224 val_224 +238 val_238 +238 val_238 +238 val_239 +238 val_239 +238 val_240 +238 val_240 +255 val_255 +255 val_255 +265 val_265 +265 val_265 +27 val_27 +27 val_27 +273 val_273 +273 val_273 +278 val_278 +278 val_278 +311 val_311 +311 val_311 +369 val_369 +369 val_369 +401 val_401 +401 val_401 +409 val_409 +409 val_409 +484 val_484 +484 val_484 +66 val_66 +66 val_66 +86 val_86 +86 val_86 +98 val_98 +98 val_98 diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-2-be7423a5e77b7289153f74bc3fd97f44 b/sql/hive/src/test/resources/golden/parallel_orderby-2-be7423a5e77b7289153f74bc3fd97f44 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-3-8ae9591fe39cd390619181e9664a92c1 b/sql/hive/src/test/resources/golden/parallel_orderby-3-8ae9591fe39cd390619181e9664a92c1 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/parallel_orderby-3-8ae9591fe39cd390619181e9664a92c1 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-4-a928b93138e9c5547c40ff2024b2b4b6 b/sql/hive/src/test/resources/golden/parallel_orderby-4-a928b93138e9c5547c40ff2024b2b4b6 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/parallel_orderby-4-a928b93138e9c5547c40ff2024b2b4b6 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-5-a58344acc57aaa38e2cb5f11a0576681 b/sql/hive/src/test/resources/golden/parallel_orderby-5-a58344acc57aaa38e2cb5f11a0576681 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/parallel_orderby-5-a58344acc57aaa38e2cb5f11a0576681 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-6-670ec1b1f28d92d72a924c29d622aa8f b/sql/hive/src/test/resources/golden/parallel_orderby-6-670ec1b1f28d92d72a924c29d622aa8f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-7-7943485bcc72b2040c45c62e45ac8853 b/sql/hive/src/test/resources/golden/parallel_orderby-7-7943485bcc72b2040c45c62e45ac8853 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-8-988591bf9ab008fdc4d71560aa57a736 b/sql/hive/src/test/resources/golden/parallel_orderby-8-988591bf9ab008fdc4d71560aa57a736 new file mode 100644 index 0000000000000..7f67251e61787 --- /dev/null +++ b/sql/hive/src/test/resources/golden/parallel_orderby-8-988591bf9ab008fdc4d71560aa57a736 @@ -0,0 +1,32 @@ +# col_name data_type comment + +key string None +value string None + +# Detailed Table Information +Database: default +Owner: marmbrus +CreateTime: Tue Apr 29 20:54:55 PDT 2014 +LastAccessTime: UNKNOWN +Protect Mode: None +Retention: 0 +Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse6323689881248298063/total_ordered +Table Type: MANAGED_TABLE +Table Parameters: + numFiles 1 + numPartitions 0 + numRows 48 + rawDataSize 512 + totalSize 560 + transient_lastDdlTime 1398830095 + +# Storage Information +SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe +InputFormat: org.apache.hadoop.mapred.TextInputFormat +OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +Compressed: No +Num Buckets: -1 +Bucket Columns: [] +Sort Columns: [] +Storage Desc Params: + serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-9-d93951df2ffc18dc09ab0ba2d46f1823 b/sql/hive/src/test/resources/golden/parallel_orderby-9-d93951df2ffc18dc09ab0ba2d46f1823 new file mode 100644 index 0000000000000..3b733e2d6c451 --- /dev/null +++ b/sql/hive/src/test/resources/golden/parallel_orderby-9-d93951df2ffc18dc09ab0ba2d46f1823 @@ -0,0 +1,48 @@ +128 val_128 +128 val_128 +150 val_150 +150 val_150 +165 val_165 +165 val_165 +193 val_193 +193 val_193 +213 val_213 +213 val_213 +213 val_213 +213 val_213 +213 val_214 +213 val_214 +224 val_224 +224 val_224 +238 val_238 +238 val_238 +238 val_239 +238 val_239 +238 val_240 +238 val_240 +255 val_255 +255 val_255 +265 val_265 +265 val_265 +27 val_27 +27 val_27 +273 val_273 +273 val_273 +278 val_278 +278 val_278 +311 val_311 +311 val_311 +369 val_369 +369 val_369 +401 val_401 +401 val_401 +409 val_409 +409 val_409 +484 val_484 +484 val_484 +66 val_66 +66 val_66 +86 val_86 +86 val_86 +98 val_98 +98 val_98 diff --git a/sql/hive/src/test/resources/golden/serde_regex-0-60462d14b99bb445b02800e9fb22760e b/sql/hive/src/test/resources/golden/serde_regex-0-60462d14b99bb445b02800e9fb22760e new file mode 100644 index 0000000000000..d00ee7786a57c --- /dev/null +++ b/sql/hive/src/test/resources/golden/serde_regex-0-60462d14b99bb445b02800e9fb22760e @@ -0,0 +1,22 @@ +ABSTRACT SYNTAX TREE: + (TOK_CREATETABLE (TOK_TABNAME serde_regex) TOK_LIKETABLE (TOK_TABCOLLIST (TOK_TABCOL host TOK_STRING) (TOK_TABCOL identity TOK_STRING) (TOK_TABCOL user TOK_STRING) (TOK_TABCOL time TOK_STRING) (TOK_TABCOL request TOK_STRING) (TOK_TABCOL status TOK_STRING) (TOK_TABCOL size TOK_INT) (TOK_TABCOL referer TOK_STRING) (TOK_TABCOL agent TOK_STRING)) (TOK_TABLESERIALIZER (TOK_SERDENAME 'org.apache.hadoop.hive.serde2.RegexSerDe' (TOK_TABLEPROPERTIES (TOK_TABLEPROPLIST (TOK_TABLEPROPERTY "input.regex" "([^ ]*) ([^ ]*) ([^ ]*) (-|\\[[^\\]]*\\]) ([^ \"]*|\"[^\"]*\") (-|[0-9]*) (-|[0-9]*)(?: ([^ \"]*|\"[^\"]*\") ([^ \"]*|\"[^\"]*\"))?"))))) TOK_TBLTEXTFILE) + +STAGE DEPENDENCIES: + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-0 + Create Table Operator: + Create Table + columns: host string, identity string, user string, time string, request string, status string, size int, referer string, agent string + if not exists: false + input format: org.apache.hadoop.mapred.TextInputFormat + # buckets: -1 + output format: org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat + serde name: org.apache.hadoop.hive.serde2.RegexSerDe + serde properties: + input.regex ([^ ]*) ([^ ]*) ([^ ]*) (-|\[[^\]]*\]) ([^ "]*|"[^"]*") (-|[0-9]*) (-|[0-9]*)(?: ([^ "]*|"[^"]*") ([^ "]*|"[^"]*"))? + name: serde_regex + isExternal: false + + diff --git a/sql/hive/src/test/resources/golden/serde_regex-1-dea03bd88cbaabcf438b398e23c139f6 b/sql/hive/src/test/resources/golden/serde_regex-1-dea03bd88cbaabcf438b398e23c139f6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/serde_regex-10-c5b3ec90419a40660e5f83736241c429 b/sql/hive/src/test/resources/golden/serde_regex-10-c5b3ec90419a40660e5f83736241c429 new file mode 100644 index 0000000000000..3e290231c27e2 --- /dev/null +++ b/sql/hive/src/test/resources/golden/serde_regex-10-c5b3ec90419a40660e5f83736241c429 @@ -0,0 +1,38 @@ +NULL 0 +NULL 0 +-1234567890.123456789 -1234567890 +-4400 4400 +-1255.49 -1255 +-1.122 -11 +-1.12 -1 +-1.12 -1 +-0.333 0 +-0.33 0 +-0.3 0 +0 0 +0 0 +0.01 0 +0.02 0 +0.1 0 +0.2 0 +0.3 0 +0.33 0 +0.333 0 +0.9999999999999999999999999 1 +1 1 +1 1 +1.12 1 +1.122 1 +2 2 +2 2 +3.14 3 +3.14 3 +3.14 3 +3.14 4 +10 10 +20 20 +100 100 +124 124 +125.2 125 +200 200 +1234567890.12345678 1234567890 diff --git a/sql/hive/src/test/resources/golden/serde_regex-11-b00710a6f7a3e5b92c2e17da54cd9ad1 b/sql/hive/src/test/resources/golden/serde_regex-11-b00710a6f7a3e5b92c2e17da54cd9ad1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/serde_regex-2-9d00484beaee46cf72b154a1351aeee9 b/sql/hive/src/test/resources/golden/serde_regex-2-9d00484beaee46cf72b154a1351aeee9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/serde_regex-3-817190d8871b70611483cd2abe2e55dc b/sql/hive/src/test/resources/golden/serde_regex-3-817190d8871b70611483cd2abe2e55dc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/serde_regex-4-c3e345183543f40a14d2dd742ebd5346 b/sql/hive/src/test/resources/golden/serde_regex-4-c3e345183543f40a14d2dd742ebd5346 new file mode 100644 index 0000000000000..c55f3dd475574 --- /dev/null +++ b/sql/hive/src/test/resources/golden/serde_regex-4-c3e345183543f40a14d2dd742ebd5346 @@ -0,0 +1,2 @@ +127.0.0.1 - frank [10/Oct/2000:13:55:36 -0700] "GET /apache_pb.gif HTTP/1.0" 200 2326 NULL NULL +127.0.0.1 - - [26/May/2009:00:00:00 +0000] "GET /someurl/?track=Blabla(Main) HTTP/1.1" 200 5864 - "Mozilla/5.0 (Windows; U; Windows NT 6.0; en-US) AppleWebKit/525.19 (KHTML, like Gecko) Chrome/1.0.154.65 Safari/525.19" diff --git a/sql/hive/src/test/resources/golden/serde_regex-5-e2d14a76a87a6014c6d2d3501a0c9129 b/sql/hive/src/test/resources/golden/serde_regex-5-e2d14a76a87a6014c6d2d3501a0c9129 new file mode 100644 index 0000000000000..8bd185bc66ebb --- /dev/null +++ b/sql/hive/src/test/resources/golden/serde_regex-5-e2d14a76a87a6014c6d2d3501a0c9129 @@ -0,0 +1,2 @@ +127.0.0.1 2326 200 [10/Oct/2000:13:55:36 -0700] +127.0.0.1 5864 200 [26/May/2009:00:00:00 +0000] diff --git a/sql/hive/src/test/resources/golden/serde_regex-6-f818821654f219d1f4e2482951fae4f1 b/sql/hive/src/test/resources/golden/serde_regex-6-f818821654f219d1f4e2482951fae4f1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/serde_regex-7-4db287576a17c0963219ca63ee0b20e0 b/sql/hive/src/test/resources/golden/serde_regex-7-4db287576a17c0963219ca63ee0b20e0 new file mode 100644 index 0000000000000..da61769c6599d --- /dev/null +++ b/sql/hive/src/test/resources/golden/serde_regex-7-4db287576a17c0963219ca63ee0b20e0 @@ -0,0 +1,22 @@ +ABSTRACT SYNTAX TREE: + (TOK_CREATETABLE (TOK_TABNAME serde_regex1) TOK_LIKETABLE (TOK_TABCOLLIST (TOK_TABCOL key TOK_DECIMAL) (TOK_TABCOL value TOK_INT)) (TOK_TABLESERIALIZER (TOK_SERDENAME 'org.apache.hadoop.hive.serde2.RegexSerDe' (TOK_TABLEPROPERTIES (TOK_TABLEPROPLIST (TOK_TABLEPROPERTY "input.regex" "([^ ]*) ([^ ]*)"))))) TOK_TBLTEXTFILE) + +STAGE DEPENDENCIES: + Stage-0 is a root stage + +STAGE PLANS: + Stage: Stage-0 + Create Table Operator: + Create Table + columns: key decimal, value int + if not exists: false + input format: org.apache.hadoop.mapred.TextInputFormat + # buckets: -1 + output format: org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat + serde name: org.apache.hadoop.hive.serde2.RegexSerDe + serde properties: + input.regex ([^ ]*) ([^ ]*) + name: serde_regex1 + isExternal: false + + diff --git a/sql/hive/src/test/resources/golden/serde_regex-8-c429ee76b751e674992f61a29c95af77 b/sql/hive/src/test/resources/golden/serde_regex-8-c429ee76b751e674992f61a29c95af77 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/serde_regex-9-f0e8d394ad18dcbd381792fe9bd8894b b/sql/hive/src/test/resources/golden/serde_regex-9-f0e8d394ad18dcbd381792fe9bd8894b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin9-0-31cd5ae86d250c1c873260c4b73d2336 b/sql/hive/src/test/resources/golden/smb_mapjoin9-0-31cd5ae86d250c1c873260c4b73d2336 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin9-1-c5c86975c400b3a1ec0f522b75635338 b/sql/hive/src/test/resources/golden/smb_mapjoin9-1-c5c86975c400b3a1ec0f522b75635338 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin9-10-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/smb_mapjoin9-10-ffe97dc8c1df3195982e38263fbe8717 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin9-10-ffe97dc8c1df3195982e38263fbe8717 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin9-11-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/smb_mapjoin9-11-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin9-11-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin9-12-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/smb_mapjoin9-12-7cccbdffc32975f8935eeba14a28147 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin9-12-7cccbdffc32975f8935eeba14a28147 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin9-13-d0eac0de35b5a7595e4567edec8e555d b/sql/hive/src/test/resources/golden/smb_mapjoin9-13-d0eac0de35b5a7595e4567edec8e555d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin9-14-f0de4fd75ca13f379fbea16010dfd3f9 b/sql/hive/src/test/resources/golden/smb_mapjoin9-14-f0de4fd75ca13f379fbea16010dfd3f9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin9-15-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin9-15-b89ea2173180c8ae423d856f943e061f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin9-15-b89ea2173180c8ae423d856f943e061f @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin9-16-5645429104736d72293601247e874df7 b/sql/hive/src/test/resources/golden/smb_mapjoin9-16-5645429104736d72293601247e874df7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin9-17-1018a99eaede08dc2b427d3fb4b91d6b b/sql/hive/src/test/resources/golden/smb_mapjoin9-17-1018a99eaede08dc2b427d3fb4b91d6b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin9-18-bda1fd3b5d0326dc3a41a8fb480b6c1c b/sql/hive/src/test/resources/golden/smb_mapjoin9-18-bda1fd3b5d0326dc3a41a8fb480b6c1c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin9-19-823e8f68baaa45d6b761b9b9890bb902 b/sql/hive/src/test/resources/golden/smb_mapjoin9-19-823e8f68baaa45d6b761b9b9890bb902 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin9-2-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin9-2-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin9-2-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin9-20-78ea4515eba2d8a79bb6895133a82051 b/sql/hive/src/test/resources/golden/smb_mapjoin9-20-78ea4515eba2d8a79bb6895133a82051 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin9-3-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin9-3-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin9-3-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin9-4-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin9-4-b89ea2173180c8ae423d856f943e061f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin9-4-b89ea2173180c8ae423d856f943e061f @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin9-5-94bc4acde9b9c9944e2eb9a9c57d3450 b/sql/hive/src/test/resources/golden/smb_mapjoin9-5-94bc4acde9b9c9944e2eb9a9c57d3450 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin9-6-6c0963a1bdf2def39b79bea21a9cdda9 b/sql/hive/src/test/resources/golden/smb_mapjoin9-6-6c0963a1bdf2def39b79bea21a9cdda9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin9-7-ba9bc547b5d2a4912f4f8567fea4009c b/sql/hive/src/test/resources/golden/smb_mapjoin9-7-ba9bc547b5d2a4912f4f8567fea4009c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin9-8-d528ff47621b8c86b370f72512a7cd3c b/sql/hive/src/test/resources/golden/smb_mapjoin9-8-d528ff47621b8c86b370f72512a7cd3c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin9-9-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/smb_mapjoin9-9-e39f59c35ebbe686a18d45d9d8bf3ab0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin9-9-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/symlink_text_input_format-0-3b2fa9592648fc07c0d43e0d7d7f9411 b/sql/hive/src/test/resources/golden/symlink_text_input_format-0-3b2fa9592648fc07c0d43e0d7d7f9411 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/symlink_text_input_format-1-d498fb503b8f47db4741fdad3d266b4a b/sql/hive/src/test/resources/golden/symlink_text_input_format-1-d498fb503b8f47db4741fdad3d266b4a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/symlink_text_input_format-2-42119039bf8023f90b7f474f235c5dd5 b/sql/hive/src/test/resources/golden/symlink_text_input_format-2-42119039bf8023f90b7f474f235c5dd5 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/symlink_text_input_format-2-42119039bf8023f90b7f474f235c5dd5 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/symlink_text_input_format-3-77b57147024eb6b28cc9f525fdaab615 b/sql/hive/src/test/resources/golden/symlink_text_input_format-3-77b57147024eb6b28cc9f525fdaab615 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/symlink_text_input_format-3-77b57147024eb6b28cc9f525fdaab615 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/symlink_text_input_format-4-676cb274a770a6b9ca86df5dc7f912d4 b/sql/hive/src/test/resources/golden/symlink_text_input_format-4-676cb274a770a6b9ca86df5dc7f912d4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/symlink_text_input_format-5-ef3052815ec41b5957627698ba06707b b/sql/hive/src/test/resources/golden/symlink_text_input_format-5-ef3052815ec41b5957627698ba06707b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/symlink_text_input_format-6-891be0baec05e358a647dcca77724446 b/sql/hive/src/test/resources/golden/symlink_text_input_format-6-891be0baec05e358a647dcca77724446 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/symlink_text_input_format-7-208bcc9c918cbeb52907c8871be19cd5 b/sql/hive/src/test/resources/golden/symlink_text_input_format-7-208bcc9c918cbeb52907c8871be19cd5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/symlink_text_input_format-8-fb2e7127e07ad9f7e97ad3df3eba3a35 b/sql/hive/src/test/resources/golden/symlink_text_input_format-8-fb2e7127e07ad9f7e97ad3df3eba3a35 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/symlink_text_input_format-9-433d5dbbcf75ff68d6897f31baa46841 b/sql/hive/src/test/resources/golden/symlink_text_input_format-9-433d5dbbcf75ff68d6897f31baa46841 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/symlink_text_input_format-9-433d5dbbcf75ff68d6897f31baa46841 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/udf_hour-4-73bfac513b993dedbe143306865a44a b/sql/hive/src/test/resources/golden/udf_hour-4-73bfac513b993dedbe143306865a44a new file mode 100644 index 0000000000000..b1bd38b62a080 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_hour-4-73bfac513b993dedbe143306865a44a @@ -0,0 +1 @@ +13 diff --git a/sql/hive/src/test/resources/golden/udf_substr-8-ba6ca6bac87ca99aabd60b8e76537ade b/sql/hive/src/test/resources/golden/udf_substr-8-ba6ca6bac87ca99aabd60b8e76537ade new file mode 100644 index 0000000000000..c14d8d4279c5f --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_substr-8-ba6ca6bac87ca99aabd60b8e76537ade @@ -0,0 +1 @@ +NULL NULL NULL NULL NULL A AB ABC ABC A AB ABC ABC B BC BC BC C C C C C C C C B BC BC BC A AB ABC ABC diff --git a/sql/hive/src/test/resources/golden/udf_substr-9-a9aa82080133620d017160f6a644455d b/sql/hive/src/test/resources/golden/udf_substr-9-a9aa82080133620d017160f6a644455d new file mode 100644 index 0000000000000..94a57d86c88fd --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_substr-9-a9aa82080133620d017160f6a644455d @@ -0,0 +1 @@ +玩 玩 玩玩玩 abc 玩玩玩 diff --git a/sql/hive/src/test/resources/golden/union25-0-f4e883ee4edf8fbb95efed8ae41cff1c b/sql/hive/src/test/resources/golden/union25-0-f4e883ee4edf8fbb95efed8ae41cff1c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union25-1-8f4a32f3c1551b4e4f22c3b776a92043 b/sql/hive/src/test/resources/golden/union25-1-8f4a32f3c1551b4e4f22c3b776a92043 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union25-2-f61103bb2045761ba95828898e63b92b b/sql/hive/src/test/resources/golden/union25-2-f61103bb2045761ba95828898e63b92b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index 0bb76f31c373d..d83732b51e9c2 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -112,6 +112,8 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "stats1.*", "stats20", "alter_merge_stats", + "columnstats.*", + // Hive seems to think 1.0 > NaN = true && 1.0 < NaN = false... which is wrong. // http://stackoverflow.com/a/1573715 @@ -176,6 +178,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { */ override def whiteList = Seq( "add_part_exist", + "add_part_multiple", "add_partition_no_whitelist", "add_partition_with_whitelist", "alias_casted_column", @@ -290,6 +293,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "disable_file_format_check", "drop_function", "drop_index", + "drop_multi_partitions", "drop_partitions_filter", "drop_partitions_filter2", "drop_partitions_filter3", @@ -302,6 +306,8 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "escape_orderby1", "escape_sortby1", "fetch_aggregation", + "fileformat_sequencefile", + "fileformat_text", "filter_join_breaktask", "filter_join_breaktask2", "groupby1", @@ -310,6 +316,10 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "groupby1_map_nomap", "groupby1_map_skew", "groupby1_noskew", + "groupby2", + "groupby2_map", + "groupby2_map_skew", + "groupby2_noskew", "groupby4", "groupby4_map", "groupby4_map_skew", @@ -333,10 +343,12 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "groupby8_noskew", "groupby9", "groupby_distinct_samekey", + "groupby_map_ppr", "groupby_multi_insert_common_distinct", "groupby_multi_single_reducer2", "groupby_mutli_insert_common_distinct", "groupby_neg_float", + "groupby_ppr", "groupby_sort_10", "groupby_sort_2", "groupby_sort_3", @@ -352,13 +364,17 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "inoutdriver", "input", "input0", + "input1", + "input10", "input11", "input11_limit", "input12", "input12_hadoop20", "input14", + "input15", "input19", "input1_limit", + "input2", "input21", "input22", "input23", @@ -367,6 +383,8 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "input26", "input28", "input2_limit", + "input3", + "input4", "input40", "input41", "input4_cb_delim", @@ -374,9 +392,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "input7", "input8", "input9", - "inputddl4", - "inputddl7", - "inputddl8", "input_limit", "input_part0", "input_part1", @@ -391,6 +406,13 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "input_part8", "input_part9", "input_testsequencefile", + "inputddl1", + "inputddl2", + "inputddl3", + "inputddl4", + "inputddl6", + "inputddl7", + "inputddl8", "insert1", "insert2_overwrite_partitions", "insert_compressed", @@ -452,7 +474,9 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "join_reorder4", "join_star", "join_view", + "lateral_view", "lateral_view_cp", + "lateral_view_outer", "lateral_view_ppd", "lineage1", "literal_double", @@ -463,6 +487,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "loadpart1", "louter_join_ppr", "mapjoin_distinct", + "mapjoin_filter_on_outerjoin", "mapjoin_mapjoin", "mapjoin_subquery", "mapjoin_subquery2", @@ -568,6 +593,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "select_unquote_and", "select_unquote_not", "select_unquote_or", + "serde_regex", "serde_reported_schema", "set_variable_sub", "show_describe_func_quotes", @@ -576,6 +602,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "skewjoinopt13", "skewjoinopt18", "skewjoinopt9", + "smb_mapjoin9", "smb_mapjoin_1", "smb_mapjoin_10", "smb_mapjoin_13", @@ -620,8 +647,11 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "udf_10_trims", "udf2", "udf6", + "udf7", "udf8", "udf9", + "udf_E", + "udf_PI", "udf_abs", "udf_acos", "udf_add", @@ -645,6 +675,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "udf_ceil", "udf_ceiling", "udf_concat", + "udf_concat_insert1", "udf_concat_insert2", "udf_concat_ws", "udf_conv", @@ -659,6 +690,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "udf_div", "udf_double", "udf_E", + "udf_elt", "udf_exp", "udf_field", "udf_find_in_set", @@ -668,9 +700,11 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "udf_from_unixtime", "udf_greaterthan", "udf_greaterthanorequal", + "udf_hash", "udf_hex", "udf_if", "udf_index", + "udf_instr", "udf_int", "udf_isnotnull", "udf_isnull", @@ -681,6 +715,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "udf_lessthanorequal", "udf_like", "udf_ln", + "udf_locate", "udf_log", "udf_log10", "udf_log2", @@ -739,9 +774,9 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "udf_trim", "udf_ucase", "udf_upper", - "udf_variance", "udf_var_pop", "udf_var_samp", + "udf_variance", "udf_weekofyear", "udf_when", "udf_xpath", @@ -767,6 +802,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "union22", "union23", "union24", + "union25", "union26", "union27", "union28", 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 d224d2ee600ec..87a92d83383ab 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 @@ -33,6 +33,9 @@ class HiveQuerySuite extends HiveComparisonTest { hiveql("FROM src SELECT key").collect() } + createQueryTest("Constant Folding Optimization for AVG_SUM_COUNT", + "SELECT AVG(0), SUM(0), COUNT(null), COUNT(value) FROM src GROUP BY key") + createQueryTest("Simple Average", "SELECT AVG(key) FROM src") diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala index cb8e8f00a7b82..c4bdf01fa3744 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala @@ -262,7 +262,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * @param windowDuration width of the window; must be a multiple of this DStream's * batching interval */ - def reduceByKeyAndWindow(reduceFunc: Function2[V, V, V], windowDuration: Duration) + def reduceByKeyAndWindow(reduceFunc: JFunction2[V, V, V], windowDuration: Duration) :JavaPairDStream[K, V] = { dstream.reduceByKeyAndWindow(reduceFunc, windowDuration) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala index b310c22b3ab78..5acf8a9a811ee 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala @@ -28,9 +28,11 @@ import org.apache.spark.annotation.DeveloperApi /** * :: DeveloperApi :: * Abstract class of a receiver that can be run on worker nodes to receive external data. A - * custom receiver can be defined by defining the functions onStart() and onStop(). onStart() + * custom receiver can be defined by defining the functions `onStart()` and `onStop()`. `onStart()` * should define the setup steps necessary to start receiving data, - * and onStop() should define the cleanup steps necessary to stop receiving data. + * and `onStop()` should define the cleanup steps necessary to stop receiving data. + * Exceptions while receiving can be handled either by restarting the receiver with `restart(...)` + * or stopped completely by `stop(...)` or * * A custom receiver in Scala would look like this. * diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala index bbf57ef9275c0..a73d6f3bf0661 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala @@ -25,7 +25,7 @@ import scala.collection.JavaConversions.mapAsScalaMap private[streaming] object RawTextHelper { - /** + /** * Splits lines and counts the words. */ def splitAndCountPartitions(iter: Iterator[String]): Iterator[(String, Long)] = { @@ -114,4 +114,3 @@ object RawTextHelper { def max(v1: Long, v2: Long) = math.max(v1, v2) } - diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala index ee0bc8b7d6a71..cd86019f63e7e 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -183,6 +183,7 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w "Received records = " + TestReceiver.counter.get() + ", " + "processed records = " + runningCount ) + Thread.sleep(100) } } diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index fc13dbecb4555..8f0ecb855718e 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -70,9 +70,6 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, private var registered = false - private val sparkUser = Option(System.getenv("SPARK_USER")).getOrElse( - SparkContext.SPARK_UNKNOWN_USER) - def run() { // Setup the directories so things go to yarn approved directories rather // then user specified and /tmp. @@ -192,7 +189,8 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, false /* initialize */ , Thread.currentThread.getContextClassLoader).getMethod("main", classOf[Array[String]]) val t = new Thread { - override def run(): Unit = SparkHadoopUtil.get.runAsUser(sparkUser) { () => + override def run() { + var successed = false try { // Copy @@ -480,6 +478,8 @@ object ApplicationMaster { def main(argStrings: Array[String]) { val args = new ApplicationMasterArguments(argStrings) - new ApplicationMaster(args).run() + SparkHadoopUtil.get.runAsSparkUser { () => + new ApplicationMaster(args).run() + } } } 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 65b7215afbd4c..a3bd91590fc25 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 @@ -29,10 +29,11 @@ import org.apache.hadoop.yarn.util.{ConverterUtils, Records} import akka.actor._ import akka.remote._ import akka.actor.Terminated -import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext} +import org.apache.spark.{Logging, SecurityManager, SparkConf} import org.apache.spark.util.{Utils, AkkaUtils} import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.scheduler.SplitInfo +import org.apache.spark.deploy.SparkHadoopUtil /** * An application master that allocates executors on behalf of a driver that is running outside @@ -279,6 +280,8 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp object ExecutorLauncher { def main(argStrings: Array[String]) { val args = new ApplicationMasterArguments(argStrings) - new ExecutorLauncher(args).run() + SparkHadoopUtil.get.runAsSparkUser { () => + new ExecutorLauncher(args).run() + } } } diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index f2be8217a2f8a..27a518ccda459 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -263,9 +263,13 @@ trait ClientBase extends Logging { distCacheMgr.setDistFilesEnv(env) distCacheMgr.setDistArchivesEnv(env) - // Allow users to specify some environment variables. - YarnSparkHadoopUtil.setEnvFromInputString(env, System.getenv("SPARK_YARN_USER_ENV"), - File.pathSeparator) + sys.env.get("SPARK_YARN_USER_ENV").foreach { userEnvs => + // Allow users to specify some environment variables. + YarnSparkHadoopUtil.setEnvFromInputString(env, userEnvs, File.pathSeparator) + + // Pass SPARK_YARN_USER_ENV itself to the AM so it can use it to set up executor environments. + env("SPARK_YARN_USER_ENV") = userEnvs + } env } @@ -322,6 +326,12 @@ trait ClientBase extends Logging { JAVA_OPTS += "-XX:CMSIncrementalDutyCycle=10" } + // SPARK_JAVA_OPTS is deprecated, but for backwards compatibility: + sys.env.get("SPARK_JAVA_OPTS").foreach { opts => + sparkConf.set("spark.executor.extraJavaOptions", opts) + sparkConf.set("spark.driver.extraJavaOptions", opts) + } + // TODO: it might be nicer to pass these as an internal environment variable rather than // as Java options, due to complications with string parsing of nested quotes. if (args.amClass == classOf[ExecutorLauncher].getName) { diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala index 7d07f6f68046a..96f8aa93394f5 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala @@ -71,8 +71,8 @@ trait ExecutorRunnableUtil extends Logging { /* else { // If no java_opts specified, default to using -XX:+CMSIncrementalMode - // It might be possible that other modes/config is being done in SPARK_JAVA_OPTS, so we dont - // want to mess with it. + // It might be possible that other modes/config is being done in spark.executor.extraJavaOptions, + // so we dont want to mess with it. // In our expts, using (default) throughput collector has severe perf ramnifications in // multi-tennent machines // The options are based on diff --git a/yarn/pom.xml b/yarn/pom.xml index 02f36627431b9..6993c89525d8c 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -28,7 +28,7 @@ yarn-parent_2.10 pom Spark Project YARN Parent POM - + org.apache.spark @@ -43,6 +43,10 @@ org.apache.hadoop hadoop-yarn-common + + org.apache.hadoop + hadoop-yarn-server-web-proxy + org.apache.hadoop hadoop-yarn-client @@ -50,7 +54,6 @@ org.apache.hadoop hadoop-client - ${yarn.version} org.scalatest @@ -128,7 +131,7 @@ - + diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 90e807160d4b6..c1dfe3f53b40b 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -71,9 +71,6 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, sparkConf.getInt("spark.yarn.max.worker.failures", math.max(args.numExecutors * 2, 3))) private var registered = false - - private val sparkUser = Option(System.getenv("SPARK_USER")).getOrElse( - SparkContext.SPARK_UNKNOWN_USER) def run() { // Setup the directories so things go to YARN approved directories rather @@ -179,8 +176,9 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, false /* initialize */ , Thread.currentThread.getContextClassLoader).getMethod("main", classOf[Array[String]]) val t = new Thread { - override def run(): Unit = SparkHadoopUtil.get.runAsUser(sparkUser) { () => - var successed = false + override def run() { + + var successed = false try { // Copy var mainArgs: Array[String] = new Array[String](args.userArgs.size) @@ -462,6 +460,8 @@ object ApplicationMaster { def main(argStrings: Array[String]) { val args = new ApplicationMasterArguments(argStrings) - new ApplicationMaster(args).run() + SparkHadoopUtil.get.runAsSparkUser { () => + new ApplicationMaster(args).run() + } } } 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 a14bb377aa133..a4ce8766d347c 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 @@ -28,12 +28,13 @@ import org.apache.hadoop.yarn.util.{ConverterUtils, Records} import akka.actor._ import akka.remote._ import akka.actor.Terminated -import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext} +import org.apache.spark.{Logging, SecurityManager, SparkConf} import org.apache.spark.util.{Utils, AkkaUtils} import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.scheduler.SplitInfo import org.apache.hadoop.yarn.client.api.AMRMClient import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest +import org.apache.spark.deploy.SparkHadoopUtil /** * An application master that allocates executors on behalf of a driver that is running outside @@ -255,6 +256,8 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp object ExecutorLauncher { def main(argStrings: Array[String]) { val args = new ApplicationMasterArguments(argStrings) - new ExecutorLauncher(args).run() + SparkHadoopUtil.get.runAsSparkUser { () => + new ExecutorLauncher(args).run() + } } }