diff --git a/.gitignore b/.gitignore index a31bf7e0091f4..34939e3a97aaa 100644 --- a/.gitignore +++ b/.gitignore @@ -1,9 +1,12 @@ *~ +*.#* +*#*# *.swp *.ipr *.iml *.iws .idea/ +.idea_modules/ sbt/*.jar .settings .cache @@ -16,9 +19,11 @@ third_party/libmesos.so third_party/libmesos.dylib conf/java-opts conf/*.sh +conf/*.cmd conf/*.properties conf/*.conf conf/*.xml +conf/slaves docs/_site docs/api target/ diff --git a/.rat-excludes b/.rat-excludes index fb6323daf9211..b14ad53720f32 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -19,7 +19,9 @@ log4j.properties log4j.properties.template metrics.properties.template slaves +slaves.template spark-env.sh +spark-env.cmd spark-env.sh.template log4j-defaults.properties bootstrap-tooltip.js @@ -58,3 +60,4 @@ dist/* .*iws logs .*scalastyle-output.xml +.*dependency-reduced-pom.xml diff --git a/assembly/pom.xml b/assembly/pom.xml index 604b1ab3de6a8..31a01e4d8e1de 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -141,7 +141,9 @@ com.google.common.** - com.google.common.base.Optional** + com/google/common/base/Absent* + com/google/common/base/Optional* + com/google/common/base/Present* @@ -347,5 +349,15 @@ + + kinesis-asl + + + org.apache.httpcomponents + httpclient + ${commons.httpclient.version} + + + diff --git a/bagel/src/test/resources/log4j.properties b/bagel/src/test/resources/log4j.properties index 30b4baa4d714a..789869f72e3b0 100644 --- a/bagel/src/test/resources/log4j.properties +++ b/bagel/src/test/resources/log4j.properties @@ -21,7 +21,7 @@ log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false log4j.appender.file.file=target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout -log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose log4j.logger.org.eclipse.jetty=WARN diff --git a/bin/compute-classpath.cmd b/bin/compute-classpath.cmd index 5ad52452a5c98..3cd0579aea8d3 100644 --- a/bin/compute-classpath.cmd +++ b/bin/compute-classpath.cmd @@ -36,7 +36,13 @@ rem Load environment variables from conf\spark-env.cmd, if it exists if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd" rem Build up classpath -set CLASSPATH=%SPARK_CLASSPATH%;%SPARK_SUBMIT_CLASSPATH%;%FWDIR%conf +set CLASSPATH=%SPARK_CLASSPATH%;%SPARK_SUBMIT_CLASSPATH% + +if not "x%SPARK_CONF_DIR%"=="x" ( + set CLASSPATH=%CLASSPATH%;%SPARK_CONF_DIR% +) else ( + set CLASSPATH=%CLASSPATH%;%FWDIR%conf +) if exist "%FWDIR%RELEASE" ( for %%d in ("%FWDIR%lib\spark-assembly*.jar") do ( diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index 0f63e36d8aeca..905bbaf99b374 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -27,8 +27,14 @@ FWDIR="$(cd "`dirname "$0"`"/..; pwd)" . "$FWDIR"/bin/load-spark-env.sh +CLASSPATH="$SPARK_CLASSPATH:$SPARK_SUBMIT_CLASSPATH" + # Build up classpath -CLASSPATH="$SPARK_CLASSPATH:$SPARK_SUBMIT_CLASSPATH:$FWDIR/conf" +if [ -n "$SPARK_CONF_DIR" ]; then + CLASSPATH="$CLASSPATH:$SPARK_CONF_DIR" +else + CLASSPATH="$CLASSPATH:$FWDIR/conf" +fi ASSEMBLY_DIR="$FWDIR/assembly/target/scala-$SCALA_VERSION" diff --git a/bin/pyspark b/bin/pyspark index 5142411e36974..6655725ef8e8e 100755 --- a/bin/pyspark +++ b/bin/pyspark @@ -52,10 +52,20 @@ fi # Figure out which Python executable to use if [[ -z "$PYSPARK_PYTHON" ]]; then - PYSPARK_PYTHON="python" + if [[ "$IPYTHON" = "1" || -n "$IPYTHON_OPTS" ]]; then + # for backward compatibility + PYSPARK_PYTHON="ipython" + else + PYSPARK_PYTHON="python" + fi fi export PYSPARK_PYTHON +if [[ -z "$PYSPARK_PYTHON_OPTS" && -n "$IPYTHON_OPTS" ]]; then + # for backward compatibility + PYSPARK_PYTHON_OPTS="$IPYTHON_OPTS" +fi + # Add the PySpark classes to the Python path: export PYTHONPATH="$SPARK_HOME/python/:$PYTHONPATH" export PYTHONPATH="$SPARK_HOME/python/lib/py4j-0.8.2.1-src.zip:$PYTHONPATH" @@ -64,11 +74,6 @@ export PYTHONPATH="$SPARK_HOME/python/lib/py4j-0.8.2.1-src.zip:$PYTHONPATH" export OLD_PYTHONSTARTUP="$PYTHONSTARTUP" export PYTHONSTARTUP="$FWDIR/python/pyspark/shell.py" -# If IPython options are specified, assume user wants to run IPython -if [[ -n "$IPYTHON_OPTS" ]]; then - IPYTHON=1 -fi - # Build up arguments list manually to preserve quotes and backslashes. # We export Spark submit arguments as an environment variable because shell.py must run as a # PYTHONSTARTUP script, which does not take in arguments. This is required for IPython notebooks. @@ -106,10 +111,5 @@ if [[ "$1" =~ \.py$ ]]; then else # PySpark shell requires special handling downstream export PYSPARK_SHELL=1 - # Only use ipython if no command line arguments were provided [SPARK-1134] - if [[ "$IPYTHON" = "1" ]]; then - exec ${PYSPARK_PYTHON:-ipython} $IPYTHON_OPTS - else - exec "$PYSPARK_PYTHON" - fi + exec "$PYSPARK_PYTHON" $PYSPARK_PYTHON_OPTS fi diff --git a/bin/pyspark2.cmd b/bin/pyspark2.cmd index 2c4b08af8d4c3..a0e66abcc26c9 100644 --- a/bin/pyspark2.cmd +++ b/bin/pyspark2.cmd @@ -33,7 +33,7 @@ for %%d in ("%FWDIR%assembly\target\scala-%SCALA_VERSION%\spark-assembly*hadoop* ) if [%FOUND_JAR%] == [0] ( echo Failed to find Spark assembly JAR. - echo You need to build Spark with sbt\sbt assembly before running this program. + echo You need to build Spark before running this program. goto exit ) :skip_build_test diff --git a/bin/run-example2.cmd b/bin/run-example2.cmd index b29bf90c64e90..b49d0dcb4ff2d 100644 --- a/bin/run-example2.cmd +++ b/bin/run-example2.cmd @@ -52,7 +52,7 @@ if exist "%FWDIR%RELEASE" ( ) if "x%SPARK_EXAMPLES_JAR%"=="x" ( echo Failed to find Spark examples assembly JAR. - echo You need to build Spark with sbt\sbt assembly before running this program. + echo You need to build Spark before running this program. goto exit ) diff --git a/bin/spark-class b/bin/spark-class index 613dc9c4566f2..e8201c18d52de 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -146,7 +146,7 @@ fi if [[ "$1" =~ org.apache.spark.tools.* ]]; then if test -z "$SPARK_TOOLS_JAR"; then echo "Failed to find Spark Tools Jar in $FWDIR/tools/target/scala-$SCALA_VERSION/" 1>&2 - echo "You need to build spark before running $1." 1>&2 + echo "You need to build Spark before running $1." 1>&2 exit 1 fi CLASSPATH="$CLASSPATH:$SPARK_TOOLS_JAR" diff --git a/bin/spark-class2.cmd b/bin/spark-class2.cmd index 6c5672819172b..da46543647efd 100644 --- a/bin/spark-class2.cmd +++ b/bin/spark-class2.cmd @@ -104,7 +104,7 @@ for %%d in ("%FWDIR%assembly\target\scala-%SCALA_VERSION%\spark-assembly*hadoop* ) if "%FOUND_JAR%"=="0" ( echo Failed to find Spark assembly JAR. - echo You need to build Spark with sbt\sbt assembly before running this program. + echo You need to build Spark before running this program. goto exit ) :skip_build_test diff --git a/bin/spark-sql b/bin/spark-sql index ae096530cad04..63d00437d508d 100755 --- a/bin/spark-sql +++ b/bin/spark-sql @@ -24,7 +24,6 @@ set -o posix CLASS="org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver" -CLASS_NOT_FOUND_EXIT_STATUS=1 # Figure out where Spark is installed FWDIR="$(cd "`dirname "$0"`"/..; pwd)" @@ -53,13 +52,4 @@ source "$FWDIR"/bin/utils.sh SUBMIT_USAGE_FUNCTION=usage gatherSparkSubmitOpts "$@" -"$FWDIR"/bin/spark-submit --class $CLASS "${SUBMISSION_OPTS[@]}" spark-internal "${APPLICATION_OPTS[@]}" -exit_status=$? - -if [[ exit_status -eq CLASS_NOT_FOUND_EXIT_STATUS ]]; then - echo - echo "Failed to load Spark SQL CLI main class $CLASS." - echo "You need to build Spark with -Phive." -fi - -exit $exit_status +exec "$FWDIR"/bin/spark-submit --class $CLASS "${SUBMISSION_OPTS[@]}" spark-internal "${APPLICATION_OPTS[@]}" diff --git a/bin/utils.sh b/bin/utils.sh index 0804b1ed9f231..22ea2b9a6d586 100755 --- a/bin/utils.sh +++ b/bin/utils.sh @@ -17,7 +17,7 @@ # limitations under the License. # -# Gather all all spark-submit options into SUBMISSION_OPTS +# Gather all spark-submit options into SUBMISSION_OPTS function gatherSparkSubmitOpts() { if [ -z "$SUBMIT_USAGE_FUNCTION" ]; then diff --git a/conf/slaves b/conf/slaves.template similarity index 100% rename from conf/slaves rename to conf/slaves.template diff --git a/core/pom.xml b/core/pom.xml index 2a81f6df289c0..a5a178079bc57 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -322,6 +322,17 @@ + + maven-clean-plugin + + + + ${basedir}/../python/build + + + true + + org.apache.maven.plugins maven-shade-plugin @@ -343,7 +354,9 @@ com.google.guava:guava + com/google/common/base/Absent* com/google/common/base/Optional* + com/google/common/base/Present* diff --git a/core/src/main/java/org/apache/spark/TaskContext.java b/core/src/main/java/org/apache/spark/TaskContext.java new file mode 100644 index 0000000000000..4e6d708af0ea7 --- /dev/null +++ b/core/src/main/java/org/apache/spark/TaskContext.java @@ -0,0 +1,269 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +import scala.Function0; +import scala.Function1; +import scala.Unit; +import scala.collection.JavaConversions; + +import org.apache.spark.annotation.DeveloperApi; +import org.apache.spark.executor.TaskMetrics; +import org.apache.spark.util.TaskCompletionListener; +import org.apache.spark.util.TaskCompletionListenerException; + +/** +* :: DeveloperApi :: +* Contextual information about a task which can be read or mutated during execution. +*/ +@DeveloperApi +public class TaskContext implements Serializable { + + private int stageId; + private int partitionId; + private long attemptId; + private boolean runningLocally; + private TaskMetrics taskMetrics; + + /** + * :: DeveloperApi :: + * Contextual information about a task which can be read or mutated during execution. + * + * @param stageId stage id + * @param partitionId index of the partition + * @param attemptId the number of attempts to execute this task + * @param runningLocally whether the task is running locally in the driver JVM + * @param taskMetrics performance metrics of the task + */ + @DeveloperApi + public TaskContext(int stageId, int partitionId, long attemptId, boolean runningLocally, + TaskMetrics taskMetrics) { + this.attemptId = attemptId; + this.partitionId = partitionId; + this.runningLocally = runningLocally; + this.stageId = stageId; + this.taskMetrics = taskMetrics; + } + + /** + * :: DeveloperApi :: + * Contextual information about a task which can be read or mutated during execution. + * + * @param stageId stage id + * @param partitionId index of the partition + * @param attemptId the number of attempts to execute this task + * @param runningLocally whether the task is running locally in the driver JVM + */ + @DeveloperApi + public TaskContext(int stageId, int partitionId, long attemptId, boolean runningLocally) { + this.attemptId = attemptId; + this.partitionId = partitionId; + this.runningLocally = runningLocally; + this.stageId = stageId; + this.taskMetrics = TaskMetrics.empty(); + } + + /** + * :: DeveloperApi :: + * Contextual information about a task which can be read or mutated during execution. + * + * @param stageId stage id + * @param partitionId index of the partition + * @param attemptId the number of attempts to execute this task + */ + @DeveloperApi + public TaskContext(int stageId, int partitionId, long attemptId) { + this.attemptId = attemptId; + this.partitionId = partitionId; + this.runningLocally = false; + this.stageId = stageId; + this.taskMetrics = TaskMetrics.empty(); + } + + private static ThreadLocal taskContext = + new ThreadLocal(); + + /** + * :: Internal API :: + * This is spark internal API, not intended to be called from user programs. + */ + public static void setTaskContext(TaskContext tc) { + taskContext.set(tc); + } + + public static TaskContext get() { + return taskContext.get(); + } + + /** :: Internal API :: */ + public static void unset() { + taskContext.remove(); + } + + // List of callback functions to execute when the task completes. + private transient List onCompleteCallbacks = + new ArrayList(); + + // Whether the corresponding task has been killed. + private volatile boolean interrupted = false; + + // Whether the task has completed. + private volatile boolean completed = false; + + /** + * Checks whether the task has completed. + */ + public boolean isCompleted() { + return completed; + } + + /** + * Checks whether the task has been killed. + */ + public boolean isInterrupted() { + return interrupted; + } + + /** + * Add a (Java friendly) listener to be executed on task completion. + * This will be called in all situation - success, failure, or cancellation. + *

+ * An example use is for HadoopRDD to register a callback to close the input stream. + */ + public TaskContext addTaskCompletionListener(TaskCompletionListener listener) { + onCompleteCallbacks.add(listener); + return this; + } + + /** + * Add a listener in the form of a Scala closure to be executed on task completion. + * This will be called in all situations - success, failure, or cancellation. + *

+ * An example use is for HadoopRDD to register a callback to close the input stream. + */ + public TaskContext addTaskCompletionListener(final Function1 f) { + onCompleteCallbacks.add(new TaskCompletionListener() { + @Override + public void onTaskCompletion(TaskContext context) { + f.apply(context); + } + }); + return this; + } + + /** + * 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. + * + * Deprecated: use addTaskCompletionListener + * + * @param f Callback function. + */ + @Deprecated + public void addOnCompleteCallback(final Function0 f) { + onCompleteCallbacks.add(new TaskCompletionListener() { + @Override + public void onTaskCompletion(TaskContext context) { + f.apply(); + } + }); + } + + /** + * ::Internal API:: + * Marks the task as completed and triggers the listeners. + */ + public void markTaskCompleted() throws TaskCompletionListenerException { + completed = true; + List errorMsgs = new ArrayList(2); + // Process complete callbacks in the reverse order of registration + List revlist = + new ArrayList(onCompleteCallbacks); + Collections.reverse(revlist); + for (TaskCompletionListener tcl: revlist) { + try { + tcl.onTaskCompletion(this); + } catch (Throwable e) { + errorMsgs.add(e.getMessage()); + } + } + + if (!errorMsgs.isEmpty()) { + throw new TaskCompletionListenerException(JavaConversions.asScalaBuffer(errorMsgs)); + } + } + + /** + * ::Internal API:: + * Marks the task for interruption, i.e. cancellation. + */ + public void markInterrupted() { + interrupted = true; + } + + @Deprecated + /** Deprecated: use getStageId() */ + public int stageId() { + return stageId; + } + + @Deprecated + /** Deprecated: use getPartitionId() */ + public int partitionId() { + return partitionId; + } + + @Deprecated + /** Deprecated: use getAttemptId() */ + public long attemptId() { + return attemptId; + } + + @Deprecated + /** Deprecated: use isRunningLocally() */ + public boolean runningLocally() { + return runningLocally; + } + + public boolean isRunningLocally() { + return runningLocally; + } + + public int getStageId() { + return stageId; + } + + public int getPartitionId() { + return partitionId; + } + + public long getAttemptId() { + return attemptId; + } + + /** ::Internal API:: */ + public TaskMetrics taskMetrics() { + return taskMetrics; + } +} diff --git a/core/src/main/resources/org/apache/spark/ui/static/webui.css b/core/src/main/resources/org/apache/spark/ui/static/webui.css index 445110d63e184..152bde5f6994f 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/webui.css +++ b/core/src/main/resources/org/apache/spark/ui/static/webui.css @@ -51,6 +51,11 @@ table.sortable thead { cursor: pointer; } +table.sortable td { + word-wrap: break-word; + max-width: 600px; +} + .progress { margin-bottom: 0px; position: relative } diff --git a/core/src/main/scala/org/apache/spark/CacheManager.scala b/core/src/main/scala/org/apache/spark/CacheManager.scala index f8584b90cabe6..d89bb50076c9a 100644 --- a/core/src/main/scala/org/apache/spark/CacheManager.scala +++ b/core/src/main/scala/org/apache/spark/CacheManager.scala @@ -168,8 +168,6 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { arr.iterator.asInstanceOf[Iterator[T]] case Right(it) => // There is not enough space to cache this partition in memory - logWarning(s"Not enough space to cache partition $key in memory! " + - s"Free memory is ${blockManager.memoryStore.freeMemory} bytes.") val returnValues = it.asInstanceOf[Iterator[T]] if (putLevel.useDisk) { logWarning(s"Persisting partition $key to disk instead.") diff --git a/core/src/main/scala/org/apache/spark/FutureAction.scala b/core/src/main/scala/org/apache/spark/FutureAction.scala index 75ea535f2f57b..e8f761eaa5799 100644 --- a/core/src/main/scala/org/apache/spark/FutureAction.scala +++ b/core/src/main/scala/org/apache/spark/FutureAction.scala @@ -83,6 +83,15 @@ trait FutureAction[T] extends Future[T] { */ @throws(classOf[Exception]) def get(): T = Await.result(this, Duration.Inf) + + /** + * Returns the job IDs run by the underlying async operation. + * + * This returns the current snapshot of the job list. Certain operations may run multiple + * jobs, so multiple calls to this method may return different lists. + */ + def jobIds: Seq[Int] + } @@ -150,8 +159,7 @@ class SimpleFutureAction[T] private[spark](jobWaiter: JobWaiter[_], resultFunc: } } - /** Get the corresponding job id for this action. */ - def jobId = jobWaiter.jobId + def jobIds = Seq(jobWaiter.jobId) } @@ -171,6 +179,8 @@ class ComplexFutureAction[T] extends FutureAction[T] { // is cancelled before the action was even run (and thus we have no thread to interrupt). @volatile private var _cancelled: Boolean = false + @volatile private var jobs: Seq[Int] = Nil + // A promise used to signal the future. private val p = promise[T]() @@ -219,6 +229,8 @@ class ComplexFutureAction[T] extends FutureAction[T] { } } + this.jobs = jobs ++ job.jobIds + // Wait for the job to complete. If the action is cancelled (with an interrupt), // cancel the job and stop the execution. This is not in a synchronized block because // Await.ready eventually waits on the monitor in FutureJob.jobWaiter. @@ -255,4 +267,7 @@ class ComplexFutureAction[T] extends FutureAction[T] { override def isCompleted: Boolean = p.isCompleted override def value: Option[Try[T]] = p.future.value + + def jobIds = jobs + } diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 51705c895a55c..4cb0bd4142435 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -18,10 +18,12 @@ package org.apache.spark import java.io._ +import java.util.concurrent.ConcurrentHashMap import java.util.zip.{GZIPInputStream, GZIPOutputStream} import scala.collection.mutable.{HashSet, HashMap, Map} import scala.concurrent.Await +import scala.collection.JavaConversions._ import akka.actor._ import akka.pattern.ask @@ -84,6 +86,9 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging * On the master, it serves as the source of map outputs recorded from ShuffleMapTasks. * On the workers, it simply serves as a cache, in which a miss triggers a fetch from the * master's corresponding HashMap. + * + * Note: because mapStatuses is accessed concurrently, subclasses should make sure it's a + * thread-safe map. */ protected val mapStatuses: Map[Int, Array[MapStatus]] @@ -339,11 +344,11 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf) * MapOutputTrackerMaster. */ private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTracker(conf) { - protected val mapStatuses = new HashMap[Int, Array[MapStatus]] + protected val mapStatuses: Map[Int, Array[MapStatus]] = + new ConcurrentHashMap[Int, Array[MapStatus]] } private[spark] object MapOutputTracker { - private val LOG_BASE = 1.1 // Serialize an array of map output locations into an efficient byte format so that we can send // it to reduce tasks. We do this by compressing the serialized bytes using GZIP. They will @@ -379,34 +384,8 @@ private[spark] object MapOutputTracker { throw new MetadataFetchFailedException( shuffleId, reduceId, "Missing an output location for shuffle " + shuffleId) } else { - (status.location, decompressSize(status.compressedSizes(reduceId))) + (status.location, status.getSizeForBlock(reduceId)) } } } - - /** - * Compress a size in bytes to 8 bits for efficient reporting of map output sizes. - * We do this by encoding the log base 1.1 of the size as an integer, which can support - * sizes up to 35 GB with at most 10% error. - */ - def compressSize(size: Long): Byte = { - if (size == 0) { - 0 - } else if (size <= 1L) { - 1 - } else { - math.min(255, math.ceil(math.log(size) / math.log(LOG_BASE)).toInt).toByte - } - } - - /** - * Decompress an 8-bit encoded block size, using the reverse operation of compressSize. - */ - def decompressSize(compressedSize: Byte): Long = { - if (compressedSize == 0) { - 0 - } else { - math.pow(LOG_BASE, compressedSize & 0xFF).toLong - } - } } diff --git a/core/src/main/scala/org/apache/spark/SecurityManager.scala b/core/src/main/scala/org/apache/spark/SecurityManager.scala index 3832a780ec4bc..0e0f1a7b2377e 100644 --- a/core/src/main/scala/org/apache/spark/SecurityManager.scala +++ b/core/src/main/scala/org/apache/spark/SecurityManager.scala @@ -103,10 +103,9 @@ import org.apache.spark.deploy.SparkHadoopUtil * and a Server, so for a particular connection is has to determine what to do. * A ConnectionId was added to be able to track connections and is used to * match up incoming messages with connections waiting for authentication. - * If its acting as a client and trying to send a message to another ConnectionManager, - * it blocks the thread calling sendMessage until the SASL negotiation has occurred. * The ConnectionManager tracks all the sendingConnections using the ConnectionId - * and waits for the response from the server and does the handshake. + * and waits for the response from the server and does the handshake before sending + * the real message. * * - HTTP for the Spark UI -> the UI was changed to use servlets so that javax servlet filters * can be used. Yarn requires a specific AmIpFilter be installed for security to work diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 428f019b02a23..396cdd1247e07 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -187,6 +187,15 @@ class SparkContext(config: SparkConf) extends Logging { val master = conf.get("spark.master") val appName = conf.get("spark.app.name") + private[spark] val isEventLogEnabled = conf.getBoolean("spark.eventLog.enabled", false) + private[spark] val eventLogDir: Option[String] = { + if (isEventLogEnabled) { + Some(conf.get("spark.eventLog.dir", EventLoggingListener.DEFAULT_LOG_DIR).stripSuffix("/")) + } else { + None + } + } + // Generate the random name for a temp folder in Tachyon // Add a timestamp as the suffix here to make it more safe val tachyonFolderName = "spark-" + randomUUID.toString() @@ -200,6 +209,7 @@ class SparkContext(config: SparkConf) extends Logging { private[spark] val listenerBus = new LiveListenerBus // Create the Spark execution environment (cache, map output tracker, etc) + conf.set("spark.executor.id", "driver") private[spark] val env = SparkEnv.create( conf, "", @@ -232,19 +242,6 @@ class SparkContext(config: SparkConf) extends Logging { /** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */ val hadoopConfiguration = SparkHadoopUtil.get.newConfiguration(conf) - // Optionally log Spark events - private[spark] val eventLogger: Option[EventLoggingListener] = { - if (conf.getBoolean("spark.eventLog.enabled", false)) { - val logger = new EventLoggingListener(appName, conf, hadoopConfiguration) - logger.start() - listenerBus.addListener(logger) - Some(logger) - } else None - } - - // At this point, all relevant SparkListeners have been registered, so begin releasing events - listenerBus.start() - val startTime = System.currentTimeMillis() // Add each JAR given through the constructor @@ -309,6 +306,29 @@ class SparkContext(config: SparkConf) extends Logging { // constructor taskScheduler.start() + val applicationId: String = taskScheduler.applicationId() + conf.set("spark.app.id", applicationId) + + val metricsSystem = env.metricsSystem + + // The metrics system for Driver need to be set spark.app.id to app ID. + // So it should start after we get app ID from the task scheduler and set spark.app.id. + metricsSystem.start() + + // Optionally log Spark events + private[spark] val eventLogger: Option[EventLoggingListener] = { + if (isEventLogEnabled) { + val logger = + new EventLoggingListener(applicationId, eventLogDir.get, conf, hadoopConfiguration) + logger.start() + listenerBus.addListener(logger) + Some(logger) + } else None + } + + // At this point, all relevant SparkListeners have been registered, so begin releasing events + listenerBus.start() + private[spark] val cleaner: Option[ContextCleaner] = { if (conf.getBoolean("spark.cleaner.referenceTracking", true)) { Some(new ContextCleaner(this)) @@ -411,8 +431,8 @@ class SparkContext(config: SparkConf) extends Logging { // Post init taskScheduler.postStartHook() - private val dagSchedulerSource = new DAGSchedulerSource(this.dagScheduler, this) - private val blockManagerSource = new BlockManagerSource(SparkEnv.get.blockManager, this) + private val dagSchedulerSource = new DAGSchedulerSource(this.dagScheduler) + private val blockManagerSource = new BlockManagerSource(SparkEnv.get.blockManager) private def initDriverMetrics() { SparkEnv.get.metricsSystem.registerSource(dagSchedulerSource) @@ -759,20 +779,20 @@ class SparkContext(config: SparkConf) extends Logging { /** * Create an [[org.apache.spark.Accumulable]] shared variable, to which tasks can add values * with `+=`. Only the driver can access the accumuable's `value`. - * @tparam T accumulator type - * @tparam R type that can be added to the accumulator + * @tparam R accumulator result type + * @tparam T type that can be added to the accumulator */ - def accumulable[T, R](initialValue: T)(implicit param: AccumulableParam[T, R]) = + def accumulable[R, T](initialValue: R)(implicit param: AccumulableParam[R, T]) = new Accumulable(initialValue, param) /** * Create an [[org.apache.spark.Accumulable]] shared variable, with a name for display in the * Spark UI. Tasks can add values to the accumuable using the `+=` operator. Only the driver can * access the accumuable's `value`. - * @tparam T accumulator type - * @tparam R type that can be added to the accumulator + * @tparam R accumulator result type + * @tparam T type that can be added to the accumulator */ - def accumulable[T, R](initialValue: T, name: String)(implicit param: AccumulableParam[T, R]) = + def accumulable[R, T](initialValue: R, name: String)(implicit param: AccumulableParam[R, T]) = new Accumulable(initialValue, param, Some(name)) /** @@ -1030,28 +1050,40 @@ class SparkContext(config: SparkConf) extends Logging { } /** - * Support function for API backtraces. + * Set the thread-local property for overriding the call sites + * of actions and RDDs. + */ + def setCallSite(shortCallSite: String) { + setLocalProperty(CallSite.SHORT_FORM, shortCallSite) + } + + /** + * Set the thread-local property for overriding the call sites + * of actions and RDDs. */ - def setCallSite(site: String) { - setLocalProperty("externalCallSite", site) + private[spark] def setCallSite(callSite: CallSite) { + setLocalProperty(CallSite.SHORT_FORM, callSite.shortForm) + setLocalProperty(CallSite.LONG_FORM, callSite.longForm) } /** - * Support function for API backtraces. + * Clear the thread-local property for overriding the call sites + * of actions and RDDs. */ def clearCallSite() { - setLocalProperty("externalCallSite", null) + setLocalProperty(CallSite.SHORT_FORM, null) + setLocalProperty(CallSite.LONG_FORM, null) } /** * Capture the current user callsite and return a formatted version for printing. If the user - * has overridden the call site, this will return the user's version. + * has overridden the call site using `setCallSite()`, this will return the user's version. */ private[spark] def getCallSite(): CallSite = { - Option(getLocalProperty("externalCallSite")) match { - case Some(callSite) => CallSite(callSite, longForm = "") - case None => Utils.getCallSite - } + Option(getLocalProperty(CallSite.SHORT_FORM)).map { case shortCallSite => + val longCallSite = Option(getLocalProperty(CallSite.LONG_FORM)).getOrElse("") + CallSite(shortCallSite, longCallSite) + }.getOrElse(Utils.getCallSite()) } /** @@ -1266,7 +1298,7 @@ class SparkContext(config: SparkConf) extends Logging { private def postApplicationStart() { // Note: this code assumes that the task scheduler has been initialized and has contacted // the cluster manager to get an application ID (in case the cluster manager provides one). - listenerBus.post(SparkListenerApplicationStart(appName, taskScheduler.applicationId(), + listenerBus.post(SparkListenerApplicationStart(appName, Some(applicationId), startTime, sparkUser)) } diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 009ed64775844..aba713cb4267a 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -43,9 +43,8 @@ import org.apache.spark.util.{AkkaUtils, Utils} * :: DeveloperApi :: * Holds all the runtime environment objects for a running Spark instance (either master or worker), * including the serializer, Akka actor system, block manager, map output tracker, etc. Currently - * Spark code finds the SparkEnv through a thread-local variable, so each thread that accesses these - * objects needs to have the right SparkEnv set. You can get the current environment with - * SparkEnv.get (e.g. after creating a SparkContext) and set it with SparkEnv.set. + * Spark code finds the SparkEnv through a global variable, so all the threads can access the same + * SparkEnv. It can be accessed by SparkEnv.get (e.g. after creating a SparkContext). * * NOTE: This is not intended for external use. This is exposed for Shark and may be made private * in a future release. @@ -119,30 +118,28 @@ class SparkEnv ( } object SparkEnv extends Logging { - private val env = new ThreadLocal[SparkEnv] - @volatile private var lastSetSparkEnv : SparkEnv = _ + @volatile private var env: SparkEnv = _ private[spark] val driverActorSystemName = "sparkDriver" private[spark] val executorActorSystemName = "sparkExecutor" def set(e: SparkEnv) { - lastSetSparkEnv = e - env.set(e) + env = e } /** - * Returns the ThreadLocal SparkEnv, if non-null. Else returns the SparkEnv - * previously set in any thread. + * Returns the SparkEnv. */ def get: SparkEnv = { - Option(env.get()).getOrElse(lastSetSparkEnv) + env } /** * Returns the ThreadLocal SparkEnv. */ + @deprecated("Use SparkEnv.get instead", "1.2") def getThreadLocal: SparkEnv = { - env.get() + env } private[spark] def create( @@ -259,11 +256,15 @@ object SparkEnv extends Logging { } val metricsSystem = if (isDriver) { + // Don't start metrics system right now for Driver. + // We need to wait for the task scheduler to give us an app ID. + // Then we can start the metrics system. MetricsSystem.createMetricsSystem("driver", conf, securityManager) } else { - MetricsSystem.createMetricsSystem("executor", conf, securityManager) + val ms = MetricsSystem.createMetricsSystem("executor", conf, securityManager) + ms.start() + ms } - metricsSystem.start() // Set the sparkFiles directory, used when downloading dependencies. In local mode, // this is a temporary directory; in distributed mode, this is the executor's current working diff --git a/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala b/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala index f6703986bdf11..376e69cd997d5 100644 --- a/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala +++ b/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala @@ -116,7 +116,7 @@ class SparkHadoopWriter(@transient jobConf: JobConf) } } } else { - logWarning ("No need to commit output of task: " + taID.value) + logInfo ("No need to commit output of task: " + taID.value) } } diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala deleted file mode 100644 index 51b3e4d5e0936..0000000000000 --- a/core/src/main/scala/org/apache/spark/TaskContext.scala +++ /dev/null @@ -1,126 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark - -import scala.collection.mutable.ArrayBuffer - -import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.executor.TaskMetrics -import org.apache.spark.util.{TaskCompletionListenerException, TaskCompletionListener} - - -/** - * :: DeveloperApi :: - * Contextual information about a task which can be read or mutated during execution. - * - * @param stageId stage id - * @param partitionId index of the partition - * @param attemptId the number of attempts to execute this task - * @param runningLocally whether the task is running locally in the driver JVM - * @param taskMetrics performance metrics of the task - */ -@DeveloperApi -class TaskContext( - val stageId: Int, - val partitionId: Int, - val attemptId: Long, - val runningLocally: Boolean = false, - private[spark] val taskMetrics: TaskMetrics = TaskMetrics.empty) - extends Serializable with Logging { - - @deprecated("use partitionId", "0.8.1") - def splitId = partitionId - - // List of callback functions to execute when the task completes. - @transient private val onCompleteCallbacks = new ArrayBuffer[TaskCompletionListener] - - // Whether the corresponding task has been killed. - @volatile private var interrupted: Boolean = false - - // Whether the task has completed. - @volatile private var completed: Boolean = false - - /** Checks whether the task has completed. */ - def isCompleted: Boolean = completed - - /** Checks whether the task has been killed. */ - def isInterrupted: Boolean = interrupted - - // TODO: Also track whether the task has completed successfully or with exception. - - /** - * Add a (Java friendly) listener to be executed on task completion. - * This will be called in all situation - success, failure, or cancellation. - * - * An example use is for HadoopRDD to register a callback to close the input stream. - */ - def addTaskCompletionListener(listener: TaskCompletionListener): this.type = { - onCompleteCallbacks += listener - this - } - - /** - * Add a listener in the form of a Scala closure to be executed on task completion. - * This will be called in all situation - success, failure, or cancellation. - * - * An example use is for HadoopRDD to register a callback to close the input stream. - */ - def addTaskCompletionListener(f: TaskContext => Unit): this.type = { - onCompleteCallbacks += new TaskCompletionListener { - override def onTaskCompletion(context: TaskContext): Unit = f(context) - } - this - } - - /** - * Add a callback function to be executed on task completion. An example use - * 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. - */ - @deprecated("use addTaskCompletionListener", "1.1.0") - def addOnCompleteCallback(f: () => Unit) { - onCompleteCallbacks += new TaskCompletionListener { - override def onTaskCompletion(context: TaskContext): Unit = f() - } - } - - /** Marks the task as completed and triggers the listeners. */ - private[spark] def markTaskCompleted(): Unit = { - completed = true - val errorMsgs = new ArrayBuffer[String](2) - // Process complete callbacks in the reverse order of registration - onCompleteCallbacks.reverse.foreach { listener => - try { - listener.onTaskCompletion(this) - } catch { - case e: Throwable => - errorMsgs += e.getMessage - logError("Error in TaskCompletionListener", e) - } - } - if (errorMsgs.nonEmpty) { - throw new TaskCompletionListenerException(errorMsgs) - } - } - - /** Marks the task for interruption, i.e. cancellation. */ - private[spark] def markInterrupted(): Unit = { - interrupted = true - } -} diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index 880f61c49726e..0846225e4f992 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -469,6 +469,22 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) fromRDD(joinResult.mapValues{case (v, w) => (JavaUtils.optionToOptional(v), w)}) } + /** + * Perform a full outer join of `this` and `other`. For each element (k, v) in `this`, the + * resulting RDD will either contain all pairs (k, (Some(v), Some(w))) for w in `other`, or + * the pair (k, (Some(v), None)) if no elements in `other` have key k. Similarly, for each + * element (k, w) in `other`, the resulting RDD will either contain all pairs + * (k, (Some(v), Some(w))) for v in `this`, or the pair (k, (None, Some(w))) if no elements + * in `this` have key k. Uses the given Partitioner to partition the output RDD. + */ + def fullOuterJoin[W](other: JavaPairRDD[K, W], partitioner: Partitioner) + : JavaPairRDD[K, (Optional[V], Optional[W])] = { + val joinResult = rdd.fullOuterJoin(other, partitioner) + fromRDD(joinResult.mapValues{ case (v, w) => + (JavaUtils.optionToOptional(v), JavaUtils.optionToOptional(w)) + }) + } + /** * Simplified version of combineByKey that hash-partitions the resulting RDD using the existing * partitioner/parallelism level. @@ -563,6 +579,38 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) fromRDD(joinResult.mapValues{case (v, w) => (JavaUtils.optionToOptional(v), w)}) } + /** + * Perform a full outer join of `this` and `other`. For each element (k, v) in `this`, the + * resulting RDD will either contain all pairs (k, (Some(v), Some(w))) for w in `other`, or + * the pair (k, (Some(v), None)) if no elements in `other` have key k. Similarly, for each + * element (k, w) in `other`, the resulting RDD will either contain all pairs + * (k, (Some(v), Some(w))) for v in `this`, or the pair (k, (None, Some(w))) if no elements + * in `this` have key k. Hash-partitions the resulting RDD using the existing partitioner/ + * parallelism level. + */ + def fullOuterJoin[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (Optional[V], Optional[W])] = { + val joinResult = rdd.fullOuterJoin(other) + fromRDD(joinResult.mapValues{ case (v, w) => + (JavaUtils.optionToOptional(v), JavaUtils.optionToOptional(w)) + }) + } + + /** + * Perform a full outer join of `this` and `other`. For each element (k, v) in `this`, the + * resulting RDD will either contain all pairs (k, (Some(v), Some(w))) for w in `other`, or + * the pair (k, (Some(v), None)) if no elements in `other` have key k. Similarly, for each + * element (k, w) in `other`, the resulting RDD will either contain all pairs + * (k, (Some(v), Some(w))) for v in `this`, or the pair (k, (None, Some(w))) if no elements + * in `this` have key k. Hash-partitions the resulting RDD into the given number of partitions. + */ + def fullOuterJoin[W](other: JavaPairRDD[K, W], numPartitions: Int) + : JavaPairRDD[K, (Optional[V], Optional[W])] = { + val joinResult = rdd.fullOuterJoin(other, numPartitions) + fromRDD(joinResult.mapValues{ case (v, w) => + (JavaUtils.optionToOptional(v), JavaUtils.optionToOptional(w)) + }) + } + /** * Return the key-value pairs in this RDD to the master as a Map. */ 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 12b345a8fa7c3..c74f86548ef85 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 @@ -196,7 +196,6 @@ private[spark] class PythonRDD( override def run(): Unit = Utils.logUncaughtExceptions { try { - SparkEnv.set(env) val stream = new BufferedOutputStream(worker.getOutputStream, bufferSize) val dataOut = new DataOutputStream(stream) // Partition index @@ -248,6 +247,11 @@ private[spark] class PythonRDD( // will kill the whole executor (see org.apache.spark.executor.Executor). _exception = e worker.shutdownOutput() + } finally { + // Release memory used by this thread for shuffles + env.shuffleMemoryManager.releaseMemoryForThisThread() + // Release memory used by this thread for unrolling blocks + env.blockManager.memoryStore.releaseUnrollMemoryForThisThread() } } } @@ -339,26 +343,34 @@ private[spark] object PythonRDD extends Logging { def readRDDFromFile(sc: JavaSparkContext, filename: String, parallelism: Int): JavaRDD[Array[Byte]] = { val file = new DataInputStream(new FileInputStream(filename)) - val objs = new collection.mutable.ArrayBuffer[Array[Byte]] try { - while (true) { - val length = file.readInt() - val obj = new Array[Byte](length) - file.readFully(obj) - objs.append(obj) + val objs = new collection.mutable.ArrayBuffer[Array[Byte]] + try { + while (true) { + val length = file.readInt() + val obj = new Array[Byte](length) + file.readFully(obj) + objs.append(obj) + } + } catch { + case eof: EOFException => {} } - } catch { - case eof: EOFException => {} + JavaRDD.fromRDD(sc.sc.parallelize(objs, parallelism)) + } finally { + file.close() } - JavaRDD.fromRDD(sc.sc.parallelize(objs, parallelism)) } def readBroadcastFromFile(sc: JavaSparkContext, filename: String): Broadcast[Array[Byte]] = { val file = new DataInputStream(new FileInputStream(filename)) - val length = file.readInt() - val obj = new Array[Byte](length) - file.readFully(obj) - sc.broadcast(obj) + try { + val length = file.readInt() + val obj = new Array[Byte](length) + file.readFully(obj) + sc.broadcast(obj) + } finally { + file.close() + } } def writeIteratorToStream[T](iter: Iterator[T], dataOut: DataOutputStream) { @@ -775,17 +787,36 @@ private[spark] object PythonRDD extends Logging { }.toJavaRDD() } + private class AutoBatchedPickler(iter: Iterator[Any]) extends Iterator[Array[Byte]] { + private val pickle = new Pickler() + private var batch = 1 + private val buffer = new mutable.ArrayBuffer[Any] + + override def hasNext(): Boolean = iter.hasNext + + override def next(): Array[Byte] = { + while (iter.hasNext && buffer.length < batch) { + buffer += iter.next() + } + val bytes = pickle.dumps(buffer.toArray) + val size = bytes.length + // let 1M < size < 10M + if (size < 1024 * 1024) { + batch *= 2 + } else if (size > 1024 * 1024 * 10 && batch > 1) { + batch /= 2 + } + buffer.clear() + bytes + } + } + /** * Convert an RDD of Java objects to an RDD of serialized Python objects, that is usable by * PySpark. */ def javaToPython(jRDD: JavaRDD[Any]): JavaRDD[Array[Byte]] = { - jRDD.rdd.mapPartitions { iter => - val pickle = new Pickler - iter.map { row => - pickle.dumps(row) - } - } + jRDD.rdd.mapPartitions { iter => new AutoBatchedPickler(iter) } } /** diff --git a/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala b/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala index 6668797f5f8be..7903457b17e13 100644 --- a/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala +++ b/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala @@ -68,8 +68,8 @@ private[python] object SerDeUtil extends Logging { construct(args ++ Array("")) } else if (args.length == 2 && args(1).isInstanceOf[String]) { val typecode = args(0).asInstanceOf[String].charAt(0) - val data: String = args(1).asInstanceOf[String] - construct(typecode, machineCodes(typecode), data.getBytes("ISO-8859-1")) + val data: Array[Byte] = args(1).asInstanceOf[String].getBytes("ISO-8859-1") + construct(typecode, machineCodes(typecode), data) } else { super.construct(args) } diff --git a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala index 942dc7d7eac87..4cd4f4f96fd16 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala @@ -163,18 +163,23 @@ private[broadcast] object HttpBroadcast extends Logging { private def write(id: Long, value: Any) { val file = getFile(id) - val out: OutputStream = { - if (compress) { - compressionCodec.compressedOutputStream(new FileOutputStream(file)) - } else { - new BufferedOutputStream(new FileOutputStream(file), bufferSize) + val fileOutputStream = new FileOutputStream(file) + try { + val out: OutputStream = { + if (compress) { + compressionCodec.compressedOutputStream(fileOutputStream) + } else { + new BufferedOutputStream(fileOutputStream, bufferSize) + } } + val ser = SparkEnv.get.serializer.newInstance() + val serOut = ser.serializeStream(out) + serOut.writeObject(value) + serOut.close() + files += file + } finally { + fileOutputStream.close() } - val ser = SparkEnv.get.serializer.newInstance() - val serOut = ser.serializeStream(out) - serOut.writeObject(value) - serOut.close() - files += file } private def read[T: ClassTag](id: Long): T = { diff --git a/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala b/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala index b66c3ba4d5fb0..79b4d7ea41a33 100644 --- a/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala @@ -54,9 +54,10 @@ object PythonRunner { val pythonPath = PythonUtils.mergePythonPaths(pathElements: _*) // Launch Python process - val builder = new ProcessBuilder(Seq(pythonExec, "-u", formattedPythonFile) ++ otherArgs) + val builder = new ProcessBuilder(Seq(pythonExec, formattedPythonFile) ++ otherArgs) val env = builder.environment() env.put("PYTHONPATH", pythonPath) + env.put("PYTHONUNBUFFERED", "YES") // value is needed to be set to a non-empty string env.put("PYSPARK_GATEWAY_PORT", "" + gatewayServer.getListeningPort) builder.redirectErrorStream(true) // Ugly but needed for stdout and stderr to synchronize val process = builder.start() 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 5ed3575816a38..f97bf67fa5a3b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -54,7 +54,7 @@ object SparkSubmit { private val SPARK_SHELL = "spark-shell" private val PYSPARK_SHELL = "pyspark-shell" - private val CLASS_NOT_FOUND_EXIT_STATUS = 1 + private val CLASS_NOT_FOUND_EXIT_STATUS = 101 // Exposed for testing private[spark] var exitFn: () => Unit = () => System.exit(-1) @@ -172,7 +172,7 @@ object SparkSubmit { // All cluster managers OptionAssigner(args.master, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES, sysProp = "spark.master"), OptionAssigner(args.name, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES, sysProp = "spark.app.name"), - OptionAssigner(args.jars, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES, sysProp = "spark.jars"), + OptionAssigner(args.jars, ALL_CLUSTER_MGRS, CLIENT, sysProp = "spark.jars"), OptionAssigner(args.driverMemory, ALL_CLUSTER_MGRS, CLIENT, sysProp = "spark.driver.memory"), OptionAssigner(args.driverExtraClassPath, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES, @@ -183,6 +183,7 @@ object SparkSubmit { sysProp = "spark.driver.extraLibraryPath"), // Standalone cluster only + OptionAssigner(args.jars, STANDALONE, CLUSTER, sysProp = "spark.jars"), OptionAssigner(args.driverMemory, STANDALONE, CLUSTER, clOption = "--memory"), OptionAssigner(args.driverCores, STANDALONE, CLUSTER, clOption = "--cores"), @@ -261,7 +262,7 @@ object SparkSubmit { } // In yarn-cluster mode, use yarn.Client as a wrapper around the user class - if (clusterManager == YARN && deployMode == CLUSTER) { + if (isYarnCluster) { childMainClass = "org.apache.spark.deploy.yarn.Client" if (args.primaryResource != SPARK_INTERNAL) { childArgs += ("--jar", args.primaryResource) @@ -279,7 +280,7 @@ object SparkSubmit { } // Read from default spark properties, if any - for ((k, v) <- args.getDefaultSparkProperties) { + for ((k, v) <- args.defaultSparkProperties) { sysProps.getOrElseUpdate(k, v) } @@ -319,6 +320,10 @@ object SparkSubmit { } catch { case e: ClassNotFoundException => e.printStackTrace(printStream) + if (childMainClass.contains("thriftserver")) { + println(s"Failed to load main class $childMainClass.") + println("You need to build Spark with -Phive.") + } System.exit(CLASS_NOT_FOUND_EXIT_STATUS) } 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 d545f58c5da7e..57b251ff47714 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -29,8 +29,9 @@ import org.apache.spark.util.Utils /** * Parses and encapsulates arguments from the spark-submit script. + * The env argument is used for testing. */ -private[spark] class SparkSubmitArguments(args: Seq[String]) { +private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, String] = sys.env) { var master: String = null var deployMode: String = null var executorMemory: String = null @@ -57,12 +58,8 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { var pyFiles: String = null val sparkProperties: HashMap[String, String] = new HashMap[String, String]() - parseOpts(args.toList) - mergeSparkProperties() - checkRequiredArguments() - - /** Return default present in the currently defined defaults file. */ - def getDefaultSparkProperties = { + /** Default properties present in the currently defined defaults file. */ + lazy val defaultSparkProperties: HashMap[String, String] = { val defaultProperties = new HashMap[String, String]() if (verbose) SparkSubmit.printStream.println(s"Using properties file: $propertiesFile") Option(propertiesFile).foreach { filename => @@ -79,6 +76,14 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { defaultProperties } + // Respect SPARK_*_MEMORY for cluster mode + driverMemory = sys.env.get("SPARK_DRIVER_MEMORY").orNull + executorMemory = sys.env.get("SPARK_EXECUTOR_MEMORY").orNull + + parseOpts(args.toList) + mergeSparkProperties() + checkRequiredArguments() + /** * Fill in any undefined values based on the default properties file or options passed in through * the '--conf' flag. @@ -86,20 +91,12 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { private def mergeSparkProperties(): Unit = { // Use common defaults file, if not specified by user if (propertiesFile == null) { - sys.env.get("SPARK_CONF_DIR").foreach { sparkConfDir => - val sep = File.separator - val defaultPath = s"${sparkConfDir}${sep}spark-defaults.conf" - val file = new File(defaultPath) - if (file.exists()) { - propertiesFile = file.getAbsolutePath - } - } - } + val sep = File.separator + val sparkHomeConfig = env.get("SPARK_HOME").map(sparkHome => s"${sparkHome}${sep}conf") + val confDir = env.get("SPARK_CONF_DIR").orElse(sparkHomeConfig) - if (propertiesFile == null) { - sys.env.get("SPARK_HOME").foreach { sparkHome => - val sep = File.separator - val defaultPath = s"${sparkHome}${sep}conf${sep}spark-defaults.conf" + confDir.foreach { sparkConfDir => + val defaultPath = s"${sparkConfDir}${sep}spark-defaults.conf" val file = new File(defaultPath) if (file.exists()) { propertiesFile = file.getAbsolutePath @@ -107,24 +104,24 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { } } - val properties = getDefaultSparkProperties + val properties = HashMap[String, String]() + properties.putAll(defaultSparkProperties) properties.putAll(sparkProperties) // Use properties file as fallback for values which have a direct analog to // arguments in this script. - master = Option(master).getOrElse(properties.get("spark.master").orNull) - executorMemory = Option(executorMemory) - .getOrElse(properties.get("spark.executor.memory").orNull) - executorCores = Option(executorCores) - .getOrElse(properties.get("spark.executor.cores").orNull) + master = Option(master).orElse(properties.get("spark.master")).orNull + executorMemory = Option(executorMemory).orElse(properties.get("spark.executor.memory")).orNull + executorCores = Option(executorCores).orElse(properties.get("spark.executor.cores")).orNull totalExecutorCores = Option(totalExecutorCores) - .getOrElse(properties.get("spark.cores.max").orNull) - name = Option(name).getOrElse(properties.get("spark.app.name").orNull) - jars = Option(jars).getOrElse(properties.get("spark.jars").orNull) + .orElse(properties.get("spark.cores.max")) + .orNull + name = Option(name).orElse(properties.get("spark.app.name")).orNull + jars = Option(jars).orElse(properties.get("spark.jars")).orNull // This supports env vars in older versions of Spark - master = Option(master).getOrElse(System.getenv("MASTER")) - deployMode = Option(deployMode).getOrElse(System.getenv("DEPLOY_MODE")) + master = Option(master).orElse(env.get("MASTER")).orNull + deployMode = Option(deployMode).orElse(env.get("DEPLOY_MODE")).orNull // Try to set main class from JAR if no --class argument is given if (mainClass == null && !isPython && primaryResource != null) { @@ -177,7 +174,7 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { } if (master.startsWith("yarn")) { - val hasHadoopEnv = sys.env.contains("HADOOP_CONF_DIR") || sys.env.contains("YARN_CONF_DIR") + val hasHadoopEnv = env.contains("HADOOP_CONF_DIR") || env.contains("YARN_CONF_DIR") if (!hasHadoopEnv && !Utils.isTesting) { throw new Exception(s"When running with master '$master' " + "either HADOOP_CONF_DIR or YARN_CONF_DIR must be set in the environment.") @@ -213,7 +210,7 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { | verbose $verbose | |Default properties from $propertiesFile: - |${getDefaultSparkProperties.mkString(" ", "\n ", "\n")} + |${defaultSparkProperties.mkString(" ", "\n ", "\n")} """.stripMargin } diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala index 38b5d8e1739d0..a64170a47bc1c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala @@ -154,7 +154,8 @@ private[spark] object SparkSubmitDriverBootstrapper { process.destroy() } } - process.waitFor() + val returnCode = process.waitFor() + sys.exit(returnCode) } } diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala index c4ef8b63b0071..d25c29113d6da 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala @@ -67,6 +67,7 @@ private[spark] class HistoryPage(parent: HistoryServer) extends WebUIPage("") { } private val appHeader = Seq( + "App ID", "App Name", "Started", "Completed", @@ -81,7 +82,8 @@ private[spark] class HistoryPage(parent: HistoryServer) extends WebUIPage("") { val duration = UIUtils.formatDuration(info.endTime - info.startTime) val lastUpdated = UIUtils.formatDate(info.lastUpdated) - {info.name} + {info.id} + {info.name} {startTime} {endTime} {duration} diff --git a/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala index aa85aa060d9c1..08a99bbe68578 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala @@ -83,15 +83,21 @@ private[spark] class FileSystemPersistenceEngine( val serialized = serializer.toBinary(value) val out = new FileOutputStream(file) - out.write(serialized) - out.close() + try { + out.write(serialized) + } finally { + out.close() + } } def deserializeFromFile[T](file: File)(implicit m: Manifest[T]): T = { val fileData = new Array[Byte](file.length().asInstanceOf[Int]) val dis = new DataInputStream(new FileInputStream(file)) - dis.readFully(fileData) - dis.close() + try { + dis.readFully(fileData) + } finally { + dis.close() + } val clazz = m.runtimeClass.asInstanceOf[Class[T]] val serializer = serialization.serializerFor(clazz) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 2a3bd6ba0b9dc..f98b531316a3d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -33,8 +33,8 @@ import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} import akka.serialization.SerializationExtension import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkException} -import org.apache.spark.deploy.{ApplicationDescription, DriverDescription, ExecutorState, - SparkHadoopUtil} +import org.apache.spark.deploy.{ApplicationDescription, DriverDescription, + ExecutorState, SparkHadoopUtil} import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.history.HistoryServer import org.apache.spark.deploy.master.DriverState.DriverState @@ -489,23 +489,24 @@ private[spark] class Master( // First schedule drivers, they take strict precedence over applications // Randomization helps balance drivers val shuffledAliveWorkers = Random.shuffle(workers.toSeq.filter(_.state == WorkerState.ALIVE)) - val aliveWorkerNum = shuffledAliveWorkers.size + val numWorkersAlive = shuffledAliveWorkers.size var curPos = 0 + for (driver <- waitingDrivers.toList) { // iterate over a copy of waitingDrivers // We assign workers to each waiting driver in a round-robin fashion. For each driver, we // start from the last worker that was assigned a driver, and continue onwards until we have // explored all alive workers. - curPos = (curPos + 1) % aliveWorkerNum - val startPos = curPos var launched = false - while (curPos != startPos && !launched) { + var numWorkersVisited = 0 + while (numWorkersVisited < numWorkersAlive && !launched) { val worker = shuffledAliveWorkers(curPos) + numWorkersVisited += 1 if (worker.memoryFree >= driver.desc.mem && worker.coresFree >= driver.desc.cores) { launchDriver(worker, driver) waitingDrivers -= driver launched = true } - curPos = (curPos + 1) % aliveWorkerNum + curPos = (curPos + 1) % numWorkersAlive } } @@ -692,16 +693,18 @@ private[spark] class Master( app.desc.appUiUrl = notFoundBasePath return false } - val fileSystem = Utils.getHadoopFileSystem(eventLogDir, + + val appEventLogDir = EventLoggingListener.getLogDirPath(eventLogDir, app.id) + val fileSystem = Utils.getHadoopFileSystem(appEventLogDir, SparkHadoopUtil.get.newConfiguration(conf)) - val eventLogInfo = EventLoggingListener.parseLoggingInfo(eventLogDir, fileSystem) + val eventLogInfo = EventLoggingListener.parseLoggingInfo(appEventLogDir, fileSystem) val eventLogPaths = eventLogInfo.logPaths val compressionCodec = eventLogInfo.compressionCodec if (eventLogPaths.isEmpty) { // Event logging is enabled for this application, but no event logs are found val title = s"Application history not found (${app.id})" - var msg = s"No event logs found for application $appName in $eventLogDir." + var msg = s"No event logs found for application $appName in $appEventLogDir." logWarning(msg) msg += " Did you specify the correct logging directory?" msg = URLEncoder.encode(msg, "UTF-8") 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 12e98fd40d6c9..2e9be2a180c68 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 @@ -30,7 +30,7 @@ import org.apache.spark.util.Utils private[spark] object CommandUtils extends Logging { def buildCommandSeq(command: Command, memory: Int, sparkHome: String): Seq[String] = { - val runner = getEnv("JAVA_HOME", command).map(_ + "/bin/java").getOrElse("java") + val runner = sys.env.get("JAVA_HOME").map(_ + "/bin/java").getOrElse("java") // SPARK-698: do not call the run.cmd script, as process.destroy() // fails to kill a process tree on Windows @@ -38,9 +38,6 @@ object CommandUtils extends Logging { command.arguments } - private def getEnv(key: String, command: Command): Option[String] = - command.environment.get(key).orElse(Option(System.getenv(key))) - /** * Attention: this must always be aligned with the environment variables in the run scripts and * the way the JAVA_OPTS are assembled there. diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala index 00a43673e5cd3..71650cd773bcf 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala @@ -42,7 +42,7 @@ private[spark] class ExecutorRunner( val workerId: String, val host: String, val sparkHome: File, - val workDir: File, + val executorDir: File, val workerUrl: String, val conf: SparkConf, var state: ExecutorState.Value) @@ -130,12 +130,6 @@ private[spark] class ExecutorRunner( */ def fetchAndRunExecutor() { try { - // Create the executor's working directory - val executorDir = new File(workDir, appId + "/" + execId) - if (!executorDir.mkdirs()) { - throw new IOException("Failed to create directory " + executorDir) - } - // Launch the process val command = getCommandSeq logInfo("Launch command: " + command.mkString("\"", "\" \"", "\"")) 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 0c454e4138c96..9b52cb06fb6fa 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 @@ -18,9 +18,11 @@ package org.apache.spark.deploy.worker import java.io.File +import java.io.IOException import java.text.SimpleDateFormat import java.util.Date +import scala.collection.JavaConversions._ import scala.collection.mutable.HashMap import scala.concurrent.duration._ import scala.language.postfixOps @@ -191,6 +193,7 @@ private[spark] class Worker( changeMaster(masterUrl, masterWebUiUrl) context.system.scheduler.schedule(0 millis, HEARTBEAT_MILLIS millis, self, SendHeartbeat) if (CLEANUP_ENABLED) { + logInfo(s"Worker cleanup enabled; old application directories will be deleted in: $workDir") context.system.scheduler.schedule(CLEANUP_INTERVAL_MILLIS millis, CLEANUP_INTERVAL_MILLIS millis, self, WorkDirCleanup) } @@ -201,10 +204,23 @@ private[spark] class Worker( case WorkDirCleanup => // Spin up a separate thread (in a future) to do the dir cleanup; don't tie up worker actor val cleanupFuture = concurrent.future { - logInfo("Cleaning up oldest application directories in " + workDir + " ...") - Utils.findOldFiles(workDir, APP_DATA_RETENTION_SECS) - .foreach(Utils.deleteRecursively) + val appDirs = workDir.listFiles() + if (appDirs == null) { + throw new IOException("ERROR: Failed to list files in " + appDirs) + } + appDirs.filter { dir => + // the directory is used by an application - check that the application is not running + // when cleaning up + val appIdFromDir = dir.getName + val isAppStillRunning = executors.values.map(_.appId).contains(appIdFromDir) + dir.isDirectory && !isAppStillRunning && + !Utils.doesDirectoryContainAnyNewFiles(dir, APP_DATA_RETENTION_SECS) + }.foreach { dir => + logInfo(s"Removing directory: ${dir.getPath}") + Utils.deleteRecursively(dir) + } } + cleanupFuture onFailure { case e: Throwable => logError("App dir cleanup failed: " + e.getMessage, e) @@ -233,8 +249,15 @@ private[spark] class Worker( } else { try { logInfo("Asked to launch executor %s/%d for %s".format(appId, execId, appDesc.name)) + + // Create the executor's working directory + val executorDir = new File(workDir, appId + "/" + execId) + if (!executorDir.mkdirs()) { + throw new IOException("Failed to create directory " + executorDir) + } + val manager = new ExecutorRunner(appId, execId, appDesc, cores_, memory_, - self, workerId, host, sparkHome, workDir, akkaUrl, conf, ExecutorState.LOADING) + self, workerId, host, sparkHome, executorDir, akkaUrl, conf, ExecutorState.LOADING) executors(appId + "/" + execId) = manager manager.start() coresUsed += cores_ @@ -242,12 +265,13 @@ private[spark] class Worker( master ! ExecutorStateChanged(appId, execId, manager.state, None, None) } catch { case e: Exception => { - logError("Failed to launch executor %s/%d for %s".format(appId, execId, appDesc.name)) + logError(s"Failed to launch executor $appId/$execId for ${appDesc.name}.", e) if (executors.contains(appId + "/" + execId)) { executors(appId + "/" + execId).kill() executors -= appId + "/" + execId } - master ! ExecutorStateChanged(appId, execId, ExecutorState.FAILED, None, None) + master ! ExecutorStateChanged(appId, execId, ExecutorState.FAILED, + Some(e.toString), None) } } } 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 13af5b6f5812d..06061edfc0844 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -106,6 +106,7 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { executorId: String, hostname: String, cores: Int, + appId: String, workerUrl: Option[String]) { SignalLogger.register(log) @@ -122,7 +123,8 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { val driver = fetcher.actorSelection(driverUrl) val timeout = AkkaUtils.askTimeout(executorConf) val fut = Patterns.ask(driver, RetrieveSparkProps, timeout) - val props = Await.result(fut, timeout).asInstanceOf[Seq[(String, String)]] + val props = Await.result(fut, timeout).asInstanceOf[Seq[(String, String)]] ++ + Seq[(String, String)](("spark.app.id", appId)) fetcher.shutdown() // Create a new ActorSystem using driver's Spark properties to run the backend. @@ -144,16 +146,16 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { def main(args: Array[String]) { args.length match { - case x if x < 4 => + case x if x < 5 => System.err.println( // Worker url is used in spark standalone mode to enforce fate-sharing with worker "Usage: CoarseGrainedExecutorBackend " + - " []") + " [] ") System.exit(1) - case 4 => - run(args(0), args(1), args(2), args(3).toInt, None) - case x if x > 4 => - run(args(0), args(1), args(2), args(3).toInt, Some(args(4))) + case 5 => + run(args(0), args(1), args(2), args(3).toInt, args(4), None) + case x if x > 5 => + run(args(0), args(1), args(2), args(3).toInt, args(4), Some(args(5))) } } } 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 acae448a9c66f..616c7e6a46368 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -24,6 +24,7 @@ import java.util.concurrent._ import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, HashMap} +import scala.util.control.NonFatal import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil @@ -73,6 +74,7 @@ private[spark] class Executor( val executorSource = new ExecutorSource(this, executorId) // Initialize Spark environment (using system properties read above) + conf.set("spark.executor.id", "executor." + executorId) private val env = { if (!isLocal) { val _env = SparkEnv.create(conf, executorId, slaveHostname, 0, @@ -146,7 +148,6 @@ private[spark] class Executor( override def run() { val startTime = System.currentTimeMillis() - SparkEnv.set(env) Thread.currentThread.setContextClassLoader(replClassLoader) val ser = SparkEnv.get.closureSerializer.newInstance() logInfo(s"Running $taskName (TID $taskId)") @@ -156,7 +157,6 @@ private[spark] class Executor( val startGCTime = gcTime try { - SparkEnv.set(env) Accumulators.clear() val (taskFiles, taskJars, taskBytes) = Task.deserializeWithDependencies(serializedTask) updateDependencies(taskFiles, taskJars) @@ -375,12 +375,17 @@ private[spark] class Executor( } val message = Heartbeat(executorId, tasksMetrics.toArray, env.blockManager.blockManagerId) - val response = AkkaUtils.askWithReply[HeartbeatResponse](message, heartbeatReceiverRef, - retryAttempts, retryIntervalMs, timeout) - if (response.reregisterBlockManager) { - logWarning("Told to re-register on heartbeat") - env.blockManager.reregister() + try { + val response = AkkaUtils.askWithReply[HeartbeatResponse](message, heartbeatReceiverRef, + retryAttempts, retryIntervalMs, timeout) + if (response.reregisterBlockManager) { + logWarning("Told to re-register on heartbeat") + env.blockManager.reregister() + } + } catch { + case NonFatal(t) => logWarning("Issue communicating with driver in heartbeater", t) } + Thread.sleep(interval) } } diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala index d6721586566c2..c4d73622c4727 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala @@ -37,8 +37,7 @@ private[spark] class ExecutorSource(val executor: Executor, executorId: String) override val metricRegistry = new MetricRegistry() - // TODO: It would be nice to pass the application name here - override val sourceName = "executor.%s".format(executorId) + override val sourceName = "executor" // Gauge for executor thread pool's actively executing task counts metricRegistry.register(MetricRegistry.name("threadpool", "activeTasks"), new Gauge[Int] { diff --git a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala index a42c8b43bbf7f..bca0b152268ad 100644 --- a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala @@ -52,7 +52,8 @@ private[spark] class MesosExecutorBackend slaveInfo: SlaveInfo) { logInfo("Registered with Mesos as executor ID " + executorInfo.getExecutorId.getValue) this.driver = driver - val properties = Utils.deserialize[Array[(String, String)]](executorInfo.getData.toByteArray) + val properties = Utils.deserialize[Array[(String, String)]](executorInfo.getData.toByteArray) ++ + Seq[(String, String)](("spark.app.id", frameworkInfo.getId.getValue)) executor = new Executor( executorInfo.getExecutorId.getValue, slaveInfo.getHostname, diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 99a88c13456df..3e49b6235aff3 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -137,7 +137,6 @@ class TaskMetrics extends Serializable { merged.localBlocksFetched += depMetrics.localBlocksFetched merged.remoteBlocksFetched += depMetrics.remoteBlocksFetched merged.remoteBytesRead += depMetrics.remoteBytesRead - merged.shuffleFinishTime = math.max(merged.shuffleFinishTime, depMetrics.shuffleFinishTime) } _shuffleReadMetrics = Some(merged) } @@ -177,11 +176,6 @@ case class InputMetrics(readMethod: DataReadMethod.Value) { */ @DeveloperApi class ShuffleReadMetrics extends Serializable { - /** - * Absolute time when this task finished reading shuffle data - */ - var shuffleFinishTime: Long = -1 - /** * Number of blocks fetched in this shuffle by this task (remote or local) */ diff --git a/core/src/main/scala/org/apache/spark/input/WholeTextFileRecordReader.scala b/core/src/main/scala/org/apache/spark/input/WholeTextFileRecordReader.scala index c3dabd2e79995..3564ab2e2a162 100644 --- a/core/src/main/scala/org/apache/spark/input/WholeTextFileRecordReader.scala +++ b/core/src/main/scala/org/apache/spark/input/WholeTextFileRecordReader.scala @@ -36,33 +36,31 @@ private[spark] class WholeTextFileRecordReader( index: Integer) extends RecordReader[String, String] { - private val path = split.getPath(index) - private val fs = path.getFileSystem(context.getConfiguration) + private[this] val path = split.getPath(index) + private[this] val fs = path.getFileSystem(context.getConfiguration) // True means the current file has been processed, then skip it. - private var processed = false + private[this] var processed = false - private val key = path.toString - private var value: String = null + private[this] val key = path.toString + private[this] var value: String = null - override def initialize(split: InputSplit, context: TaskAttemptContext) = {} + override def initialize(split: InputSplit, context: TaskAttemptContext): Unit = {} - override def close() = {} + override def close(): Unit = {} - override def getProgress = if (processed) 1.0f else 0.0f + override def getProgress: Float = if (processed) 1.0f else 0.0f - override def getCurrentKey = key + override def getCurrentKey: String = key - override def getCurrentValue = value + override def getCurrentValue: String = value - override def nextKeyValue = { + override def nextKeyValue(): Boolean = { if (!processed) { val fileIn = fs.open(path) val innerBuffer = ByteStreams.toByteArray(fileIn) - value = new Text(innerBuffer).toString Closeables.close(fileIn, false) - processed = true true } else { diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala index 6ef817d0e587e..5dd67b0cbf683 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala @@ -63,15 +63,18 @@ import org.apache.spark.metrics.source.Source * * [options] is the specific property of this source or sink. */ -private[spark] class MetricsSystem private (val instance: String, - conf: SparkConf, securityMgr: SecurityManager) extends Logging { +private[spark] class MetricsSystem private ( + val instance: String, + conf: SparkConf, + securityMgr: SecurityManager) + extends Logging { - val confFile = conf.get("spark.metrics.conf", null) - val metricsConfig = new MetricsConfig(Option(confFile)) + private[this] val confFile = conf.get("spark.metrics.conf", null) + private[this] val metricsConfig = new MetricsConfig(Option(confFile)) - val sinks = new mutable.ArrayBuffer[Sink] - val sources = new mutable.ArrayBuffer[Source] - val registry = new MetricRegistry() + private val sinks = new mutable.ArrayBuffer[Sink] + private val sources = new mutable.ArrayBuffer[Source] + private val registry = new MetricRegistry() // Treat MetricsServlet as a special sink as it should be exposed to add handlers to web ui private var metricsServlet: Option[MetricsServlet] = None @@ -80,10 +83,10 @@ private[spark] class MetricsSystem private (val instance: String, def getServletHandlers = metricsServlet.map(_.getHandlers).getOrElse(Array()) metricsConfig.initialize() - registerSources() - registerSinks() def start() { + registerSources() + registerSinks() sinks.foreach(_.start) } @@ -91,14 +94,43 @@ private[spark] class MetricsSystem private (val instance: String, sinks.foreach(_.stop) } - def report(): Unit = { + def report() { sinks.foreach(_.report()) } + /** + * Build a name that uniquely identifies each metric source. + * The name is structured as follows: ... + * If either ID is not available, this defaults to just using . + * + * @param source Metric source to be named by this method. + * @return An unique metric name for each combination of + * application, executor/driver and metric source. + */ + def buildRegistryName(source: Source): String = { + val appId = conf.getOption("spark.app.id") + val executorId = conf.getOption("spark.executor.id") + val defaultName = MetricRegistry.name(source.sourceName) + + if (instance == "driver" || instance == "executor") { + if (appId.isDefined && executorId.isDefined) { + MetricRegistry.name(appId.get, executorId.get, source.sourceName) + } else { + // Only Driver and Executor are set spark.app.id and spark.executor.id. + // For instance, Master and Worker are not related to a specific application. + val warningMsg = s"Using default name $defaultName for source because %s is not set." + if (appId.isEmpty) { logWarning(warningMsg.format("spark.app.id")) } + if (executorId.isEmpty) { logWarning(warningMsg.format("spark.executor.id")) } + defaultName + } + } else { defaultName } + } + def registerSource(source: Source) { sources += source try { - registry.register(source.sourceName, source.metricRegistry) + val regName = buildRegistryName(source) + registry.register(regName, source.metricRegistry) } catch { case e: IllegalArgumentException => logInfo("Metrics already registered", e) } @@ -106,8 +138,9 @@ private[spark] class MetricsSystem private (val instance: String, def removeSource(source: Source) { sources -= source + val regName = buildRegistryName(source) registry.removeMatching(new MetricFilter { - def matches(name: String, metric: Metric): Boolean = name.startsWith(source.sourceName) + def matches(name: String, metric: Metric): Boolean = name.startsWith(regName) }) } @@ -122,7 +155,7 @@ private[spark] class MetricsSystem private (val instance: String, val source = Class.forName(classPath).newInstance() registerSource(source.asInstanceOf[Source]) } catch { - case e: Exception => logError("Source class " + classPath + " cannot be instantialized", e) + case e: Exception => logError("Source class " + classPath + " cannot be instantiated", e) } } } @@ -155,8 +188,8 @@ private[spark] object MetricsSystem { val SINK_REGEX = "^sink\\.(.+)\\.(.+)".r val SOURCE_REGEX = "^source\\.(.+)\\.(.+)".r - val MINIMAL_POLL_UNIT = TimeUnit.SECONDS - val MINIMAL_POLL_PERIOD = 1 + private[this] val MINIMAL_POLL_UNIT = TimeUnit.SECONDS + private[this] val MINIMAL_POLL_PERIOD = 1 def checkMinimalPollingPeriod(pollUnit: TimeUnit, pollPeriod: Int) { val period = MINIMAL_POLL_UNIT.convert(pollPeriod, pollUnit) @@ -166,7 +199,8 @@ private[spark] object MetricsSystem { } } - def createMetricsSystem(instance: String, conf: SparkConf, - securityMgr: SecurityManager): MetricsSystem = + def createMetricsSystem( + instance: String, conf: SparkConf, securityMgr: SecurityManager): MetricsSystem = { new MetricsSystem(instance, conf, securityMgr) + } } diff --git a/core/src/main/scala/org/apache/spark/network/ManagedBuffer.scala b/core/src/main/scala/org/apache/spark/network/ManagedBuffer.scala index e990c1da6730f..a4409181ec907 100644 --- a/core/src/main/scala/org/apache/spark/network/ManagedBuffer.scala +++ b/core/src/main/scala/org/apache/spark/network/ManagedBuffer.scala @@ -17,15 +17,17 @@ package org.apache.spark.network -import java.io.{FileInputStream, RandomAccessFile, File, InputStream} +import java.io._ import java.nio.ByteBuffer import java.nio.channels.FileChannel import java.nio.channels.FileChannel.MapMode +import scala.util.Try + import com.google.common.io.ByteStreams import io.netty.buffer.{ByteBufInputStream, ByteBuf} -import org.apache.spark.util.ByteBufferInputStream +import org.apache.spark.util.{ByteBufferInputStream, Utils} /** @@ -71,18 +73,47 @@ final class FileSegmentManagedBuffer(val file: File, val offset: Long, val lengt try { channel = new RandomAccessFile(file, "r").getChannel channel.map(MapMode.READ_ONLY, offset, length) + } catch { + case e: IOException => + Try(channel.size).toOption match { + case Some(fileLen) => + throw new IOException(s"Error in reading $this (actual file length $fileLen)", e) + case None => + throw new IOException(s"Error in opening $this", e) + } } finally { if (channel != null) { - channel.close() + Utils.tryLog(channel.close()) } } } override def inputStream(): InputStream = { - val is = new FileInputStream(file) - is.skip(offset) - ByteStreams.limit(is, length) + var is: FileInputStream = null + try { + is = new FileInputStream(file) + is.skip(offset) + ByteStreams.limit(is, length) + } catch { + case e: IOException => + if (is != null) { + Utils.tryLog(is.close()) + } + Try(file.length).toOption match { + case Some(fileLen) => + throw new IOException(s"Error in reading $this (actual file length $fileLen)", e) + case None => + throw new IOException(s"Error in opening $this", e) + } + case e: Throwable => + if (is != null) { + Utils.tryLog(is.close()) + } + throw e + } } + + override def toString: String = s"${getClass.getName}($file, $offset, $length)" } diff --git a/core/src/main/scala/org/apache/spark/network/nio/Connection.scala b/core/src/main/scala/org/apache/spark/network/nio/Connection.scala index 74074a8dcbfff..f368209980f93 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/Connection.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/Connection.scala @@ -20,23 +20,27 @@ package org.apache.spark.network.nio import java.net._ import java.nio._ import java.nio.channels._ +import java.util.LinkedList import org.apache.spark._ -import scala.collection.mutable.{ArrayBuffer, HashMap, Queue} +import scala.collection.mutable.{ArrayBuffer, HashMap} private[nio] abstract class Connection(val channel: SocketChannel, val selector: Selector, - val socketRemoteConnectionManagerId: ConnectionManagerId, val connectionId: ConnectionId) + val socketRemoteConnectionManagerId: ConnectionManagerId, val connectionId: ConnectionId, + val securityMgr: SecurityManager) extends Logging { var sparkSaslServer: SparkSaslServer = null var sparkSaslClient: SparkSaslClient = null - def this(channel_ : SocketChannel, selector_ : Selector, id_ : ConnectionId) = { + def this(channel_ : SocketChannel, selector_ : Selector, id_ : ConnectionId, + securityMgr_ : SecurityManager) = { this(channel_, selector_, ConnectionManagerId.fromSocketAddress( - channel_.socket.getRemoteSocketAddress.asInstanceOf[InetSocketAddress]), id_) + channel_.socket.getRemoteSocketAddress.asInstanceOf[InetSocketAddress]), + id_, securityMgr_) } channel.configureBlocking(false) @@ -52,14 +56,6 @@ abstract class Connection(val channel: SocketChannel, val selector: Selector, val remoteAddress = getRemoteAddress() - /** - * Used to synchronize client requests: client's work-related requests must - * wait until SASL authentication completes. - */ - private val authenticated = new Object() - - def getAuthenticated(): Object = authenticated - def isSaslComplete(): Boolean def resetForceReregister(): Boolean @@ -192,22 +188,22 @@ abstract class Connection(val channel: SocketChannel, val selector: Selector, private[nio] class SendingConnection(val address: InetSocketAddress, selector_ : Selector, - remoteId_ : ConnectionManagerId, id_ : ConnectionId) - extends Connection(SocketChannel.open, selector_, remoteId_, id_) { + remoteId_ : ConnectionManagerId, id_ : ConnectionId, + securityMgr_ : SecurityManager) + extends Connection(SocketChannel.open, selector_, remoteId_, id_, securityMgr_) { def isSaslComplete(): Boolean = { if (sparkSaslClient != null) sparkSaslClient.isComplete() else false } private class Outbox { - val messages = new Queue[Message]() + val messages = new LinkedList[Message]() val defaultChunkSize = 65536 var nextMessageToBeUsed = 0 def addMessage(message: Message) { messages.synchronized { - /* messages += message */ - messages.enqueue(message) + messages.add(message) logDebug("Added [" + message + "] to outbox for sending to " + "[" + getRemoteConnectionManagerId() + "]") } @@ -218,10 +214,27 @@ class SendingConnection(val address: InetSocketAddress, selector_ : Selector, while (!messages.isEmpty) { /* nextMessageToBeUsed = nextMessageToBeUsed % messages.size */ /* val message = messages(nextMessageToBeUsed) */ - val message = messages.dequeue() + + val message = if (securityMgr.isAuthenticationEnabled() && !isSaslComplete()) { + // only allow sending of security messages until sasl is complete + var pos = 0 + var securityMsg: Message = null + while (pos < messages.size() && securityMsg == null) { + if (messages.get(pos).isSecurityNeg) { + securityMsg = messages.remove(pos) + } + pos = pos + 1 + } + // didn't find any security messages and auth isn't completed so return + if (securityMsg == null) return None + securityMsg + } else { + messages.removeFirst() + } + val chunk = message.getChunkForSending(defaultChunkSize) if (chunk.isDefined) { - messages.enqueue(message) + messages.add(message) nextMessageToBeUsed = nextMessageToBeUsed + 1 if (!message.started) { logDebug( @@ -273,6 +286,15 @@ class SendingConnection(val address: InetSocketAddress, selector_ : Selector, changeConnectionKeyInterest(DEFAULT_INTEREST) } + def registerAfterAuth(): Unit = { + outbox.synchronized { + needForceReregister = true + } + if (channel.isConnected) { + registerInterest() + } + } + def send(message: Message) { outbox.synchronized { outbox.addMessage(message) @@ -415,8 +437,9 @@ class SendingConnection(val address: InetSocketAddress, selector_ : Selector, private[spark] class ReceivingConnection( channel_ : SocketChannel, selector_ : Selector, - id_ : ConnectionId) - extends Connection(channel_, selector_, id_) { + id_ : ConnectionId, + securityMgr_ : SecurityManager) + extends Connection(channel_, selector_, id_, securityMgr_) { def isSaslComplete(): Boolean = { if (sparkSaslServer != null) sparkSaslServer.isComplete() else false @@ -460,7 +483,7 @@ private[spark] class ReceivingConnection( if (currId != null) currId else super.getRemoteConnectionManagerId() } - // The reciever's remote address is the local socket on remote side : which is NOT + // The receiver's remote address is the local socket on remote side : which is NOT // the connection manager id of the receiver. // We infer that from the messages we receive on the receiver socket. private def processConnectionManagerId(header: MessageChunkHeader) { diff --git a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala index 09d3ea306515b..01cd27a907eea 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala @@ -32,7 +32,7 @@ import scala.concurrent.{Await, ExecutionContext, Future, Promise} import scala.language.postfixOps import org.apache.spark._ -import org.apache.spark.util.{SystemClock, Utils} +import org.apache.spark.util.Utils private[nio] class ConnectionManager( @@ -65,8 +65,6 @@ private[nio] class ConnectionManager( private val selector = SelectorProvider.provider.openSelector() private val ackTimeoutMonitor = new Timer("AckTimeoutMonitor", true) - // default to 30 second timeout waiting for authentication - private val authTimeout = conf.getInt("spark.core.connection.auth.wait.timeout", 30) private val ackTimeout = conf.getInt("spark.core.connection.ack.wait.timeout", 60) private val handleMessageExecutor = new ThreadPoolExecutor( @@ -409,7 +407,8 @@ private[nio] class ConnectionManager( while (newChannel != null) { try { val newConnectionId = new ConnectionId(id, idCount.getAndIncrement.intValue) - val newConnection = new ReceivingConnection(newChannel, selector, newConnectionId) + val newConnection = new ReceivingConnection(newChannel, selector, newConnectionId, + securityManager) newConnection.onReceive(receiveMessage) addListeners(newConnection) addConnection(newConnection) @@ -501,7 +500,7 @@ private[nio] class ConnectionManager( def changeConnectionKeyInterest(connection: Connection, ops: Int) { keyInterestChangeRequests += ((connection.key, ops)) - // so that registerations happen ! + // so that registrations happen ! wakeupSelector() } @@ -527,9 +526,8 @@ private[nio] class ConnectionManager( if (waitingConn.isSaslComplete()) { logDebug("Client sasl completed for id: " + waitingConn.connectionId) connectionsAwaitingSasl -= waitingConn.connectionId - waitingConn.getAuthenticated().synchronized { - waitingConn.getAuthenticated().notifyAll() - } + waitingConn.registerAfterAuth() + wakeupSelector() return } else { var replyToken : Array[Byte] = null @@ -538,9 +536,8 @@ private[nio] class ConnectionManager( if (waitingConn.isSaslComplete()) { logDebug("Client sasl completed after evaluate for id: " + waitingConn.connectionId) connectionsAwaitingSasl -= waitingConn.connectionId - waitingConn.getAuthenticated().synchronized { - waitingConn.getAuthenticated().notifyAll() - } + waitingConn.registerAfterAuth() + wakeupSelector() return } val securityMsgResp = SecurityMessage.fromResponse(replyToken, @@ -574,9 +571,11 @@ private[nio] class ConnectionManager( } replyToken = connection.sparkSaslServer.response(securityMsg.getToken) if (connection.isSaslComplete()) { - logDebug("Server sasl completed: " + connection.connectionId) + logDebug("Server sasl completed: " + connection.connectionId + + " for: " + connectionId) } else { - logDebug("Server sasl not completed: " + connection.connectionId) + logDebug("Server sasl not completed: " + connection.connectionId + + " for: " + connectionId) } if (replyToken != null) { val securityMsgResp = SecurityMessage.fromResponse(replyToken, @@ -723,7 +722,8 @@ private[nio] class ConnectionManager( if (message == null) throw new Exception("Error creating security message") connectionsAwaitingSasl += ((conn.connectionId, conn)) sendSecurityMessage(connManagerId, message) - logDebug("adding connectionsAwaitingSasl id: " + conn.connectionId) + logDebug("adding connectionsAwaitingSasl id: " + conn.connectionId + + " to: " + connManagerId) } catch { case e: Exception => { logError("Error getting first response from the SaslClient.", e) @@ -744,7 +744,7 @@ private[nio] class ConnectionManager( val inetSocketAddress = new InetSocketAddress(connManagerId.host, connManagerId.port) val newConnectionId = new ConnectionId(id, idCount.getAndIncrement.intValue) val newConnection = new SendingConnection(inetSocketAddress, selector, connManagerId, - newConnectionId) + newConnectionId, securityManager) logInfo("creating new sending connection for security! " + newConnectionId ) registerRequests.enqueue(newConnection) @@ -769,61 +769,23 @@ private[nio] class ConnectionManager( connectionManagerId.port) val newConnectionId = new ConnectionId(id, idCount.getAndIncrement.intValue) val newConnection = new SendingConnection(inetSocketAddress, selector, connectionManagerId, - newConnectionId) + newConnectionId, securityManager) logTrace("creating new sending connection: " + newConnectionId) registerRequests.enqueue(newConnection) newConnection } val connection = connectionsById.getOrElseUpdate(connectionManagerId, startNewConnection()) - if (authEnabled) { - checkSendAuthFirst(connectionManagerId, connection) - } + message.senderAddress = id.toSocketAddress() logDebug("Before Sending [" + message + "] to [" + connectionManagerId + "]" + " " + "connectionid: " + connection.connectionId) if (authEnabled) { - // if we aren't authenticated yet lets block the senders until authentication completes - try { - connection.getAuthenticated().synchronized { - val clock = SystemClock - val startTime = clock.getTime() - - while (!connection.isSaslComplete()) { - logDebug("getAuthenticated wait connectionid: " + connection.connectionId) - // have timeout in case remote side never responds - connection.getAuthenticated().wait(500) - if (((clock.getTime() - startTime) >= (authTimeout * 1000)) - && (!connection.isSaslComplete())) { - // took to long to authenticate the connection, something probably went wrong - throw new Exception("Took to long for authentication to " + connectionManagerId + - ", waited " + authTimeout + "seconds, failing.") - } - } - } - } catch { - case e: Exception => logError("Exception while waiting for authentication.", e) - - // need to tell sender it failed - messageStatuses.synchronized { - val s = messageStatuses.get(message.id) - s match { - case Some(msgStatus) => { - messageStatuses -= message.id - logInfo("Notifying " + msgStatus.connectionManagerId) - msgStatus.markDone(None) - } - case None => { - logError("no messageStatus for failed message id: " + message.id) - } - } - } - } + checkSendAuthFirst(connectionManagerId, connection) } logDebug("Sending [" + message + "] to [" + connectionManagerId + "]") connection.send(message) - wakeupSelector() } @@ -832,7 +794,7 @@ private[nio] class ConnectionManager( } /** - * Send a message and block until an acknowldgment is received or an error occurs. + * Send a message and block until an acknowledgment is received or an error occurs. * @param connectionManagerId the message's destination * @param message the message being sent * @return a Future that either returns the acknowledgment message or captures an exception. diff --git a/core/src/main/scala/org/apache/spark/network/nio/NioBlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/nio/NioBlockTransferService.scala index 59958ee894230..b389b9a2022c6 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/NioBlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/NioBlockTransferService.scala @@ -200,6 +200,6 @@ final class NioBlockTransferService(conf: SparkConf, securityManager: SecurityMa val buffer = blockDataManager.getBlockData(blockId).orNull logDebug("GetBlock " + blockId + " used " + Utils.getUsedTimeMs(startTimeMs) + " and got buffer " + buffer) - buffer.nioByteBuffer() + if (buffer == null) null else buffer.nioByteBuffer() } } diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index 036dcc49664ef..6b63eb23e9ee1 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -23,6 +23,7 @@ import java.io.EOFException import scala.collection.immutable.Map import scala.reflect.ClassTag +import scala.collection.mutable.ListBuffer import org.apache.hadoop.conf.{Configurable, Configuration} import org.apache.hadoop.mapred.FileSplit @@ -43,6 +44,7 @@ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.executor.{DataReadMethod, InputMetrics} import org.apache.spark.rdd.HadoopRDD.HadoopMapPartitionsWithSplitRDD import org.apache.spark.util.{NextIterator, Utils} +import org.apache.spark.scheduler.{HostTaskLocation, HDFSCacheTaskLocation} /** @@ -194,7 +196,7 @@ class HadoopRDD[K, V]( val jobConf = getJobConf() val inputFormat = getInputFormat(jobConf) HadoopRDD.addLocalConfiguration(new SimpleDateFormat("yyyyMMddHHmm").format(createTime), - context.stageId, theSplit.index, context.attemptId.toInt, jobConf) + context.getStageId, theSplit.index, context.getAttemptId.toInt, jobConf) reader = inputFormat.getRecordReader(split.inputSplit.value, jobConf, Reporter.NULL) // Register an on-task-completion callback to close the input stream. @@ -249,9 +251,21 @@ class HadoopRDD[K, V]( } override def getPreferredLocations(split: Partition): Seq[String] = { - // TODO: Filtering out "localhost" in case of file:// URLs - val hadoopSplit = split.asInstanceOf[HadoopPartition] - hadoopSplit.inputSplit.value.getLocations.filter(_ != "localhost") + val hsplit = split.asInstanceOf[HadoopPartition].inputSplit.value + val locs: Option[Seq[String]] = HadoopRDD.SPLIT_INFO_REFLECTIONS match { + case Some(c) => + try { + val lsplit = c.inputSplitWithLocationInfo.cast(hsplit) + val infos = c.getLocationInfo.invoke(lsplit).asInstanceOf[Array[AnyRef]] + Some(HadoopRDD.convertSplitLocationInfo(infos)) + } catch { + case e: Exception => + logDebug("Failed to use InputSplitWithLocations.", e) + None + } + case None => None + } + locs.getOrElse(hsplit.getLocations.filter(_ != "localhost")) } override def checkpoint() { @@ -261,7 +275,7 @@ class HadoopRDD[K, V]( def getConf: Configuration = getJobConf() } -private[spark] object HadoopRDD { +private[spark] object HadoopRDD extends Logging { /** Constructing Configuration objects is not threadsafe, use this lock to serialize. */ val CONFIGURATION_INSTANTIATION_LOCK = new Object() @@ -309,4 +323,42 @@ private[spark] object HadoopRDD { f(inputSplit, firstParent[T].iterator(split, context)) } } + + private[spark] class SplitInfoReflections { + val inputSplitWithLocationInfo = + Class.forName("org.apache.hadoop.mapred.InputSplitWithLocationInfo") + val getLocationInfo = inputSplitWithLocationInfo.getMethod("getLocationInfo") + val newInputSplit = Class.forName("org.apache.hadoop.mapreduce.InputSplit") + val newGetLocationInfo = newInputSplit.getMethod("getLocationInfo") + val splitLocationInfo = Class.forName("org.apache.hadoop.mapred.SplitLocationInfo") + val isInMemory = splitLocationInfo.getMethod("isInMemory") + val getLocation = splitLocationInfo.getMethod("getLocation") + } + + private[spark] val SPLIT_INFO_REFLECTIONS: Option[SplitInfoReflections] = try { + Some(new SplitInfoReflections) + } catch { + case e: Exception => + logDebug("SplitLocationInfo and other new Hadoop classes are " + + "unavailable. Using the older Hadoop location info code.", e) + None + } + + private[spark] def convertSplitLocationInfo(infos: Array[AnyRef]): Seq[String] = { + val out = ListBuffer[String]() + infos.foreach { loc => { + val locationStr = HadoopRDD.SPLIT_INFO_REFLECTIONS.get. + getLocation.invoke(loc).asInstanceOf[String] + if (locationStr != "localhost") { + if (HadoopRDD.SPLIT_INFO_REFLECTIONS.get.isInMemory. + invoke(loc).asInstanceOf[Boolean]) { + logDebug("Partition " + locationStr + " is cached by Hadoop.") + out += new HDFSCacheTaskLocation(locationStr).toString + } else { + out += new HostTaskLocation(locationStr).toString + } + } + }} + out.seq + } } diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala index 4c84b3f62354d..0cccdefc5ee09 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -173,9 +173,21 @@ class NewHadoopRDD[K, V]( new NewHadoopMapPartitionsWithSplitRDD(this, f, preservesPartitioning) } - override def getPreferredLocations(split: Partition): Seq[String] = { - val theSplit = split.asInstanceOf[NewHadoopPartition] - theSplit.serializableHadoopSplit.value.getLocations.filter(_ != "localhost") + override def getPreferredLocations(hsplit: Partition): Seq[String] = { + val split = hsplit.asInstanceOf[NewHadoopPartition].serializableHadoopSplit.value + val locs = HadoopRDD.SPLIT_INFO_REFLECTIONS match { + case Some(c) => + try { + val infos = c.newGetLocationInfo.invoke(split).asInstanceOf[Array[AnyRef]] + Some(HadoopRDD.convertSplitLocationInfo(infos)) + } catch { + case e : Exception => + logDebug("Failed to use InputSplit#getLocationInfo.", e) + None + } + case None => None + } + locs.getOrElse(split.getLocations.filter(_ != "localhost")) } def getConf: Configuration = confBroadcast.value.value diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index f6d9d12fe9006..0d97506450a7f 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -86,7 +86,8 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) } val aggregator = new Aggregator[K, V, C](createCombiner, mergeValue, mergeCombiners) if (self.partitioner == Some(partitioner)) { - self.mapPartitionsWithContext((context, iter) => { + self.mapPartitions(iter => { + val context = TaskContext.get() new InterruptibleIterator(context, aggregator.combineValuesByKey(iter, context)) }, preservesPartitioning = true) } else { @@ -419,6 +420,8 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) /** * Group the values for each key in the RDD into a single sequence. Allows controlling the * partitioning of the resulting key-value pair RDD by passing a Partitioner. + * The ordering of elements within each group is not guaranteed, and may even differ + * each time the resulting RDD is evaluated. * * Note: This operation may be very expensive. If you are grouping in order to perform an * aggregation (such as a sum or average) over each key, using [[PairRDDFunctions.aggregateByKey]] @@ -438,7 +441,8 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) /** * Group the values for each key in the RDD into a single sequence. Hash-partitions the - * resulting RDD with into `numPartitions` partitions. + * resulting RDD with into `numPartitions` partitions. The ordering of elements within + * each group is not guaranteed, and may even differ each time the resulting RDD is evaluated. * * Note: This operation may be very expensive. If you are grouping in order to perform an * aggregation (such as a sum or average) over each key, using [[PairRDDFunctions.aggregateByKey]] @@ -506,6 +510,23 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) } } + /** + * Perform a full outer join of `this` and `other`. For each element (k, v) in `this`, the + * resulting RDD will either contain all pairs (k, (Some(v), Some(w))) for w in `other`, or + * the pair (k, (Some(v), None)) if no elements in `other` have key k. Similarly, for each + * element (k, w) in `other`, the resulting RDD will either contain all pairs + * (k, (Some(v), Some(w))) for v in `this`, or the pair (k, (None, Some(w))) if no elements + * in `this` have key k. Uses the given Partitioner to partition the output RDD. + */ + def fullOuterJoin[W](other: RDD[(K, W)], partitioner: Partitioner) + : RDD[(K, (Option[V], Option[W]))] = { + this.cogroup(other, partitioner).flatMapValues { + case (vs, Seq()) => vs.map(v => (Some(v), None)) + case (Seq(), ws) => ws.map(w => (None, Some(w))) + case (vs, ws) => for (v <- vs; w <- ws) yield (Some(v), Some(w)) + } + } + /** * Simplified version of combineByKey that hash-partitions the resulting RDD using the * existing partitioner/parallelism level. @@ -517,7 +538,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) /** * Group the values for each key in the RDD into a single sequence. Hash-partitions the - * resulting RDD with the existing partitioner/parallelism level. + * resulting RDD with the existing partitioner/parallelism level. The ordering of elements + * within each group is not guaranteed, and may even differ each time the resulting RDD is + * evaluated. * * Note: This operation may be very expensive. If you are grouping in order to perform an * aggregation (such as a sum or average) over each key, using [[PairRDDFunctions.aggregateByKey]] @@ -585,6 +608,31 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) rightOuterJoin(other, new HashPartitioner(numPartitions)) } + /** + * Perform a full outer join of `this` and `other`. For each element (k, v) in `this`, the + * resulting RDD will either contain all pairs (k, (Some(v), Some(w))) for w in `other`, or + * the pair (k, (Some(v), None)) if no elements in `other` have key k. Similarly, for each + * element (k, w) in `other`, the resulting RDD will either contain all pairs + * (k, (Some(v), Some(w))) for v in `this`, or the pair (k, (None, Some(w))) if no elements + * in `this` have key k. Hash-partitions the resulting RDD using the existing partitioner/ + * parallelism level. + */ + def fullOuterJoin[W](other: RDD[(K, W)]): RDD[(K, (Option[V], Option[W]))] = { + fullOuterJoin(other, defaultPartitioner(self, other)) + } + + /** + * Perform a full outer join of `this` and `other`. For each element (k, v) in `this`, the + * resulting RDD will either contain all pairs (k, (Some(v), Some(w))) for w in `other`, or + * the pair (k, (Some(v), None)) if no elements in `other` have key k. Similarly, for each + * element (k, w) in `other`, the resulting RDD will either contain all pairs + * (k, (Some(v), Some(w))) for v in `this`, or the pair (k, (None, Some(w))) if no elements + * in `this` have key k. Hash-partitions the resulting RDD into the given number of partitions. + */ + def fullOuterJoin[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (Option[V], Option[W]))] = { + fullOuterJoin(other, new HashPartitioner(numPartitions)) + } + /** * Return the key-value pairs in this RDD to the master as a Map. * @@ -872,7 +920,12 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) hadoopConf.set("mapred.output.compression.codec", c.getCanonicalName) hadoopConf.set("mapred.output.compression.type", CompressionType.BLOCK.toString) } - hadoopConf.setOutputCommitter(classOf[FileOutputCommitter]) + + // Use configured output committer if already set + if (conf.getOutputCommitter == null) { + hadoopConf.setOutputCommitter(classOf[FileOutputCommitter]) + } + FileOutputFormat.setOutputPath(hadoopConf, SparkHadoopWriter.createPathFromString(path, hadoopConf)) saveAsHadoopDataset(hadoopConf) @@ -903,9 +956,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) val writeShard = (context: TaskContext, iter: Iterator[(K,V)]) => { // Hadoop wants a 32-bit task attempt ID, so if ours is bigger than Int.MaxValue, roll it // around by taking a mod. We expect that no task will be attempted 2 billion times. - val attemptNumber = (context.attemptId % Int.MaxValue).toInt + val attemptNumber = (context.getAttemptId % Int.MaxValue).toInt /* "reduce task" */ - val attemptId = newTaskAttemptID(jobtrackerID, stageId, isMap = false, context.partitionId, + val attemptId = newTaskAttemptID(jobtrackerID, stageId, isMap = false, context.getPartitionId, attemptNumber) val hadoopContext = newTaskAttemptContext(wrappedConf.value, attemptId) val format = outfmt.newInstance @@ -974,9 +1027,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) val writeToFile = (context: TaskContext, iter: Iterator[(K, V)]) => { // Hadoop wants a 32-bit task attempt ID, so if ours is bigger than Int.MaxValue, roll it // around by taking a mod. We expect that no task will be attempted 2 billion times. - val attemptNumber = (context.attemptId % Int.MaxValue).toInt + val attemptNumber = (context.getAttemptId % Int.MaxValue).toInt - writer.setup(context.stageId, context.partitionId, attemptNumber) + writer.setup(context.getStageId, context.getPartitionId, attemptNumber) writer.open() try { var count = 0 diff --git a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala index 5d77d37378458..56ac7a69be0d3 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala @@ -131,7 +131,6 @@ private[spark] class PipedRDD[T: ClassTag]( // Start a thread to feed the process input from our parent's iterator new Thread("stdin writer for " + command) { override def run() { - SparkEnv.set(env) val out = new PrintWriter(proc.getOutputStream) // input the pipe context firstly 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 a9b905b0d1a63..2aba40d152e3e 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -17,7 +17,7 @@ package org.apache.spark.rdd -import java.util.Random +import java.util.{Properties, Random} import scala.collection.{mutable, Map} import scala.collection.mutable.ArrayBuffer @@ -41,7 +41,7 @@ import org.apache.spark.partial.CountEvaluator import org.apache.spark.partial.GroupedCountEvaluator import org.apache.spark.partial.PartialResult import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.{BoundedPriorityQueue, Utils} +import org.apache.spark.util.{BoundedPriorityQueue, Utils, CallSite} import org.apache.spark.util.collection.OpenHashMap import org.apache.spark.util.random.{BernoulliSampler, PoissonSampler, SamplingUtils} @@ -208,7 +208,7 @@ abstract class RDD[T: ClassTag]( } /** - * Get the preferred locations of a partition (as hostnames), taking into account whether the + * Get the preferred locations of a partition, taking into account whether the * RDD is checkpointed. */ final def preferredLocations(split: Partition): Seq[String] = { @@ -509,7 +509,8 @@ abstract class RDD[T: ClassTag]( /** * Return an RDD of grouped items. Each group consists of a key and a sequence of elements - * mapping to that key. + * mapping to that key. The ordering of elements within each group is not guaranteed, and + * may even differ each time the resulting RDD is evaluated. * * Note: This operation may be very expensive. If you are grouping in order to perform an * aggregation (such as a sum or average) over each key, using [[PairRDDFunctions.aggregateByKey]] @@ -520,7 +521,8 @@ abstract class RDD[T: ClassTag]( /** * Return an RDD of grouped elements. Each group consists of a key and a sequence of elements - * mapping to that key. + * mapping to that key. The ordering of elements within each group is not guaranteed, and + * may even differ each time the resulting RDD is evaluated. * * Note: This operation may be very expensive. If you are grouping in order to perform an * aggregation (such as a sum or average) over each key, using [[PairRDDFunctions.aggregateByKey]] @@ -531,7 +533,8 @@ abstract class RDD[T: ClassTag]( /** * Return an RDD of grouped items. Each group consists of a key and a sequence of elements - * mapping to that key. + * mapping to that key. The ordering of elements within each group is not guaranteed, and + * may even differ each time the resulting RDD is evaluated. * * Note: This operation may be very expensive. If you are grouping in order to perform an * aggregation (such as a sum or average) over each key, using [[PairRDDFunctions.aggregateByKey]] @@ -619,6 +622,7 @@ abstract class RDD[T: ClassTag]( * should be `false` unless this is a pair RDD and the input function doesn't modify the keys. */ @DeveloperApi + @deprecated("use TaskContext.get", "1.2.0") def mapPartitionsWithContext[U: ClassTag]( f: (TaskContext, Iterator[T]) => Iterator[U], preservesPartitioning: Boolean = false): RDD[U] = { @@ -1027,8 +1031,14 @@ abstract class RDD[T: ClassTag]( * 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. + * + * Note that some RDDs, such as those returned by groupBy(), do not guarantee order of + * elements in a partition. The index assigned to each element is therefore not guaranteed, + * and may even change if the RDD is reevaluated. If a fixed ordering is required to guarantee + * the same index assignments, you should sort the RDD with sortByKey() or save it to a file. */ def zipWithIndex(): RDD[(T, Long)] = new ZippedWithIndexRDD(this) @@ -1036,6 +1046,11 @@ abstract class RDD[T: ClassTag]( * 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]]. + * + * Note that some RDDs, such as those returned by groupBy(), do not guarantee order of + * elements in a partition. The unique ID assigned to each element is therefore not guaranteed, + * and may even change if the RDD is reevaluated. If a fixed ordering is required to guarantee + * the same index assignments, you should sort the RDD with sortByKey() or save it to a file. */ def zipWithUniqueId(): RDD[(T, Long)] = { val n = this.partitions.size.toLong @@ -1224,7 +1239,8 @@ abstract class RDD[T: ClassTag]( private var storageLevel: StorageLevel = StorageLevel.NONE /** User code that created this RDD (e.g. `textFile`, `parallelize`). */ - @transient private[spark] val creationSite = Utils.getCallSite + @transient private[spark] val creationSite = sc.getCallSite() + private[spark] def getCreationSite: String = Option(creationSite).map(_.shortForm).getOrElse("") private[spark] def elementClassTag: ClassTag[T] = classTag[T] diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index b2774dfc47553..788eb1ff4e455 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -630,16 +630,17 @@ class DAGScheduler( protected def runLocallyWithinThread(job: ActiveJob) { var jobResult: JobResult = JobSucceeded try { - SparkEnv.set(env) val rdd = job.finalStage.rdd val split = rdd.partitions(job.partitions(0)) val taskContext = - new TaskContext(job.finalStage.id, job.partitions(0), 0, runningLocally = true) + new TaskContext(job.finalStage.id, job.partitions(0), 0, true) + TaskContext.setTaskContext(taskContext) try { val result = job.func(taskContext, rdd.iterator(split, taskContext)) job.listener.taskSucceeded(0, result) } finally { taskContext.markTaskCompleted() + TaskContext.unset() } } catch { case e: Exception => @@ -1207,7 +1208,7 @@ class DAGScheduler( .format(job.jobId, stageId)) } else if (jobsForStage.get.size == 1) { if (!stageIdToStage.contains(stageId)) { - logError("Missing Stage for stage with id $stageId") + logError(s"Missing Stage for stage with id $stageId") } else { // This is the only job that uses this stage, so fail the stage if it is running. val stage = stageIdToStage(stageId) @@ -1301,7 +1302,7 @@ class DAGScheduler( // If the RDD has some placement preferences (as is the case for input RDDs), get those val rddPrefs = rdd.preferredLocations(rdd.partitions(partition)).toList if (!rddPrefs.isEmpty) { - return rddPrefs.map(host => TaskLocation(host)) + return rddPrefs.map(TaskLocation(_)) } // If the RDD has narrow dependencies, pick the first partition of the first narrow dep // that has any placement preferences. Ideally we would choose based on transfer sizes, diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala index 94944399b134a..12668b6c0988e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala @@ -22,10 +22,10 @@ import com.codahale.metrics.{Gauge,MetricRegistry} import org.apache.spark.SparkContext import org.apache.spark.metrics.source.Source -private[spark] class DAGSchedulerSource(val dagScheduler: DAGScheduler, sc: SparkContext) +private[spark] class DAGSchedulerSource(val dagScheduler: DAGScheduler) extends Source { override val metricRegistry = new MetricRegistry() - override val sourceName = "%s.DAGScheduler".format(sc.appName) + override val sourceName = "DAGScheduler" metricRegistry.register(MetricRegistry.name("stage", "failedStages"), new Gauge[Int] { override def getValue: Int = dagScheduler.failedStages.size diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index 64b32ae0edaac..100c9ba9b7809 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -43,38 +43,29 @@ import org.apache.spark.util.{FileLogger, JsonProtocol, Utils} * spark.eventLog.buffer.kb - Buffer size to use when writing to output streams */ private[spark] class EventLoggingListener( - appName: String, + appId: String, + logBaseDir: String, sparkConf: SparkConf, hadoopConf: Configuration) extends SparkListener with Logging { import EventLoggingListener._ - def this(appName: String, sparkConf: SparkConf) = - this(appName, sparkConf, SparkHadoopUtil.get.newConfiguration(sparkConf)) + def this(appId: String, logBaseDir: String, sparkConf: SparkConf) = + this(appId, logBaseDir, sparkConf, SparkHadoopUtil.get.newConfiguration(sparkConf)) 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("[ :/]", "-").replaceAll("[${}'\"]", "_") - .toLowerCase + "-" + System.currentTimeMillis - val logDir = Utils.resolveURI(logBaseDir) + "/" + name.stripSuffix("/") - + val logDir = EventLoggingListener.getLogDirPath(logBaseDir, appId) + val logDirName: String = logDir.split("/").last 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] - /** - * Return only the unique application directory without the base directory. - */ - def getApplicationLogDir(): String = { - name - } - /** * Begin logging events. * If compression is used, log a file that indicates which compression library is used. @@ -184,6 +175,18 @@ private[spark] object EventLoggingListener extends Logging { } else "" } + /** + * Return a file-system-safe path to the log directory for the given application. + * + * @param logBaseDir A base directory for the path to the log directory for given application. + * @param appId A unique app ID. + * @return A path which consists of file-system-safe characters. + */ + def getLogDirPath(logBaseDir: String, appId: String): String = { + val name = appId.replaceAll("[ :/]", "-").replaceAll("[${}'\"]", "_").toLowerCase + Utils.resolveURI(logBaseDir) + "/" + name.stripSuffix("/") + } + /** * Parse the event logging information associated with the logs in the given directory. * diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala index 4d6b5c81883b6..54904bffdf10b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala @@ -20,15 +20,12 @@ package org.apache.spark.scheduler import java.io.{File, FileNotFoundException, IOException, PrintWriter} import java.text.SimpleDateFormat import java.util.{Date, Properties} -import java.util.concurrent.LinkedBlockingQueue import scala.collection.mutable.HashMap import org.apache.spark._ import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.executor.{DataReadMethod, TaskMetrics} -import org.apache.spark.rdd.RDD -import org.apache.spark.storage.StorageLevel +import org.apache.spark.executor.TaskMetrics /** * :: DeveloperApi :: @@ -62,24 +59,16 @@ class JobLogger(val user: String, val logDirName: String) extends SparkListener private val dateFormat = new ThreadLocal[SimpleDateFormat]() { override def initialValue() = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") } - private val eventQueue = new LinkedBlockingQueue[SparkListenerEvent] createLogDir() - // The following 5 functions are used only in testing. - private[scheduler] def getLogDir = logDir - private[scheduler] def getJobIdToPrintWriter = jobIdToPrintWriter - private[scheduler] def getStageIdToJobId = stageIdToJobId - private[scheduler] def getJobIdToStageIds = jobIdToStageIds - private[scheduler] def getEventQueue = eventQueue - /** Create a folder for log files, the folder's name is the creation time of jobLogger */ protected def createLogDir() { val dir = new File(logDir + "/" + logDirName + "/") if (dir.exists()) { return } - if (dir.mkdirs() == false) { + if (!dir.mkdirs()) { // JobLogger should throw a exception rather than continue to construct this object. throw new IOException("create log directory error:" + logDir + "/" + logDirName + "/") } @@ -171,7 +160,6 @@ class JobLogger(val user: String, val logDirName: String) extends SparkListener } val shuffleReadMetrics = taskMetrics.shuffleReadMetrics match { case Some(metrics) => - " SHUFFLE_FINISH_TIME=" + metrics.shuffleFinishTime + " BLOCK_FETCHED_TOTAL=" + metrics.totalBlocksFetched + " BLOCK_FETCHED_LOCAL=" + metrics.localBlocksFetched + " BLOCK_FETCHED_REMOTE=" + metrics.remoteBlocksFetched + @@ -262,7 +250,7 @@ class JobLogger(val user: String, val logDirName: String) extends SparkListener protected def recordJobProperties(jobId: Int, properties: Properties) { if (properties != null) { val description = properties.getProperty(SparkContext.SPARK_JOB_DESCRIPTION, "") - jobLogInfo(jobId, description, false) + jobLogInfo(jobId, description, withTime = false) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala index d3f63ff92ac6f..e25096ea92d70 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala @@ -24,22 +24,123 @@ import org.apache.spark.storage.BlockManagerId /** * Result returned by a ShuffleMapTask to a scheduler. Includes the block manager address that the * task ran on as well as the sizes of outputs for each reducer, for passing on to the reduce tasks. - * The map output sizes are compressed using MapOutputTracker.compressSize. */ -private[spark] class MapStatus(var location: BlockManagerId, var compressedSizes: Array[Byte]) - extends Externalizable { +private[spark] sealed trait MapStatus { + /** Location where this task was run. */ + def location: BlockManagerId - def this() = this(null, null) // For deserialization only + /** Estimated size for the reduce block, in bytes. */ + def getSizeForBlock(reduceId: Int): Long +} + + +private[spark] object MapStatus { + + def apply(loc: BlockManagerId, uncompressedSizes: Array[Long]): MapStatus = { + if (uncompressedSizes.length > 2000) { + new HighlyCompressedMapStatus(loc, uncompressedSizes) + } else { + new CompressedMapStatus(loc, uncompressedSizes) + } + } + + private[this] val LOG_BASE = 1.1 + + /** + * Compress a size in bytes to 8 bits for efficient reporting of map output sizes. + * We do this by encoding the log base 1.1 of the size as an integer, which can support + * sizes up to 35 GB with at most 10% error. + */ + def compressSize(size: Long): Byte = { + if (size == 0) { + 0 + } else if (size <= 1L) { + 1 + } else { + math.min(255, math.ceil(math.log(size) / math.log(LOG_BASE)).toInt).toByte + } + } + + /** + * Decompress an 8-bit encoded block size, using the reverse operation of compressSize. + */ + def decompressSize(compressedSize: Byte): Long = { + if (compressedSize == 0) { + 0 + } else { + math.pow(LOG_BASE, compressedSize & 0xFF).toLong + } + } +} + + +/** + * A [[MapStatus]] implementation that tracks the size of each block. Size for each block is + * represented using a single byte. + * + * @param loc location where the task is being executed. + * @param compressedSizes size of the blocks, indexed by reduce partition id. + */ +private[spark] class CompressedMapStatus( + private[this] var loc: BlockManagerId, + private[this] var compressedSizes: Array[Byte]) + extends MapStatus with Externalizable { + + protected def this() = this(null, null.asInstanceOf[Array[Byte]]) // For deserialization only + + def this(loc: BlockManagerId, uncompressedSizes: Array[Long]) { + this(loc, uncompressedSizes.map(MapStatus.compressSize)) + } - def writeExternal(out: ObjectOutput) { - location.writeExternal(out) + override def location: BlockManagerId = loc + + override def getSizeForBlock(reduceId: Int): Long = { + MapStatus.decompressSize(compressedSizes(reduceId)) + } + + override def writeExternal(out: ObjectOutput): Unit = { + loc.writeExternal(out) out.writeInt(compressedSizes.length) out.write(compressedSizes) } - def readExternal(in: ObjectInput) { - location = BlockManagerId(in) - compressedSizes = new Array[Byte](in.readInt()) + override def readExternal(in: ObjectInput): Unit = { + loc = BlockManagerId(in) + val len = in.readInt() + compressedSizes = new Array[Byte](len) in.readFully(compressedSizes) } } + + +/** + * A [[MapStatus]] implementation that only stores the average size of the blocks. + * + * @param loc location where the task is being executed. + * @param avgSize average size of all the blocks + */ +private[spark] class HighlyCompressedMapStatus( + private[this] var loc: BlockManagerId, + private[this] var avgSize: Long) + extends MapStatus with Externalizable { + + def this(loc: BlockManagerId, uncompressedSizes: Array[Long]) { + this(loc, uncompressedSizes.sum / uncompressedSizes.length) + } + + protected def this() = this(null, 0L) // For deserialization only + + override def location: BlockManagerId = loc + + override def getSizeForBlock(reduceId: Int): Long = avgSize + + override def writeExternal(out: ObjectOutput): Unit = { + loc.writeExternal(out) + out.writeLong(avgSize) + } + + override def readExternal(in: ObjectInput): Unit = { + loc = BlockManagerId(in) + avgSize = in.readLong() + } +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala index 2ccbd8edeb028..4a9ff918afe25 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala @@ -58,11 +58,7 @@ private[spark] class ResultTask[T, U]( ByteBuffer.wrap(taskBinary.value), Thread.currentThread.getContextClassLoader) metrics = Some(context.taskMetrics) - try { - func(context, rdd.iterator(partition, context)) - } finally { - context.markTaskCompleted() - } + func(context, rdd.iterator(partition, context)) } // This is only callable on the driver side. diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala index a0be8307eff27..992c477493d8e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala @@ -23,6 +23,8 @@ package org.apache.spark.scheduler * machines become available and can launch tasks on them. */ private[spark] trait SchedulerBackend { + private val appId = "spark-application-" + System.currentTimeMillis + def start(): Unit def stop(): Unit def reviveOffers(): Unit @@ -33,10 +35,10 @@ private[spark] trait SchedulerBackend { def isReady(): Boolean = true /** - * The application ID associated with the job, if any. + * Get an application ID associated with the job. * - * @return The application ID, or None if the backend does not provide an ID. + * @return An application ID */ - def applicationId(): Option[String] = None + def applicationId(): String = appId } 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 381eff2147e95..79709089c0da4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala @@ -69,12 +69,15 @@ private[spark] class ShuffleMapTask( return writer.stop(success = true).get } catch { case e: Exception => - if (writer != null) { - writer.stop(success = false) + try { + if (writer != null) { + writer.stop(success = false) + } + } catch { + case e: Exception => + log.debug("Could not stop writer", e) } throw e - } finally { - context.markTaskCompleted() } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index 6aa0cca06878d..c6e47c84a0cb2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -45,13 +45,19 @@ import org.apache.spark.util.Utils private[spark] abstract class Task[T](val stageId: Int, var partitionId: Int) extends Serializable { final def run(attemptId: Long): T = { - context = new TaskContext(stageId, partitionId, attemptId, runningLocally = false) + context = new TaskContext(stageId, partitionId, attemptId, false) + TaskContext.setTaskContext(context) context.taskMetrics.hostname = Utils.localHostName() taskThread = Thread.currentThread() if (_killed) { kill(interruptThread = false) } - runTask(context) + try { + runTask(context) + } finally { + context.markTaskCompleted() + TaskContext.unset() + } } def runTask(context: TaskContext): T @@ -92,7 +98,7 @@ private[spark] abstract class Task[T](val stageId: Int, var partitionId: Int) ex if (interruptThread && taskThread != null) { taskThread.interrupt() } - } + } } /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskLocation.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskLocation.scala index 67c9a6760b1b3..10c685f29d3ac 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskLocation.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskLocation.scala @@ -22,13 +22,51 @@ package org.apache.spark.scheduler * In the latter case, we will prefer to launch the task on that executorID, but our next level * of preference will be executors on the same host if this is not possible. */ -private[spark] -class TaskLocation private (val host: String, val executorId: Option[String]) extends Serializable { - override def toString: String = "TaskLocation(" + host + ", " + executorId + ")" +private[spark] sealed trait TaskLocation { + def host: String +} + +/** + * A location that includes both a host and an executor id on that host. + */ +private [spark] case class ExecutorCacheTaskLocation(override val host: String, + val executorId: String) extends TaskLocation { +} + +/** + * A location on a host. + */ +private [spark] case class HostTaskLocation(override val host: String) extends TaskLocation { + override def toString = host +} + +/** + * A location on a host that is cached by HDFS. + */ +private [spark] case class HDFSCacheTaskLocation(override val host: String) + extends TaskLocation { + override def toString = TaskLocation.inMemoryLocationTag + host } private[spark] object TaskLocation { - def apply(host: String, executorId: String) = new TaskLocation(host, Some(executorId)) + // We identify hosts on which the block is cached with this prefix. Because this prefix contains + // underscores, which are not legal characters in hostnames, there should be no potential for + // confusion. See RFC 952 and RFC 1123 for information about the format of hostnames. + val inMemoryLocationTag = "hdfs_cache_" + + def apply(host: String, executorId: String) = new ExecutorCacheTaskLocation(host, executorId) - def apply(host: String) = new TaskLocation(host, None) + /** + * Create a TaskLocation from a string returned by getPreferredLocations. + * These strings have the form [hostname] or hdfs_cache_[hostname], depending on whether the + * location is cached. + */ + def apply(str: String) = { + val hstr = str.stripPrefix(inMemoryLocationTag) + if (hstr.equals(str)) { + new HostTaskLocation(str) + } else { + new HostTaskLocation(hstr) + } + } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala index df59f444b7a0e..3f345ceeaaf7a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala @@ -19,6 +19,8 @@ package org.apache.spark.scheduler import java.nio.ByteBuffer +import scala.util.control.NonFatal + import org.apache.spark._ import org.apache.spark.TaskState.TaskState import org.apache.spark.serializer.SerializerInstance @@ -32,7 +34,7 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul private val THREADS = sparkEnv.conf.getInt("spark.resultGetter.threads", 4) private val getTaskResultExecutor = Utils.newDaemonFixedThreadPool( - THREADS, "Result resolver thread") + THREADS, "task-result-getter") protected val serializer = new ThreadLocal[SerializerInstance] { override def initialValue(): SerializerInstance = { @@ -70,7 +72,8 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul case cnf: ClassNotFoundException => val loader = Thread.currentThread.getContextClassLoader taskSetManager.abort("ClassNotFound with classloader: " + loader) - case ex: Exception => + // Matching NonFatal so we don't catch the ControlThrowable from the "return" above. + case NonFatal(ex) => logError("Exception while getting task result", ex) taskSetManager.abort("Exception while getting task result: %s".format(ex)) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala index 1c1ce666eab0f..a129a434c9a1a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala @@ -31,6 +31,8 @@ import org.apache.spark.storage.BlockManagerId */ private[spark] trait TaskScheduler { + private val appId = "spark-application-" + System.currentTimeMillis + def rootPool: Pool def schedulingMode: SchedulingMode @@ -66,10 +68,10 @@ private[spark] trait TaskScheduler { blockManagerId: BlockManagerId): Boolean /** - * The application ID associated with the job, if any. + * Get an application ID associated with the job. * - * @return The application ID, or None if the backend does not provide an ID. + * @return An application ID */ - def applicationId(): Option[String] = None + def applicationId(): String = appId } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 633e892554c50..6d697e3d003f6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -216,8 +216,6 @@ private[spark] class TaskSchedulerImpl( * that tasks are balanced across the cluster. */ def resourceOffers(offers: Seq[WorkerOffer]): Seq[Seq[TaskDescription]] = synchronized { - SparkEnv.set(sc.env) - // Mark each slave as alive and remember its hostname // Also track if new executor is added var newExecAvail = false @@ -492,7 +490,7 @@ private[spark] class TaskSchedulerImpl( } } - override def applicationId(): Option[String] = backend.applicationId() + override def applicationId(): String = backend.applicationId() } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index d9d53faf843ff..a6c23fc85a1b0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -181,8 +181,24 @@ private[spark] class TaskSetManager( } for (loc <- tasks(index).preferredLocations) { - for (execId <- loc.executorId) { - addTo(pendingTasksForExecutor.getOrElseUpdate(execId, new ArrayBuffer)) + loc match { + case e: ExecutorCacheTaskLocation => + addTo(pendingTasksForExecutor.getOrElseUpdate(e.executorId, new ArrayBuffer)) + case e: HDFSCacheTaskLocation => { + val exe = sched.getExecutorsAliveOnHost(loc.host) + exe match { + case Some(set) => { + for (e <- set) { + addTo(pendingTasksForExecutor.getOrElseUpdate(e, new ArrayBuffer)) + } + logInfo(s"Pending task $index has a cached location at ${e.host} " + + ", where there are executors " + set.mkString(",")) + } + case None => logDebug(s"Pending task $index has a cached location at ${e.host} " + + ", but there are no executors alive there.") + } + } + case _ => Unit } addTo(pendingTasksForHost.getOrElseUpdate(loc.host, new ArrayBuffer)) for (rack <- sched.getRackForHost(loc.host)) { @@ -283,7 +299,10 @@ private[spark] class TaskSetManager( // on multiple nodes when we replicate cached blocks, as in Spark Streaming for (index <- speculatableTasks if canRunOnHost(index)) { val prefs = tasks(index).preferredLocations - val executors = prefs.flatMap(_.executorId) + val executors = prefs.flatMap(_ match { + case e: ExecutorCacheTaskLocation => Some(e.executorId) + case _ => None + }); if (executors.contains(execId)) { speculatableTasks -= index return Some((index, TaskLocality.PROCESS_LOCAL)) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala index 6abf6d930c155..fb8160abc59db 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala @@ -66,7 +66,7 @@ private[spark] object CoarseGrainedClusterMessages { case class RemoveExecutor(executorId: String, reason: String) extends CoarseGrainedClusterMessage - case class AddWebUIFilter(filterName:String, filterParams: String, proxyBase :String) + case class AddWebUIFilter(filterName:String, filterParams: Map[String, String], proxyBase :String) extends CoarseGrainedClusterMessage } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 9a0cb1c6c6ccd..59aed6b72fe42 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -62,15 +62,9 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A val createTime = System.currentTimeMillis() class DriverActor(sparkProperties: Seq[(String, String)]) extends Actor with ActorLogReceive { - override protected def log = CoarseGrainedSchedulerBackend.this.log - - private val executorActor = new HashMap[String, ActorRef] - private val executorAddress = new HashMap[String, Address] - private val executorHost = new HashMap[String, String] - private val freeCores = new HashMap[String, Int] - private val totalCores = new HashMap[String, Int] private val addressToExecutorId = new HashMap[Address, String] + private val executorDataMap = new HashMap[String, ExecutorData] override def preStart() { // Listen for remote client disconnection events, since they don't go through Akka's watch() @@ -85,16 +79,14 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A def receiveWithLogging = { case RegisterExecutor(executorId, hostPort, cores) => Utils.checkHostPort(hostPort, "Host port expected " + hostPort) - if (executorActor.contains(executorId)) { + if (executorDataMap.contains(executorId)) { sender ! RegisterExecutorFailed("Duplicate executor ID: " + executorId) } else { logInfo("Registered executor: " + sender + " with ID " + executorId) sender ! RegisteredExecutor - executorActor(executorId) = sender - executorHost(executorId) = Utils.parseHostPort(hostPort)._1 - totalCores(executorId) = cores - freeCores(executorId) = cores - executorAddress(executorId) = sender.path.address + executorDataMap.put(executorId, new ExecutorData(sender, sender.path.address, + Utils.parseHostPort(hostPort)._1, cores, cores)) + addressToExecutorId(sender.path.address) = executorId totalCoreCount.addAndGet(cores) totalRegisteredExecutors.addAndGet(1) @@ -104,13 +96,14 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A case StatusUpdate(executorId, taskId, state, data) => scheduler.statusUpdate(taskId, state, data.value) if (TaskState.isFinished(state)) { - if (executorActor.contains(executorId)) { - freeCores(executorId) += scheduler.CPUS_PER_TASK - makeOffers(executorId) - } else { - // Ignoring the update since we don't know about the executor. - val msg = "Ignored task status update (%d state %s) from unknown executor %s with ID %s" - logWarning(msg.format(taskId, state, sender, executorId)) + executorDataMap.get(executorId) match { + case Some(executorInfo) => + executorInfo.freeCores += scheduler.CPUS_PER_TASK + makeOffers(executorId) + case None => + // Ignoring the update since we don't know about the executor. + logWarning(s"Ignored task status update ($taskId state $state) " + + "from unknown executor $sender with ID $executorId") } } @@ -118,7 +111,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A makeOffers() case KillTask(taskId, executorId, interruptThread) => - executorActor(executorId) ! KillTask(taskId, executorId, interruptThread) + executorDataMap(executorId).executorActor ! KillTask(taskId, executorId, interruptThread) case StopDriver => sender ! true @@ -126,8 +119,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A case StopExecutors => logInfo("Asking each executor to shut down") - for (executor <- executorActor.values) { - executor ! StopExecutor + for ((_, executorData) <- executorDataMap) { + executorData.executorActor ! StopExecutor } sender ! true @@ -138,6 +131,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A case AddWebUIFilter(filterName, filterParams, proxyBase) => addWebUIFilter(filterName, filterParams, proxyBase) sender ! true + case DisassociatedEvent(_, address, _) => addressToExecutorId.get(address).foreach(removeExecutor(_, "remote Akka client disassociated")) @@ -148,14 +142,16 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A // Make fake resource offers on all executors def makeOffers() { - launchTasks(scheduler.resourceOffers( - executorHost.toArray.map {case (id, host) => new WorkerOffer(id, host, freeCores(id))})) + launchTasks(scheduler.resourceOffers(executorDataMap.map { case (id, executorData) => + new WorkerOffer(id, executorData.executorHost, executorData.freeCores) + }.toSeq)) } // Make fake resource offers on just one executor def makeOffers(executorId: String) { + val executorData = executorDataMap(executorId) launchTasks(scheduler.resourceOffers( - Seq(new WorkerOffer(executorId, executorHost(executorId), freeCores(executorId))))) + Seq(new WorkerOffer(executorId, executorData.executorHost, executorData.freeCores)))) } // Launch tasks returned by a set of resource offers @@ -179,25 +175,21 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A } } else { - freeCores(task.executorId) -= scheduler.CPUS_PER_TASK - executorActor(task.executorId) ! LaunchTask(new SerializableBuffer(serializedTask)) + val executorData = executorDataMap(task.executorId) + executorData.freeCores -= scheduler.CPUS_PER_TASK + executorData.executorActor ! LaunchTask(new SerializableBuffer(serializedTask)) } } } // Remove a disconnected slave from the cluster def removeExecutor(executorId: String, reason: String) { - if (executorActor.contains(executorId)) { - logInfo("Executor " + executorId + " disconnected, so removing it") - val numCores = totalCores(executorId) - executorActor -= executorId - executorHost -= executorId - addressToExecutorId -= executorAddress(executorId) - executorAddress -= executorId - totalCores -= executorId - freeCores -= executorId - totalCoreCount.addAndGet(-numCores) - scheduler.executorLost(executorId, SlaveLost(reason)) + executorDataMap.get(executorId) match { + case Some(executorInfo) => + executorDataMap -= executorId + totalCoreCount.addAndGet(-executorInfo.totalCores) + scheduler.executorLost(executorId, SlaveLost(reason)) + case None => logError(s"Asked to remove non existant executor $executorId") } } } @@ -283,15 +275,17 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A } // Add filters to the SparkUI - def addWebUIFilter(filterName: String, filterParams: String, proxyBase: String) { + def addWebUIFilter(filterName: String, filterParams: Map[String, String], proxyBase: String) { if (proxyBase != null && proxyBase.nonEmpty) { System.setProperty("spark.ui.proxyBase", proxyBase) } - if (Seq(filterName, filterParams).forall(t => t != null && t.nonEmpty)) { + val hasFilter = (filterName != null && filterName.nonEmpty && + filterParams != null && filterParams.nonEmpty) + if (hasFilter) { logInfo(s"Add WebUI Filter. $filterName, $filterParams, $proxyBase") conf.set("spark.ui.filters", filterName) - conf.set(s"spark.$filterName.params", filterParams) + filterParams.foreach { case (k, v) => conf.set(s"spark.$filterName.param.$k", v) } scheduler.sc.ui.foreach { ui => JettyUtils.addFilters(ui.getHandlers, conf) } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala new file mode 100644 index 0000000000000..b71bd5783d6df --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.scheduler.cluster + +import akka.actor.{Address, ActorRef} + +/** + * Grouping of data for an executor used by CoarseGrainedSchedulerBackend. + * + * @param executorActor The ActorRef representing this executor + * @param executorAddress The network address of this executor + * @param executorHost The hostname that this executor is running on + * @param freeCores The current number of cores available for work on the executor + * @param totalCores The total number of cores available to the executor + */ +private[cluster] class ExecutorData( + val executorActor: ActorRef, + val executorAddress: Address, + val executorHost: String , + var freeCores: Int, + val totalCores: Int +) 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 2f45d192e1d4d..ed209d195ec9d 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 @@ -34,7 +34,7 @@ private[spark] class SparkDeploySchedulerBackend( var client: AppClient = null var stopping = false var shutdownCallback : (SparkDeploySchedulerBackend) => Unit = _ - var appId: String = _ + @volatile var appId: String = _ val registrationLock = new Object() var registrationDone = false @@ -68,9 +68,8 @@ private[spark] class SparkDeploySchedulerBackend( val command = Command("org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.executorEnvs, classPathEntries, libraryPathEntries, javaOpts) val appUIAddress = sc.ui.map(_.appUIAddress).getOrElse("") - val eventLogDir = sc.eventLogger.map(_.logDir) val appDesc = new ApplicationDescription(sc.appName, maxCores, sc.executorMemory, command, - appUIAddress, eventLogDir) + appUIAddress, sc.eventLogDir) client = new AppClient(sc.env.actorSystem, masters, appDesc, this, conf) client.start() @@ -129,7 +128,11 @@ private[spark] class SparkDeploySchedulerBackend( totalCoreCount.get() >= totalExpectedCores * minRegisteredRatio } - override def applicationId(): Option[String] = Option(appId) + override def applicationId(): String = + Option(appId).getOrElse { + logWarning("Application ID is not initialized yet.") + super.applicationId + } private def waitForRegistration() = { registrationLock.synchronized { diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index 64568409dbafd..90828578cd88f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -76,6 +76,8 @@ private[spark] class CoarseMesosSchedulerBackend( var nextMesosTaskId = 0 + @volatile var appId: String = _ + def newMesosTaskId(): Int = { val id = nextMesosTaskId nextMesosTaskId += 1 @@ -167,7 +169,8 @@ private[spark] class CoarseMesosSchedulerBackend( override def offerRescinded(d: SchedulerDriver, o: OfferID) {} override def registered(d: SchedulerDriver, frameworkId: FrameworkID, masterInfo: MasterInfo) { - logInfo("Registered as framework ID " + frameworkId.getValue) + appId = frameworkId.getValue + logInfo("Registered as framework ID " + appId) registeredLock.synchronized { isRegistered = true registeredLock.notifyAll() @@ -198,7 +201,9 @@ private[spark] class CoarseMesosSchedulerBackend( val slaveId = offer.getSlaveId.toString val mem = getResource(offer.getResourcesList, "mem") val cpus = getResource(offer.getResourcesList, "cpus").toInt - if (totalCoresAcquired < maxCores && mem >= sc.executorMemory && cpus >= 1 && + if (totalCoresAcquired < maxCores && + mem >= MemoryUtils.calculateTotalMemory(sc) && + cpus >= 1 && failuresBySlaveId.getOrElse(slaveId, 0) < MAX_SLAVE_FAILURES && !slaveIdsWithExecutors.contains(slaveId)) { // Launch an executor on the slave @@ -214,7 +219,8 @@ private[spark] class CoarseMesosSchedulerBackend( .setCommand(createCommand(offer, cpusToUse + extraCoresPerSlave)) .setName("Task " + taskId) .addResources(createResource("cpus", cpusToUse)) - .addResources(createResource("mem", sc.executorMemory)) + .addResources(createResource("mem", + MemoryUtils.calculateTotalMemory(sc))) .build() d.launchTasks( Collections.singleton(offer.getId), Collections.singletonList(task), filters) @@ -310,4 +316,10 @@ private[spark] class CoarseMesosSchedulerBackend( slaveLost(d, s) } + override def applicationId(): String = + Option(appId).getOrElse { + logWarning("Application ID is not initialized yet.") + super.applicationId + } + } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MemoryUtils.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MemoryUtils.scala new file mode 100644 index 0000000000000..5101ec8352e79 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MemoryUtils.scala @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.scheduler.cluster.mesos + +import org.apache.spark.SparkContext + +private[spark] object MemoryUtils { + // These defaults copied from YARN + val OVERHEAD_FRACTION = 1.07 + val OVERHEAD_MINIMUM = 384 + + def calculateTotalMemory(sc: SparkContext) = { + math.max( + sc.conf.getOption("spark.mesos.executor.memoryOverhead") + .getOrElse(OVERHEAD_MINIMUM.toString) + .toInt + sc.executorMemory, + OVERHEAD_FRACTION * sc.executorMemory + ) + } +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index a9ef126f5de0e..e0f2fd622f54c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -30,7 +30,7 @@ import org.apache.mesos._ import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, TaskState => MesosTaskState, _} import org.apache.spark.{Logging, SparkContext, SparkException, TaskState} -import org.apache.spark.scheduler.{ExecutorExited, ExecutorLossReason, SchedulerBackend, SlaveLost, TaskDescription, TaskSchedulerImpl, WorkerOffer} +import org.apache.spark.scheduler._ import org.apache.spark.util.Utils /** @@ -62,6 +62,8 @@ private[spark] class MesosSchedulerBackend( var classLoader: ClassLoader = null + @volatile var appId: String = _ + override def start() { synchronized { classLoader = Thread.currentThread.getContextClassLoader @@ -124,15 +126,24 @@ private[spark] class MesosSchedulerBackend( command.setValue("cd %s*; ./sbin/spark-executor".format(basename)) command.addUris(CommandInfo.URI.newBuilder().setValue(uri)) } + val cpus = Resource.newBuilder() + .setName("cpus") + .setType(Value.Type.SCALAR) + .setScalar(Value.Scalar.newBuilder() + .setValue(scheduler.CPUS_PER_TASK).build()) + .build() val memory = Resource.newBuilder() .setName("mem") .setType(Value.Type.SCALAR) - .setScalar(Value.Scalar.newBuilder().setValue(sc.executorMemory).build()) + .setScalar( + Value.Scalar.newBuilder() + .setValue(MemoryUtils.calculateTotalMemory(sc)).build()) .build() ExecutorInfo.newBuilder() .setExecutorId(ExecutorID.newBuilder().setValue(execId).build()) .setCommand(command) .setData(ByteString.copyFrom(createExecArg())) + .addResources(cpus) .addResources(memory) .build() } @@ -168,7 +179,8 @@ private[spark] class MesosSchedulerBackend( override def registered(d: SchedulerDriver, frameworkId: FrameworkID, masterInfo: MasterInfo) { val oldClassLoader = setClassLoader() try { - logInfo("Registered as framework ID " + frameworkId.getValue) + appId = frameworkId.getValue + logInfo("Registered as framework ID " + appId) registeredLock.synchronized { isRegistered = true registeredLock.notifyAll() @@ -204,18 +216,31 @@ private[spark] class MesosSchedulerBackend( val offerableWorkers = new ArrayBuffer[WorkerOffer] val offerableIndices = new HashMap[String, Int] - def enoughMemory(o: Offer) = { + def sufficientOffer(o: Offer) = { val mem = getResource(o.getResourcesList, "mem") + val cpus = getResource(o.getResourcesList, "cpus") val slaveId = o.getSlaveId.getValue - mem >= sc.executorMemory || slaveIdsWithExecutors.contains(slaveId) + (mem >= MemoryUtils.calculateTotalMemory(sc) && + // need at least 1 for executor, 1 for task + cpus >= 2 * scheduler.CPUS_PER_TASK) || + (slaveIdsWithExecutors.contains(slaveId) && + cpus >= scheduler.CPUS_PER_TASK) } - for ((offer, index) <- offers.zipWithIndex if enoughMemory(offer)) { - offerableIndices.put(offer.getSlaveId.getValue, index) + for ((offer, index) <- offers.zipWithIndex if sufficientOffer(offer)) { + val slaveId = offer.getSlaveId.getValue + offerableIndices.put(slaveId, index) + val cpus = if (slaveIdsWithExecutors.contains(slaveId)) { + getResource(offer.getResourcesList, "cpus").toInt + } else { + // If the executor doesn't exist yet, subtract CPU for executor + getResource(offer.getResourcesList, "cpus").toInt - + scheduler.CPUS_PER_TASK + } offerableWorkers += new WorkerOffer( offer.getSlaveId.getValue, offer.getHostname, - getResource(offer.getResourcesList, "cpus").toInt) + cpus) } // Call into the TaskSchedulerImpl @@ -347,7 +372,20 @@ private[spark] class MesosSchedulerBackend( recordSlaveLost(d, slaveId, ExecutorExited(status)) } + override def killTask(taskId: Long, executorId: String, interruptThread: Boolean): Unit = { + driver.killTask( + TaskID.newBuilder() + .setValue(taskId.toString).build() + ) + } + // TODO: query Mesos for number of cores override def defaultParallelism() = sc.conf.getInt("spark.default.parallelism", 8) + override def applicationId(): String = + Option(appId).getOrElse { + logWarning("Application ID is not initialized yet.") + super.applicationId + } + } diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala index 9ea25c2bc7090..58b78f041cd85 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala @@ -88,6 +88,7 @@ private[spark] class LocalActor( private[spark] class LocalBackend(scheduler: TaskSchedulerImpl, val totalCores: Int) extends SchedulerBackend with ExecutorBackend { + private val appId = "local-" + System.currentTimeMillis var localActor: ActorRef = null override def start() { @@ -115,4 +116,6 @@ private[spark] class LocalBackend(scheduler: TaskSchedulerImpl, val totalCores: localActor ! StatusUpdate(taskId, state, serializedData) } + override def applicationId(): String = appId + } diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala index 4b9454d75abb7..746ed33b54c00 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala @@ -103,13 +103,11 @@ private[spark] class HashShuffleWriter[K, V]( private def commitWritesAndBuildStatus(): MapStatus = { // Commit the writes. Get the size of each bucket block (total block size). - val compressedSizes = shuffle.writers.map { writer: BlockObjectWriter => + val sizes: Array[Long] = shuffle.writers.map { writer: BlockObjectWriter => writer.commitAndClose() - val size = writer.fileSegment().length - MapOutputTracker.compressSize(size) + writer.fileSegment().length } - - new MapStatus(blockManager.blockManagerId, compressedSizes) + MapStatus(blockManager.blockManagerId, sizes) } private def revertWrites(): Unit = { diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala index 89a78d6982ba0..927481b72cf4f 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala @@ -70,8 +70,7 @@ private[spark] class SortShuffleWriter[K, V, C]( val partitionLengths = sorter.writePartitionedFile(blockId, context, outputFile) shuffleBlockManager.writeIndexFile(dep.shuffleId, mapId, partitionLengths) - mapStatus = new MapStatus(blockManager.blockManagerId, - partitionLengths.map(MapOutputTracker.compressSize)) + mapStatus = MapStatus(blockManager.blockManagerId, partitionLengths) } /** Close this writer, passing along whether the map completed */ diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index d1bee3d2c033c..3f5d06e1aeee7 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -22,6 +22,7 @@ import java.nio.{ByteBuffer, MappedByteBuffer} import scala.concurrent.ExecutionContext.Implicits.global +import scala.collection.mutable import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.concurrent.{Await, Future} import scala.concurrent.duration._ @@ -112,6 +113,11 @@ private[spark] class BlockManager( private val broadcastCleaner = new MetadataCleaner( MetadataCleanerType.BROADCAST_VARS, this.dropOldBroadcastBlocks, conf) + // Field related to peer block managers that are necessary for block replication + @volatile private var cachedPeers: Seq[BlockManagerId] = _ + private val peerFetchLock = new Object + private var lastPeerFetchTime = 0L + initialize() /* The compression codec to use. Note that the "lazy" val is necessary because we want to delay @@ -787,31 +793,111 @@ private[spark] class BlockManager( } /** - * Replicate block to another node. + * Get peer block managers in the system. + */ + private def getPeers(forceFetch: Boolean): Seq[BlockManagerId] = { + peerFetchLock.synchronized { + val cachedPeersTtl = conf.getInt("spark.storage.cachedPeersTtl", 60 * 1000) // milliseconds + val timeout = System.currentTimeMillis - lastPeerFetchTime > cachedPeersTtl + if (cachedPeers == null || forceFetch || timeout) { + cachedPeers = master.getPeers(blockManagerId).sortBy(_.hashCode) + lastPeerFetchTime = System.currentTimeMillis + logDebug("Fetched peers from master: " + cachedPeers.mkString("[", ",", "]")) + } + cachedPeers + } + } + + /** + * Replicate block to another node. Not that this is a blocking call that returns after + * the block has been replicated. */ - @volatile var cachedPeers: Seq[BlockManagerId] = null private def replicate(blockId: BlockId, data: ByteBuffer, level: StorageLevel): Unit = { + val maxReplicationFailures = conf.getInt("spark.storage.maxReplicationFailures", 1) + val numPeersToReplicateTo = level.replication - 1 + val peersForReplication = new ArrayBuffer[BlockManagerId] + val peersReplicatedTo = new ArrayBuffer[BlockManagerId] + val peersFailedToReplicateTo = new ArrayBuffer[BlockManagerId] val tLevel = StorageLevel( level.useDisk, level.useMemory, level.useOffHeap, level.deserialized, 1) - if (cachedPeers == null) { - cachedPeers = master.getPeers(blockManagerId, level.replication - 1) + val startTime = System.currentTimeMillis + val random = new Random(blockId.hashCode) + + var replicationFailed = false + var failures = 0 + var done = false + + // Get cached list of peers + peersForReplication ++= getPeers(forceFetch = false) + + // Get a random peer. Note that this selection of a peer is deterministic on the block id. + // So assuming the list of peers does not change and no replication failures, + // if there are multiple attempts in the same node to replicate the same block, + // the same set of peers will be selected. + def getRandomPeer(): Option[BlockManagerId] = { + // If replication had failed, then force update the cached list of peers and remove the peers + // that have been already used + if (replicationFailed) { + peersForReplication.clear() + peersForReplication ++= getPeers(forceFetch = true) + peersForReplication --= peersReplicatedTo + peersForReplication --= peersFailedToReplicateTo + } + if (!peersForReplication.isEmpty) { + Some(peersForReplication(random.nextInt(peersForReplication.size))) + } else { + None + } } - for (peer: BlockManagerId <- cachedPeers) { - val start = System.nanoTime - data.rewind() - logDebug(s"Try to replicate $blockId once; The size of the data is ${data.limit()} Bytes. " + - s"To node: $peer") - try { - blockTransferService.uploadBlockSync( - peer.host, peer.port, blockId.toString, new NioByteBufferManagedBuffer(data), tLevel) - } catch { - case e: Exception => - logError(s"Failed to replicate block to $peer", e) + // One by one choose a random peer and try uploading the block to it + // If replication fails (e.g., target peer is down), force the list of cached peers + // to be re-fetched from driver and then pick another random peer for replication. Also + // temporarily black list the peer for which replication failed. + // + // This selection of a peer and replication is continued in a loop until one of the + // following 3 conditions is fulfilled: + // (i) specified number of peers have been replicated to + // (ii) too many failures in replicating to peers + // (iii) no peer left to replicate to + // + while (!done) { + getRandomPeer() match { + case Some(peer) => + try { + val onePeerStartTime = System.currentTimeMillis + data.rewind() + logTrace(s"Trying to replicate $blockId of ${data.limit()} bytes to $peer") + blockTransferService.uploadBlockSync( + peer.host, peer.port, blockId.toString, new NioByteBufferManagedBuffer(data), tLevel) + logTrace(s"Replicated $blockId of ${data.limit()} bytes to $peer in %f ms" + .format((System.currentTimeMillis - onePeerStartTime))) + peersReplicatedTo += peer + peersForReplication -= peer + replicationFailed = false + if (peersReplicatedTo.size == numPeersToReplicateTo) { + done = true // specified number of peers have been replicated to + } + } catch { + case e: Exception => + logWarning(s"Failed to replicate $blockId to $peer, failure #$failures", e) + failures += 1 + replicationFailed = true + peersFailedToReplicateTo += peer + if (failures > maxReplicationFailures) { // too many failures in replcating to peers + done = true + } + } + case None => // no peer left to replicate to + done = true } - - logDebug("Replicating BlockId %s once used %fs; The size of the data is %d bytes." - .format(blockId, (System.nanoTime - start) / 1e6, data.limit())) + } + val timeTakeMs = (System.currentTimeMillis - startTime) + logDebug(s"Replicating $blockId of ${data.limit()} bytes to " + + s"${peersReplicatedTo.size} peer(s) took $timeTakeMs ms") + if (peersReplicatedTo.size < numPeersToReplicateTo) { + logWarning(s"Block $blockId replicated to only " + + s"${peersReplicatedTo.size} peer(s) instead of $numPeersToReplicateTo peers") } } 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 d4487fce49ab6..142285094342c 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala @@ -59,6 +59,8 @@ class BlockManagerId private ( def port: Int = port_ + def isDriver: Boolean = (executorId == "") + override def writeExternal(out: ObjectOutput) { out.writeUTF(executorId_) out.writeUTF(host_) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala index 2e262594b3538..d08e1419e3e41 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -84,13 +84,8 @@ class BlockManagerMaster( } /** Get ids of other nodes in the cluster from the driver */ - def getPeers(blockManagerId: BlockManagerId, numPeers: Int): Seq[BlockManagerId] = { - val result = askDriverWithReply[Seq[BlockManagerId]](GetPeers(blockManagerId, numPeers)) - if (result.length != numPeers) { - throw new SparkException( - "Error getting peers, only got " + result.size + " instead of " + numPeers) - } - result + def getPeers(blockManagerId: BlockManagerId): Seq[BlockManagerId] = { + askDriverWithReply[Seq[BlockManagerId]](GetPeers(blockManagerId)) } /** 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 1a6c7cb24f9ac..6a06257ed0c08 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -83,8 +83,8 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus case GetLocationsMultipleBlockIds(blockIds) => sender ! getLocationsMultipleBlockIds(blockIds) - case GetPeers(blockManagerId, size) => - sender ! getPeers(blockManagerId, size) + case GetPeers(blockManagerId) => + sender ! getPeers(blockManagerId) case GetMemoryStatus => sender ! memoryStatus @@ -173,11 +173,10 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus * from the executors, but not from the driver. */ private def removeBroadcast(broadcastId: Long, removeFromDriver: Boolean): Future[Seq[Int]] = { - // TODO: Consolidate usages of import context.dispatcher val removeMsg = RemoveBroadcast(broadcastId, removeFromDriver) val requiredBlockManagers = blockManagerInfo.values.filter { info => - removeFromDriver || info.blockManagerId.executorId != "" + removeFromDriver || !info.blockManagerId.isDriver } Future.sequence( requiredBlockManagers.map { bm => @@ -212,7 +211,7 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus val minSeenTime = now - slaveTimeout val toRemove = new mutable.HashSet[BlockManagerId] for (info <- blockManagerInfo.values) { - if (info.lastSeenMs < minSeenTime && info.blockManagerId.executorId != "") { + if (info.lastSeenMs < minSeenTime && !info.blockManagerId.isDriver) { logWarning("Removing BlockManager " + info.blockManagerId + " with no recent heart beats: " + (now - info.lastSeenMs) + "ms exceeds " + slaveTimeout + "ms") toRemove += info.blockManagerId @@ -232,7 +231,7 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus */ private def heartbeatReceived(blockManagerId: BlockManagerId): Boolean = { if (!blockManagerInfo.contains(blockManagerId)) { - blockManagerId.executorId == "" && !isLocal + blockManagerId.isDriver && !isLocal } else { blockManagerInfo(blockManagerId).updateLastSeenMs() true @@ -355,7 +354,7 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus tachyonSize: Long) { if (!blockManagerInfo.contains(blockManagerId)) { - if (blockManagerId.executorId == "" && !isLocal) { + if (blockManagerId.isDriver && !isLocal) { // We intentionally do not register the master (except in local mode), // so we should not indicate failure. sender ! true @@ -403,16 +402,14 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus blockIds.map(blockId => getLocations(blockId)) } - private def getPeers(blockManagerId: BlockManagerId, size: Int): Seq[BlockManagerId] = { - val peers: Array[BlockManagerId] = blockManagerInfo.keySet.toArray - - val selfIndex = peers.indexOf(blockManagerId) - if (selfIndex == -1) { - throw new SparkException("Self index for " + blockManagerId + " not found") + /** Get the list of the peers of the given block manager */ + private def getPeers(blockManagerId: BlockManagerId): Seq[BlockManagerId] = { + val blockManagerIds = blockManagerInfo.keySet + if (blockManagerIds.contains(blockManagerId)) { + blockManagerIds.filterNot { _.isDriver }.filterNot { _ == blockManagerId }.toSeq + } else { + Seq.empty } - - // Note that this logic will select the same node multiple times if there aren't enough peers - Array.tabulate[BlockManagerId](size) { i => peers((selfIndex + i + 1) % peers.length) }.toSeq } } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala index 2ba16b8476600..3db5dd9774ae8 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala @@ -88,7 +88,7 @@ private[spark] object BlockManagerMessages { case class GetLocationsMultipleBlockIds(blockIds: Array[BlockId]) extends ToBlockManagerMaster - case class GetPeers(blockManagerId: BlockManagerId, size: Int) extends ToBlockManagerMaster + case class GetPeers(blockManagerId: BlockManagerId) extends ToBlockManagerMaster case class RemoveExecutor(execId: String) extends ToBlockManagerMaster diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveActor.scala index 14ae2f38c5670..8462871e798a5 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveActor.scala @@ -58,9 +58,9 @@ class BlockManagerSlaveActor( SparkEnv.get.shuffleManager.unregisterShuffle(shuffleId) } - case RemoveBroadcast(broadcastId, tellMaster) => + case RemoveBroadcast(broadcastId, _) => doAsync[Int]("removing broadcast " + broadcastId, sender) { - blockManager.removeBroadcast(broadcastId, tellMaster) + blockManager.removeBroadcast(broadcastId, tellMaster = true) } case GetBlockStatus(blockId, _) => diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala index 49fea6d9e2a76..8569c6f3cbbc3 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala @@ -22,10 +22,10 @@ import com.codahale.metrics.{Gauge,MetricRegistry} import org.apache.spark.SparkContext import org.apache.spark.metrics.source.Source -private[spark] class BlockManagerSource(val blockManager: BlockManager, sc: SparkContext) +private[spark] class BlockManagerSource(val blockManager: BlockManager) extends Source { override val metricRegistry = new MetricRegistry() - override val sourceName = "%s.BlockManager".format(sc.appName) + override val sourceName = "BlockManager" metricRegistry.register(MetricRegistry.name("memory", "maxMem_MB"), new Gauge[Long] { override def getValue: Long = { diff --git a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala index e9304f6bb45d0..bac459e835a3f 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala @@ -73,7 +73,21 @@ private[spark] class DiskStore(blockManager: BlockManager, diskManager: DiskBloc val startTime = System.currentTimeMillis val file = diskManager.getFile(blockId) val outputStream = new FileOutputStream(file) - blockManager.dataSerializeStream(blockId, outputStream, values) + try { + try { + blockManager.dataSerializeStream(blockId, outputStream, values) + } finally { + // Close outputStream here because it should be closed before file is deleted. + outputStream.close() + } + } catch { + case e: Throwable => + if (file.exists()) { + file.delete() + } + throw e + } + val length = file.length val timeTaken = System.currentTimeMillis - startTime diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index 0a09c24d61879..edbc729c17ade 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -132,8 +132,6 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) PutResult(res.size, res.data, droppedBlocks) case Right(iteratorValues) => // Not enough space to unroll this block; drop to disk if applicable - logWarning(s"Not enough space to store block $blockId in memory! " + - s"Free memory is $freeMemory bytes.") if (level.useDisk && allowPersistToDisk) { logWarning(s"Persisting block $blockId to disk instead.") val res = blockManager.diskStore.putIterator(blockId, iteratorValues, level, returnValues) @@ -265,6 +263,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) Left(vector.toArray) } else { // We ran out of space while unrolling the values for this block + logUnrollFailureMessage(blockId, vector.estimateSize()) Right(vector.iterator ++ values) } @@ -424,7 +423,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) * Reserve additional memory for unrolling blocks used by this thread. * Return whether the request is granted. */ - private[spark] def reserveUnrollMemoryForThisThread(memory: Long): Boolean = { + def reserveUnrollMemoryForThisThread(memory: Long): Boolean = { accountingLock.synchronized { val granted = freeMemory > currentUnrollMemory + memory if (granted) { @@ -439,7 +438,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) * Release memory used by this thread for unrolling blocks. * If the amount is not specified, remove the current thread's allocation altogether. */ - private[spark] def releaseUnrollMemoryForThisThread(memory: Long = -1L): Unit = { + def releaseUnrollMemoryForThisThread(memory: Long = -1L): Unit = { val threadId = Thread.currentThread().getId accountingLock.synchronized { if (memory < 0) { @@ -457,16 +456,50 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) /** * Return the amount of memory currently occupied for unrolling blocks across all threads. */ - private[spark] def currentUnrollMemory: Long = accountingLock.synchronized { + def currentUnrollMemory: Long = accountingLock.synchronized { unrollMemoryMap.values.sum } /** * Return the amount of memory currently occupied for unrolling blocks by this thread. */ - private[spark] def currentUnrollMemoryForThisThread: Long = accountingLock.synchronized { + def currentUnrollMemoryForThisThread: Long = accountingLock.synchronized { unrollMemoryMap.getOrElse(Thread.currentThread().getId, 0L) } + + /** + * Return the number of threads currently unrolling blocks. + */ + def numThreadsUnrolling: Int = accountingLock.synchronized { unrollMemoryMap.keys.size } + + /** + * Log information about current memory usage. + */ + def logMemoryUsage(): Unit = { + val blocksMemory = currentMemory + val unrollMemory = currentUnrollMemory + val totalMemory = blocksMemory + unrollMemory + logInfo( + s"Memory use = ${Utils.bytesToString(blocksMemory)} (blocks) + " + + s"${Utils.bytesToString(unrollMemory)} (scratch space shared across " + + s"$numThreadsUnrolling thread(s)) = ${Utils.bytesToString(totalMemory)}. " + + s"Storage limit = ${Utils.bytesToString(maxMemory)}." + ) + } + + /** + * Log a warning for failing to unroll a block. + * + * @param blockId ID of the block we are trying to unroll. + * @param finalVectorSize Final size of the vector before unrolling failed. + */ + def logUnrollFailureMessage(blockId: BlockId, finalVectorSize: Long): Unit = { + logWarning( + s"Not enough space to cache $blockId in memory! " + + s"(computed ${Utils.bytesToString(finalVectorSize)} so far)" + ) + logMemoryUsage() + } } private[spark] case class ResultWithDroppedBlocks( diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala index d868758a7f549..71b276b5f18e4 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala @@ -121,7 +121,7 @@ final class ShuffleBlockFetcherIterator( } override def onBlockFetchFailure(e: Throwable): Unit = { - logError("Failed to get block(s) from ${req.address.host}:${req.address.port}", e) + logError(s"Failed to get block(s) from ${req.address.host}:${req.address.port}", e) // Note that there is a chance that some blocks have been fetched successfully, but we // still add them to the failed queue. This is fine because when the caller see a // FetchFailedException, it is going to fail the entire task anyway. diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index 6b4689291097f..2a27d49d2de05 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -21,9 +21,7 @@ import java.net.{InetSocketAddress, URL} import javax.servlet.DispatcherType import javax.servlet.http.{HttpServlet, HttpServletRequest, HttpServletResponse} -import scala.annotation.tailrec import scala.language.implicitConversions -import scala.util.{Failure, Success, Try} import scala.xml.Node import org.eclipse.jetty.server.Server @@ -147,15 +145,19 @@ private[spark] object JettyUtils extends Logging { val holder : FilterHolder = new FilterHolder() holder.setClassName(filter) // Get any parameters for each filter - val paramName = "spark." + filter + ".params" - val params = conf.get(paramName, "").split(',').map(_.trim()).toSet - params.foreach { - case param : String => + conf.get("spark." + filter + ".params", "").split(',').map(_.trim()).toSet.foreach { + param: String => if (!param.isEmpty) { val parts = param.split("=") if (parts.length == 2) holder.setInitParameter(parts(0), parts(1)) } } + + val prefix = s"spark.$filter.param." + conf.getAll + .filter { case (k, v) => k.length() > prefix.length() && k.startsWith(prefix) } + .foreach { case (k, v) => holder.setInitParameter(k.substring(prefix.length()), v) } + val enumDispatcher = java.util.EnumSet.of(DispatcherType.ASYNC, DispatcherType.ERROR, DispatcherType.FORWARD, DispatcherType.INCLUDE, DispatcherType.REQUEST) handlers.foreach { case(handler) => handler.addFilter(holder, "/*", enumDispatcher) } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index db01be596e073..2414e4c65237e 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -103,7 +103,7 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { val taskHeaders: Seq[String] = Seq( - "Index", "ID", "Attempt", "Status", "Locality Level", "Executor", + "Index", "ID", "Attempt", "Status", "Locality Level", "Executor ID / Host", "Launch Time", "Duration", "GC Time", "Accumulators") ++ {if (hasInput) Seq("Input") else Nil} ++ {if (hasShuffleRead) Seq("Shuffle Read") else Nil} ++ @@ -282,7 +282,7 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { } {info.status} {info.taskLocality} - {info.host} + {info.executorId} / {info.host} {UIUtils.formatDate(new Date(info.launchTime))} {formatDuration} 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 c4dddb2d1037e..5b2e7d3a7edb9 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -25,7 +25,6 @@ import scala.collection.Map import org.json4s.DefaultFormats import org.json4s.JsonDSL._ import org.json4s.JsonAST._ -import org.json4s.jackson.JsonMethods._ import org.apache.spark.executor.{DataReadMethod, InputMetrics, ShuffleReadMetrics, @@ -255,7 +254,6 @@ private[spark] object JsonProtocol { } def shuffleReadMetricsToJson(shuffleReadMetrics: ShuffleReadMetrics): JValue = { - ("Shuffle Finish Time" -> shuffleReadMetrics.shuffleFinishTime) ~ ("Remote Blocks Fetched" -> shuffleReadMetrics.remoteBlocksFetched) ~ ("Local Blocks Fetched" -> shuffleReadMetrics.localBlocksFetched) ~ ("Fetch Wait Time" -> shuffleReadMetrics.fetchWaitTime) ~ @@ -590,7 +588,6 @@ private[spark] object JsonProtocol { def shuffleReadMetricsFromJson(json: JValue): ShuffleReadMetrics = { val metrics = new ShuffleReadMetrics - metrics.shuffleFinishTime = (json \ "Shuffle Finish Time").extract[Long] metrics.remoteBlocksFetched = (json \ "Remote Blocks Fetched").extract[Int] metrics.localBlocksFetched = (json \ "Local Blocks Fetched").extract[Int] metrics.fetchWaitTime = (json \ "Fetch Wait Time").extract[Long] 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 c76b7af18481d..3d307b3c16d3e 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -23,7 +23,7 @@ import java.nio.ByteBuffer import java.util.{Properties, Locale, Random, UUID} import java.util.concurrent.{ThreadFactory, ConcurrentHashMap, Executors, ThreadPoolExecutor} -import org.apache.log4j.PropertyConfigurator +import org.eclipse.jetty.util.MultiException import scala.collection.JavaConversions._ import scala.collection.Map @@ -37,18 +37,23 @@ import com.google.common.io.Files import com.google.common.util.concurrent.ThreadFactoryBuilder import org.apache.commons.lang3.SystemUtils import org.apache.hadoop.conf.Configuration +import org.apache.log4j.PropertyConfigurator import org.apache.hadoop.fs.{FileSystem, FileUtil, Path} import org.json4s._ import tachyon.client.{TachyonFile,TachyonFS} import org.apache.spark._ -import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.executor.ExecutorUncaughtExceptionHandler import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance} /** CallSite represents a place in user code. It can have a short and a long form. */ private[spark] case class CallSite(shortForm: String, longForm: String) +private[spark] object CallSite { + val SHORT_FORM = "callSite.short" + val LONG_FORM = "callSite.long" +} + /** * Various utility methods used by Spark. */ @@ -81,7 +86,7 @@ private[spark] object Utils extends Logging { ois.readObject.asInstanceOf[T] } - /** Deserialize a Long value (used for {@link org.apache.spark.api.python.PythonPartitioner}) */ + /** Deserialize a Long value (used for [[org.apache.spark.api.python.PythonPartitioner]]) */ def deserializeLongValue(bytes: Array[Byte]) : Long = { // Note: we assume that we are given a Long value encoded in network (big-endian) byte order var result = bytes(7) & 0xFFL @@ -148,7 +153,7 @@ private[spark] object Utils extends Logging { def classForName(className: String) = Class.forName(className, true, getContextOrSparkClassLoader) /** - * Primitive often used when writing {@link java.nio.ByteBuffer} to {@link java.io.DataOutput}. + * Primitive often used when writing [[java.nio.ByteBuffer]] to [[java.io.DataOutput]] */ def writeByteBuffer(bb: ByteBuffer, out: ObjectOutput) = { if (bb.hasArray) { @@ -328,7 +333,7 @@ private[spark] object Utils extends Logging { val tempFile = File.createTempFile("fetchFileTemp", null, new File(tempDir)) val targetFile = new File(targetDir, filename) val uri = new URI(url) - val fileOverwrite = conf.getBoolean("spark.files.overwrite", false) + val fileOverwrite = conf.getBoolean("spark.files.overwrite", defaultValue = false) uri.getScheme match { case "http" | "https" | "ftp" => logInfo("Fetching " + url + " to " + tempFile) @@ -350,7 +355,7 @@ private[spark] object Utils extends Logging { uc.connect() val in = uc.getInputStream() val out = new FileOutputStream(tempFile) - Utils.copyStream(in, out, true) + Utils.copyStream(in, out, closeStreams = true) if (targetFile.exists && !Files.equal(tempFile, targetFile)) { if (fileOverwrite) { targetFile.delete() @@ -397,7 +402,7 @@ private[spark] object Utils extends Logging { val fs = getHadoopFileSystem(uri, hadoopConf) val in = fs.open(new Path(uri)) val out = new FileOutputStream(tempFile) - Utils.copyStream(in, out, true) + Utils.copyStream(in, out, closeStreams = true) if (targetFile.exists && !Files.equal(tempFile, targetFile)) { if (fileOverwrite) { targetFile.delete() @@ -661,7 +666,7 @@ private[spark] object Utils extends Logging { */ def deleteRecursively(file: File) { if (file != null) { - if ((file.isDirectory) && !isSymlink(file)) { + if (file.isDirectory() && !isSymlink(file)) { for (child <- listFilesSafely(file)) { deleteRecursively(child) } @@ -696,26 +701,27 @@ private[spark] object Utils extends Logging { new File(file.getParentFile().getCanonicalFile(), file.getName()) } - if (fileInCanonicalDir.getCanonicalFile().equals(fileInCanonicalDir.getAbsoluteFile())) { - return false - } else { - return true - } + !fileInCanonicalDir.getCanonicalFile().equals(fileInCanonicalDir.getAbsoluteFile()) } /** - * Finds all the files in a directory whose last modified time is older than cutoff seconds. - * @param dir must be the path to a directory, or IllegalArgumentException is thrown - * @param cutoff measured in seconds. Files older than this are returned. + * Determines if a directory contains any files newer than cutoff seconds. + * + * @param dir must be the path to a directory, or IllegalArgumentException is thrown + * @param cutoff measured in seconds. Returns true if there are any files or directories in the + * given directory whose last modified time is later than this many seconds ago */ - def findOldFiles(dir: File, cutoff: Long): Seq[File] = { - val currentTimeMillis = System.currentTimeMillis - if (dir.isDirectory) { - val files = listFilesSafely(dir) - files.filter { file => file.lastModified < (currentTimeMillis - cutoff * 1000) } - } else { - throw new IllegalArgumentException(dir + " is not a directory!") + def doesDirectoryContainAnyNewFiles(dir: File, cutoff: Long): Boolean = { + if (!dir.isDirectory) { + throw new IllegalArgumentException("$dir is not a directory!") } + val filesAndDirs = dir.listFiles() + val cutoffTimeInMillis = System.currentTimeMillis - (cutoff * 1000) + + filesAndDirs.exists(_.lastModified() > cutoffTimeInMillis) || + filesAndDirs.filter(_.isDirectory).exists( + subdir => doesDirectoryContainAnyNewFiles(subdir, cutoff) + ) } /** @@ -799,7 +805,7 @@ private[spark] object Utils extends Logging { .start() new Thread("read stdout for " + command(0)) { override def run() { - for (line <- Source.fromInputStream(process.getInputStream).getLines) { + for (line <- Source.fromInputStream(process.getInputStream).getLines()) { System.err.println(line) } } @@ -813,8 +819,10 @@ private[spark] object Utils extends Logging { /** * Execute a command and get its output, throwing an exception if it yields a code other than 0. */ - def executeAndGetOutput(command: Seq[String], workingDir: File = new File("."), - extraEnvironment: Map[String, String] = Map.empty): String = { + def executeAndGetOutput( + command: Seq[String], + workingDir: File = new File("."), + extraEnvironment: Map[String, String] = Map.empty): String = { val builder = new ProcessBuilder(command: _*) .directory(workingDir) val environment = builder.environment() @@ -824,7 +832,7 @@ private[spark] object Utils extends Logging { val process = builder.start() new Thread("read stderr for " + command(0)) { override def run() { - for (line <- Source.fromInputStream(process.getErrorStream).getLines) { + for (line <- Source.fromInputStream(process.getErrorStream).getLines()) { System.err.println(line) } } @@ -832,7 +840,7 @@ private[spark] object Utils extends Logging { val output = new StringBuffer val stdoutThread = new Thread("read stdout for " + command(0)) { override def run() { - for (line <- Source.fromInputStream(process.getInputStream).getLines) { + for (line <- Source.fromInputStream(process.getInputStream).getLines()) { output.append(line) } } @@ -841,8 +849,8 @@ private[spark] object Utils extends Logging { val exitCode = process.waitFor() stdoutThread.join() // Wait for it to finish reading output if (exitCode != 0) { - logError(s"Process $command exited with code $exitCode: ${output}") - throw new SparkException("Process " + command + " exited with code " + exitCode) + logError(s"Process $command exited with code $exitCode: $output") + throw new SparkException(s"Process $command exited with code $exitCode") } output.toString } @@ -855,29 +863,37 @@ private[spark] object Utils extends Logging { try { block } catch { + case e: ControlThrowable => throw e case t: Throwable => ExecutorUncaughtExceptionHandler.uncaughtException(t) } } - /** - * A regular expression to match classes of the "core" Spark API that we want to skip when - * finding the call site of a method. - */ - private val SPARK_CLASS_REGEX = """^org\.apache\.spark(\.api\.java)?(\.util)?(\.rdd)?\.[A-Z]""".r + /** Default filtering function for finding call sites using `getCallSite`. */ + private def coreExclusionFunction(className: String): Boolean = { + // A regular expression to match classes of the "core" Spark API that we want to skip when + // finding the call site of a method. + val SPARK_CORE_CLASS_REGEX = """^org\.apache\.spark(\.api\.java)?(\.util)?(\.rdd)?\.[A-Z]""".r + val SCALA_CLASS_REGEX = """^scala""".r + val isSparkCoreClass = SPARK_CORE_CLASS_REGEX.findFirstIn(className).isDefined + val isScalaClass = SCALA_CLASS_REGEX.findFirstIn(className).isDefined + // If the class is a Spark internal class or a Scala class, then exclude. + isSparkCoreClass || isScalaClass + } /** * When called inside a class in the spark package, returns the name of the user code class * (outside the spark package) that called into Spark, as well as which Spark method they called. * This is used, for example, to tell users where in their code each RDD got created. + * + * @param skipClass Function that is used to exclude non-user-code classes. */ - def getCallSite: CallSite = { - val trace = Thread.currentThread.getStackTrace() - .filterNot { ste:StackTraceElement => - // When running under some profilers, the current stack trace might contain some bogus - // frames. This is intended to ensure that we don't crash in these situations by - // ignoring any frames that we can't examine. - (ste == null || ste.getMethodName == null || ste.getMethodName.contains("getStackTrace")) - } + def getCallSite(skipClass: String => Boolean = coreExclusionFunction): CallSite = { + val trace = Thread.currentThread.getStackTrace().filterNot { ste: StackTraceElement => + // When running under some profilers, the current stack trace might contain some bogus + // frames. This is intended to ensure that we don't crash in these situations by + // ignoring any frames that we can't examine. + ste == null || ste.getMethodName == null || ste.getMethodName.contains("getStackTrace") + } // Keep crawling up the stack trace until we find the first function not inside of the spark // package. We track the last (shallowest) contiguous Spark method. This might be an RDD @@ -891,7 +907,7 @@ private[spark] object Utils extends Logging { for (el <- trace) { if (insideSpark) { - if (SPARK_CLASS_REGEX.findFirstIn(el.getClassName).isDefined) { + if (skipClass(el.getClassName)) { lastSparkMethod = if (el.getMethodName == "") { // Spark method is a constructor; get its class name el.getClassName.substring(el.getClassName.lastIndexOf('.') + 1) @@ -911,7 +927,7 @@ private[spark] object Utils extends Logging { } val callStackDepth = System.getProperty("spark.callstack.depth", "20").toInt CallSite( - shortForm = "%s at %s:%s".format(lastSparkMethod, firstUserFile, firstUserLine), + shortForm = s"$lastSparkMethod at $firstUserFile:$firstUserLine", longForm = callStack.take(callStackDepth).mkString("\n")) } @@ -1014,7 +1030,7 @@ private[spark] object Utils extends Logging { false } - def isSpace(c: Char): Boolean = { + private def isSpace(c: Char): Boolean = { " \t\r\n".indexOf(c) != -1 } @@ -1166,7 +1182,7 @@ private[spark] object Utils extends Logging { } import scala.sys.process._ (linkCmd + src.getAbsolutePath() + " " + dst.getPath() + cmdSuffix) lines_! - ProcessLogger(line => (logInfo(line))) + ProcessLogger(line => logInfo(line)) } @@ -1247,7 +1263,7 @@ private[spark] object Utils extends Logging { val startTime = System.currentTimeMillis while (!terminated) { try { - process.exitValue + process.exitValue() terminated = true } catch { case e: IllegalThreadStateException => @@ -1291,6 +1307,20 @@ private[spark] object Utils extends Logging { } } + /** Executes the given block in a Try, logging any uncaught exceptions. */ + def tryLog[T](f: => T): Try[T] = { + try { + val res = f + scala.util.Success(res) + } catch { + case ct: ControlThrowable => + throw ct + case t: Throwable => + logError(s"Uncaught exception in thread ${Thread.currentThread().getName}", t) + scala.util.Failure(t) + } + } + /** Returns true if the given exception was fatal. See docs for scala.util.control.NonFatal. */ def isFatalError(e: Throwable): Boolean = { e match { @@ -1382,15 +1412,15 @@ private[spark] object Utils extends Logging { } /** - * Default number of retries in binding to a port. + * Default maximum number of retries when binding to a port before giving up. */ val portMaxRetries: Int = { if (sys.props.contains("spark.testing")) { // Set a higher number of retries for tests... - sys.props.get("spark.ports.maxRetries").map(_.toInt).getOrElse(100) + sys.props.get("spark.port.maxRetries").map(_.toInt).getOrElse(100) } else { Option(SparkEnv.get) - .flatMap(_.conf.getOption("spark.ports.maxRetries")) + .flatMap(_.conf.getOption("spark.port.maxRetries")) .map(_.toInt) .getOrElse(16) } @@ -1414,7 +1444,12 @@ private[spark] object Utils extends Logging { val serviceString = if (serviceName.isEmpty) "" else s" '$serviceName'" for (offset <- 0 to maxRetries) { // Do not increment port if startPort is 0, which is treated as a special port - val tryPort = if (startPort == 0) startPort else (startPort + offset) % 65536 + val tryPort = if (startPort == 0) { + startPort + } else { + // If the new port wraps around, do not try a privilege port + ((startPort + offset - 1024) % (65536 - 1024)) + 1024 + } try { val (service, port) = startService(tryPort) logInfo(s"Successfully started service$serviceString on port $port.") @@ -1447,6 +1482,7 @@ private[spark] object Utils extends Logging { return true } isBindCollision(e.getCause) + case e: MultiException => e.getThrowables.exists(isBindCollision) case e: Exception => isBindCollision(e.getCause) case _ => false } diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index 8a015c1d26a96..0c088da46aa5e 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -66,23 +66,19 @@ class ExternalAppendOnlyMap[K, V, C]( mergeCombiners: (C, C) => C, serializer: Serializer = SparkEnv.get.serializer, blockManager: BlockManager = SparkEnv.get.blockManager) - extends Iterable[(K, C)] with Serializable with Logging { + extends Iterable[(K, C)] + with Serializable + with Logging + with Spillable[SizeTracker] { private var currentMap = new SizeTrackingAppendOnlyMap[K, C] private val spilledMaps = new ArrayBuffer[DiskMapIterator] private val sparkConf = SparkEnv.get.conf private val diskBlockManager = blockManager.diskBlockManager - private val shuffleMemoryManager = SparkEnv.get.shuffleMemoryManager // Number of pairs inserted since last spill; note that we count them even if a value is merged // with a previous key in case we're doing something like groupBy where the result grows - private var elementsRead = 0L - - // Number of in-memory pairs inserted before tracking the map's shuffle memory usage - private val trackMemoryThreshold = 1000 - - // How much of the shared memory pool this collection has claimed - private var myMemoryThreshold = 0L + protected[this] var elementsRead = 0L /** * Size of object batches when reading/writing from serializers. @@ -95,11 +91,7 @@ class ExternalAppendOnlyMap[K, V, C]( */ private val serializerBatchSize = sparkConf.getLong("spark.shuffle.spill.batchSize", 10000) - // How many times we have spilled so far - private var spillCount = 0 - // Number of bytes spilled in total - private var _memoryBytesSpilled = 0L private var _diskBytesSpilled = 0L private val fileBufferSize = sparkConf.getInt("spark.shuffle.file.buffer.kb", 32) * 1024 @@ -136,19 +128,8 @@ class ExternalAppendOnlyMap[K, V, C]( while (entries.hasNext) { curEntry = entries.next() - if (elementsRead > trackMemoryThreshold && elementsRead % 32 == 0 && - currentMap.estimateSize() >= myMemoryThreshold) - { - // Claim up to double our current memory from the shuffle memory pool - val currentMemory = currentMap.estimateSize() - val amountToRequest = 2 * currentMemory - myMemoryThreshold - val granted = shuffleMemoryManager.tryToAcquire(amountToRequest) - myMemoryThreshold += granted - if (myMemoryThreshold <= currentMemory) { - // We were granted too little memory to grow further (either tryToAcquire returned 0, - // or we already had more memory than myMemoryThreshold); spill the current collection - spill(currentMemory) // Will also release memory back to ShuffleMemoryManager - } + if (maybeSpill(currentMap, currentMap.estimateSize())) { + currentMap = new SizeTrackingAppendOnlyMap[K, C] } currentMap.changeValue(curEntry._1, update) elementsRead += 1 @@ -171,11 +152,7 @@ class ExternalAppendOnlyMap[K, V, C]( /** * Sort the existing contents of the in-memory map and spill them to a temporary file on disk. */ - private def spill(mapSize: Long): Unit = { - spillCount += 1 - val threadId = Thread.currentThread().getId - logInfo("Thread %d spilling in-memory map of %d MB to disk (%d time%s so far)" - .format(threadId, mapSize / (1024 * 1024), spillCount, if (spillCount > 1) "s" else "")) + override protected[this] def spill(collection: SizeTracker): Unit = { val (blockId, file) = diskBlockManager.createTempBlock() curWriteMetrics = new ShuffleWriteMetrics() var writer = blockManager.getDiskWriter(blockId, file, serializer, fileBufferSize, @@ -231,18 +208,11 @@ class ExternalAppendOnlyMap[K, V, C]( } } - currentMap = new SizeTrackingAppendOnlyMap[K, C] spilledMaps.append(new DiskMapIterator(file, blockId, batchSizes)) - // Release our memory back to the shuffle pool so that other threads can grab it - shuffleMemoryManager.release(myMemoryThreshold) - myMemoryThreshold = 0L - elementsRead = 0 - _memoryBytesSpilled += mapSize } - def memoryBytesSpilled: Long = _memoryBytesSpilled def diskBytesSpilled: Long = _diskBytesSpilled /** diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala index 782b979e2e93d..644fa36818647 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala @@ -79,14 +79,14 @@ private[spark] class ExternalSorter[K, V, C]( aggregator: Option[Aggregator[K, V, C]] = None, partitioner: Option[Partitioner] = None, ordering: Option[Ordering[K]] = None, - serializer: Option[Serializer] = None) extends Logging { + serializer: Option[Serializer] = None) + extends Logging with Spillable[SizeTrackingPairCollection[(Int, K), C]] { private val numPartitions = partitioner.map(_.numPartitions).getOrElse(1) private val shouldPartition = numPartitions > 1 private val blockManager = SparkEnv.get.blockManager private val diskBlockManager = blockManager.diskBlockManager - private val shuffleMemoryManager = SparkEnv.get.shuffleMemoryManager private val ser = Serializer.getSerializer(serializer) private val serInstance = ser.newInstance() @@ -115,22 +115,14 @@ private[spark] class ExternalSorter[K, V, C]( // Number of pairs read from input since last spill; note that we count them even if a value is // merged with a previous key in case we're doing something like groupBy where the result grows - private var elementsRead = 0L - - // What threshold of elementsRead we start estimating map size at. - private val trackMemoryThreshold = 1000 + protected[this] var elementsRead = 0L // Total spilling statistics - private var spillCount = 0 - private var _memoryBytesSpilled = 0L private var _diskBytesSpilled = 0L // Write metrics for current spill private var curWriteMetrics: ShuffleWriteMetrics = _ - // How much of the shared memory pool this collection has claimed - private var myMemoryThreshold = 0L - // If there are fewer than spark.shuffle.sort.bypassMergeThreshold partitions and we don't need // local aggregation and sorting, write numPartitions files directly and just concatenate them // at the end. This avoids doing serialization and deserialization twice to merge together the @@ -152,7 +144,7 @@ private[spark] class ExternalSorter[K, V, C]( override def compare(a: K, b: K): Int = { val h1 = if (a == null) 0 else a.hashCode() val h2 = if (b == null) 0 else b.hashCode() - h1 - h2 + if (h1 < h2) -1 else if (h1 == h2) 0 else 1 } }) @@ -209,7 +201,7 @@ private[spark] class ExternalSorter[K, V, C]( elementsRead += 1 kv = records.next() map.changeValue((getPartition(kv._1), kv._1), update) - maybeSpill(usingMap = true) + maybeSpillCollection(usingMap = true) } } else { // Stick values into our buffer @@ -217,7 +209,7 @@ private[spark] class ExternalSorter[K, V, C]( elementsRead += 1 val kv = records.next() buffer.insert((getPartition(kv._1), kv._1), kv._2.asInstanceOf[C]) - maybeSpill(usingMap = false) + maybeSpillCollection(usingMap = false) } } } @@ -227,61 +219,31 @@ private[spark] class ExternalSorter[K, V, C]( * * @param usingMap whether we're using a map or buffer as our current in-memory collection */ - private def maybeSpill(usingMap: Boolean): Unit = { + private def maybeSpillCollection(usingMap: Boolean): Unit = { if (!spillingEnabled) { return } - val collection: SizeTrackingPairCollection[(Int, K), C] = if (usingMap) map else buffer - - // TODO: factor this out of both here and ExternalAppendOnlyMap - if (elementsRead > trackMemoryThreshold && elementsRead % 32 == 0 && - collection.estimateSize() >= myMemoryThreshold) - { - // Claim up to double our current memory from the shuffle memory pool - val currentMemory = collection.estimateSize() - val amountToRequest = 2 * currentMemory - myMemoryThreshold - val granted = shuffleMemoryManager.tryToAcquire(amountToRequest) - myMemoryThreshold += granted - if (myMemoryThreshold <= currentMemory) { - // We were granted too little memory to grow further (either tryToAcquire returned 0, - // or we already had more memory than myMemoryThreshold); spill the current collection - spill(currentMemory, usingMap) // Will also release memory back to ShuffleMemoryManager + if (usingMap) { + if (maybeSpill(map, map.estimateSize())) { + map = new SizeTrackingAppendOnlyMap[(Int, K), C] + } + } else { + if (maybeSpill(buffer, buffer.estimateSize())) { + buffer = new SizeTrackingPairBuffer[(Int, K), C] } } } /** * Spill the current in-memory collection to disk, adding a new file to spills, and clear it. - * - * @param usingMap whether we're using a map or buffer as our current in-memory collection */ - private def spill(memorySize: Long, usingMap: Boolean): Unit = { - val collection: SizeTrackingPairCollection[(Int, K), C] = if (usingMap) map else buffer - val memorySize = collection.estimateSize() - - spillCount += 1 - val threadId = Thread.currentThread().getId - logInfo("Thread %d spilling in-memory batch of %d MB to disk (%d spill%s so far)" - .format(threadId, memorySize / (1024 * 1024), spillCount, if (spillCount > 1) "s" else "")) - + override protected[this] def spill(collection: SizeTrackingPairCollection[(Int, K), C]): Unit = { if (bypassMergeSort) { spillToPartitionFiles(collection) } else { spillToMergeableFile(collection) } - - if (usingMap) { - map = new SizeTrackingAppendOnlyMap[(Int, K), C] - } else { - buffer = new SizeTrackingPairBuffer[(Int, K), C] - } - - // Release our memory back to the shuffle pool so that other threads can grab it - shuffleMemoryManager.release(myMemoryThreshold) - myMemoryThreshold = 0 - - _memoryBytesSpilled += memorySize } /** @@ -804,8 +766,6 @@ private[spark] class ExternalSorter[K, V, C]( } } - def memoryBytesSpilled: Long = _memoryBytesSpilled - def diskBytesSpilled: Long = _diskBytesSpilled /** diff --git a/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala b/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala new file mode 100644 index 0000000000000..d7dccd4af8c6e --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala @@ -0,0 +1,111 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.collection + +import org.apache.spark.Logging +import org.apache.spark.SparkEnv + +/** + * Spills contents of an in-memory collection to disk when the memory threshold + * has been exceeded. + */ +private[spark] trait Spillable[C] { + + this: Logging => + + /** + * Spills the current in-memory collection to disk, and releases the memory. + * + * @param collection collection to spill to disk + */ + protected def spill(collection: C): Unit + + // Number of elements read from input since last spill + protected var elementsRead: Long + + // Memory manager that can be used to acquire/release memory + private[this] val shuffleMemoryManager = SparkEnv.get.shuffleMemoryManager + + // What threshold of elementsRead we start estimating collection size at + private[this] val trackMemoryThreshold = 1000 + + // How much of the shared memory pool this collection has claimed + private[this] var myMemoryThreshold = 0L + + // Number of bytes spilled in total + private[this] var _memoryBytesSpilled = 0L + + // Number of spills + private[this] var _spillCount = 0 + + /** + * Spills the current in-memory collection to disk if needed. Attempts to acquire more + * memory before spilling. + * + * @param collection collection to spill to disk + * @param currentMemory estimated size of the collection in bytes + * @return true if `collection` was spilled to disk; false otherwise + */ + protected def maybeSpill(collection: C, currentMemory: Long): Boolean = { + if (elementsRead > trackMemoryThreshold && elementsRead % 32 == 0 && + currentMemory >= myMemoryThreshold) { + // Claim up to double our current memory from the shuffle memory pool + val amountToRequest = 2 * currentMemory - myMemoryThreshold + val granted = shuffleMemoryManager.tryToAcquire(amountToRequest) + myMemoryThreshold += granted + if (myMemoryThreshold <= currentMemory) { + // We were granted too little memory to grow further (either tryToAcquire returned 0, + // or we already had more memory than myMemoryThreshold); spill the current collection + _spillCount += 1 + logSpillage(currentMemory) + + spill(collection) + + // Keep track of spills, and release memory + _memoryBytesSpilled += currentMemory + releaseMemoryForThisThread() + return true + } + } + false + } + + /** + * @return number of bytes spilled in total + */ + def memoryBytesSpilled: Long = _memoryBytesSpilled + + /** + * Release our memory back to the shuffle pool so that other threads can grab it. + */ + private def releaseMemoryForThisThread(): Unit = { + shuffleMemoryManager.release(myMemoryThreshold) + myMemoryThreshold = 0L + } + + /** + * Prints a standard log message detailing spillage. + * + * @param size number of bytes spilled + */ + @inline private def logSpillage(size: Long) { + val threadId = Thread.currentThread().getId + logInfo("Thread %d spilling in-memory map of %d MB to disk (%d time%s so far)" + .format(threadId, size / (1024 * 1024), _spillCount, if (_spillCount > 1) "s" else "")) + } +} diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index b8574dfb42e6b..4a078435447e5 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -776,7 +776,7 @@ public void persist() { @Test public void iterator() { JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5), 2); - TaskContext context = new TaskContext(0, 0, 0, false, new TaskMetrics()); + TaskContext context = new TaskContext(0, 0, 0L, false, new TaskMetrics()); Assert.assertEquals(1, rdd.iterator(rdd.partitions().get(0), context).next().intValue()); } @@ -1307,4 +1307,30 @@ public void collectUnderlyingScalaRDD() { SomeCustomClass[] collected = (SomeCustomClass[]) rdd.rdd().retag(SomeCustomClass.class).collect(); Assert.assertEquals(data.size(), collected.length); } + + /** + * Test for SPARK-3647. This test needs to use the maven-built assembly to trigger the issue, + * since that's the only artifact where Guava classes have been relocated. + */ + @Test + public void testGuavaOptional() { + // Stop the context created in setUp() and start a local-cluster one, to force usage of the + // assembly. + sc.stop(); + JavaSparkContext localCluster = new JavaSparkContext("local-cluster[1,1,512]", "JavaAPISuite"); + try { + JavaRDD rdd1 = localCluster.parallelize(Arrays.asList(1, 2, null), 3); + JavaRDD> rdd2 = rdd1.map( + new Function>() { + @Override + public Optional call(Integer i) { + return Optional.fromNullable(i); + } + }); + rdd2.collect(); + } finally { + localCluster.stop(); + } + } + } diff --git a/core/src/test/java/org/apache/spark/util/JavaTaskCompletionListenerImpl.java b/core/src/test/java/org/apache/spark/util/JavaTaskCompletionListenerImpl.java index af34cdb03e4d1..0944bf8cd5c71 100644 --- a/core/src/test/java/org/apache/spark/util/JavaTaskCompletionListenerImpl.java +++ b/core/src/test/java/org/apache/spark/util/JavaTaskCompletionListenerImpl.java @@ -30,10 +30,9 @@ public class JavaTaskCompletionListenerImpl implements TaskCompletionListener { public void onTaskCompletion(TaskContext context) { context.isCompleted(); context.isInterrupted(); - context.stageId(); - context.partitionId(); - context.runningLocally(); - context.taskMetrics(); + context.getStageId(); + context.getPartitionId(); + context.isRunningLocally(); context.addTaskCompletionListener(this); } } diff --git a/core/src/test/resources/log4j.properties b/core/src/test/resources/log4j.properties index 26b73a1b39744..9dd05f17f012b 100644 --- a/core/src/test/resources/log4j.properties +++ b/core/src/test/resources/log4j.properties @@ -21,7 +21,7 @@ log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false log4j.appender.file.file=target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout -log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose log4j.logger.org.eclipse.jetty=WARN diff --git a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala index 90dcadcffd091..d735010d7c9d5 100644 --- a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala @@ -94,7 +94,7 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar } whenExecuting(blockManager) { - val context = new TaskContext(0, 0, 0, runningLocally = true) + val context = new TaskContext(0, 0, 0, true) val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY) assert(value.toList === List(1, 2, 3, 4)) } diff --git a/core/src/test/scala/org/apache/spark/FutureActionSuite.scala b/core/src/test/scala/org/apache/spark/FutureActionSuite.scala new file mode 100644 index 0000000000000..db9c25fc457a4 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/FutureActionSuite.scala @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark + +import scala.concurrent.Await +import scala.concurrent.duration.Duration + +import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} + +import org.apache.spark.SparkContext._ + +class FutureActionSuite extends FunSuite with BeforeAndAfter with Matchers with LocalSparkContext { + + before { + sc = new SparkContext("local", "FutureActionSuite") + } + + test("simple async action") { + val rdd = sc.parallelize(1 to 10, 2) + val job = rdd.countAsync() + val res = Await.result(job, Duration.Inf) + res should be (10) + job.jobIds.size should be (1) + } + + test("complex async action") { + val rdd = sc.parallelize(1 to 15, 3) + val job = rdd.takeAsync(10) + val res = Await.result(job, Duration.Inf) + res should be (1 to 10) + job.jobIds.size should be (2) + } + +} diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index 5369169811f81..1fef79ad1001f 100644 --- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala @@ -23,32 +23,13 @@ import akka.actor._ import akka.testkit.TestActorRef import org.scalatest.FunSuite -import org.apache.spark.scheduler.MapStatus +import org.apache.spark.scheduler.{CompressedMapStatus, MapStatus} import org.apache.spark.shuffle.FetchFailedException import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.AkkaUtils class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { private val conf = new SparkConf - test("compressSize") { - assert(MapOutputTracker.compressSize(0L) === 0) - assert(MapOutputTracker.compressSize(1L) === 1) - assert(MapOutputTracker.compressSize(2L) === 8) - assert(MapOutputTracker.compressSize(10L) === 25) - assert((MapOutputTracker.compressSize(1000000L) & 0xFF) === 145) - assert((MapOutputTracker.compressSize(1000000000L) & 0xFF) === 218) - // This last size is bigger than we can encode in a byte, so check that we just return 255 - assert((MapOutputTracker.compressSize(1000000000000000000L) & 0xFF) === 255) - } - - test("decompressSize") { - assert(MapOutputTracker.decompressSize(0) === 0) - for (size <- Seq(2L, 10L, 100L, 50000L, 1000000L, 1000000000L)) { - val size2 = MapOutputTracker.decompressSize(MapOutputTracker.compressSize(size)) - assert(size2 >= 0.99 * size && size2 <= 1.11 * size, - "size " + size + " decompressed to " + size2 + ", which is out of range") - } - } test("master start and stop") { val actorSystem = ActorSystem("test") @@ -65,14 +46,12 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker, conf))) tracker.registerShuffle(10, 2) assert(tracker.containsShuffle(10)) - val compressedSize1000 = MapOutputTracker.compressSize(1000L) - val compressedSize10000 = MapOutputTracker.compressSize(10000L) - val size1000 = MapOutputTracker.decompressSize(compressedSize1000) - val size10000 = MapOutputTracker.decompressSize(compressedSize10000) - tracker.registerMapOutput(10, 0, new MapStatus(BlockManagerId("a", "hostA", 1000), - Array(compressedSize1000, compressedSize10000))) - tracker.registerMapOutput(10, 1, new MapStatus(BlockManagerId("b", "hostB", 1000), - Array(compressedSize10000, compressedSize1000))) + val size1000 = MapStatus.decompressSize(MapStatus.compressSize(1000L)) + val size10000 = MapStatus.decompressSize(MapStatus.compressSize(10000L)) + tracker.registerMapOutput(10, 0, MapStatus(BlockManagerId("a", "hostA", 1000), + Array(1000L, 10000L))) + tracker.registerMapOutput(10, 1, MapStatus(BlockManagerId("b", "hostB", 1000), + Array(10000L, 1000L))) val statuses = tracker.getServerStatuses(10, 0) assert(statuses.toSeq === Seq((BlockManagerId("a", "hostA", 1000), size1000), (BlockManagerId("b", "hostB", 1000), size10000))) @@ -84,11 +63,11 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { val tracker = new MapOutputTrackerMaster(conf) tracker.trackerActor = actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker, conf))) tracker.registerShuffle(10, 2) - val compressedSize1000 = MapOutputTracker.compressSize(1000L) - val compressedSize10000 = MapOutputTracker.compressSize(10000L) - tracker.registerMapOutput(10, 0, new MapStatus(BlockManagerId("a", "hostA", 1000), + val compressedSize1000 = MapStatus.compressSize(1000L) + val compressedSize10000 = MapStatus.compressSize(10000L) + tracker.registerMapOutput(10, 0, MapStatus(BlockManagerId("a", "hostA", 1000), Array(compressedSize1000, compressedSize10000))) - tracker.registerMapOutput(10, 1, new MapStatus(BlockManagerId("b", "hostB", 1000), + tracker.registerMapOutput(10, 1, MapStatus(BlockManagerId("b", "hostB", 1000), Array(compressedSize10000, compressedSize1000))) assert(tracker.containsShuffle(10)) assert(tracker.getServerStatuses(10, 0).nonEmpty) @@ -103,11 +82,11 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { tracker.trackerActor = actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker, conf))) tracker.registerShuffle(10, 2) - val compressedSize1000 = MapOutputTracker.compressSize(1000L) - val compressedSize10000 = MapOutputTracker.compressSize(10000L) - tracker.registerMapOutput(10, 0, new MapStatus(BlockManagerId("a", "hostA", 1000), + val compressedSize1000 = MapStatus.compressSize(1000L) + val compressedSize10000 = MapStatus.compressSize(10000L) + tracker.registerMapOutput(10, 0, MapStatus(BlockManagerId("a", "hostA", 1000), Array(compressedSize1000, compressedSize1000, compressedSize1000))) - tracker.registerMapOutput(10, 1, new MapStatus(BlockManagerId("b", "hostB", 1000), + tracker.registerMapOutput(10, 1, MapStatus(BlockManagerId("b", "hostB", 1000), Array(compressedSize10000, compressedSize1000, compressedSize1000))) // As if we had two simultaneous fetch failures @@ -142,10 +121,9 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { slaveTracker.updateEpoch(masterTracker.getEpoch) intercept[FetchFailedException] { slaveTracker.getServerStatuses(10, 0) } - val compressedSize1000 = MapOutputTracker.compressSize(1000L) - val size1000 = MapOutputTracker.decompressSize(compressedSize1000) - masterTracker.registerMapOutput(10, 0, new MapStatus( - BlockManagerId("a", "hostA", 1000), Array(compressedSize1000))) + val size1000 = MapStatus.decompressSize(MapStatus.compressSize(1000L)) + masterTracker.registerMapOutput(10, 0, MapStatus( + BlockManagerId("a", "hostA", 1000), Array(1000L))) masterTracker.incrementEpoch() slaveTracker.updateEpoch(masterTracker.getEpoch) assert(slaveTracker.getServerStatuses(10, 0).toSeq === @@ -173,8 +151,8 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { // Frame size should be ~123B, and no exception should be thrown masterTracker.registerShuffle(10, 1) - masterTracker.registerMapOutput(10, 0, new MapStatus( - BlockManagerId("88", "mph", 1000), Array.fill[Byte](10)(0))) + masterTracker.registerMapOutput(10, 0, MapStatus( + BlockManagerId("88", "mph", 1000), Array.fill[Long](10)(0))) masterActor.receive(GetMapOutputStatuses(10)) } @@ -194,8 +172,8 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { // being sent. masterTracker.registerShuffle(20, 100) (0 until 100).foreach { i => - masterTracker.registerMapOutput(20, i, new MapStatus( - BlockManagerId("999", "mps", 1000), Array.fill[Byte](4000000)(0))) + masterTracker.registerMapOutput(20, i, new CompressedMapStatus( + BlockManagerId("999", "mps", 1000), Array.fill[Long](4000000)(0))) } intercept[SparkException] { masterActor.receive(GetMapOutputStatuses(20)) } } diff --git a/core/src/test/scala/org/apache/spark/PartitioningSuite.scala b/core/src/test/scala/org/apache/spark/PartitioningSuite.scala index fc0cee3e8749d..646ede30ae6ff 100644 --- a/core/src/test/scala/org/apache/spark/PartitioningSuite.scala +++ b/core/src/test/scala/org/apache/spark/PartitioningSuite.scala @@ -193,11 +193,13 @@ class PartitioningSuite extends FunSuite with SharedSparkContext with PrivateMet assert(grouped2.join(grouped4).partitioner === grouped4.partitioner) assert(grouped2.leftOuterJoin(grouped4).partitioner === grouped4.partitioner) assert(grouped2.rightOuterJoin(grouped4).partitioner === grouped4.partitioner) + assert(grouped2.fullOuterJoin(grouped4).partitioner === grouped4.partitioner) assert(grouped2.cogroup(grouped4).partitioner === grouped4.partitioner) assert(grouped2.join(reduced2).partitioner === grouped2.partitioner) assert(grouped2.leftOuterJoin(reduced2).partitioner === grouped2.partitioner) assert(grouped2.rightOuterJoin(reduced2).partitioner === grouped2.partitioner) + assert(grouped2.fullOuterJoin(reduced2).partitioner === grouped2.partitioner) assert(grouped2.cogroup(reduced2).partitioner === grouped2.partitioner) assert(grouped2.map(_ => 1).partitioner === None) @@ -218,6 +220,7 @@ class PartitioningSuite extends FunSuite with SharedSparkContext with PrivateMet assert(intercept[SparkException]{ arrPairs.join(arrPairs) }.getMessage.contains("array")) assert(intercept[SparkException]{ arrPairs.leftOuterJoin(arrPairs) }.getMessage.contains("array")) assert(intercept[SparkException]{ arrPairs.rightOuterJoin(arrPairs) }.getMessage.contains("array")) + assert(intercept[SparkException]{ arrPairs.fullOuterJoin(arrPairs) }.getMessage.contains("array")) assert(intercept[SparkException]{ arrPairs.groupByKey() }.getMessage.contains("array")) assert(intercept[SparkException]{ arrPairs.countByKey() }.getMessage.contains("array")) assert(intercept[SparkException]{ arrPairs.countByKeyApprox(1) }.getMessage.contains("array")) diff --git a/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala b/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala index 978a6ded80829..acaf321de52fb 100644 --- a/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala +++ b/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala @@ -132,7 +132,7 @@ class BroadcastSuite extends FunSuite with LocalSparkContext { val statuses = bmm.getBlockStatus(blockId, askSlaves = true) assert(statuses.size === 1) statuses.head match { case (bm, status) => - assert(bm.executorId === "", "Block should only be on the driver") + assert(bm.isDriver, "Block should only be on the driver") assert(status.storageLevel === StorageLevel.MEMORY_AND_DISK) assert(status.memSize > 0, "Block should be in memory store on the driver") assert(status.diskSize === 0, "Block should not be in disk store on the driver") diff --git a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala index 2a58c6a40d8e4..3f1cd0752e766 100644 --- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala @@ -115,11 +115,13 @@ class JsonProtocolSuite extends FunSuite { workerInfo.lastHeartbeat = JsonConstants.currTimeInMillis workerInfo } + def createExecutorRunner(): ExecutorRunner = { new ExecutorRunner("appId", 123, createAppDesc(), 4, 1234, null, "workerId", "host", new File("sparkHome"), new File("workDir"), "akka://worker", new SparkConf, ExecutorState.RUNNING) } + def createDriverRunner(): DriverRunner = { new DriverRunner(new SparkConf(), "driverId", new File("workDir"), new File("sparkHome"), createDriverDesc(), null, "akka://worker") 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 22b369a829418..4cba90e8f2afe 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.deploy -import java.io.{File, OutputStream, PrintStream} +import java.io._ import scala.collection.mutable.ArrayBuffer @@ -26,6 +26,7 @@ import org.apache.spark.deploy.SparkSubmit._ import org.apache.spark.util.Utils import org.scalatest.FunSuite import org.scalatest.Matchers +import com.google.common.io.Files class SparkSubmitSuite extends FunSuite with Matchers { def beforeAll() { @@ -154,6 +155,7 @@ class SparkSubmitSuite extends FunSuite with Matchers { sysProps("spark.app.name") should be ("beauty") sysProps("spark.shuffle.spill") should be ("false") sysProps("SPARK_SUBMIT") should be ("true") + sysProps.keys should not contain ("spark.jars") } test("handles YARN client mode") { @@ -305,6 +307,21 @@ class SparkSubmitSuite extends FunSuite with Matchers { runSparkSubmit(args) } + test("SPARK_CONF_DIR overrides spark-defaults.conf") { + forConfDir(Map("spark.executor.memory" -> "2.3g")) { path => + val unusedJar = TestUtils.createJarWithClasses(Seq.empty) + val args = Seq( + "--class", SimpleApplicationTest.getClass.getName.stripSuffix("$"), + "--name", "testApp", + "--master", "local", + unusedJar.toString) + val appArgs = new SparkSubmitArguments(args, Map("SPARK_CONF_DIR" -> path)) + assert(appArgs.propertiesFile != null) + assert(appArgs.propertiesFile.startsWith(path)) + appArgs.executorMemory should be ("2.3g") + } + } + // NOTE: This is an expensive operation in terms of time (10 seconds+). Use sparingly. def runSparkSubmit(args: Seq[String]): String = { val sparkHome = sys.props.getOrElse("spark.test.home", fail("spark.test.home is not set!")) @@ -313,6 +330,22 @@ class SparkSubmitSuite extends FunSuite with Matchers { new File(sparkHome), Map("SPARK_TESTING" -> "1", "SPARK_HOME" -> sparkHome)) } + + def forConfDir(defaults: Map[String, String]) (f: String => Unit) = { + val tmpDir = Files.createTempDir() + + val defaultsConf = new File(tmpDir.getAbsolutePath, "spark-defaults.conf") + val writer = new OutputStreamWriter(new FileOutputStream(defaultsConf)) + for ((key, value) <- defaults) writer.write(s"$key $value\n") + + writer.close() + + try { + f(tmpDir.getAbsolutePath) + } finally { + Utils.deleteRecursively(tmpDir) + } + } } object JarCreationTest { diff --git a/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala b/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala index 96a5a1231813e..3925f0ccbdbf0 100644 --- a/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala +++ b/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala @@ -17,42 +17,171 @@ package org.apache.spark.metrics -import org.scalatest.{BeforeAndAfter, FunSuite} +import org.scalatest.{BeforeAndAfter, FunSuite, PrivateMethodTester} + import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.deploy.master.MasterSource +import org.apache.spark.metrics.source.Source + +import com.codahale.metrics.MetricRegistry + +import scala.collection.mutable.ArrayBuffer -class MetricsSystemSuite extends FunSuite with BeforeAndAfter { +class MetricsSystemSuite extends FunSuite with BeforeAndAfter with PrivateMethodTester{ var filePath: String = _ var conf: SparkConf = null var securityMgr: SecurityManager = null before { - filePath = getClass.getClassLoader.getResource("test_metrics_system.properties").getFile() + filePath = getClass.getClassLoader.getResource("test_metrics_system.properties").getFile conf = new SparkConf(false).set("spark.metrics.conf", filePath) securityMgr = new SecurityManager(conf) } test("MetricsSystem with default config") { val metricsSystem = MetricsSystem.createMetricsSystem("default", conf, securityMgr) - val sources = metricsSystem.sources - val sinks = metricsSystem.sinks + metricsSystem.start() + val sources = PrivateMethod[ArrayBuffer[Source]]('sources) + val sinks = PrivateMethod[ArrayBuffer[Source]]('sinks) - assert(sources.length === 0) - assert(sinks.length === 0) - assert(!metricsSystem.getServletHandlers.isEmpty) + assert(metricsSystem.invokePrivate(sources()).length === 0) + assert(metricsSystem.invokePrivate(sinks()).length === 0) + assert(metricsSystem.getServletHandlers.nonEmpty) } test("MetricsSystem with sources add") { val metricsSystem = MetricsSystem.createMetricsSystem("test", conf, securityMgr) - val sources = metricsSystem.sources - val sinks = metricsSystem.sinks + metricsSystem.start() + val sources = PrivateMethod[ArrayBuffer[Source]]('sources) + val sinks = PrivateMethod[ArrayBuffer[Source]]('sinks) - assert(sources.length === 0) - assert(sinks.length === 1) - assert(!metricsSystem.getServletHandlers.isEmpty) + assert(metricsSystem.invokePrivate(sources()).length === 0) + assert(metricsSystem.invokePrivate(sinks()).length === 1) + assert(metricsSystem.getServletHandlers.nonEmpty) val source = new MasterSource(null) metricsSystem.registerSource(source) - assert(sources.length === 1) + assert(metricsSystem.invokePrivate(sources()).length === 1) + } + + test("MetricsSystem with Driver instance") { + val source = new Source { + override val sourceName = "dummySource" + override val metricRegistry = new MetricRegistry() + } + + val appId = "testId" + val executorId = "driver" + conf.set("spark.app.id", appId) + conf.set("spark.executor.id", executorId) + + val instanceName = "driver" + val driverMetricsSystem = MetricsSystem.createMetricsSystem(instanceName, conf, securityMgr) + + val metricName = driverMetricsSystem.buildRegistryName(source) + assert(metricName === s"$appId.$executorId.${source.sourceName}") + } + + test("MetricsSystem with Driver instance and spark.app.id is not set") { + val source = new Source { + override val sourceName = "dummySource" + override val metricRegistry = new MetricRegistry() + } + + val executorId = "driver" + conf.set("spark.executor.id", executorId) + + val instanceName = "driver" + val driverMetricsSystem = MetricsSystem.createMetricsSystem(instanceName, conf, securityMgr) + + val metricName = driverMetricsSystem.buildRegistryName(source) + assert(metricName === source.sourceName) + } + + test("MetricsSystem with Driver instance and spark.executor.id is not set") { + val source = new Source { + override val sourceName = "dummySource" + override val metricRegistry = new MetricRegistry() + } + + val appId = "testId" + conf.set("spark.app.id", appId) + + val instanceName = "driver" + val driverMetricsSystem = MetricsSystem.createMetricsSystem(instanceName, conf, securityMgr) + + val metricName = driverMetricsSystem.buildRegistryName(source) + assert(metricName === source.sourceName) + } + + test("MetricsSystem with Executor instance") { + val source = new Source { + override val sourceName = "dummySource" + override val metricRegistry = new MetricRegistry() + } + + val appId = "testId" + val executorId = "executor.1" + conf.set("spark.app.id", appId) + conf.set("spark.executor.id", executorId) + + val instanceName = "executor" + val driverMetricsSystem = MetricsSystem.createMetricsSystem(instanceName, conf, securityMgr) + + val metricName = driverMetricsSystem.buildRegistryName(source) + assert(metricName === s"$appId.$executorId.${source.sourceName}") + } + + test("MetricsSystem with Executor instance and spark.app.id is not set") { + val source = new Source { + override val sourceName = "dummySource" + override val metricRegistry = new MetricRegistry() + } + + val executorId = "executor.1" + conf.set("spark.executor.id", executorId) + + val instanceName = "executor" + val driverMetricsSystem = MetricsSystem.createMetricsSystem(instanceName, conf, securityMgr) + + val metricName = driverMetricsSystem.buildRegistryName(source) + assert(metricName === source.sourceName) + } + + test("MetricsSystem with Executor instance and spark.executor.id is not set") { + val source = new Source { + override val sourceName = "dummySource" + override val metricRegistry = new MetricRegistry() + } + + val appId = "testId" + conf.set("spark.app.id", appId) + + val instanceName = "executor" + val driverMetricsSystem = MetricsSystem.createMetricsSystem(instanceName, conf, securityMgr) + + val metricName = driverMetricsSystem.buildRegistryName(source) + assert(metricName === source.sourceName) + } + + test("MetricsSystem with instance which is neither Driver nor Executor") { + val source = new Source { + override val sourceName = "dummySource" + override val metricRegistry = new MetricRegistry() + } + + val appId = "testId" + val executorId = "dummyExecutorId" + conf.set("spark.app.id", appId) + conf.set("spark.executor.id", executorId) + + val instanceName = "testInstance" + val driverMetricsSystem = MetricsSystem.createMetricsSystem(instanceName, conf, securityMgr) + + val metricName = driverMetricsSystem.buildRegistryName(source) + + // Even if spark.app.id and spark.executor.id are set, they are not used for the metric name. + assert(metricName != s"$appId.$executorId.${source.sourceName}") + assert(metricName === source.sourceName) } } diff --git a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala index 63d3ddb4af98a..75b01191901b8 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala @@ -17,17 +17,21 @@ package org.apache.spark.rdd -import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.HashSet +import org.apache.hadoop.fs.FileSystem +import org.apache.hadoop.mapred._ +import org.apache.hadoop.util.Progressable + +import scala.collection.mutable.{ArrayBuffer, HashSet} import scala.util.Random -import org.scalatest.FunSuite import com.google.common.io.Files -import org.apache.hadoop.mapreduce._ -import org.apache.hadoop.conf.{Configuration, Configurable} - -import org.apache.spark.SparkContext._ +import org.apache.hadoop.conf.{Configurable, Configuration} +import org.apache.hadoop.mapreduce.{JobContext => NewJobContext, OutputCommitter => NewOutputCommitter, +OutputFormat => NewOutputFormat, RecordWriter => NewRecordWriter, +TaskAttemptContext => NewTaskAttempContext} import org.apache.spark.{Partitioner, SharedSparkContext} +import org.apache.spark.SparkContext._ +import org.scalatest.FunSuite class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { test("aggregateByKey") { @@ -294,6 +298,21 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { )) } + test("fullOuterJoin") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) + val joined = rdd1.fullOuterJoin(rdd2).collect() + assert(joined.size === 6) + assert(joined.toSet === Set( + (1, (Some(1), Some('x'))), + (1, (Some(2), Some('x'))), + (2, (Some(1), Some('y'))), + (2, (Some(1), Some('z'))), + (3, (Some(1), None)), + (4, (None, Some('w'))) + )) + } + test("join with no matches") { val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) val rdd2 = sc.parallelize(Array((4, 'x'), (5, 'y'), (5, 'z'), (6, 'w'))) @@ -467,7 +486,7 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { val pairs = sc.parallelize(Array((new Integer(1), new Integer(1)))) // No error, non-configurable formats still work - pairs.saveAsNewAPIHadoopFile[FakeFormat]("ignored") + pairs.saveAsNewAPIHadoopFile[NewFakeFormat]("ignored") /* Check that configurable formats get configured: @@ -478,6 +497,17 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { pairs.saveAsNewAPIHadoopFile[ConfigTestFormat]("ignored") } + test("saveAsHadoopFile should respect configured output committers") { + val pairs = sc.parallelize(Array((new Integer(1), new Integer(1)))) + val conf = new JobConf() + conf.setOutputCommitter(classOf[FakeOutputCommitter]) + + FakeOutputCommitter.ran = false + pairs.saveAsHadoopFile("ignored", pairs.keyClass, pairs.valueClass, classOf[FakeOutputFormat], conf) + + assert(FakeOutputCommitter.ran, "OutputCommitter was never called") + } + test("lookup") { val pairs = sc.parallelize(Array((1,2), (3,4), (5,6), (5,7))) @@ -621,40 +651,86 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { and the test will therefore throw InstantiationException when saveAsNewAPIHadoopFile tries to instantiate them with Class.newInstance. */ + +/* + * Original Hadoop API + */ class FakeWriter extends RecordWriter[Integer, Integer] { + override def write(key: Integer, value: Integer): Unit = () + + override def close(reporter: Reporter): Unit = () +} + +class FakeOutputCommitter() extends OutputCommitter() { + override def setupJob(jobContext: JobContext): Unit = () + + override def needsTaskCommit(taskContext: TaskAttemptContext): Boolean = true + + override def setupTask(taskContext: TaskAttemptContext): Unit = () + + override def commitTask(taskContext: TaskAttemptContext): Unit = { + FakeOutputCommitter.ran = true + () + } + + override def abortTask(taskContext: TaskAttemptContext): Unit = () +} + +/* + * Used to communicate state between the test harness and the OutputCommitter. + */ +object FakeOutputCommitter { + var ran = false +} + +class FakeOutputFormat() extends OutputFormat[Integer, Integer]() { + override def getRecordWriter( + ignored: FileSystem, + job: JobConf, name: String, + progress: Progressable): RecordWriter[Integer, Integer] = { + new FakeWriter() + } - def close(p1: TaskAttemptContext) = () + override def checkOutputSpecs(ignored: FileSystem, job: JobConf): Unit = () +} + +/* + * New-style Hadoop API + */ +class NewFakeWriter extends NewRecordWriter[Integer, Integer] { + + def close(p1: NewTaskAttempContext) = () def write(p1: Integer, p2: Integer) = () } -class FakeCommitter extends OutputCommitter { - def setupJob(p1: JobContext) = () +class NewFakeCommitter extends NewOutputCommitter { + def setupJob(p1: NewJobContext) = () - def needsTaskCommit(p1: TaskAttemptContext): Boolean = false + def needsTaskCommit(p1: NewTaskAttempContext): Boolean = false - def setupTask(p1: TaskAttemptContext) = () + def setupTask(p1: NewTaskAttempContext) = () - def commitTask(p1: TaskAttemptContext) = () + def commitTask(p1: NewTaskAttempContext) = () - def abortTask(p1: TaskAttemptContext) = () + def abortTask(p1: NewTaskAttempContext) = () } -class FakeFormat() extends OutputFormat[Integer, Integer]() { +class NewFakeFormat() extends NewOutputFormat[Integer, Integer]() { - def checkOutputSpecs(p1: JobContext) = () + def checkOutputSpecs(p1: NewJobContext) = () - def getRecordWriter(p1: TaskAttemptContext): RecordWriter[Integer, Integer] = { - new FakeWriter() + def getRecordWriter(p1: NewTaskAttempContext): NewRecordWriter[Integer, Integer] = { + new NewFakeWriter() } - def getOutputCommitter(p1: TaskAttemptContext): OutputCommitter = { - new FakeCommitter() + def getOutputCommitter(p1: NewTaskAttempContext): NewOutputCommitter = { + new NewFakeCommitter() } } -class ConfigTestFormat() extends FakeFormat() with Configurable { +class ConfigTestFormat() extends NewFakeFormat() with Configurable { var setConfCalled = false def setConf(p1: Configuration) = { @@ -664,7 +740,7 @@ class ConfigTestFormat() extends FakeFormat() with Configurable { def getConf: Configuration = null - override def getRecordWriter(p1: TaskAttemptContext): RecordWriter[Integer, Integer] = { + override def getRecordWriter(p1: NewTaskAttempContext): NewRecordWriter[Integer, Integer] = { assert(setConfCalled, "setConf was never called") super.getRecordWriter(p1) } diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index c1b501a75c8b8..465c1a8a43a79 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -193,6 +193,7 @@ class RDDSuite extends FunSuite with SharedSparkContext { assert(rdd.join(emptyKv).collect().size === 0) assert(rdd.rightOuterJoin(emptyKv).collect().size === 0) assert(rdd.leftOuterJoin(emptyKv).collect().size === 2) + assert(rdd.fullOuterJoin(emptyKv).collect().size === 2) assert(rdd.cogroup(emptyKv).collect().size === 2) assert(rdd.union(emptyKv).collect().size === 2) } diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index aa73469b6acd8..a2e4f712db55b 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -740,7 +740,7 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F } private def makeMapStatus(host: String, reduces: Int): MapStatus = - new MapStatus(makeBlockManagerId(host), Array.fill[Byte](reduces)(2)) + MapStatus(makeBlockManagerId(host), Array.fill[Long](reduces)(2)) private def makeBlockManagerId(host: String): BlockManagerId = BlockManagerId("exec-" + host, host, 12345) diff --git a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala index e5315bc93e217..3efa85431876b 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -169,7 +169,9 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter { // Verify logging directory exists val conf = getLoggingConf(logDirPath, compressionCodec) - val eventLogger = new EventLoggingListener("test", conf) + val logBaseDir = conf.get("spark.eventLog.dir") + val appId = EventLoggingListenerSuite.getUniqueApplicationId + val eventLogger = new EventLoggingListener(appId, logBaseDir, conf) eventLogger.start() val logPath = new Path(eventLogger.logDir) assert(fileSystem.exists(logPath)) @@ -209,7 +211,9 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter { // Verify that all information is correctly parsed before stop() val conf = getLoggingConf(logDirPath, compressionCodec) - val eventLogger = new EventLoggingListener("test", conf) + val logBaseDir = conf.get("spark.eventLog.dir") + val appId = EventLoggingListenerSuite.getUniqueApplicationId + val eventLogger = new EventLoggingListener(appId, logBaseDir, conf) eventLogger.start() var eventLoggingInfo = EventLoggingListener.parseLoggingInfo(eventLogger.logDir, fileSystem) assertInfoCorrect(eventLoggingInfo, loggerStopped = false) @@ -228,7 +232,9 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter { */ private def testEventLogging(compressionCodec: Option[String] = None) { val conf = getLoggingConf(logDirPath, compressionCodec) - val eventLogger = new EventLoggingListener("test", conf) + val logBaseDir = conf.get("spark.eventLog.dir") + val appId = EventLoggingListenerSuite.getUniqueApplicationId + val eventLogger = new EventLoggingListener(appId, logBaseDir, conf) val listenerBus = new LiveListenerBus val applicationStart = SparkListenerApplicationStart("Greatest App (N)ever", None, 125L, "Mickey") @@ -408,4 +414,6 @@ object EventLoggingListenerSuite { } conf } + + def getUniqueApplicationId = "test-" + System.currentTimeMillis } diff --git a/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala new file mode 100644 index 0000000000000..79e04f046e4c4 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala @@ -0,0 +1,92 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.scheduler + +import org.apache.spark.storage.BlockManagerId +import org.scalatest.FunSuite + +import org.apache.spark.SparkConf +import org.apache.spark.serializer.JavaSerializer + + +class MapStatusSuite extends FunSuite { + + test("compressSize") { + assert(MapStatus.compressSize(0L) === 0) + assert(MapStatus.compressSize(1L) === 1) + assert(MapStatus.compressSize(2L) === 8) + assert(MapStatus.compressSize(10L) === 25) + assert((MapStatus.compressSize(1000000L) & 0xFF) === 145) + assert((MapStatus.compressSize(1000000000L) & 0xFF) === 218) + // This last size is bigger than we can encode in a byte, so check that we just return 255 + assert((MapStatus.compressSize(1000000000000000000L) & 0xFF) === 255) + } + + test("decompressSize") { + assert(MapStatus.decompressSize(0) === 0) + for (size <- Seq(2L, 10L, 100L, 50000L, 1000000L, 1000000000L)) { + val size2 = MapStatus.decompressSize(MapStatus.compressSize(size)) + assert(size2 >= 0.99 * size && size2 <= 1.11 * size, + "size " + size + " decompressed to " + size2 + ", which is out of range") + } + } + + test("large tasks should use " + classOf[HighlyCompressedMapStatus].getName) { + val sizes = Array.fill[Long](2001)(150L) + val status = MapStatus(null, sizes) + assert(status.isInstanceOf[HighlyCompressedMapStatus]) + assert(status.getSizeForBlock(10) === 150L) + assert(status.getSizeForBlock(50) === 150L) + assert(status.getSizeForBlock(99) === 150L) + assert(status.getSizeForBlock(2000) === 150L) + } + + test(classOf[HighlyCompressedMapStatus].getName + ": estimated size is within 10%") { + val sizes = Array.tabulate[Long](50) { i => i.toLong } + val loc = BlockManagerId("a", "b", 10) + val status = MapStatus(loc, sizes) + val ser = new JavaSerializer(new SparkConf) + val buf = ser.newInstance().serialize(status) + val status1 = ser.newInstance().deserialize[MapStatus](buf) + assert(status1.location == loc) + for (i <- 0 until sizes.length) { + // make sure the estimated size is within 10% of the input; note that we skip the very small + // sizes because the compression is very lossy there. + val estimate = status1.getSizeForBlock(i) + if (estimate > 100) { + assert(math.abs(estimate - sizes(i)) * 10 <= sizes(i), + s"incorrect estimated size $estimate, original was ${sizes(i)}") + } + } + } + + test(classOf[HighlyCompressedMapStatus].getName + ": estimated size should be the average size") { + val sizes = Array.tabulate[Long](3000) { i => i.toLong } + val avg = sizes.sum / sizes.length + val loc = BlockManagerId("a", "b", 10) + val status = MapStatus(loc, sizes) + val ser = new JavaSerializer(new SparkConf) + val buf = ser.newInstance().serialize(status) + val status1 = ser.newInstance().deserialize[MapStatus](buf) + assert(status1.location == loc) + for (i <- 0 until 3000) { + val estimate = status1.getSizeForBlock(i) + assert(estimate === avg) + } + } +} diff --git a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala index 7ab351d1b4d24..48114feee6233 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala @@ -155,7 +155,8 @@ class ReplayListenerSuite extends FunSuite with BeforeAndAfter { * 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) { + private class EventMonster(conf: SparkConf) + extends EventLoggingListener("test", "testdir", conf) { logger.close() } diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 93e8ddacf8865..c0b07649eb6dd 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -642,6 +642,28 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { assert(manager.resourceOffer("execC", "host3", ANY) !== None) } + test("Test that locations with HDFSCacheTaskLocation are treated as PROCESS_LOCAL.") { + // Regression test for SPARK-2931 + sc = new SparkContext("local", "test") + val sched = new FakeTaskScheduler(sc, + ("execA", "host1"), ("execB", "host2"), ("execC", "host3")) + val taskSet = FakeTask.createTaskSet(3, + Seq(HostTaskLocation("host1")), + Seq(HostTaskLocation("host2")), + Seq(HDFSCacheTaskLocation("host3"))) + val clock = new FakeClock + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) + assert(manager.myLocalityLevels.sameElements(Array(PROCESS_LOCAL, NODE_LOCAL, ANY))) + sched.removeExecutor("execA") + manager.executorAdded() + assert(manager.myLocalityLevels.sameElements(Array(PROCESS_LOCAL, NODE_LOCAL, ANY))) + sched.removeExecutor("execB") + manager.executorAdded() + assert(manager.myLocalityLevels.sameElements(Array(PROCESS_LOCAL, NODE_LOCAL, ANY))) + sched.removeExecutor("execC") + manager.executorAdded() + assert(manager.myLocalityLevels.sameElements(Array(ANY))) + } def createTaskResult(id: Int): DirectTaskResult[Int] = { val valueSer = SparkEnv.get.serializer.newInstance() diff --git a/core/src/test/scala/org/apache/spark/serializer/ProactiveClosureSerializationSuite.scala b/core/src/test/scala/org/apache/spark/serializer/ProactiveClosureSerializationSuite.scala index aad6599589420..d037e2c19a64d 100644 --- a/core/src/test/scala/org/apache/spark/serializer/ProactiveClosureSerializationSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/ProactiveClosureSerializationSuite.scala @@ -50,8 +50,7 @@ class ProactiveClosureSerializationSuite extends FunSuite with SharedSparkContex "flatMap" -> xflatMap _, "filter" -> xfilter _, "mapPartitions" -> xmapPartitions _, - "mapPartitionsWithIndex" -> xmapPartitionsWithIndex _, - "mapPartitionsWithContext" -> xmapPartitionsWithContext _)) { + "mapPartitionsWithIndex" -> xmapPartitionsWithIndex _)) { val (name, xf) = transformation test(s"$name transformations throw proactive serialization exceptions") { @@ -78,8 +77,5 @@ class ProactiveClosureSerializationSuite extends FunSuite with SharedSparkContex private def xmapPartitionsWithIndex(x: RDD[String], uc: UnserializableClass): RDD[String] = x.mapPartitionsWithIndex((_, it) => it.map(y=>uc.op(y))) - - private def xmapPartitionsWithContext(x: RDD[String], uc: UnserializableClass): RDD[String] = - x.mapPartitionsWithContext((_, it) => it.map(y=>uc.op(y))) } diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala new file mode 100644 index 0000000000000..1f1d53a1ee3b0 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala @@ -0,0 +1,418 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.storage + +import scala.collection.mutable.ArrayBuffer +import scala.concurrent.duration._ +import scala.language.implicitConversions +import scala.language.postfixOps + +import akka.actor.{ActorSystem, Props} +import org.mockito.Mockito.{mock, when} +import org.scalatest.{BeforeAndAfter, FunSuite, Matchers, PrivateMethodTester} +import org.scalatest.concurrent.Eventually._ + +import org.apache.spark.{MapOutputTrackerMaster, SecurityManager, SparkConf} +import org.apache.spark.network.BlockTransferService +import org.apache.spark.network.nio.NioBlockTransferService +import org.apache.spark.scheduler.LiveListenerBus +import org.apache.spark.serializer.KryoSerializer +import org.apache.spark.shuffle.hash.HashShuffleManager +import org.apache.spark.storage.StorageLevel._ +import org.apache.spark.util.{AkkaUtils, SizeEstimator} + +/** Testsuite that tests block replication in BlockManager */ +class BlockManagerReplicationSuite extends FunSuite with Matchers with BeforeAndAfter { + + private val conf = new SparkConf(false) + var actorSystem: ActorSystem = null + var master: BlockManagerMaster = null + val securityMgr = new SecurityManager(conf) + val mapOutputTracker = new MapOutputTrackerMaster(conf) + val shuffleManager = new HashShuffleManager(conf) + + // List of block manager created during an unit test, so that all of the them can be stopped + // after the unit test. + val allStores = new ArrayBuffer[BlockManager] + + // Reuse a serializer across tests to avoid creating a new thread-local buffer on each test + conf.set("spark.kryoserializer.buffer.mb", "1") + val serializer = new KryoSerializer(conf) + + // Implicitly convert strings to BlockIds for test clarity. + implicit def StringToBlockId(value: String): BlockId = new TestBlockId(value) + + private def makeBlockManager(maxMem: Long, name: String = ""): BlockManager = { + val transfer = new NioBlockTransferService(conf, securityMgr) + val store = new BlockManager(name, actorSystem, master, serializer, maxMem, conf, + mapOutputTracker, shuffleManager, transfer) + allStores += store + store + } + + before { + val (actorSystem, boundPort) = AkkaUtils.createActorSystem( + "test", "localhost", 0, conf = conf, securityManager = securityMgr) + this.actorSystem = actorSystem + + conf.set("spark.authenticate", "false") + conf.set("spark.driver.port", boundPort.toString) + conf.set("spark.storage.unrollFraction", "0.4") + conf.set("spark.storage.unrollMemoryThreshold", "512") + + // to make a replication attempt to inactive store fail fast + conf.set("spark.core.connection.ack.wait.timeout", "1") + // to make cached peers refresh frequently + conf.set("spark.storage.cachedPeersTtl", "10") + + master = new BlockManagerMaster( + actorSystem.actorOf(Props(new BlockManagerMasterActor(true, conf, new LiveListenerBus))), + conf, true) + allStores.clear() + } + + after { + allStores.foreach { _.stop() } + allStores.clear() + actorSystem.shutdown() + actorSystem.awaitTermination() + actorSystem = null + master = null + } + + + test("get peers with addition and removal of block managers") { + val numStores = 4 + val stores = (1 to numStores - 1).map { i => makeBlockManager(1000, s"store$i") } + val storeIds = stores.map { _.blockManagerId }.toSet + assert(master.getPeers(stores(0).blockManagerId).toSet === + storeIds.filterNot { _ == stores(0).blockManagerId }) + assert(master.getPeers(stores(1).blockManagerId).toSet === + storeIds.filterNot { _ == stores(1).blockManagerId }) + assert(master.getPeers(stores(2).blockManagerId).toSet === + storeIds.filterNot { _ == stores(2).blockManagerId }) + + // Add driver store and test whether it is filtered out + val driverStore = makeBlockManager(1000, "") + assert(master.getPeers(stores(0).blockManagerId).forall(!_.isDriver)) + assert(master.getPeers(stores(1).blockManagerId).forall(!_.isDriver)) + assert(master.getPeers(stores(2).blockManagerId).forall(!_.isDriver)) + + // Add a new store and test whether get peers returns it + val newStore = makeBlockManager(1000, s"store$numStores") + assert(master.getPeers(stores(0).blockManagerId).toSet === + storeIds.filterNot { _ == stores(0).blockManagerId } + newStore.blockManagerId) + assert(master.getPeers(stores(1).blockManagerId).toSet === + storeIds.filterNot { _ == stores(1).blockManagerId } + newStore.blockManagerId) + assert(master.getPeers(stores(2).blockManagerId).toSet === + storeIds.filterNot { _ == stores(2).blockManagerId } + newStore.blockManagerId) + assert(master.getPeers(newStore.blockManagerId).toSet === storeIds) + + // Remove a store and test whether get peers returns it + val storeIdToRemove = stores(0).blockManagerId + master.removeExecutor(storeIdToRemove.executorId) + assert(!master.getPeers(stores(1).blockManagerId).contains(storeIdToRemove)) + assert(!master.getPeers(stores(2).blockManagerId).contains(storeIdToRemove)) + assert(!master.getPeers(newStore.blockManagerId).contains(storeIdToRemove)) + + // Test whether asking for peers of a unregistered block manager id returns empty list + assert(master.getPeers(stores(0).blockManagerId).isEmpty) + assert(master.getPeers(BlockManagerId("", "", 1)).isEmpty) + } + + + test("block replication - 2x replication") { + testReplication(2, + Seq(MEMORY_ONLY, MEMORY_ONLY_SER, DISK_ONLY, MEMORY_AND_DISK_2, MEMORY_AND_DISK_SER_2) + ) + } + + test("block replication - 3x replication") { + // Generate storage levels with 3x replication + val storageLevels = { + Seq(MEMORY_ONLY, MEMORY_ONLY_SER, DISK_ONLY, MEMORY_AND_DISK, MEMORY_AND_DISK_SER).map { + level => StorageLevel( + level.useDisk, level.useMemory, level.useOffHeap, level.deserialized, 3) + } + } + testReplication(3, storageLevels) + } + + test("block replication - mixed between 1x to 5x") { + // Generate storage levels with varying replication + val storageLevels = Seq( + MEMORY_ONLY, + MEMORY_ONLY_SER_2, + StorageLevel(true, false, false, false, 3), + StorageLevel(true, true, false, true, 4), + StorageLevel(true, true, false, false, 5), + StorageLevel(true, true, false, true, 4), + StorageLevel(true, false, false, false, 3), + MEMORY_ONLY_SER_2, + MEMORY_ONLY + ) + testReplication(5, storageLevels) + } + + test("block replication - 2x replication without peers") { + intercept[org.scalatest.exceptions.TestFailedException] { + testReplication(1, + Seq(StorageLevel.MEMORY_AND_DISK_2, StorageLevel(true, false, false, false, 3))) + } + } + + test("block replication - deterministic node selection") { + val blockSize = 1000 + val storeSize = 10000 + val stores = (1 to 5).map { + i => makeBlockManager(storeSize, s"store$i") + } + val storageLevel2x = StorageLevel.MEMORY_AND_DISK_2 + val storageLevel3x = StorageLevel(true, true, false, true, 3) + val storageLevel4x = StorageLevel(true, true, false, true, 4) + + def putBlockAndGetLocations(blockId: String, level: StorageLevel): Set[BlockManagerId] = { + stores.head.putSingle(blockId, new Array[Byte](blockSize), level) + val locations = master.getLocations(blockId).sortBy { _.executorId }.toSet + stores.foreach { _.removeBlock(blockId) } + master.removeBlock(blockId) + locations + } + + // Test if two attempts to 2x replication returns same set of locations + val a1Locs = putBlockAndGetLocations("a1", storageLevel2x) + assert(putBlockAndGetLocations("a1", storageLevel2x) === a1Locs, + "Inserting a 2x replicated block second time gave different locations from the first") + + // Test if two attempts to 3x replication returns same set of locations + val a2Locs3x = putBlockAndGetLocations("a2", storageLevel3x) + assert(putBlockAndGetLocations("a2", storageLevel3x) === a2Locs3x, + "Inserting a 3x replicated block second time gave different locations from the first") + + // Test if 2x replication of a2 returns a strict subset of the locations of 3x replication + val a2Locs2x = putBlockAndGetLocations("a2", storageLevel2x) + assert( + a2Locs2x.subsetOf(a2Locs3x), + "Inserting a with 2x replication gave locations that are not a subset of locations" + + s" with 3x replication [3x: ${a2Locs3x.mkString(",")}; 2x: ${a2Locs2x.mkString(",")}" + ) + + // Test if 4x replication of a2 returns a strict superset of the locations of 3x replication + val a2Locs4x = putBlockAndGetLocations("a2", storageLevel4x) + assert( + a2Locs3x.subsetOf(a2Locs4x), + "Inserting a with 4x replication gave locations that are not a superset of locations " + + s"with 3x replication [3x: ${a2Locs3x.mkString(",")}; 4x: ${a2Locs4x.mkString(",")}" + ) + + // Test if 3x replication of two different blocks gives two different sets of locations + val a3Locs3x = putBlockAndGetLocations("a3", storageLevel3x) + assert(a3Locs3x !== a2Locs3x, "Two blocks gave same locations with 3x replication") + } + + test("block replication - replication failures") { + /* + Create a system of three block managers / stores. One of them (say, failableStore) + cannot receive blocks. So attempts to use that as replication target fails. + + +-----------/fails/-----------> failableStore + | + normalStore + | + +-----------/works/-----------> anotherNormalStore + + We are first going to add a normal block manager (i.e. normalStore) and the failable block + manager (i.e. failableStore), and test whether 2x replication fails to create two + copies of a block. Then we are going to add another normal block manager + (i.e., anotherNormalStore), and test that now 2x replication works as the + new store will be used for replication. + */ + + // Add a normal block manager + val store = makeBlockManager(10000, "store") + + // Insert a block with 2x replication and return the number of copies of the block + def replicateAndGetNumCopies(blockId: String): Int = { + store.putSingle(blockId, new Array[Byte](1000), StorageLevel.MEMORY_AND_DISK_2) + val numLocations = master.getLocations(blockId).size + allStores.foreach { _.removeBlock(blockId) } + numLocations + } + + // Add a failable block manager with a mock transfer service that does not + // allow receiving of blocks. So attempts to use it as a replication target will fail. + val failableTransfer = mock(classOf[BlockTransferService]) // this wont actually work + when(failableTransfer.hostName).thenReturn("some-hostname") + when(failableTransfer.port).thenReturn(1000) + val failableStore = new BlockManager("failable-store", actorSystem, master, serializer, + 10000, conf, mapOutputTracker, shuffleManager, failableTransfer) + allStores += failableStore // so that this gets stopped after test + assert(master.getPeers(store.blockManagerId).toSet === Set(failableStore.blockManagerId)) + + // Test that 2x replication fails by creating only one copy of the block + assert(replicateAndGetNumCopies("a1") === 1) + + // Add another normal block manager and test that 2x replication works + makeBlockManager(10000, "anotherStore") + eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { + assert(replicateAndGetNumCopies("a2") === 2) + } + } + + test("block replication - addition and deletion of block managers") { + val blockSize = 1000 + val storeSize = 10000 + val initialStores = (1 to 2).map { i => makeBlockManager(storeSize, s"store$i") } + + // Insert a block with given replication factor and return the number of copies of the block\ + def replicateAndGetNumCopies(blockId: String, replicationFactor: Int): Int = { + val storageLevel = StorageLevel(true, true, false, true, replicationFactor) + initialStores.head.putSingle(blockId, new Array[Byte](blockSize), storageLevel) + val numLocations = master.getLocations(blockId).size + allStores.foreach { _.removeBlock(blockId) } + numLocations + } + + // 2x replication should work, 3x replication should only replicate 2x + assert(replicateAndGetNumCopies("a1", 2) === 2) + assert(replicateAndGetNumCopies("a2", 3) === 2) + + // Add another store, 3x replication should work now, 4x replication should only replicate 3x + val newStore1 = makeBlockManager(storeSize, s"newstore1") + eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { + assert(replicateAndGetNumCopies("a3", 3) === 3) + } + assert(replicateAndGetNumCopies("a4", 4) === 3) + + // Add another store, 4x replication should work now + val newStore2 = makeBlockManager(storeSize, s"newstore2") + eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { + assert(replicateAndGetNumCopies("a5", 4) === 4) + } + + // Remove all but the 1st store, 2x replication should fail + (initialStores.tail ++ Seq(newStore1, newStore2)).foreach { + store => + master.removeExecutor(store.blockManagerId.executorId) + store.stop() + } + assert(replicateAndGetNumCopies("a6", 2) === 1) + + // Add new stores, 3x replication should work + val newStores = (3 to 5).map { + i => makeBlockManager(storeSize, s"newstore$i") + } + eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { + assert(replicateAndGetNumCopies("a7", 3) === 3) + } + } + + /** + * Test replication of blocks with different storage levels (various combinations of + * memory, disk & serialization). For each storage level, this function tests every store + * whether the block is present and also tests the master whether its knowledge of blocks + * is correct. Then it also drops the block from memory of each store (using LRU) and + * again checks whether the master's knowledge gets updated. + */ + private def testReplication(maxReplication: Int, storageLevels: Seq[StorageLevel]) { + import org.apache.spark.storage.StorageLevel._ + + assert(maxReplication > 1, + s"Cannot test replication factor $maxReplication") + + // storage levels to test with the given replication factor + + val storeSize = 10000 + val blockSize = 1000 + + // As many stores as the replication factor + val stores = (1 to maxReplication).map { + i => makeBlockManager(storeSize, s"store$i") + } + + storageLevels.foreach { storageLevel => + // Put the block into one of the stores + val blockId = new TestBlockId( + "block-with-" + storageLevel.description.replace(" ", "-").toLowerCase) + stores(0).putSingle(blockId, new Array[Byte](blockSize), storageLevel) + + // Assert that master know two locations for the block + val blockLocations = master.getLocations(blockId).map(_.executorId).toSet + assert(blockLocations.size === storageLevel.replication, + s"master did not have ${storageLevel.replication} locations for $blockId") + + // Test state of the stores that contain the block + stores.filter { + testStore => blockLocations.contains(testStore.blockManagerId.executorId) + }.foreach { testStore => + val testStoreName = testStore.blockManagerId.executorId + assert(testStore.getLocal(blockId).isDefined, s"$blockId was not found in $testStoreName") + assert(master.getLocations(blockId).map(_.executorId).toSet.contains(testStoreName), + s"master does not have status for ${blockId.name} in $testStoreName") + + val blockStatus = master.getBlockStatus(blockId)(testStore.blockManagerId) + + // Assert that block status in the master for this store has expected storage level + assert( + blockStatus.storageLevel.useDisk === storageLevel.useDisk && + blockStatus.storageLevel.useMemory === storageLevel.useMemory && + blockStatus.storageLevel.useOffHeap === storageLevel.useOffHeap && + blockStatus.storageLevel.deserialized === storageLevel.deserialized, + s"master does not know correct storage level for ${blockId.name} in $testStoreName") + + // Assert that the block status in the master for this store has correct memory usage info + assert(!blockStatus.storageLevel.useMemory || blockStatus.memSize >= blockSize, + s"master does not know size of ${blockId.name} stored in memory of $testStoreName") + + + // If the block is supposed to be in memory, then drop the copy of the block in + // this store test whether master is updated with zero memory usage this store + if (storageLevel.useMemory) { + // Force the block to be dropped by adding a number of dummy blocks + (1 to 10).foreach { + i => + testStore.putSingle(s"dummy-block-$i", new Array[Byte](1000), MEMORY_ONLY_SER) + } + (1 to 10).foreach { + i => testStore.removeBlock(s"dummy-block-$i") + } + + val newBlockStatusOption = master.getBlockStatus(blockId).get(testStore.blockManagerId) + + // Assert that the block status in the master either does not exist (block removed + // from every store) or has zero memory usage for this store + assert( + newBlockStatusOption.isEmpty || newBlockStatusOption.get.memSize === 0, + s"after dropping, master does not know size of ${blockId.name} " + + s"stored in memory of $testStoreName" + ) + } + + // If the block is supposed to be in disk (after dropping or otherwise, then + // test whether master has correct disk usage for this store + if (storageLevel.useDisk) { + assert(master.getBlockStatus(blockId)(testStore.blockManagerId).diskSize >= blockSize, + s"after dropping, master does not know size of ${blockId.name} " + + s"stored in disk of $testStoreName" + ) + } + } + master.removeBlock(blockId) + } + } +} diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index e251660dae5de..9d96202a3e7ac 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -21,8 +21,6 @@ import java.nio.{ByteBuffer, MappedByteBuffer} import java.util.Arrays import java.util.concurrent.TimeUnit -import org.apache.spark.network.nio.NioBlockTransferService - import scala.collection.mutable.ArrayBuffer import scala.concurrent.Await import scala.concurrent.duration._ @@ -35,13 +33,13 @@ import akka.util.Timeout import org.mockito.Mockito.{mock, when} -import org.scalatest.{BeforeAndAfter, FunSuite, PrivateMethodTester} +import org.scalatest.{BeforeAndAfter, FunSuite, Matchers, PrivateMethodTester} import org.scalatest.concurrent.Eventually._ import org.scalatest.concurrent.Timeouts._ -import org.scalatest.Matchers import org.apache.spark.{MapOutputTrackerMaster, SecurityManager, SparkConf} import org.apache.spark.executor.DataReadMethod +import org.apache.spark.network.nio.NioBlockTransferService import org.apache.spark.scheduler.LiveListenerBus import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} import org.apache.spark.shuffle.hash.HashShuffleManager @@ -189,7 +187,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter store = makeBlockManager(2000, "exec1") store2 = makeBlockManager(2000, "exec2") - val peers = master.getPeers(store.blockManagerId, 1) + val peers = master.getPeers(store.blockManagerId) assert(peers.size === 1, "master did not return the other manager as a peer") assert(peers.head === store2.blockManagerId, "peer returned by master is not the other manager") @@ -448,7 +446,6 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter val list2DiskGet = store.get("list2disk") assert(list2DiskGet.isDefined, "list2memory expected to be in store") assert(list2DiskGet.get.data.size === 3) - System.out.println(list2DiskGet) // We don't know the exact size of the data on disk, but it should certainly be > 0. assert(list2DiskGet.get.inputMetrics.bytesRead > 0) assert(list2DiskGet.get.inputMetrics.readMethod === DataReadMethod.Disk) diff --git a/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala index 76bf4cfd11267..7bca1711ae226 100644 --- a/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala @@ -106,10 +106,9 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext { masterTracker.incrementEpoch() slaveTracker.updateEpoch(masterTracker.getEpoch) - val compressedSize1000 = MapOutputTracker.compressSize(1000L) - val size1000 = MapOutputTracker.decompressSize(compressedSize1000) - masterTracker.registerMapOutput(10, 0, new MapStatus( - BlockManagerId("a", "hostA", 1000), Array(compressedSize1000))) + val size1000 = MapStatus.decompressSize(MapStatus.compressSize(1000L)) + masterTracker.registerMapOutput(10, 0, + MapStatus(BlockManagerId("a", "hostA", 1000), Array(1000L))) masterTracker.incrementEpoch() slaveTracker.updateEpoch(masterTracker.getEpoch) @@ -157,10 +156,9 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext { masterTracker.incrementEpoch() slaveTracker.updateEpoch(masterTracker.getEpoch) - val compressedSize1000 = MapOutputTracker.compressSize(1000L) - val size1000 = MapOutputTracker.decompressSize(compressedSize1000) - masterTracker.registerMapOutput(10, 0, new MapStatus( - BlockManagerId("a", "hostA", 1000), Array(compressedSize1000))) + val size1000 = MapStatus.decompressSize(MapStatus.compressSize(1000L)) + masterTracker.registerMapOutput(10, 0, MapStatus( + BlockManagerId("a", "hostA", 1000), Array(1000L))) masterTracker.incrementEpoch() slaveTracker.updateEpoch(masterTracker.getEpoch) diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 2b45d8b695853..f1f88c5fd3634 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -356,7 +356,6 @@ class JsonProtocolSuite extends FunSuite { } private def assertEquals(metrics1: ShuffleReadMetrics, metrics2: ShuffleReadMetrics) { - assert(metrics1.shuffleFinishTime === metrics2.shuffleFinishTime) assert(metrics1.remoteBlocksFetched === metrics2.remoteBlocksFetched) assert(metrics1.localBlocksFetched === metrics2.localBlocksFetched) assert(metrics1.fetchWaitTime === metrics2.fetchWaitTime) @@ -568,7 +567,6 @@ class JsonProtocolSuite extends FunSuite { t.inputMetrics = Some(inputMetrics) } else { val sr = new ShuffleReadMetrics - sr.shuffleFinishTime = b + c sr.remoteBytesRead = b + d sr.localBlocksFetched = e sr.fetchWaitTime = a + d @@ -806,7 +804,6 @@ class JsonProtocolSuite extends FunSuite { | "Memory Bytes Spilled": 800, | "Disk Bytes Spilled": 0, | "Shuffle Read Metrics": { - | "Shuffle Finish Time": 900, | "Remote Blocks Fetched": 800, | "Local Blocks Fetched": 700, | "Fetch Wait Time": 900, diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala index 70d423ba8a04d..e63d9d085e385 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -189,17 +189,28 @@ class UtilsSuite extends FunSuite { assert(Utils.getIteratorSize(iterator) === 5L) } - test("findOldFiles") { + test("doesDirectoryContainFilesNewerThan") { // create some temporary directories and files val parent: File = Utils.createTempDir() val child1: File = Utils.createTempDir(parent.getCanonicalPath) // The parent directory has two child directories val child2: File = Utils.createTempDir(parent.getCanonicalPath) - // set the last modified time of child1 to 10 secs old - child1.setLastModified(System.currentTimeMillis() - (1000 * 10)) + val child3: File = Utils.createTempDir(child1.getCanonicalPath) + // set the last modified time of child1 to 30 secs old + child1.setLastModified(System.currentTimeMillis() - (1000 * 30)) - val result = Utils.findOldFiles(parent, 5) // find files older than 5 secs - assert(result.size.equals(1)) - assert(result(0).getCanonicalPath.equals(child1.getCanonicalPath)) + // although child1 is old, child2 is still new so return true + assert(Utils.doesDirectoryContainAnyNewFiles(parent, 5)) + + child2.setLastModified(System.currentTimeMillis - (1000 * 30)) + assert(Utils.doesDirectoryContainAnyNewFiles(parent, 5)) + + parent.setLastModified(System.currentTimeMillis - (1000 * 30)) + // although parent and its immediate children are new, child3 is still old + // we expect a full recursive search for new files. + assert(Utils.doesDirectoryContainAnyNewFiles(parent, 5)) + + child3.setLastModified(System.currentTimeMillis - (1000 * 30)) + assert(!Utils.doesDirectoryContainAnyNewFiles(parent, 5)) } test("resolveURI") { diff --git a/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala index 706faed980f31..f26e40fbd4b36 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala @@ -24,6 +24,8 @@ import org.scalatest.{PrivateMethodTester, FunSuite} import org.apache.spark._ import org.apache.spark.SparkContext._ +import scala.util.Random + class ExternalSorterSuite extends FunSuite with LocalSparkContext with PrivateMethodTester { private def createSparkConf(loadDefaults: Boolean): SparkConf = { val conf = new SparkConf(loadDefaults) @@ -707,4 +709,57 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext with PrivateMe Some(agg), Some(new HashPartitioner(FEW_PARTITIONS)), None, None) assertDidNotBypassMergeSort(sorter4) } + + test("sort without breaking sorting contracts") { + val conf = createSparkConf(true) + conf.set("spark.shuffle.memoryFraction", "0.01") + conf.set("spark.shuffle.manager", "sort") + sc = new SparkContext("local-cluster[1,1,512]", "test", conf) + + // Using wrongOrdering to show integer overflow introduced exception. + val rand = new Random(100L) + val wrongOrdering = new Ordering[String] { + override def compare(a: String, b: String) = { + val h1 = if (a == null) 0 else a.hashCode() + val h2 = if (b == null) 0 else b.hashCode() + h1 - h2 + } + } + + val testData = Array.tabulate(100000) { _ => rand.nextInt().toString } + + val sorter1 = new ExternalSorter[String, String, String]( + None, None, Some(wrongOrdering), None) + val thrown = intercept[IllegalArgumentException] { + sorter1.insertAll(testData.iterator.map(i => (i, i))) + sorter1.iterator + } + + assert(thrown.getClass() === classOf[IllegalArgumentException]) + assert(thrown.getMessage().contains("Comparison method violates its general contract")) + sorter1.stop() + + // Using aggregation and external spill to make sure ExternalSorter using + // partitionKeyComparator. + def createCombiner(i: String) = ArrayBuffer(i) + def mergeValue(c: ArrayBuffer[String], i: String) = c += i + def mergeCombiners(c1: ArrayBuffer[String], c2: ArrayBuffer[String]) = c1 ++= c2 + + val agg = new Aggregator[String, String, ArrayBuffer[String]]( + createCombiner, mergeValue, mergeCombiners) + + val sorter2 = new ExternalSorter[String, String, ArrayBuffer[String]]( + Some(agg), None, None, None) + sorter2.insertAll(testData.iterator.map(i => (i, i))) + + // To validate the hash ordering of key + var minKey = Int.MinValue + sorter2.iterator.foreach { case (k, v) => + val h = k.hashCode() + assert(h >= minKey) + minKey = h + } + + sorter2.stop() + } } diff --git a/dev/check-license b/dev/check-license index 9ff0929e9a5e8..72b1013479964 100755 --- a/dev/check-license +++ b/dev/check-license @@ -20,11 +20,10 @@ acquire_rat_jar () { - URL1="http://search.maven.org/remotecontent?filepath=org/apache/rat/apache-rat/${RAT_VERSION}/apache-rat-${RAT_VERSION}.jar" - URL2="http://repo1.maven.org/maven2/org/apache/rat/apache-rat/${RAT_VERSION}/apache-rat-${RAT_VERSION}.jar" + URL="http://repo1.maven.org/maven2/org/apache/rat/apache-rat/${RAT_VERSION}/apache-rat-${RAT_VERSION}.jar" JAR="$rat_jar" - + if [[ ! -f "$rat_jar" ]]; then # Download rat launch jar if it hasn't been downloaded yet if [ ! -f "$JAR" ]; then @@ -32,15 +31,17 @@ acquire_rat_jar () { printf "Attempting to fetch rat\n" JAR_DL="${JAR}.part" if hash curl 2>/dev/null; then - (curl --silent "${URL1}" > "$JAR_DL" || curl --silent "${URL2}" > "$JAR_DL") && mv "$JAR_DL" "$JAR" + curl --silent "${URL}" > "$JAR_DL" && mv "$JAR_DL" "$JAR" elif hash wget 2>/dev/null; then - (wget --quiet ${URL1} -O "$JAR_DL" || wget --quiet ${URL2} -O "$JAR_DL") && mv "$JAR_DL" "$JAR" + wget --quiet ${URL} -O "$JAR_DL" && mv "$JAR_DL" "$JAR" else printf "You do not have curl or wget installed, please install rat manually.\n" exit -1 fi fi - if [ ! -f "$JAR" ]; then + + unzip -tq $JAR &> /dev/null + if [ $? -ne 0 ]; then # We failed to download printf "Our attempt to download rat locally to ${JAR} failed. Please install rat manually.\n" exit -1 @@ -55,7 +56,7 @@ cd "$FWDIR" if test -x "$JAVA_HOME/bin/java"; then declare java_cmd="$JAVA_HOME/bin/java" -else +else declare java_cmd=java fi diff --git a/dev/merge_spark_pr.py b/dev/merge_spark_pr.py index a8e92e36fe0d8..02ac20984add9 100755 --- a/dev/merge_spark_pr.py +++ b/dev/merge_spark_pr.py @@ -73,11 +73,10 @@ def fail(msg): def run_cmd(cmd): + print cmd if isinstance(cmd, list): - print " ".join(cmd) return subprocess.check_output(cmd) else: - print cmd return subprocess.check_output(cmd.split(" ")) diff --git a/dev/run-tests b/dev/run-tests index 53148d23f385f..4be2baaf48cd1 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -24,6 +24,16 @@ cd "$FWDIR" # Remove work directory rm -rf ./work +source "$FWDIR/dev/run-tests-codes.sh" + +CURRENT_BLOCK=$BLOCK_GENERAL + +function handle_error () { + echo "[error] Got a return code of $? on line $1 of the run-tests script." + exit $CURRENT_BLOCK +} + + # Build against the right verison of Hadoop. { if [ -n "$AMPLAB_JENKINS_BUILD_PROFILE" ]; then @@ -91,26 +101,34 @@ if [ -n "$AMPLAB_JENKINS" ]; then fi fi -# Fail fast -set -e set -o pipefail +trap 'handle_error $LINENO' ERR echo "" echo "=========================================================================" echo "Running Apache RAT checks" echo "=========================================================================" + +CURRENT_BLOCK=$BLOCK_RAT + ./dev/check-license echo "" echo "=========================================================================" echo "Running Scala style checks" echo "=========================================================================" + +CURRENT_BLOCK=$BLOCK_SCALA_STYLE + ./dev/lint-scala echo "" echo "=========================================================================" echo "Running Python style checks" echo "=========================================================================" + +CURRENT_BLOCK=$BLOCK_PYTHON_STYLE + ./dev/lint-python echo "" @@ -118,6 +136,8 @@ echo "=========================================================================" echo "Building Spark" echo "=========================================================================" +CURRENT_BLOCK=$BLOCK_BUILD + { # We always build with Hive because the PySpark Spark SQL tests need it. BUILD_MVN_PROFILE_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive" @@ -127,6 +147,8 @@ echo "=========================================================================" # NOTE: echo "q" is needed because sbt on encountering a build file with failure #+ (either resolution or compilation) prompts the user for input either q, r, etc #+ to quit or retry. This echo is there to make it not block. + # NOTE: Do not quote $BUILD_MVN_PROFILE_ARGS or else it will be interpreted as a + #+ single argument! # QUESTION: Why doesn't 'yes "q"' work? # QUESTION: Why doesn't 'grep -v -e "^\[info\] Resolving"' work? echo -e "q\n" \ @@ -139,27 +161,35 @@ echo "=========================================================================" echo "Running Spark unit tests" echo "=========================================================================" +CURRENT_BLOCK=$BLOCK_SPARK_UNIT_TESTS + { # If the Spark SQL tests are enabled, run the tests with the Hive profiles enabled. + # This must be a single argument, as it is. if [ -n "$_RUN_SQL_TESTS" ]; then SBT_MAVEN_PROFILES_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive" fi if [ -n "$_SQL_TESTS_ONLY" ]; then - SBT_MAVEN_TEST_ARGS="catalyst/test sql/test hive/test" + # This must be an array of individual arguments. Otherwise, having one long string + #+ will be interpreted as a single test, which doesn't work. + SBT_MAVEN_TEST_ARGS=("catalyst/test" "sql/test" "hive/test" "hive-thriftserver/test") else - SBT_MAVEN_TEST_ARGS="test" + SBT_MAVEN_TEST_ARGS=("test") fi - echo "[info] Running Spark tests with these arguments: $SBT_MAVEN_PROFILES_ARGS $SBT_MAVEN_TEST_ARGS" + echo "[info] Running Spark tests with these arguments: $SBT_MAVEN_PROFILES_ARGS ${SBT_MAVEN_TEST_ARGS[@]}" # NOTE: echo "q" is needed because sbt on encountering a build file with failure #+ (either resolution or compilation) prompts the user for input either q, r, etc #+ to quit or retry. This echo is there to make it not block. + # NOTE: Do not quote $SBT_MAVEN_PROFILES_ARGS or else it will be interpreted as a + #+ single argument! + #+ "${SBT_MAVEN_TEST_ARGS[@]}" is cool because it's an array. # QUESTION: Why doesn't 'yes "q"' work? # QUESTION: Why doesn't 'grep -v -e "^\[info\] Resolving"' work? echo -e "q\n" \ - | sbt/sbt "$SBT_MAVEN_PROFILES_ARGS" "$SBT_MAVEN_TEST_ARGS" \ + | sbt/sbt $SBT_MAVEN_PROFILES_ARGS "${SBT_MAVEN_TEST_ARGS[@]}" \ | grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" } @@ -167,10 +197,16 @@ echo "" echo "=========================================================================" echo "Running PySpark tests" echo "=========================================================================" + +CURRENT_BLOCK=$BLOCK_PYSPARK_UNIT_TESTS + ./python/run-tests echo "" echo "=========================================================================" echo "Detecting binary incompatibilites with MiMa" echo "=========================================================================" + +CURRENT_BLOCK=$BLOCK_MIMA + ./dev/mima diff --git a/dev/run-tests-codes.sh b/dev/run-tests-codes.sh new file mode 100644 index 0000000000000..1348e0609dda4 --- /dev/null +++ b/dev/run-tests-codes.sh @@ -0,0 +1,27 @@ +#!/usr/bin/env bash + +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +readonly BLOCK_GENERAL=10 +readonly BLOCK_RAT=11 +readonly BLOCK_SCALA_STYLE=12 +readonly BLOCK_PYTHON_STYLE=13 +readonly BLOCK_BUILD=14 +readonly BLOCK_SPARK_UNIT_TESTS=15 +readonly BLOCK_PYSPARK_UNIT_TESTS=16 +readonly BLOCK_MIMA=17 diff --git a/dev/run-tests-jenkins b/dev/run-tests-jenkins index 06c3781eb3ccf..451f3b771cc76 100755 --- a/dev/run-tests-jenkins +++ b/dev/run-tests-jenkins @@ -26,9 +26,23 @@ FWDIR="$(cd `dirname $0`/..; pwd)" cd "$FWDIR" +source "$FWDIR/dev/run-tests-codes.sh" + COMMENTS_URL="https://api.github.com/repos/apache/spark/issues/$ghprbPullId/comments" PULL_REQUEST_URL="https://github.com/apache/spark/pull/$ghprbPullId" +# Important Environment Variables +# --- +# $ghprbActualCommit +#+ This is the hash of the most recent commit in the PR. +#+ The merge-base of this and master is the commit from which the PR was branched. +# $sha1 +#+ If the patch merges cleanly, this is a reference to the merge commit hash +#+ (e.g. "origin/pr/2606/merge"). +#+ If the patch does not merge cleanly, it is equal to $ghprbActualCommit. +#+ The merge-base of this and master in the case of a clean merge is the most recent commit +#+ against master. + COMMIT_URL="https://github.com/apache/spark/commit/${ghprbActualCommit}" # GitHub doesn't auto-link short hashes when submitted via the API, unfortunately. :( SHORT_COMMIT_HASH="${ghprbActualCommit:0:7}" @@ -84,42 +98,46 @@ function post_message () { fi } + +# We diff master...$ghprbActualCommit because that gets us changes introduced in the PR +#+ and not anything else added to master since the PR was branched. + # check PR merge-ability and check for new public classes { if [ "$sha1" == "$ghprbActualCommit" ]; then - merge_note=" * This patch **does not** merge cleanly!" + merge_note=" * This patch **does not merge cleanly**." else merge_note=" * This patch merges cleanly." + fi + + source_files=$( + git diff master...$ghprbActualCommit --name-only `# diff patch against master from branch point` \ + | grep -v -e "\/test" `# ignore files in test directories` \ + | grep -e "\.py$" -e "\.java$" -e "\.scala$" `# include only code files` \ + | tr "\n" " " + ) + new_public_classes=$( + git diff master...$ghprbActualCommit ${source_files} `# diff patch against master from branch point` \ + | grep "^\+" `# filter in only added lines` \ + | sed -r -e "s/^\+//g" `# remove the leading +` \ + | grep -e "trait " -e "class " `# filter in lines with these key words` \ + | grep -e "{" -e "(" `# filter in lines with these key words, too` \ + | grep -v -e "\@\@" -e "private" `# exclude lines with these words` \ + | grep -v -e "^// " -e "^/\*" -e "^ \* " `# exclude comment lines` \ + | sed -r -e "s/\{.*//g" `# remove from the { onwards` \ + | sed -r -e "s/\}//g" `# just in case, remove }; they mess the JSON` \ + | sed -r -e "s/\"/\\\\\"/g" `# escape double quotes; they mess the JSON` \ + | sed -r -e "s/^(.*)$/\`\1\`/g" `# surround with backticks for style` \ + | sed -r -e "s/^/ \* /g" `# prepend ' *' to start of line` \ + | sed -r -e "s/$/\\\n/g" `# append newline to end of line` \ + | tr -d "\n" `# remove actual LF characters` + ) - source_files=$( - git diff master --name-only \ - | grep -v -e "\/test" `# ignore files in test directories` \ - | grep -e "\.py$" -e "\.java$" -e "\.scala$" `# include only code files` \ - | tr "\n" " " - ) - new_public_classes=$( - git diff master ${source_files} `# diff this patch against master and...` \ - | grep "^\+" `# filter in only added lines` \ - | sed -r -e "s/^\+//g" `# remove the leading +` \ - | grep -e "trait " -e "class " `# filter in lines with these key words` \ - | grep -e "{" -e "(" `# filter in lines with these key words, too` \ - | grep -v -e "\@\@" -e "private" `# exclude lines with these words` \ - | grep -v -e "^// " -e "^/\*" -e "^ \* " `# exclude comment lines` \ - | sed -r -e "s/\{.*//g" `# remove from the { onwards` \ - | sed -r -e "s/\}//g" `# just in case, remove }; they mess the JSON` \ - | sed -r -e "s/\"/\\\\\"/g" `# escape double quotes; they mess the JSON` \ - | sed -r -e "s/^(.*)$/\`\1\`/g" `# surround with backticks for style` \ - | sed -r -e "s/^/ \* /g" `# prepend ' *' to start of line` \ - | sed -r -e "s/$/\\\n/g" `# append newline to end of line` \ - | tr -d "\n" `# remove actual LF characters` - ) - - if [ "$new_public_classes" == "" ]; then - public_classes_note=" * This patch adds no public classes." - else - public_classes_note=" * This patch adds the following public classes _(experimental)_:" - public_classes_note="${public_classes_note}\n${new_public_classes}" - fi + if [ -z "$new_public_classes" ]; then + public_classes_note=" * This patch adds no public classes." + else + public_classes_note=" * This patch adds the following public classes _(experimental)_:" + public_classes_note="${public_classes_note}\n${new_public_classes}" fi } @@ -141,16 +159,36 @@ function post_message () { test_result="$?" if [ "$test_result" -eq "124" ]; then - fail_message="**[Tests timed out](${BUILD_URL}consoleFull)** after \ - a configured wait of \`${TESTS_TIMEOUT}\`." + fail_message="**[Tests timed out](${BUILD_URL}consoleFull)** \ + for PR $ghprbPullId at commit [\`${SHORT_COMMIT_HASH}\`](${COMMIT_URL}) \ + after a configured wait of \`${TESTS_TIMEOUT}\`." + post_message "$fail_message" exit $test_result + elif [ "$test_result" -eq "0" ]; then + test_result_note=" * This patch **passes all tests**." else - if [ "$test_result" -eq "0" ]; then - test_result_note=" * This patch **passes** unit tests." + if [ "$test_result" -eq "$BLOCK_GENERAL" ]; then + failing_test="some tests" + elif [ "$test_result" -eq "$BLOCK_RAT" ]; then + failing_test="RAT tests" + elif [ "$test_result" -eq "$BLOCK_SCALA_STYLE" ]; then + failing_test="Scala style tests" + elif [ "$test_result" -eq "$BLOCK_PYTHON_STYLE" ]; then + failing_test="Python style tests" + elif [ "$test_result" -eq "$BLOCK_BUILD" ]; then + failing_test="to build" + elif [ "$test_result" -eq "$BLOCK_SPARK_UNIT_TESTS" ]; then + failing_test="Spark unit tests" + elif [ "$test_result" -eq "$BLOCK_PYSPARK_UNIT_TESTS" ]; then + failing_test="PySpark unit tests" + elif [ "$test_result" -eq "$BLOCK_MIMA" ]; then + failing_test="MiMa tests" else - test_result_note=" * This patch **fails** unit tests." + failing_test="some tests" fi + + test_result_note=" * This patch **fails $failing_test**." fi } diff --git a/docs/README.md b/docs/README.md index fdc89d2eb767a..79708c3df9106 100644 --- a/docs/README.md +++ b/docs/README.md @@ -20,12 +20,16 @@ In this directory you will find textfiles formatted using Markdown, with an ".md read those text files directly if you want. Start with index.md. 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). -If not already installed, you need to install `kramdown` and `jekyll-redirect-from` Gems -with `sudo gem install kramdown jekyll-redirect-from`. -Execute `jekyll build` from the `docs/` directory. Compiling the site with Jekyll will create a directory +`Jekyll` and a few dependencies must be installed for this to work. We recommend +installing via the Ruby Gem dependency manager. Since the exact HTML output +varies between versions of Jekyll and its dependencies, we list specific versions here +in some cases: + + $ sudo gem install jekyll -v 1.4.3 + $ sudo gem uninstall kramdown -v 1.4.1 + $ sudo gem install jekyll-redirect-from + +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: diff --git a/docs/_config.yml b/docs/_config.yml index d3ea2625c7448..7bc3a78e2d265 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -3,6 +3,11 @@ markdown: kramdown gems: - jekyll-redirect-from +# For some reason kramdown seems to behave differently on different +# OS/packages wrt encoding. So we hard code this config. +kramdown: + entity_output: numeric + # These allow the documentation to be updated with nerw releases # of Spark, Scala, and Mesos. SPARK_VERSION: 1.0.0-SNAPSHOT diff --git a/docs/building-spark.md b/docs/building-spark.md index 2378092d4a1a8..b2940ee4029e8 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -169,7 +169,22 @@ compilation. More advanced developers may wish to use SBT. The SBT build is derived from the Maven POM files, and so the same Maven profiles and variables can be set to control the SBT build. For example: - sbt/sbt -Pyarn -Phadoop-2.3 compile + sbt/sbt -Pyarn -Phadoop-2.3 assembly + +# Testing with SBT + +Some of the tests require Spark to be packaged first, so always run `sbt/sbt assembly` the first time. The following is an example of a correct (build, test) sequence: + + sbt/sbt -Pyarn -Phadoop-2.3 -Phive assembly + sbt/sbt -Pyarn -Phadoop-2.3 -Phive test + +To run only a specific test suite as follows: + + sbt/sbt -Pyarn -Phadoop-2.3 -Phive "test-only org.apache.spark.repl.ReplSuite" + +To run test suites of a specific sub project as follows: + + sbt/sbt -Pyarn -Phadoop-2.3 -Phive core/test # Speeding up Compilation with Zinc diff --git a/docs/configuration.md b/docs/configuration.md index 99faf51c6f3db..1c33855365170 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -206,6 +206,25 @@ Apart from these, the following properties are also available, and may be useful used during aggregation goes above this amount, it will spill the data into disks. + + spark.python.profile + false + + Enable profiling in Python worker, the profile result will show up by `sc.show_profiles()`, + or it will be displayed before the driver exiting. It also can be dumped into disk by + `sc.dump_profiles(path)`. If some of the profile results had been displayed maually, + they will not be displayed automatically before driver exiting. + + + + spark.python.profile.dump + (none) + + The directory which is used to dump the profile result before driver exiting. + The results will be dumped as separated file for each RDD. They can be loaded + by ptats.Stats(). If this is specified, the profile result will not be displayed + automatically. + spark.python.worker.reuse true @@ -234,6 +253,17 @@ Apart from these, the following properties are also available, and may be useful spark.executor.uri. + + spark.mesos.executor.memoryOverhead + executor memory * 0.07, with minimum of 384 + + This value is an additive for spark.executor.memory, specified in MiB, + which is used to calculate the total Mesos task memory. A value of 384 + implies a 384MiB overhead. Additionally, there is a hard-coded 7% minimum + overhead. The final overhead will be the larger of either + `spark.mesos.executor.memoryOverhead` or 7% of `spark.executor.memory`. + + #### Shuffle Behavior @@ -394,10 +424,11 @@ Apart from these, the following properties are also available, and may be useful spark.io.compression.codec snappy - The codec used to compress internal data such as RDD partitions and shuffle outputs. By default, - Spark provides three codecs: lz4, lzf, and snappy. You - can also use fully qualified class names to specify the codec, e.g. - org.apache.spark.io.LZ4CompressionCodec, + The codec used to compress internal data such as RDD partitions, broadcast variables and + shuffle outputs. By default, Spark provides three codecs: lz4, lzf, + and snappy. You can also use fully qualified class names to specify the codec, + e.g. + org.apache.spark.io.LZ4CompressionCodec, org.apache.spark.io.LZFCompressionCodec, and org.apache.spark.io.SnappyCompressionCodec. @@ -657,7 +688,7 @@ Apart from these, the following properties are also available, and may be useful spark.port.maxRetries 16 - Maximum number of retries when binding to a port before giving up. + Default maximum number of retries when binding to a port before giving up. @@ -1088,3 +1119,10 @@ compute `SPARK_LOCAL_IP` by looking up the IP of a specific network interface. Spark uses [log4j](http://logging.apache.org/log4j/) for logging. You can configure it by adding a `log4j.properties` file in the `conf` directory. One way to start is to copy the existing `log4j.properties.template` located there. + +# Overriding configuration directory + +To specify a different configuration directory other than the default "SPARK_HOME/conf", +you can set SPARK_CONF_DIR. Spark will use the the configuration files (spark-defaults.conf, spark-env.sh, log4j.properties, etc) +from this directory. + diff --git a/docs/ec2-scripts.md b/docs/ec2-scripts.md index b2ca6a9b48f32..530798f2b8022 100644 --- a/docs/ec2-scripts.md +++ b/docs/ec2-scripts.md @@ -48,6 +48,15 @@ by looking for the "Name" tag of the instance in the Amazon EC2 Console. key pair, `` is the number of slave nodes to launch (try 1 at first), and `` is the name to give to your cluster. + + For example: + + ```bash + export AWS_SECRET_ACCESS_KEY=AaBbCcDdEeFGgHhIiJjKkLlMmNnOoPpQqRrSsTtU +export AWS_ACCESS_KEY_ID=ABCDEFG1234567890123 +./spark-ec2 --key-pair=awskey --identity-file=awskey.pem --region=us-west-1 --zone=us-west-1a --spark-version=1.1.0 launch my-spark-cluster + ``` + - After everything launches, check that the cluster scheduler is up and sees all the slaves by going to its web UI, which will be printed at the end of the script (typically `http://:8080`). @@ -55,27 +64,27 @@ by looking for the "Name" tag of the instance in the Amazon EC2 Console. You can also run `./spark-ec2 --help` to see more usage options. The following options are worth pointing out: -- `--instance-type=` can be used to specify an EC2 +- `--instance-type=` can be used to specify an EC2 instance type to use. For now, the script only supports 64-bit instance types, and the default type is `m1.large` (which has 2 cores and 7.5 GB RAM). Refer to the Amazon pages about [EC2 instance types](http://aws.amazon.com/ec2/instance-types) and [EC2 pricing](http://aws.amazon.com/ec2/#pricing) for information about other instance types. -- `--region=` specifies an EC2 region in which to launch +- `--region=` specifies an EC2 region in which to launch instances. The default region is `us-east-1`. -- `--zone=` can be used to specify an EC2 availability zone +- `--zone=` can be used to specify an EC2 availability zone to launch instances in. Sometimes, you will get an error because there is not enough capacity in one zone, and you should try to launch in another. -- `--ebs-vol-size=GB` will attach an EBS volume with a given amount +- `--ebs-vol-size=` will attach an EBS volume with a given amount of space to each node so that you can have a persistent HDFS cluster on your nodes across cluster restarts (see below). -- `--spot-price=PRICE` will launch the worker nodes as +- `--spot-price=` will launch the worker nodes as [Spot Instances](http://aws.amazon.com/ec2/spot-instances/), bidding for the given maximum price (in dollars). -- `--spark-version=VERSION` will pre-load the cluster with the - specified version of Spark. VERSION can be a version number +- `--spark-version=` will pre-load the cluster with the + specified version of Spark. The `` can be a version number (e.g. "0.7.3") or a specific git hash. By default, a recent version will be used. - If one of your launches fails due to e.g. not having the right @@ -137,11 +146,11 @@ cost you any EC2 cycles, but ***will*** continue to cost money for EBS storage. - To stop one of your clusters, go into the `ec2` directory and run -`./spark-ec2 stop `. +`./spark-ec2 --region= stop `. - To restart it later, run -`./spark-ec2 -i start `. +`./spark-ec2 -i --region= start `. - To ultimately destroy the cluster and stop consuming EBS space, run -`./spark-ec2 destroy ` as described in the previous +`./spark-ec2 --region= destroy ` as described in the previous section. # Limitations diff --git a/docs/mllib-clustering.md b/docs/mllib-clustering.md index dfd9cd572888c..d10bd63746629 100644 --- a/docs/mllib-clustering.md +++ b/docs/mllib-clustering.md @@ -52,7 +52,7 @@ import org.apache.spark.mllib.linalg.Vectors // Load and parse the data val data = sc.textFile("data/mllib/kmeans_data.txt") -val parsedData = data.map(s => Vectors.dense(s.split(' ').map(_.toDouble))) +val parsedData = data.map(s => Vectors.dense(s.split(' ').map(_.toDouble))).cache() // Cluster the data into two classes using KMeans val numClusters = 2 @@ -100,6 +100,7 @@ public class KMeansExample { } } ); + parsedData.cache(); // Cluster the data into two classes using KMeans int numClusters = 2; diff --git a/docs/mllib-feature-extraction.md b/docs/mllib-feature-extraction.md index 44f0f76220b6e..1511ae6dda4ed 100644 --- a/docs/mllib-feature-extraction.md +++ b/docs/mllib-feature-extraction.md @@ -68,7 +68,7 @@ val sc: SparkContext = ... val documents: RDD[Seq[String]] = sc.textFile("...").map(_.split(" ").toSeq) val hashingTF = new HashingTF() -val tf: RDD[Vector] = hasingTF.transform(documents) +val tf: RDD[Vector] = hashingTF.transform(documents) {% endhighlight %} While applying `HashingTF` only needs a single pass to the data, applying `IDF` needs two passes: @@ -82,6 +82,21 @@ tf.cache() val idf = new IDF().fit(tf) val tfidf: RDD[Vector] = idf.transform(tf) {% endhighlight %} + +MLLib's IDF implementation provides an option for ignoring terms which occur in less than a +minimum number of documents. In such cases, the IDF for these terms is set to 0. This feature +can be used by passing the `minDocFreq` value to the IDF constructor. + +{% highlight scala %} +import org.apache.spark.mllib.feature.IDF + +// ... continue from the previous example +tf.cache() +val idf = new IDF(minDocFreq = 2).fit(tf) +val tfidf: RDD[Vector] = idf.transform(tf) +{% endhighlight %} + + diff --git a/docs/mllib-linear-methods.md b/docs/mllib-linear-methods.md index 9137f9dc1b692..d31bec3e1bd01 100644 --- a/docs/mllib-linear-methods.md +++ b/docs/mllib-linear-methods.md @@ -396,7 +396,7 @@ val data = sc.textFile("data/mllib/ridge-data/lpsa.data") val parsedData = data.map { line => val parts = line.split(',') LabeledPoint(parts(0).toDouble, Vectors.dense(parts(1).split(' ').map(_.toDouble))) -} +}.cache() // Building the model val numIterations = 100 @@ -455,6 +455,7 @@ public class LinearRegression { } } ); + parsedData.cache(); // Building the model int numIterations = 100; @@ -470,7 +471,7 @@ public class LinearRegression { } } ); - JavaRDD MSE = new JavaDoubleRDD(valuesAndPreds.map( + double MSE = new JavaDoubleRDD(valuesAndPreds.map( new Function, Object>() { public Object call(Tuple2 pair) { return Math.pow(pair._1() - pair._2(), 2.0); @@ -553,8 +554,8 @@ but in practice you will likely want to use unlabeled vectors for test data. {% highlight scala %} -val trainingData = ssc.textFileStream('/training/data/dir').map(LabeledPoint.parse) -val testData = ssc.textFileStream('/testing/data/dir').map(LabeledPoint.parse) +val trainingData = ssc.textFileStream("/training/data/dir").map(LabeledPoint.parse).cache() +val testData = ssc.textFileStream("/testing/data/dir").map(LabeledPoint.parse) {% endhighlight %} diff --git a/docs/mllib-optimization.md b/docs/mllib-optimization.md index 26ce5f3c501ff..45141c235be90 100644 --- a/docs/mllib-optimization.md +++ b/docs/mllib-optimization.md @@ -217,6 +217,7 @@ import org.apache.spark.mllib.evaluation.BinaryClassificationMetrics import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.util.MLUtils import org.apache.spark.mllib.classification.LogisticRegressionModel +import org.apache.spark.mllib.optimization.{LBFGS, LogisticGradient, SquaredL2Updater} val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") val numFeatures = data.take(1)(0).features.size diff --git a/docs/programming-guide.md b/docs/programming-guide.md index 624cc744dfd51..8e8cc1dd983f8 100644 --- a/docs/programming-guide.md +++ b/docs/programming-guide.md @@ -211,17 +211,17 @@ For a complete list of options, run `pyspark --help`. Behind the scenes, 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`: +use IPython, set the `PYSPARK_PYTHON` variable to `ipython` when running `bin/pyspark`: {% highlight bash %} -$ IPYTHON=1 ./bin/pyspark +$ PYSPARK_PYTHON=ipython ./bin/pyspark {% endhighlight %} -You can customize the `ipython` command by setting `IPYTHON_OPTS`. For example, to launch +You can customize the `ipython` command by setting `PYSPARK_PYTHON_OPTS`. For example, to launch the [IPython Notebook](http://ipython.org/notebook.html) with PyLab plot support: {% highlight bash %} -$ IPYTHON_OPTS="notebook --pylab inline" ./bin/pyspark +$ PYSPARK_PYTHON=ipython PYSPARK_PYTHON_OPTS="notebook --pylab inline" ./bin/pyspark {% endhighlight %} @@ -286,7 +286,7 @@ We describe operations on distributed datasets later on. -One important parameter for parallel collections is the number of *slices* to cut the dataset into. Spark will run one task for each slice of the cluster. Typically you want 2-4 slices for each CPU in your cluster. Normally, Spark tries to set the number of slices automatically based on your cluster. However, you can also set it manually by passing it as a second parameter to `parallelize` (e.g. `sc.parallelize(data, 10)`). +One important parameter for parallel collections is the number of *partitions* to cut the dataset into. Spark will run one task for each partition of the cluster. Typically you want 2-4 partitions for each CPU in your cluster. Normally, Spark tries to set the number of partitions automatically based on your cluster. However, you can also set it manually by passing it as a second parameter to `parallelize` (e.g. `sc.parallelize(data, 10)`). Note: some places in the code use the term slices (a synonym for partitions) to maintain backward compatibility. ## External Datasets @@ -311,7 +311,7 @@ Some notes on reading files with Spark: * All of Spark's file-based input methods, including `textFile`, support running on directories, compressed files, and wildcards as well. For example, you can use `textFile("/my/directory")`, `textFile("/my/directory/*.txt")`, and `textFile("/my/directory/*.gz")`. -* The `textFile` method also takes an optional second argument for controlling the number of slices of the file. By default, Spark creates one slice for each block of the file (blocks being 64MB by default in HDFS), but you can also ask for a higher number of slices by passing a larger value. Note that you cannot have fewer slices than blocks. +* The `textFile` method also takes an optional second argument for controlling the number of partitions of the file. By default, Spark creates one partition for each block of the file (blocks being 64MB by default in HDFS), but you can also ask for a higher number of partitions by passing a larger value. Note that you cannot have fewer partitions than blocks. Apart from text files, Spark's Scala API also supports several other data formats: @@ -343,7 +343,7 @@ Some notes on reading files with Spark: * All of Spark's file-based input methods, including `textFile`, support running on directories, compressed files, and wildcards as well. For example, you can use `textFile("/my/directory")`, `textFile("/my/directory/*.txt")`, and `textFile("/my/directory/*.gz")`. -* The `textFile` method also takes an optional second argument for controlling the number of slices of the file. By default, Spark creates one slice for each block of the file (blocks being 64MB by default in HDFS), but you can also ask for a higher number of slices by passing a larger value. Note that you cannot have fewer slices than blocks. +* The `textFile` method also takes an optional second argument for controlling the number of partitions of the file. By default, Spark creates one partition for each block of the file (blocks being 64MB by default in HDFS), but you can also ask for a higher number of partitions by passing a larger value. Note that you cannot have fewer partitions than blocks. Apart from text files, Spark's Java API also supports several other data formats: @@ -375,7 +375,7 @@ Some notes on reading files with Spark: * All of Spark's file-based input methods, including `textFile`, support running on directories, compressed files, and wildcards as well. For example, you can use `textFile("/my/directory")`, `textFile("/my/directory/*.txt")`, and `textFile("/my/directory/*.gz")`. -* The `textFile` method also takes an optional second argument for controlling the number of slices of the file. By default, Spark creates one slice for each block of the file (blocks being 64MB by default in HDFS), but you can also ask for a higher number of slices by passing a larger value. Note that you cannot have fewer slices than blocks. +* The `textFile` method also takes an optional second argument for controlling the number of partitions of the file. By default, Spark creates one partition for each block of the file (blocks being 64MB by default in HDFS), but you can also ask for a higher number of partitions by passing a larger value. Note that you cannot have fewer partitions than blocks. Apart from text files, Spark's Python API also supports several other data formats: @@ -883,7 +883,7 @@ for details. groupByKey([numTasks]) When called on a dataset of (K, V) pairs, returns a dataset of (K, Iterable<V>) pairs.
- Note: If you are grouping in order to perform an aggregation (such as a sum or + Note: If you are grouping in order to perform an aggregation (such as a sum or average) over each key, using reduceByKey or combineByKey will yield much better performance.
@@ -906,7 +906,7 @@ for details. join(otherDataset, [numTasks]) When called on datasets of type (K, V) and (K, W), returns a dataset of (K, (V, W)) pairs with all pairs of elements for each key. - Outer joins are also supported through leftOuterJoin and rightOuterJoin. + Outer joins are supported through leftOuterJoin, rightOuterJoin, and fullOuterJoin. diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 74bcc2eeb65f6..695813a2ba881 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -79,16 +79,16 @@ Most of the configs are the same for Spark on YARN as for other deployment modes spark.yarn.executor.memoryOverhead - 384 + executorMemory * 0.07, with minimum of 384 - The amount of off heap memory (in megabytes) to be allocated per executor. This is memory that accounts for things like VM overheads, interned strings, other native overheads, etc. + The amount of off heap memory (in megabytes) to be allocated per executor. This is memory that accounts for things like VM overheads, interned strings, other native overheads, etc. This tends to grow with the executor size (typically 6-10%). spark.yarn.driver.memoryOverhead - 384 + driverMemory * 0.07, with minimum of 384 - The amount of off heap memory (in megabytes) to be allocated per driver. This is memory that accounts for things like VM overheads, interned strings, other native overheads, etc. + The amount of off heap memory (in megabytes) to be allocated per driver. This is memory that accounts for things like VM overheads, interned strings, other native overheads, etc. This tends to grow with the container size (typically 6-10%). @@ -205,6 +205,8 @@ Note that for the first option, both executors and the application master will s log4j configuration, which may cause issues when they run on the same node (e.g. trying to write to the same log file). +If you need a reference to the proper location to put log files in the YARN so that YARN can properly display and aggregate them, use "${spark.yarn.app.container.log.dir}" in your log4j.properties. For example, log4j.appender.file_appender.File=${spark.yarn.app.container.log.dir}/spark.log. For streaming application, configuring RollingFileAppender and setting file location to YARN's log directory will avoid disk overflow caused by large log file, and logs can be accessed using YARN's log utility. + # 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. diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index c791c81f8bfd0..a3028aa86dc45 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -62,7 +62,12 @@ Finally, the following configuration options can be passed to the master and wor # Cluster Launch Scripts -To launch a Spark standalone cluster with the launch scripts, you need to create a file called `conf/slaves` in your Spark directory, which should contain the hostnames of all the machines where you would like to start Spark workers, one per line. The master machine must be able to access each of the slave machines via password-less `ssh` (using a private key). For testing, you can just put `localhost` in this file. +To launch a Spark standalone cluster with the launch scripts, you should create a file called conf/slaves in your Spark directory, +which must contain the hostnames of all the machines where you intend to start Spark workers, one per line. +If conf/slaves does not exist, the launch scripts defaults to a single machine (localhost), which is useful for testing. +Note, the master machine accesses each of the worker machines via ssh. By default, ssh is run in parallel and requires password-less (using a private key) access to be setup. +If you do not have a password-less setup, you can set the environment variable SPARK_SSH_FOREGROUND and serially provide a password for each worker. + Once you've set up this file, you can launch or stop your cluster with the following shell scripts, based on Hadoop's deploy scripts, and available in `SPARK_HOME/bin`: @@ -242,14 +247,16 @@ To run an interactive Spark shell against the cluster, run the following command ./bin/spark-shell --master spark://IP:PORT -You can also pass an option `--cores ` to control the number of cores that spark-shell uses on the cluster. +You can also pass an option `--total-executor-cores ` to control the number of cores that spark-shell uses on the cluster. # Launching Compiled Spark Applications The [`spark-submit` script](submitting-applications.html) provides the most straightforward way to submit a compiled Spark application to the cluster. For standalone clusters, Spark currently -only supports deploying the driver inside the client process that is submitting the application -(`client` deploy mode). +supports two deploy modes. In `client` mode, the driver is launched in the same process as the +client that submits the application. In `cluster` mode, however, the driver is launched from one +of the Worker processes inside the cluster, and the client process exits as soon as it fulfills +its responsibility of submitting the application without waiting for the application to finish. If your application is launched through Spark submit, then the application jar is automatically distributed to all worker nodes. For any additional jars that your application depends on, you @@ -307,7 +314,7 @@ tight firewall settings. For a complete list of ports to configure, see the By default, standalone scheduling clusters are resilient to Worker failures (insofar as Spark itself is resilient to losing work by moving it to other workers). However, the scheduler uses a Master to make scheduling decisions, and this (by default) creates a single point of failure: if the Master crashes, no new applications can be created. In order to circumvent this, we have two high availability schemes, detailed below. -# Standby Masters with ZooKeeper +## Standby Masters with ZooKeeper **Overview** @@ -347,7 +354,7 @@ There's an important distinction to be made between "registering with a Master" Due to this property, new Masters can be created at any time, and the only thing you need to worry about is that _new_ applications and Workers can find it to register with in case it becomes the leader. Once registered, you're taken care of. -# Single-Node Recovery with Local File System +## Single-Node Recovery with Local File System **Overview** diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 5212e19c41349..368c3d0008b07 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -605,7 +605,7 @@ Spark SQL can automatically infer the schema of a JSON dataset and load it as a This conversion can be done using one of two methods in a SQLContext: * `jsonFile` - loads data from a directory of JSON files where each line of the files is a JSON object. -* `jsonRdd` - loads data from an existing RDD where each element of the RDD is a string containing a JSON object. +* `jsonRDD` - loads data from an existing RDD where each element of the RDD is a string containing a JSON object. {% highlight scala %} // sc is an existing SparkContext. @@ -620,8 +620,8 @@ val people = sqlContext.jsonFile(path) // The inferred schema can be visualized using the printSchema() method. people.printSchema() // root -// |-- age: IntegerType -// |-- name: StringType +// |-- age: integer (nullable = true) +// |-- name: string (nullable = true) // Register this SchemaRDD as a table. people.registerTempTable("people") @@ -643,7 +643,7 @@ Spark SQL can automatically infer the schema of a JSON dataset and load it as a This conversion can be done using one of two methods in a JavaSQLContext : * `jsonFile` - loads data from a directory of JSON files where each line of the files is a JSON object. -* `jsonRdd` - loads data from an existing RDD where each element of the RDD is a string containing a JSON object. +* `jsonRDD` - loads data from an existing RDD where each element of the RDD is a string containing a JSON object. {% highlight java %} // sc is an existing JavaSparkContext. @@ -658,8 +658,8 @@ JavaSchemaRDD people = sqlContext.jsonFile(path); // The inferred schema can be visualized using the printSchema() method. people.printSchema(); // root -// |-- age: IntegerType -// |-- name: StringType +// |-- age: integer (nullable = true) +// |-- name: string (nullable = true) // Register this JavaSchemaRDD as a table. people.registerTempTable("people"); @@ -681,7 +681,7 @@ Spark SQL can automatically infer the schema of a JSON dataset and load it as a This conversion can be done using one of two methods in a SQLContext: * `jsonFile` - loads data from a directory of JSON files where each line of the files is a JSON object. -* `jsonRdd` - loads data from an existing RDD where each element of the RDD is a string containing a JSON object. +* `jsonRDD` - loads data from an existing RDD where each element of the RDD is a string containing a JSON object. {% highlight python %} # sc is an existing SparkContext. @@ -697,8 +697,8 @@ people = sqlContext.jsonFile(path) # The inferred schema can be visualized using the printSchema() method. people.printSchema() # root -# |-- age: IntegerType -# |-- name: StringType +# |-- age: integer (nullable = true) +# |-- name: string (nullable = true) # Register this SchemaRDD as a table. people.registerTempTable("people") @@ -732,7 +732,7 @@ Configuration of Hive is done by placing your `hive-site.xml` file in `conf/`.
When working with Hive one must construct a `HiveContext`, which inherits from `SQLContext`, and -adds support for finding tables in in the MetaStore and writing queries using HiveQL. Users who do +adds support for finding tables in the MetaStore and writing queries using HiveQL. Users who do not have an existing Hive deployment can still create a HiveContext. When not configured by the hive-site.xml, the context automatically creates `metastore_db` and `warehouse` in the current directory. @@ -753,7 +753,7 @@ sqlContext.sql("FROM src SELECT key, value").collect().foreach(println)
When working with Hive one must construct a `JavaHiveContext`, which inherits from `JavaSQLContext`, and -adds support for finding tables in in the MetaStore and writing queries using HiveQL. In addition to +adds support for finding tables in the MetaStore and writing queries using HiveQL. In addition to the `sql` method a `JavaHiveContext` also provides an `hql` methods, which allows queries to be expressed in HiveQL. @@ -774,7 +774,7 @@ Row[] results = sqlContext.sql("FROM src SELECT key, value").collect();
When working with Hive one must construct a `HiveContext`, which inherits from `SQLContext`, and -adds support for finding tables in in the MetaStore and writing queries using HiveQL. In addition to +adds support for finding tables in the MetaStore and writing queries using HiveQL. In addition to the `sql` method a `HiveContext` also provides an `hql` methods, which allows queries to be expressed in HiveQL. @@ -872,12 +872,12 @@ that these options will be deprecated in future release as more optimizations ar Spark SQL also supports interfaces for running SQL queries directly without the need to write any code. -## Running the Thrift JDBC server +## Running the Thrift JDBC/ODBC server -The Thrift JDBC server implemented here corresponds to the [`HiveServer2`](https://cwiki.apache.org/confluence/display/Hive/Setting+Up+HiveServer2) +The Thrift JDBC/ODBC server implemented here corresponds to the [`HiveServer2`](https://cwiki.apache.org/confluence/display/Hive/Setting+Up+HiveServer2) in Hive 0.12. You can test the JDBC server with the beeline script that comes with either Spark or Hive 0.12. -To start the JDBC server, run the following in the Spark directory: +To start the JDBC/ODBC server, run the following in the Spark directory: ./sbin/start-thriftserver.sh @@ -906,11 +906,11 @@ or system properties: ``` {% endhighlight %} -Now you can use beeline to test the Thrift JDBC server: +Now you can use beeline to test the Thrift JDBC/ODBC server: ./bin/beeline -Connect to the JDBC server in beeline with: +Connect to the JDBC/ODBC server in beeline with: beeline> !connect jdbc:hive2://localhost:10000 @@ -1394,7 +1394,7 @@ please use factory methods provided in StructType - org.apache.spark.sql.api.java + org.apache.spark.sql.api.java.Row DataType.createStructType(fields)
Note: fields is a List or an array of StructFields. diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 41f170580f452..5c21e912ea160 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -138,7 +138,7 @@ import org.apache.spark.streaming.api.java.*; import scala.Tuple2; // Create a local StreamingContext with two working thread and batch interval of 1 second -val conf = new SparkConf().setMaster("local[2]").setAppName("NetworkWordCount") +SparkConf conf = new SparkConf().setMaster("local[2]").setAppName("NetworkWordCount") JavaStreamingContext jssc = new JavaStreamingContext(conf, new Duration(1000)) {% endhighlight %} diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index abac71eaca595..941dfb988b9fb 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -38,7 +38,7 @@ from boto.ec2.blockdevicemapping import BlockDeviceMapping, BlockDeviceType, EBSBlockDeviceType from boto import ec2 -DEFAULT_SPARK_VERSION = "1.0.0" +DEFAULT_SPARK_VERSION = "1.1.0" # A URL prefix from which to fetch AMI information AMI_PREFIX = "https://raw.github.com/mesos/spark-ec2/v2/ami-list" @@ -217,8 +217,15 @@ def is_active(instance): # Return correct versions of Spark and Shark, given the supplied Spark version def get_spark_shark_version(opts): spark_shark_map = { - "0.7.3": "0.7.1", "0.8.0": "0.8.0", "0.8.1": "0.8.1", "0.9.0": "0.9.0", "0.9.1": "0.9.1", - "1.0.0": "1.0.0" + "0.7.3": "0.7.1", + "0.8.0": "0.8.0", + "0.8.1": "0.8.1", + "0.9.0": "0.9.0", + "0.9.1": "0.9.1", + "1.0.0": "1.0.0", + "1.0.1": "1.0.1", + "1.0.2": "1.0.2", + "1.1.0": "1.1.0", } version = opts.spark_version.replace("v", "") if version not in spark_shark_map: @@ -227,49 +234,49 @@ def get_spark_shark_version(opts): return (version, spark_shark_map[version]) -# Attempt to resolve an appropriate AMI given the architecture and -# region of the request. -# Information regarding Amazon Linux AMI instance type was update on 2014-6-20: -# http://aws.amazon.com/amazon-linux-ami/instance-type-matrix/ +# Attempt to resolve an appropriate AMI given the architecture and region of the request. +# Source: http://aws.amazon.com/amazon-linux-ami/instance-type-matrix/ +# Last Updated: 2014-06-20 +# For easy maintainability, please keep this manually-inputted dictionary sorted by key. def get_spark_ami(opts): instance_types = { - "m1.small": "pvm", - "m1.medium": "pvm", - "m1.large": "pvm", - "m1.xlarge": "pvm", - "t1.micro": "pvm", "c1.medium": "pvm", "c1.xlarge": "pvm", - "m2.xlarge": "pvm", - "m2.2xlarge": "pvm", - "m2.4xlarge": "pvm", + "c3.2xlarge": "pvm", + "c3.4xlarge": "pvm", + "c3.8xlarge": "pvm", + "c3.large": "pvm", + "c3.xlarge": "pvm", "cc1.4xlarge": "hvm", "cc2.8xlarge": "hvm", "cg1.4xlarge": "hvm", - "hs1.8xlarge": "pvm", - "hi1.4xlarge": "pvm", - "m3.medium": "hvm", - "m3.large": "hvm", - "m3.xlarge": "hvm", - "m3.2xlarge": "hvm", "cr1.8xlarge": "hvm", - "i2.xlarge": "hvm", + "hi1.4xlarge": "pvm", + "hs1.8xlarge": "pvm", "i2.2xlarge": "hvm", "i2.4xlarge": "hvm", "i2.8xlarge": "hvm", - "c3.large": "pvm", - "c3.xlarge": "pvm", - "c3.2xlarge": "pvm", - "c3.4xlarge": "pvm", - "c3.8xlarge": "pvm", - "r3.large": "hvm", - "r3.xlarge": "hvm", + "i2.xlarge": "hvm", + "m1.large": "pvm", + "m1.medium": "pvm", + "m1.small": "pvm", + "m1.xlarge": "pvm", + "m2.2xlarge": "pvm", + "m2.4xlarge": "pvm", + "m2.xlarge": "pvm", + "m3.2xlarge": "hvm", + "m3.large": "hvm", + "m3.medium": "hvm", + "m3.xlarge": "hvm", "r3.2xlarge": "hvm", "r3.4xlarge": "hvm", "r3.8xlarge": "hvm", + "r3.large": "hvm", + "r3.xlarge": "hvm", + "t1.micro": "pvm", + "t2.medium": "hvm", "t2.micro": "hvm", "t2.small": "hvm", - "t2.medium": "hvm" } if opts.instance_type in instance_types: instance_type = instance_types[opts.instance_type] @@ -505,9 +512,10 @@ def tag_instance(instance, name): for i in range(0, 5): try: instance.add_tag(key='Name', value=name) + break except: print "Failed attempt %i of 5 to tag %s" % ((i + 1), name) - if (i == 5): + if i == 5: raise "Error - failed max attempts to add name tag" time.sleep(5) @@ -529,7 +537,7 @@ def get_existing_cluster(conn, opts, cluster_name, die_on_error=True): for res in reservations: active = [i for i in res.instances if is_active(i)] for instance in active: - if (instance.tags.get(u'Name') is None): + if instance.tags.get(u'Name') is None: tag_instance(instance, name) # Now proceed to detect master and slaves instances. reservations = conn.get_all_instances() @@ -544,7 +552,7 @@ def get_existing_cluster(conn, opts, cluster_name, die_on_error=True): elif name.startswith(cluster_name + "-slave"): slave_nodes.append(inst) if any((master_nodes, slave_nodes)): - print ("Found %d master(s), %d slaves" % (len(master_nodes), len(slave_nodes))) + print "Found %d master(s), %d slaves" % (len(master_nodes), len(slave_nodes)) if master_nodes != [] or not die_on_error: return (master_nodes, slave_nodes) else: @@ -623,45 +631,45 @@ def wait_for_cluster(conn, wait_secs, master_nodes, slave_nodes): # Get number of local disks available for a given EC2 instance type. def get_num_disks(instance_type): - # From http://docs.aws.amazon.com/AWSEC2/latest/UserGuide/InstanceStorage.html - # Updated 2014-6-20 + # Source: http://docs.aws.amazon.com/AWSEC2/latest/UserGuide/InstanceStorage.html + # Last Updated: 2014-06-20 + # For easy maintainability, please keep this manually-inputted dictionary sorted by key. disks_by_instance = { - "m1.small": 1, - "m1.medium": 1, - "m1.large": 2, - "m1.xlarge": 4, - "t1.micro": 1, "c1.medium": 1, "c1.xlarge": 4, - "m2.xlarge": 1, - "m2.2xlarge": 1, - "m2.4xlarge": 2, + "c3.2xlarge": 2, + "c3.4xlarge": 2, + "c3.8xlarge": 2, + "c3.large": 2, + "c3.xlarge": 2, "cc1.4xlarge": 2, "cc2.8xlarge": 4, "cg1.4xlarge": 2, - "hs1.8xlarge": 24, "cr1.8xlarge": 2, + "g2.2xlarge": 1, "hi1.4xlarge": 2, - "m3.medium": 1, - "m3.large": 1, - "m3.xlarge": 2, - "m3.2xlarge": 2, - "i2.xlarge": 1, + "hs1.8xlarge": 24, "i2.2xlarge": 2, "i2.4xlarge": 4, "i2.8xlarge": 8, - "c3.large": 2, - "c3.xlarge": 2, - "c3.2xlarge": 2, - "c3.4xlarge": 2, - "c3.8xlarge": 2, - "r3.large": 1, - "r3.xlarge": 1, + "i2.xlarge": 1, + "m1.large": 2, + "m1.medium": 1, + "m1.small": 1, + "m1.xlarge": 4, + "m2.2xlarge": 1, + "m2.4xlarge": 2, + "m2.xlarge": 1, + "m3.2xlarge": 2, + "m3.large": 1, + "m3.medium": 1, + "m3.xlarge": 2, "r3.2xlarge": 1, "r3.4xlarge": 1, "r3.8xlarge": 2, - "g2.2xlarge": 1, - "t1.micro": 0 + "r3.large": 1, + "r3.xlarge": 1, + "t1.micro": 0, } if instance_type in disks_by_instance: return disks_by_instance[instance_type] @@ -784,7 +792,7 @@ def ssh(host, opts, command): ssh_command(opts) + ['-t', '-t', '%s@%s' % (opts.user, host), stringify_command(command)]) except subprocess.CalledProcessError as e: - if (tries > 5): + if tries > 5: # If this was an ssh failure, provide the user with hints. if e.returncode == 255: raise UsageError( @@ -819,18 +827,18 @@ def ssh_read(host, opts, command): ssh_command(opts) + ['%s@%s' % (opts.user, host), stringify_command(command)]) -def ssh_write(host, opts, command, input): +def ssh_write(host, opts, command, arguments): tries = 0 while True: proc = subprocess.Popen( ssh_command(opts) + ['%s@%s' % (opts.user, host), stringify_command(command)], stdin=subprocess.PIPE) - proc.stdin.write(input) + proc.stdin.write(arguments) proc.stdin.close() status = proc.wait() if status == 0: break - elif (tries > 5): + elif tries > 5: raise RuntimeError("ssh_write failed with error %s" % proc.returncode) else: print >> stderr, \ diff --git a/examples/pom.xml b/examples/pom.xml index 2b561857f9f33..eb49a0e5af22d 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -43,6 +43,11 @@ spark-streaming-kinesis-asl_${scala.binary.version} ${project.version} + + org.apache.httpcomponents + httpclient + ${commons.httpclient.version} + diff --git a/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java b/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java index 11157d7573fae..0f07cb4098325 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java @@ -31,7 +31,6 @@ * Usage: JavaSparkPi [slices] */ public final class JavaSparkPi { - public static void main(String[] args) throws Exception { SparkConf sparkConf = new SparkConf().setAppName("JavaSparkPi"); @@ -61,5 +60,7 @@ public Integer call(Integer integer, Integer integer2) { }); System.out.println("Pi is roughly " + 4.0 * count / n); + + jsc.stop(); } } 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 898297dc658ba..01c77bd44337e 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 @@ -61,7 +61,8 @@ public static void main(String[] args) throws Exception { // Load a text file and convert each line to a Java Bean. JavaRDD people = ctx.textFile("examples/src/main/resources/people.txt").map( new Function() { - public Person call(String line) throws Exception { + @Override + public Person call(String line) { String[] parts = line.split(","); Person person = new Person(); @@ -82,6 +83,7 @@ public Person call(String line) throws Exception { // The results of SQL queries are SchemaRDDs and support all the normal RDD operations. // The columns of a row in the result can be accessed by ordinal. List teenagerNames = teenagers.map(new Function() { + @Override public String call(Row row) { return "Name: " + row.getString(0); } @@ -104,6 +106,7 @@ public String call(Row row) { JavaSchemaRDD teenagers2 = sqlCtx.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19"); teenagerNames = teenagers2.map(new Function() { + @Override public String call(Row row) { return "Name: " + row.getString(0); } @@ -136,6 +139,7 @@ public String call(Row row) { // The results of SQL queries are JavaSchemaRDDs and support all the normal RDD operations. // The columns of a row in the result can be accessed by ordinal. teenagerNames = teenagers3.map(new Function() { + @Override public String call(Row row) { return "Name: " + row.getString(0); } }).collect(); for (String name: teenagerNames) { @@ -162,6 +166,7 @@ public String call(Row row) { JavaSchemaRDD peopleWithCity = sqlCtx.sql("SELECT name, address.city FROM people2"); List nameAndCity = peopleWithCity.map(new Function() { + @Override public String call(Row row) { return "Name: " + row.getString(0) + ", City: " + row.getString(1); } @@ -169,5 +174,7 @@ public String call(Row row) { for (String name: nameAndCity) { System.out.println(name); } + + ctx.stop(); } } diff --git a/examples/src/main/python/als.py b/examples/src/main/python/als.py index 5b1fa4d997eeb..70b6146e39a87 100755 --- a/examples/src/main/python/als.py +++ b/examples/src/main/python/als.py @@ -54,7 +54,7 @@ def update(i, vec, mat, ratings): if __name__ == "__main__": """ - Usage: als [M] [U] [F] [iterations] [slices]" + Usage: als [M] [U] [F] [iterations] [partitions]" """ print >> sys.stderr, """WARN: This is a naive implementation of ALS and is given as an @@ -66,10 +66,10 @@ def update(i, vec, mat, ratings): U = int(sys.argv[2]) if len(sys.argv) > 2 else 500 F = int(sys.argv[3]) if len(sys.argv) > 3 else 10 ITERATIONS = int(sys.argv[4]) if len(sys.argv) > 4 else 5 - slices = int(sys.argv[5]) if len(sys.argv) > 5 else 2 + partitions = int(sys.argv[5]) if len(sys.argv) > 5 else 2 - print "Running ALS with M=%d, U=%d, F=%d, iters=%d, slices=%d\n" % \ - (M, U, F, ITERATIONS, slices) + print "Running ALS with M=%d, U=%d, F=%d, iters=%d, partitions=%d\n" % \ + (M, U, F, ITERATIONS, partitions) R = matrix(rand(M, F)) * matrix(rand(U, F).T) ms = matrix(rand(M, F)) @@ -80,7 +80,7 @@ def update(i, vec, mat, ratings): usb = sc.broadcast(us) for i in range(ITERATIONS): - ms = sc.parallelize(range(M), slices) \ + ms = sc.parallelize(range(M), partitions) \ .map(lambda x: update(x, msb.value[x, :], usb.value, Rb.value)) \ .collect() # collect() returns a list, so array ends up being @@ -88,7 +88,7 @@ def update(i, vec, mat, ratings): ms = matrix(np.array(ms)[:, :, 0]) msb = sc.broadcast(ms) - us = sc.parallelize(range(U), slices) \ + us = sc.parallelize(range(U), partitions) \ .map(lambda x: update(x, usb.value[x, :], msb.value, Rb.value.T)) \ .collect() us = matrix(np.array(us)[:, :, 0]) diff --git a/examples/src/main/python/avro_inputformat.py b/examples/src/main/python/avro_inputformat.py index cfda8d8327aa3..4626bbb7e3b02 100644 --- a/examples/src/main/python/avro_inputformat.py +++ b/examples/src/main/python/avro_inputformat.py @@ -78,3 +78,5 @@ output = avro_rdd.map(lambda x: x[0]).collect() for k in output: print k + + sc.stop() diff --git a/examples/src/main/python/parquet_inputformat.py b/examples/src/main/python/parquet_inputformat.py new file mode 100644 index 0000000000000..fa4c20ab20281 --- /dev/null +++ b/examples/src/main/python/parquet_inputformat.py @@ -0,0 +1,61 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import sys + +from pyspark import SparkContext + +""" +Read data file users.parquet in local Spark distro: + +$ cd $SPARK_HOME +$ export AVRO_PARQUET_JARS=/path/to/parquet-avro-1.5.0.jar +$ ./bin/spark-submit --driver-class-path /path/to/example/jar \\ + --jars $AVRO_PARQUET_JARS \\ + ./examples/src/main/python/parquet_inputformat.py \\ + examples/src/main/resources/users.parquet +<...lots of log output...> +{u'favorite_color': None, u'name': u'Alyssa', u'favorite_numbers': [3, 9, 15, 20]} +{u'favorite_color': u'red', u'name': u'Ben', u'favorite_numbers': []} +<...more log output...> +""" +if __name__ == "__main__": + if len(sys.argv) != 2: + print >> sys.stderr, """ + Usage: parquet_inputformat.py + + Run with example jar: + ./bin/spark-submit --driver-class-path /path/to/example/jar \\ + /path/to/examples/parquet_inputformat.py + Assumes you have Parquet data stored in . + """ + exit(-1) + + path = sys.argv[1] + sc = SparkContext(appName="ParquetInputFormat") + + parquet_rdd = sc.newAPIHadoopFile( + path, + 'parquet.avro.AvroParquetInputFormat', + 'java.lang.Void', + 'org.apache.avro.generic.IndexedRecord', + valueConverter='org.apache.spark.examples.pythonconverters.IndexedRecordToJavaConverter') + output = parquet_rdd.map(lambda x: x[1]).collect() + for k in output: + print k + + sc.stop() diff --git a/examples/src/main/python/pi.py b/examples/src/main/python/pi.py index ee9036adfa281..a7c74e969cdb9 100755 --- a/examples/src/main/python/pi.py +++ b/examples/src/main/python/pi.py @@ -24,18 +24,18 @@ if __name__ == "__main__": """ - Usage: pi [slices] + Usage: pi [partitions] """ sc = SparkContext(appName="PythonPi") - slices = int(sys.argv[1]) if len(sys.argv) > 1 else 2 - n = 100000 * slices + partitions = int(sys.argv[1]) if len(sys.argv) > 1 else 2 + n = 100000 * partitions def f(_): x = random() * 2 - 1 y = random() * 2 - 1 return 1 if x ** 2 + y ** 2 < 1 else 0 - count = sc.parallelize(xrange(1, n + 1), slices).map(f).reduce(add) + count = sc.parallelize(xrange(1, n + 1), partitions).map(f).reduce(add) print "Pi is roughly %f" % (4.0 * count / n) sc.stop() diff --git a/examples/src/main/python/sql.py b/examples/src/main/python/sql.py new file mode 100644 index 0000000000000..eefa022f1927c --- /dev/null +++ b/examples/src/main/python/sql.py @@ -0,0 +1,73 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import os + +from pyspark import SparkContext +from pyspark.sql import SQLContext +from pyspark.sql import Row, StructField, StructType, StringType, IntegerType + + +if __name__ == "__main__": + sc = SparkContext(appName="PythonSQL") + sqlContext = SQLContext(sc) + + # RDD is created from a list of rows + some_rdd = sc.parallelize([Row(name="John", age=19), + Row(name="Smith", age=23), + Row(name="Sarah", age=18)]) + # Infer schema from the first row, create a SchemaRDD and print the schema + some_schemardd = sqlContext.inferSchema(some_rdd) + some_schemardd.printSchema() + + # Another RDD is created from a list of tuples + another_rdd = sc.parallelize([("John", 19), ("Smith", 23), ("Sarah", 18)]) + # Schema with two fields - person_name and person_age + schema = StructType([StructField("person_name", StringType(), False), + StructField("person_age", IntegerType(), False)]) + # Create a SchemaRDD by applying the schema to the RDD and print the schema + another_schemardd = sqlContext.applySchema(another_rdd, schema) + another_schemardd.printSchema() + # root + # |-- age: integer (nullable = true) + # |-- name: string (nullable = true) + + # A JSON dataset is pointed to by path. + # The path can be either a single text file or a directory storing text files. + path = os.environ['SPARK_HOME'] + "examples/src/main/resources/people.json" + # Create a SchemaRDD from the file(s) pointed to by path + people = sqlContext.jsonFile(path) + # root + # |-- person_name: string (nullable = false) + # |-- person_age: integer (nullable = false) + + # The inferred schema can be visualized using the printSchema() method. + people.printSchema() + # root + # |-- age: IntegerType + # |-- name: StringType + + # Register this SchemaRDD as a table. + people.registerAsTable("people") + + # SQL statements can be run by using the sql methods provided by sqlContext + teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") + + for each in teenagers.collect(): + print each[0] + + sc.stop() diff --git a/examples/src/main/python/transitive_closure.py b/examples/src/main/python/transitive_closure.py index bf331b542c438..00a281bfb6506 100755 --- a/examples/src/main/python/transitive_closure.py +++ b/examples/src/main/python/transitive_closure.py @@ -37,11 +37,11 @@ def generateGraph(): if __name__ == "__main__": """ - Usage: transitive_closure [slices] + Usage: transitive_closure [partitions] """ sc = SparkContext(appName="PythonTransitiveClosure") - slices = int(sys.argv[1]) if len(sys.argv) > 1 else 2 - tc = sc.parallelize(generateGraph(), slices).cache() + partitions = int(sys.argv[1]) if len(sys.argv) > 1 else 2 + tc = sc.parallelize(generateGraph(), partitions).cache() # Linear transitive closure: each round grows paths by one edge, # by joining the graph's edges with the already-discovered paths. diff --git a/examples/src/main/resources/full_user.avsc b/examples/src/main/resources/full_user.avsc new file mode 100644 index 0000000000000..04e7ba2dca4f6 --- /dev/null +++ b/examples/src/main/resources/full_user.avsc @@ -0,0 +1 @@ +{"type": "record", "namespace": "example.avro", "name": "User", "fields": [{"type": "string", "name": "name"}, {"type": ["string", "null"], "name": "favorite_color"}, {"type": {"items": "int", "type": "array"}, "name": "favorite_numbers"}]} \ No newline at end of file diff --git a/examples/src/main/resources/users.parquet b/examples/src/main/resources/users.parquet new file mode 100644 index 0000000000000..aa527338c43a8 Binary files /dev/null and b/examples/src/main/resources/users.parquet differ diff --git a/examples/src/main/scala/org/apache/spark/examples/CassandraCQLTest.scala b/examples/src/main/scala/org/apache/spark/examples/CassandraCQLTest.scala index 71f53af68f4d3..11d5c92c5952d 100644 --- a/examples/src/main/scala/org/apache/spark/examples/CassandraCQLTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/CassandraCQLTest.scala @@ -136,5 +136,7 @@ object CassandraCQLTest { classOf[CqlOutputFormat], job.getConfiguration() ) + + sc.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/CassandraTest.scala b/examples/src/main/scala/org/apache/spark/examples/CassandraTest.scala index 91ba364a346a5..ec689474aecb0 100644 --- a/examples/src/main/scala/org/apache/spark/examples/CassandraTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/CassandraTest.scala @@ -126,6 +126,8 @@ object CassandraTest { } }.saveAsNewAPIHadoopFile("casDemo", classOf[ByteBuffer], classOf[List[Mutation]], classOf[ColumnFamilyOutputFormat], job.getConfiguration) + + sc.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala index efd91bb054981..15f6678648b29 100644 --- a/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala @@ -44,11 +44,11 @@ object GroupByTest { arr1(i) = (ranGen.nextInt(Int.MaxValue), byteArr) } arr1 - }.cache + }.cache() // Enforce that everything has been calculated and in cache - pairs1.count + pairs1.count() - println(pairs1.groupByKey(numReducers).count) + println(pairs1.groupByKey(numReducers).count()) sc.stop() } diff --git a/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala b/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala index 4c655b84fde2e..74620ad007d83 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala @@ -79,5 +79,7 @@ object LogQuery { .reduceByKey((a, b) => a.merge(b)) .collect().foreach{ case (user, query) => println("%s\t%s".format(user, query))} + + sc.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRank.scala b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRank.scala index 235c3bf820244..e4db3ec51313d 100644 --- a/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRank.scala +++ b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRank.scala @@ -21,7 +21,6 @@ import org.apache.spark._ import org.apache.spark.SparkContext._ import org.apache.spark.bagel._ -import org.apache.spark.bagel.Bagel._ import scala.xml.{XML,NodeSeq} @@ -78,9 +77,9 @@ object WikipediaPageRank { (id, new PRVertex(1.0 / numVertices, outEdges)) }) if (usePartitioner) { - vertices = vertices.partitionBy(new HashPartitioner(sc.defaultParallelism)).cache + vertices = vertices.partitionBy(new HashPartitioner(sc.defaultParallelism)).cache() } else { - vertices = vertices.cache + vertices = vertices.cache() } println("Done parsing input file.") @@ -100,7 +99,9 @@ object WikipediaPageRank { (result .filter { case (id, vertex) => vertex.value >= threshold } .map { case (id, vertex) => "%s\t%s\n".format(id, vertex.value) } - .collect.mkString) + .collect().mkString) println(top) + + sc.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/CosineSimilarity.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/CosineSimilarity.scala new file mode 100644 index 0000000000000..6a3b0241ced7f --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/CosineSimilarity.scala @@ -0,0 +1,107 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.mllib + +import scopt.OptionParser + +import org.apache.spark.SparkContext._ +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.linalg.distributed.{MatrixEntry, RowMatrix} +import org.apache.spark.{SparkConf, SparkContext} + +/** + * Compute the similar columns of a matrix, using cosine similarity. + * + * The input matrix must be stored in row-oriented dense format, one line per row with its entries + * separated by space. For example, + * {{{ + * 0.5 1.0 + * 2.0 3.0 + * 4.0 5.0 + * }}} + * represents a 3-by-2 matrix, whose first row is (0.5, 1.0). + * + * Example invocation: + * + * bin/run-example mllib.CosineSimilarity \ + * --threshold 0.1 data/mllib/sample_svm_data.txt + */ +object CosineSimilarity { + case class Params(inputFile: String = null, threshold: Double = 0.1) + + def main(args: Array[String]) { + val defaultParams = Params() + + val parser = new OptionParser[Params]("CosineSimilarity") { + head("CosineSimilarity: an example app.") + opt[Double]("threshold") + .required() + .text(s"threshold similarity: to tradeoff computation vs quality estimate") + .action((x, c) => c.copy(threshold = x)) + arg[String]("") + .required() + .text(s"input file, one row per line, space-separated") + .action((x, c) => c.copy(inputFile = x)) + note( + """ + |For example, the following command runs this app on a dataset: + | + | ./bin/spark-submit --class org.apache.spark.examples.mllib.CosineSimilarity \ + | examplesjar.jar \ + | --threshold 0.1 data/mllib/sample_svm_data.txt + """.stripMargin) + } + + parser.parse(args, defaultParams).map { params => + run(params) + } getOrElse { + System.exit(1) + } + } + + def run(params: Params) { + val conf = new SparkConf().setAppName("CosineSimilarity") + val sc = new SparkContext(conf) + + // Load and parse the data file. + val rows = sc.textFile(params.inputFile).map { line => + val values = line.split(' ').map(_.toDouble) + Vectors.dense(values) + }.cache() + val mat = new RowMatrix(rows) + + // Compute similar columns perfectly, with brute force. + val exact = mat.columnSimilarities() + + // Compute similar columns with estimation using DIMSUM + val approx = mat.columnSimilarities(params.threshold) + + val exactEntries = exact.entries.map { case MatrixEntry(i, j, u) => ((i, j), u) } + val approxEntries = approx.entries.map { case MatrixEntry(i, j, v) => ((i, j), v) } + val MAE = exactEntries.leftOuterJoin(approxEntries).values.map { + case (u, Some(v)) => + math.abs(u - v) + case (u, None) => + math.abs(u) + }.mean() + + println(s"Average absolute error in estimate is: $MAE") + + sc.stop() + } +} diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala index 4683e6eb966be..4adc91d2fbe65 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala @@ -21,16 +21,18 @@ import scopt.OptionParser import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.SparkContext._ +import org.apache.spark.mllib.evaluation.MulticlassMetrics import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.tree.{DecisionTree, impurity} +import org.apache.spark.mllib.tree.{RandomForest, DecisionTree, impurity} import org.apache.spark.mllib.tree.configuration.{Algo, Strategy} import org.apache.spark.mllib.tree.configuration.Algo._ -import org.apache.spark.mllib.tree.model.DecisionTreeModel +import org.apache.spark.mllib.tree.model.{RandomForestModel, DecisionTreeModel} import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD +import org.apache.spark.util.Utils /** - * An example runner for decision tree. Run with + * An example runner for decision trees and random forests. Run with * {{{ * ./bin/run-example org.apache.spark.examples.mllib.DecisionTreeRunner [options] * }}} @@ -50,6 +52,7 @@ object DecisionTreeRunner { case class Params( input: String = null, + testInput: String = "", dataFormat: String = "libsvm", algo: Algo = Classification, maxDepth: Int = 5, @@ -57,6 +60,8 @@ object DecisionTreeRunner { maxBins: Int = 32, minInstancesPerNode: Int = 1, minInfoGain: Double = 0.0, + numTrees: Int = 1, + featureSubsetStrategy: String = "auto", fracTest: Double = 0.2) def main(args: Array[String]) { @@ -79,19 +84,33 @@ object DecisionTreeRunner { .action((x, c) => c.copy(maxBins = x)) opt[Int]("minInstancesPerNode") .text(s"min number of instances required at child nodes to create the parent split," + - s" default: ${defaultParams.minInstancesPerNode}") + s" default: ${defaultParams.minInstancesPerNode}") .action((x, c) => c.copy(minInstancesPerNode = x)) opt[Double]("minInfoGain") .text(s"min info gain required to create a split, default: ${defaultParams.minInfoGain}") .action((x, c) => c.copy(minInfoGain = x)) + opt[Int]("numTrees") + .text(s"number of trees (1 = decision tree, 2+ = random forest)," + + s" default: ${defaultParams.numTrees}") + .action((x, c) => c.copy(numTrees = x)) + opt[String]("featureSubsetStrategy") + .text(s"feature subset sampling strategy" + + s" (${RandomForest.supportedFeatureSubsetStrategies.mkString(", ")}}), " + + s"default: ${defaultParams.featureSubsetStrategy}") + .action((x, c) => c.copy(featureSubsetStrategy = x)) opt[Double]("fracTest") - .text(s"fraction of data to hold out for testing, default: ${defaultParams.fracTest}") + .text(s"fraction of data to hold out for testing. If given option testInput, " + + s"this option is ignored. default: ${defaultParams.fracTest}") .action((x, c) => c.copy(fracTest = x)) + opt[String]("testInput") + .text(s"input path to test dataset. If given, option fracTest is ignored." + + s" default: ${defaultParams.testInput}") + .action((x, c) => c.copy(testInput = x)) opt[String]("") .text("data format: libsvm (default), dense (deprecated in Spark v1.1)") .action((x, c) => c.copy(dataFormat = x)) arg[String]("") - .text("input paths to labeled examples in dense format (label,f0 f1 f2 ...)") + .text("input path to labeled examples") .required() .action((x, c) => c.copy(input = x)) checkConfig { params => @@ -128,7 +147,7 @@ object DecisionTreeRunner { case "libsvm" => MLUtils.loadLibSVMFile(sc, params.input).cache() } // For classification, re-index classes if needed. - val (examples, numClasses) = params.algo match { + val (examples, classIndexMap, numClasses) = params.algo match { case Classification => { // classCounts: class --> # examples in class val classCounts = origExamples.map(_.label).countByValue() @@ -157,16 +176,40 @@ object DecisionTreeRunner { val frac = classCounts(c) / numExamples.toDouble println(s"$c\t$frac\t${classCounts(c)}") } - (examples, numClasses) + (examples, classIndexMap, numClasses) } case Regression => - (origExamples, 0) + (origExamples, null, 0) case _ => throw new IllegalArgumentException("Algo ${params.algo} not supported.") } - // Split into training, test. - val splits = examples.randomSplit(Array(1.0 - params.fracTest, params.fracTest)) + // Create training, test sets. + val splits = if (params.testInput != "") { + // Load testInput. + val origTestExamples = params.dataFormat match { + case "dense" => MLUtils.loadLabeledPoints(sc, params.testInput) + case "libsvm" => MLUtils.loadLibSVMFile(sc, params.testInput) + } + params.algo match { + case Classification => { + // classCounts: class --> # examples in class + val testExamples = { + if (classIndexMap.isEmpty) { + origTestExamples + } else { + origTestExamples.map(lp => LabeledPoint(classIndexMap(lp.label), lp.features)) + } + } + Array(examples, testExamples) + } + case Regression => + Array(examples, origTestExamples) + } + } else { + // Split input into training, test. + examples.randomSplit(Array(1.0 - params.fracTest, params.fracTest)) + } val training = splits(0).cache() val test = splits(1).cache() val numTraining = training.count() @@ -191,38 +234,78 @@ object DecisionTreeRunner { numClassesForClassification = numClasses, minInstancesPerNode = params.minInstancesPerNode, minInfoGain = params.minInfoGain) - val model = DecisionTree.train(training, strategy) - - println(model) - - if (params.algo == Classification) { - val accuracy = accuracyScore(model, test) - println(s"Test accuracy = $accuracy") - } - - if (params.algo == Regression) { - val mse = meanSquaredError(model, test) - println(s"Test mean squared error = $mse") + if (params.numTrees == 1) { + val model = DecisionTree.train(training, strategy) + if (model.numNodes < 20) { + println(model.toDebugString) // Print full model. + } else { + println(model) // Print model summary. + } + if (params.algo == Classification) { + val trainAccuracy = + new MulticlassMetrics(training.map(lp => (model.predict(lp.features), lp.label))) + .precision + println(s"Train accuracy = $trainAccuracy") + val testAccuracy = + new MulticlassMetrics(test.map(lp => (model.predict(lp.features), lp.label))).precision + println(s"Test accuracy = $testAccuracy") + } + if (params.algo == Regression) { + val trainMSE = meanSquaredError(model, training) + println(s"Train mean squared error = $trainMSE") + val testMSE = meanSquaredError(model, test) + println(s"Test mean squared error = $testMSE") + } + } else { + val randomSeed = Utils.random.nextInt() + if (params.algo == Classification) { + val model = RandomForest.trainClassifier(training, strategy, params.numTrees, + params.featureSubsetStrategy, randomSeed) + if (model.totalNumNodes < 30) { + println(model.toDebugString) // Print full model. + } else { + println(model) // Print model summary. + } + val trainAccuracy = + new MulticlassMetrics(training.map(lp => (model.predict(lp.features), lp.label))) + .precision + println(s"Train accuracy = $trainAccuracy") + val testAccuracy = + new MulticlassMetrics(test.map(lp => (model.predict(lp.features), lp.label))).precision + println(s"Test accuracy = $testAccuracy") + } + if (params.algo == Regression) { + val model = RandomForest.trainRegressor(training, strategy, params.numTrees, + params.featureSubsetStrategy, randomSeed) + if (model.totalNumNodes < 30) { + println(model.toDebugString) // Print full model. + } else { + println(model) // Print model summary. + } + val trainMSE = meanSquaredError(model, training) + println(s"Train mean squared error = $trainMSE") + val testMSE = meanSquaredError(model, test) + println(s"Test mean squared error = $testMSE") + } } sc.stop() } /** - * Calculates the classifier accuracy. + * Calculates the mean squared error for regression. */ - private def accuracyScore( - model: DecisionTreeModel, - data: RDD[LabeledPoint]): Double = { - val correctCount = data.filter(y => model.predict(y.features) == y.label).count() - val count = data.count() - correctCount.toDouble / count + private def meanSquaredError(tree: DecisionTreeModel, data: RDD[LabeledPoint]): Double = { + data.map { y => + val err = tree.predict(y.features) - y.label + err * err + }.mean() } /** * Calculates the mean squared error for regression. */ - private def meanSquaredError(tree: DecisionTreeModel, data: RDD[LabeledPoint]): Double = { + private def meanSquaredError(tree: RandomForestModel, data: RDD[LabeledPoint]): Double = { data.map { y => val err = tree.predict(y.features) - y.label err * err diff --git a/examples/src/main/scala/org/apache/spark/examples/pythonconverters/AvroConverters.scala b/examples/src/main/scala/org/apache/spark/examples/pythonconverters/AvroConverters.scala index 1b25983a38453..a11890d6f2b1c 100644 --- a/examples/src/main/scala/org/apache/spark/examples/pythonconverters/AvroConverters.scala +++ b/examples/src/main/scala/org/apache/spark/examples/pythonconverters/AvroConverters.scala @@ -30,21 +30,28 @@ import org.apache.spark.api.python.Converter import org.apache.spark.SparkException -/** - * Implementation of [[org.apache.spark.api.python.Converter]] that converts - * an Avro Record wrapped in an AvroKey (or AvroValue) to a Java Map. It tries - * to work with all 3 Avro data mappings (Generic, Specific and Reflect). - */ -class AvroWrapperToJavaConverter extends Converter[Any, Any] { - override def convert(obj: Any): Any = { +object AvroConversionUtil extends Serializable { + def fromAvro(obj: Any, schema: Schema): Any = { if (obj == null) { return null } - obj.asInstanceOf[AvroWrapper[_]].datum() match { - case null => null - case record: IndexedRecord => unpackRecord(record) - case other => throw new SparkException( - s"Unsupported top-level Avro data type ${other.getClass.getName}") + schema.getType match { + case UNION => unpackUnion(obj, schema) + case ARRAY => unpackArray(obj, schema) + case FIXED => unpackFixed(obj, schema) + case MAP => unpackMap(obj, schema) + case BYTES => unpackBytes(obj) + case RECORD => unpackRecord(obj) + case STRING => obj.toString + case ENUM => obj.toString + case NULL => obj + case BOOLEAN => obj + case DOUBLE => obj + case FLOAT => obj + case INT => obj + case LONG => obj + case other => throw new SparkException( + s"Unknown Avro schema type ${other.getName}") } } @@ -103,28 +110,37 @@ class AvroWrapperToJavaConverter extends Converter[Any, Any] { "Unions may only consist of a concrete type and null") } } +} - def fromAvro(obj: Any, schema: Schema): Any = { +/** + * Implementation of [[org.apache.spark.api.python.Converter]] that converts + * an Avro IndexedRecord (e.g., derived from AvroParquetInputFormat) to a Java Map. + */ +class IndexedRecordToJavaConverter extends Converter[IndexedRecord, JMap[String, Any]]{ + override def convert(record: IndexedRecord): JMap[String, Any] = { + if (record == null) { + return null + } + val map = new java.util.HashMap[String, Any] + AvroConversionUtil.unpackRecord(record) + } +} + +/** + * Implementation of [[org.apache.spark.api.python.Converter]] that converts + * an Avro Record wrapped in an AvroKey (or AvroValue) to a Java Map. It tries + * to work with all 3 Avro data mappings (Generic, Specific and Reflect). + */ +class AvroWrapperToJavaConverter extends Converter[Any, Any] { + override def convert(obj: Any): Any = { if (obj == null) { return null } - schema.getType match { - case UNION => unpackUnion(obj, schema) - case ARRAY => unpackArray(obj, schema) - case FIXED => unpackFixed(obj, schema) - case MAP => unpackMap(obj, schema) - case BYTES => unpackBytes(obj) - case RECORD => unpackRecord(obj) - case STRING => obj.toString - case ENUM => obj.toString - case NULL => obj - case BOOLEAN => obj - case DOUBLE => obj - case FLOAT => obj - case INT => obj - case LONG => obj - case other => throw new SparkException( - s"Unknown Avro schema type ${other.getName}") + obj.asInstanceOf[AvroWrapper[_]].datum() match { + case null => null + case record: IndexedRecord => AvroConversionUtil.unpackRecord(record) + case other => throw new SparkException( + s"Unsupported top-level Avro data type ${other.getClass.getName}") } } } diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala b/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala index d56d64c564200..2e98b2dc30b80 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala @@ -51,7 +51,7 @@ object RDDRelation { val rddFromSql = sql("SELECT key, value FROM records WHERE key < 10") println("Result of RDD.map:") - rddFromSql.map(row => s"Key: ${row(0)}, Value: ${row(1)}").collect.foreach(println) + rddFromSql.map(row => s"Key: ${row(0)}, Value: ${row(1)}").collect().foreach(println) // Queries can also be written using a LINQ-like Scala DSL. rdd.where('key === 1).orderBy('value.asc).select('key).collect().foreach(println) @@ -68,5 +68,7 @@ object RDDRelation { // These files can also be registered as tables. parquetFile.registerTempTable("parquetFile") sql("SELECT * FROM parquetFile").collect().foreach(println) + + sc.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala index 3423fac0ad303..e26f213e8afa8 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala @@ -39,7 +39,7 @@ object HiveFromSpark { // Queries are expressed in HiveQL println("Result of 'SELECT *': ") - sql("SELECT * FROM src").collect.foreach(println) + sql("SELECT * FROM src").collect().foreach(println) // Aggregation queries are also supported. val count = sql("SELECT COUNT(*) FROM src").collect().head.getLong(0) @@ -61,5 +61,7 @@ object HiveFromSpark { // Queries can then join RDD data with data stored in Hive. println("Result of SELECT *:") sql("SELECT * FROM records r JOIN src s ON r.key = s.key").collect().foreach(println) + + sc.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala index 566ba6f911e02..c9e1511278ede 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala @@ -53,8 +53,8 @@ object KafkaWordCount { val ssc = new StreamingContext(sparkConf, Seconds(2)) ssc.checkpoint("checkpoint") - val topicpMap = topics.split(",").map((_,numThreads.toInt)).toMap - val lines = KafkaUtils.createStream(ssc, zkQuorum, group, topicpMap).map(_._2) + val topicMap = topics.split(",").map((_,numThreads.toInt)).toMap + val lines = KafkaUtils.createStream(ssc, zkQuorum, group, topicMap).map(_._2) val words = lines.flatMap(_.split(" ")) val wordCounts = words.map(x => (x, 1L)) .reduceByKeyAndWindow(_ + _, _ - _, Minutes(10), Seconds(2), 2) diff --git a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkAvroCallbackHandler.scala b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkAvroCallbackHandler.scala index e77cf7bfa54d0..3c656a381bd9b 100644 --- a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkAvroCallbackHandler.scala +++ b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkAvroCallbackHandler.scala @@ -16,7 +16,7 @@ */ package org.apache.spark.streaming.flume.sink -import java.util.concurrent.{ConcurrentHashMap, Executors} +import java.util.concurrent.{CountDownLatch, ConcurrentHashMap, Executors} import java.util.concurrent.atomic.AtomicLong import scala.collection.JavaConversions._ @@ -58,8 +58,12 @@ private[flume] class SparkAvroCallbackHandler(val threads: Int, val channel: Cha private val seqBase = RandomStringUtils.randomAlphanumeric(8) private val seqCounter = new AtomicLong(0) + @volatile private var stopped = false + @volatile private var isTest = false + private var testLatch: CountDownLatch = null + /** * Returns a bunch of events to Spark over Avro RPC. * @param n Maximum number of events to return in a batch @@ -90,6 +94,9 @@ private[flume] class SparkAvroCallbackHandler(val threads: Int, val channel: Cha val processor = new TransactionProcessor( channel, seq, n, transactionTimeout, backOffInterval, this) sequenceNumberToProcessor.put(seq, processor) + if (isTest) { + processor.countDownWhenBatchAcked(testLatch) + } Some(processor) } else { None @@ -141,6 +148,11 @@ private[flume] class SparkAvroCallbackHandler(val threads: Int, val channel: Cha } } + private[sink] def countDownWhenBatchAcked(latch: CountDownLatch) { + testLatch = latch + isTest = true + } + /** * Shuts down the executor used to process transactions. */ diff --git a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSink.scala b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSink.scala index 98ae7d783aec8..14dffb15fef98 100644 --- a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSink.scala +++ b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSink.scala @@ -138,6 +138,16 @@ class SparkSink extends AbstractSink with Logging with Configurable { throw new RuntimeException("Server was not started!") ) } + + /** + * Pass in a [[CountDownLatch]] for testing purposes. This batch is counted down when each + * batch is received. The test can simply call await on this latch till the expected number of + * batches are received. + * @param latch + */ + private[flume] def countdownWhenBatchReceived(latch: CountDownLatch) { + handler.foreach(_.countDownWhenBatchAcked(latch)) + } } /** diff --git a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/TransactionProcessor.scala b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/TransactionProcessor.scala index 13f3aa94be414..ea45b14294df9 100644 --- a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/TransactionProcessor.scala +++ b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/TransactionProcessor.scala @@ -62,6 +62,10 @@ private class TransactionProcessor(val channel: Channel, val seqNum: String, @volatile private var stopped = false + @volatile private var isTest = false + + private var testLatch: CountDownLatch = null + // The transaction that this processor would handle var txOpt: Option[Transaction] = None @@ -182,6 +186,9 @@ private class TransactionProcessor(val channel: Channel, val seqNum: String, rollbackAndClose(tx, close = false) // tx will be closed later anyway } finally { tx.close() + if (isTest) { + testLatch.countDown() + } } } else { logWarning("Spark could not commit transaction, NACK received. Rolling back transaction.") @@ -237,4 +244,9 @@ private class TransactionProcessor(val channel: Channel, val seqNum: String, processAckOrNack() null } + + private[sink] def countDownWhenBatchAcked(latch: CountDownLatch) { + testLatch = latch + isTest = true + } } diff --git a/external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala b/external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala index 75a6668c6210b..a2b2cc6149d95 100644 --- a/external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala +++ b/external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala @@ -38,7 +38,7 @@ class SparkSinkSuite extends FunSuite { val channelCapacity = 5000 test("Success with ack") { - val (channel, sink) = initializeChannelAndSink() + val (channel, sink, latch) = initializeChannelAndSink() channel.start() sink.start() @@ -51,6 +51,7 @@ class SparkSinkSuite extends FunSuite { val events = client.getEventBatch(1000) client.ack(events.getSequenceNumber) assert(events.getEvents.size() === 1000) + latch.await(1, TimeUnit.SECONDS) assertChannelIsEmpty(channel) sink.stop() channel.stop() @@ -58,7 +59,7 @@ class SparkSinkSuite extends FunSuite { } test("Failure with nack") { - val (channel, sink) = initializeChannelAndSink() + val (channel, sink, latch) = initializeChannelAndSink() channel.start() sink.start() putEvents(channel, eventsPerBatch) @@ -70,6 +71,7 @@ class SparkSinkSuite extends FunSuite { val events = client.getEventBatch(1000) assert(events.getEvents.size() === 1000) client.nack(events.getSequenceNumber) + latch.await(1, TimeUnit.SECONDS) assert(availableChannelSlots(channel) === 4000) sink.stop() channel.stop() @@ -77,7 +79,7 @@ class SparkSinkSuite extends FunSuite { } test("Failure with timeout") { - val (channel, sink) = initializeChannelAndSink(Map(SparkSinkConfig + val (channel, sink, latch) = initializeChannelAndSink(Map(SparkSinkConfig .CONF_TRANSACTION_TIMEOUT -> 1.toString)) channel.start() sink.start() @@ -88,7 +90,7 @@ class SparkSinkSuite extends FunSuite { val (transceiver, client) = getTransceiverAndClient(address, 1)(0) val events = client.getEventBatch(1000) assert(events.getEvents.size() === 1000) - Thread.sleep(1000) + latch.await(1, TimeUnit.SECONDS) assert(availableChannelSlots(channel) === 4000) sink.stop() channel.stop() @@ -106,7 +108,7 @@ class SparkSinkSuite extends FunSuite { def testMultipleConsumers(failSome: Boolean): Unit = { implicit val executorContext = ExecutionContext .fromExecutorService(Executors.newFixedThreadPool(5)) - val (channel, sink) = initializeChannelAndSink() + val (channel, sink, latch) = initializeChannelAndSink(Map.empty, 5) channel.start() sink.start() (1 to 5).foreach(_ => putEvents(channel, eventsPerBatch)) @@ -136,7 +138,7 @@ class SparkSinkSuite extends FunSuite { } }) batchCounter.await() - TimeUnit.SECONDS.sleep(1) // Allow the sink to commit the transactions. + latch.await(1, TimeUnit.SECONDS) executorContext.shutdown() if(failSome) { assert(availableChannelSlots(channel) === 3000) @@ -148,8 +150,8 @@ class SparkSinkSuite extends FunSuite { transceiversAndClients.foreach(x => x._1.close()) } - private def initializeChannelAndSink(overrides: Map[String, String] = Map.empty): (MemoryChannel, - SparkSink) = { + private def initializeChannelAndSink(overrides: Map[String, String] = Map.empty, + batchCounter: Int = 1): (MemoryChannel, SparkSink, CountDownLatch) = { val channel = new MemoryChannel() val channelContext = new Context() @@ -165,7 +167,9 @@ class SparkSinkSuite extends FunSuite { sinkContext.put(SparkSinkConfig.CONF_PORT, 0.toString) sink.configure(sinkContext) sink.setChannel(channel) - (channel, sink) + val latch = new CountDownLatch(batchCounter) + sink.countdownWhenBatchReceived(latch) + (channel, sink, latch) } private def putEvents(ch: MemoryChannel, count: Int): Unit = { diff --git a/external/flume/src/test/resources/log4j.properties b/external/flume/src/test/resources/log4j.properties index 45d2ec676df66..4411d6e20c52a 100644 --- a/external/flume/src/test/resources/log4j.properties +++ b/external/flume/src/test/resources/log4j.properties @@ -22,7 +22,7 @@ log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false log4j.appender.file.file=target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout -log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose log4j.logger.org.eclipse.jetty=WARN diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala index 6ee7ac974b4a0..33235d150b4a5 100644 --- a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala +++ b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala @@ -31,7 +31,7 @@ import org.apache.flume.source.avro.{AvroFlumeEvent, AvroSourceProtocol} import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.{TestOutputStream, StreamingContext, TestSuiteBase} import org.apache.spark.streaming.util.ManualClock -import org.apache.spark.streaming.api.java.JavaReceiverInputDStream +import org.apache.spark.util.Utils import org.jboss.netty.channel.ChannelPipeline import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory @@ -41,21 +41,26 @@ import org.jboss.netty.handler.codec.compression._ class FlumeStreamSuite extends TestSuiteBase { test("flume input stream") { - runFlumeStreamTest(false, 9998) + runFlumeStreamTest(false) } test("flume input compressed stream") { - runFlumeStreamTest(true, 9997) + runFlumeStreamTest(true) } - def runFlumeStreamTest(enableDecompression: Boolean, testPort: Int) { + def runFlumeStreamTest(enableDecompression: Boolean) { // Set up the streaming context and input streams val ssc = new StreamingContext(conf, batchDuration) - val flumeStream: JavaReceiverInputDStream[SparkFlumeEvent] = - FlumeUtils.createStream(ssc, "localhost", testPort, StorageLevel.MEMORY_AND_DISK, enableDecompression) + val (flumeStream, testPort) = + Utils.startServiceOnPort(9997, (trialPort: Int) => { + val dstream = FlumeUtils.createStream( + ssc, "localhost", trialPort, StorageLevel.MEMORY_AND_DISK, enableDecompression) + (dstream, trialPort) + }) + val outputBuffer = new ArrayBuffer[Seq[SparkFlumeEvent]] with SynchronizedBuffer[Seq[SparkFlumeEvent]] - val outputStream = new TestOutputStream(flumeStream.receiverInputDStream, outputBuffer) + val outputStream = new TestOutputStream(flumeStream, outputBuffer) outputStream.register() ssc.start() @@ -63,13 +68,13 @@ class FlumeStreamSuite extends TestSuiteBase { val input = Seq(1, 2, 3, 4, 5) Thread.sleep(1000) val transceiver = new NettyTransceiver(new InetSocketAddress("localhost", testPort)) - var client: AvroSourceProtocol = null; - + var client: AvroSourceProtocol = null + if (enableDecompression) { client = SpecificRequestor.getClient( classOf[AvroSourceProtocol], new NettyTransceiver(new InetSocketAddress("localhost", testPort), - new CompressionChannelFactory(6))); + new CompressionChannelFactory(6))) } else { client = SpecificRequestor.getClient( classOf[AvroSourceProtocol], transceiver) diff --git a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java index 0571454c01dae..efb0099c7c850 100644 --- a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java +++ b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java @@ -81,7 +81,7 @@ public void testKafkaStream() throws InterruptedException { Predef.>conforms())); HashMap kafkaParams = new HashMap(); - kafkaParams.put("zookeeper.connect", testSuite.zkConnect()); + kafkaParams.put("zookeeper.connect", testSuite.zkHost() + ":" + testSuite.zkPort()); kafkaParams.put("group.id", "test-consumer-" + KafkaTestUtils.random().nextInt(10000)); kafkaParams.put("auto.offset.reset", "smallest"); diff --git a/external/kafka/src/test/resources/log4j.properties b/external/kafka/src/test/resources/log4j.properties index 45d2ec676df66..4411d6e20c52a 100644 --- a/external/kafka/src/test/resources/log4j.properties +++ b/external/kafka/src/test/resources/log4j.properties @@ -22,7 +22,7 @@ log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false log4j.appender.file.file=target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout -log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose log4j.logger.org.eclipse.jetty=WARN diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala index c0b55e9340253..6943326eb750e 100644 --- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala +++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala @@ -24,7 +24,7 @@ import java.util.{Properties, Random} import scala.collection.mutable import kafka.admin.CreateTopicCommand -import kafka.common.TopicAndPartition +import kafka.common.{KafkaException, TopicAndPartition} import kafka.producer.{KeyedMessage, ProducerConfig, Producer} import kafka.utils.ZKStringSerializer import kafka.serializer.{StringDecoder, StringEncoder} @@ -42,14 +42,13 @@ import org.apache.spark.util.Utils class KafkaStreamSuite extends TestSuiteBase { import KafkaTestUtils._ - val zkConnect = "localhost:2181" + val zkHost = "localhost" + var zkPort: Int = 0 val zkConnectionTimeout = 6000 val zkSessionTimeout = 6000 - val brokerPort = 9092 - val brokerProps = getBrokerConfig(brokerPort, zkConnect) - val brokerConf = new KafkaConfig(brokerProps) - + protected var brokerPort = 9092 + protected var brokerConf: KafkaConfig = _ protected var zookeeper: EmbeddedZookeeper = _ protected var zkClient: ZkClient = _ protected var server: KafkaServer = _ @@ -59,16 +58,35 @@ class KafkaStreamSuite extends TestSuiteBase { override def beforeFunction() { // Zookeeper server startup - zookeeper = new EmbeddedZookeeper(zkConnect) + zookeeper = new EmbeddedZookeeper(s"$zkHost:$zkPort") + // Get the actual zookeeper binding port + zkPort = zookeeper.actualPort logInfo("==================== 0 ====================") - zkClient = new ZkClient(zkConnect, zkSessionTimeout, zkConnectionTimeout, ZKStringSerializer) + + zkClient = new ZkClient(s"$zkHost:$zkPort", zkSessionTimeout, zkConnectionTimeout, + ZKStringSerializer) logInfo("==================== 1 ====================") // Kafka broker startup - server = new KafkaServer(brokerConf) - logInfo("==================== 2 ====================") - server.startup() - logInfo("==================== 3 ====================") + var bindSuccess: Boolean = false + while(!bindSuccess) { + try { + val brokerProps = getBrokerConfig(brokerPort, s"$zkHost:$zkPort") + brokerConf = new KafkaConfig(brokerProps) + server = new KafkaServer(brokerConf) + logInfo("==================== 2 ====================") + server.startup() + logInfo("==================== 3 ====================") + bindSuccess = true + } catch { + case e: KafkaException => + if (e.getMessage != null && e.getMessage.contains("Socket server failed to bind to")) { + brokerPort += 1 + } + case e: Exception => throw new Exception("Kafka server create failed", e) + } + } + Thread.sleep(2000) logInfo("==================== 4 ====================") super.beforeFunction() @@ -92,7 +110,7 @@ class KafkaStreamSuite extends TestSuiteBase { createTopic(topic) produceAndSendMessage(topic, sent) - val kafkaParams = Map("zookeeper.connect" -> zkConnect, + val kafkaParams = Map("zookeeper.connect" -> s"$zkHost:$zkPort", "group.id" -> s"test-consumer-${random.nextInt(10000)}", "auto.offset.reset" -> "smallest") @@ -200,6 +218,8 @@ object KafkaTestUtils { factory.configure(new InetSocketAddress(ip, port), 16) factory.startup(zookeeper) + val actualPort = factory.getLocalPort + def shutdown() { factory.shutdown() Utils.deleteRecursively(snapshotDir) diff --git a/external/mqtt/src/test/resources/log4j.properties b/external/mqtt/src/test/resources/log4j.properties index 45d2ec676df66..4411d6e20c52a 100644 --- a/external/mqtt/src/test/resources/log4j.properties +++ b/external/mqtt/src/test/resources/log4j.properties @@ -22,7 +22,7 @@ log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false log4j.appender.file.file=target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout -log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose log4j.logger.org.eclipse.jetty=WARN diff --git a/external/twitter/src/test/resources/log4j.properties b/external/twitter/src/test/resources/log4j.properties index 45d2ec676df66..4411d6e20c52a 100644 --- a/external/twitter/src/test/resources/log4j.properties +++ b/external/twitter/src/test/resources/log4j.properties @@ -22,7 +22,7 @@ log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false log4j.appender.file.file=target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout -log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose log4j.logger.org.eclipse.jetty=WARN diff --git a/external/zeromq/src/test/resources/log4j.properties b/external/zeromq/src/test/resources/log4j.properties index 45d2ec676df66..4411d6e20c52a 100644 --- a/external/zeromq/src/test/resources/log4j.properties +++ b/external/zeromq/src/test/resources/log4j.properties @@ -22,7 +22,7 @@ log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false log4j.appender.file.file=target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout -log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose log4j.logger.org.eclipse.jetty=WARN diff --git a/extras/java8-tests/src/test/resources/log4j.properties b/extras/java8-tests/src/test/resources/log4j.properties index 180beaa8cc5a7..bb0ab319a0080 100644 --- a/extras/java8-tests/src/test/resources/log4j.properties +++ b/extras/java8-tests/src/test/resources/log4j.properties @@ -21,7 +21,7 @@ log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false log4j.appender.file.file=target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout -log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose log4j.logger.org.eclipse.jetty=WARN diff --git a/extras/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java b/extras/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java index aa917d0575c4c..b0bff27a61c19 100644 --- a/extras/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java +++ b/extras/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java @@ -71,6 +71,9 @@ * org.apache.spark.examples.streaming.JavaKinesisWordCountASL mySparkStream \ * https://kinesis.us-east-1.amazonaws.com * + * Note that number of workers/threads should be 1 more than the number of receivers. + * This leaves one thread available for actually processing the data. + * * There is a companion helper class called KinesisWordCountProducerASL which puts dummy data * onto the Kinesis stream. * Usage instructions for KinesisWordCountProducerASL are provided in the class definition. @@ -114,12 +117,8 @@ public static void main(String[] args) { /* In this example, we're going to create 1 Kinesis Worker/Receiver/DStream for each shard */ int numStreams = numShards; - /* Must add 1 more thread than the number of receivers or the output won't show properly from the driver */ - int numSparkThreads = numStreams + 1; - /* Setup the Spark config. */ - SparkConf sparkConfig = new SparkConf().setAppName("KinesisWordCount").setMaster( - "local[" + numSparkThreads + "]"); + SparkConf sparkConfig = new SparkConf().setAppName("KinesisWordCount"); /* Kinesis checkpoint interval. Same as batchInterval for this example. */ Duration checkpointInterval = batchInterval; diff --git a/extras/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala b/extras/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala index fffd90de08240..32da0858d1a1d 100644 --- a/extras/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala +++ b/extras/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala @@ -65,6 +65,10 @@ import org.apache.log4j.Level * org.apache.spark.examples.streaming.KinesisWordCountASL mySparkStream \ * https://kinesis.us-east-1.amazonaws.com * + * + * Note that number of workers/threads should be 1 more than the number of receivers. + * This leaves one thread available for actually processing the data. + * * There is a companion helper class below called KinesisWordCountProducerASL which puts * dummy data onto the Kinesis stream. * Usage instructions for KinesisWordCountProducerASL are provided in that class definition. @@ -97,17 +101,10 @@ private object KinesisWordCountASL extends Logging { /* In this example, we're going to create 1 Kinesis Worker/Receiver/DStream for each shard. */ val numStreams = numShards - /* - * numSparkThreads should be 1 more thread than the number of receivers. - * This leaves one thread available for actually processing the data. - */ - val numSparkThreads = numStreams + 1 - /* Setup the and SparkConfig and StreamingContext */ /* Spark Streaming batch interval */ - val batchInterval = Milliseconds(2000) + val batchInterval = Milliseconds(2000) val sparkConfig = new SparkConf().setAppName("KinesisWordCount") - .setMaster(s"local[$numSparkThreads]") val ssc = new StreamingContext(sparkConfig, batchInterval) /* Kinesis checkpoint interval. Same as batchInterval for this example. */ diff --git a/extras/kinesis-asl/src/test/resources/log4j.properties b/extras/kinesis-asl/src/test/resources/log4j.properties index e01e049595475..d9d08f68687d3 100644 --- a/extras/kinesis-asl/src/test/resources/log4j.properties +++ b/extras/kinesis-asl/src/test/resources/log4j.properties @@ -20,7 +20,7 @@ log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false log4j.appender.file.file=target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout -log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose log4j.logger.org.eclipse.jetty=WARN diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala index 02afaa987d40d..d0dd45dba618e 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala @@ -254,7 +254,7 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Seriali * Picks a random vertex from the graph and returns its ID. */ def pickRandomVertex(): VertexId = { - val probability = 50 / graph.numVertices + val probability = 50.0 / graph.numVertices var found = false var retVal: VertexId = null.asInstanceOf[VertexId] while (!found) { 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 04fbc9dbab8d1..2c8b245955d12 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala @@ -392,7 +392,7 @@ object VertexRDD { */ def apply[VD: ClassTag]( vertices: RDD[(VertexId, VD)], edges: EdgeRDD[_, _], defaultVal: VD): VertexRDD[VD] = { - VertexRDD(vertices, edges, defaultVal, (a, b) => b) + VertexRDD(vertices, edges, defaultVal, (a, b) => a) } /** @@ -419,7 +419,7 @@ object VertexRDD { (vertexIter, routingTableIter) => val routingTable = if (routingTableIter.hasNext) routingTableIter.next() else RoutingTablePartition.empty - Iterator(ShippableVertexPartition(vertexIter, routingTable, defaultVal)) + Iterator(ShippableVertexPartition(vertexIter, routingTable, defaultVal, mergeFunc)) } new VertexRDD(vertexPartitions) } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala index dca54b8a7da86..5412d720475dc 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala @@ -36,7 +36,7 @@ private[graphx] object ShippableVertexPartition { /** Construct a `ShippableVertexPartition` from the given vertices without any routing table. */ def apply[VD: ClassTag](iter: Iterator[(VertexId, VD)]): ShippableVertexPartition[VD] = - apply(iter, RoutingTablePartition.empty, null.asInstanceOf[VD]) + apply(iter, RoutingTablePartition.empty, null.asInstanceOf[VD], (a, b) => a) /** * Construct a `ShippableVertexPartition` from the given vertices with the specified routing @@ -44,10 +44,28 @@ object ShippableVertexPartition { */ def apply[VD: ClassTag]( iter: Iterator[(VertexId, VD)], routingTable: RoutingTablePartition, defaultVal: VD) - : ShippableVertexPartition[VD] = { - val fullIter = iter ++ routingTable.iterator.map(vid => (vid, defaultVal)) - val (index, values, mask) = VertexPartitionBase.initFrom(fullIter, (a: VD, b: VD) => a) - new ShippableVertexPartition(index, values, mask, routingTable) + : ShippableVertexPartition[VD] = + apply(iter, routingTable, defaultVal, (a, b) => a) + + /** + * Construct a `ShippableVertexPartition` from the given vertices with the specified routing + * table, filling in missing vertices mentioned in the routing table using `defaultVal`, + * and merging duplicate vertex atrribute with mergeFunc. + */ + def apply[VD: ClassTag]( + iter: Iterator[(VertexId, VD)], routingTable: RoutingTablePartition, defaultVal: VD, + mergeFunc: (VD, VD) => VD): ShippableVertexPartition[VD] = { + val map = new GraphXPrimitiveKeyOpenHashMap[VertexId, VD] + // Merge the given vertices using mergeFunc + iter.foreach { pair => + map.setMerge(pair._1, pair._2, mergeFunc) + } + // Fill in missing vertices mentioned in the routing table + routingTable.iterator.foreach { vid => + map.changeValue(vid, defaultVal, identity) + } + + new ShippableVertexPartition(map.keySet, map._values, map.keySet.getBitSet, routingTable) } import scala.language.implicitConversions diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/StronglyConnectedComponents.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/StronglyConnectedComponents.scala index 46da38eeb725a..8dd958033b338 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/StronglyConnectedComponents.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/StronglyConnectedComponents.scala @@ -75,7 +75,7 @@ object StronglyConnectedComponents { sccWorkGraph, Long.MaxValue, activeDirection = EdgeDirection.Out)( (vid, myScc, neighborScc) => (math.min(myScc._1, neighborScc), myScc._2), e => { - if (e.srcId < e.dstId) { + if (e.srcAttr._1 < e.dstAttr._1) { Iterator((e.dstId, e.srcAttr._1)) } else { Iterator() diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala index b8309289fe475..8a13c74221546 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala @@ -118,7 +118,7 @@ object GraphGenerators { val Z = rand.nextGaussian() X = math.exp(mu + sigma*Z) } - math.round(X.toFloat) + math.floor(X).toInt } /** diff --git a/graphx/src/test/resources/log4j.properties b/graphx/src/test/resources/log4j.properties index 26b73a1b39744..9dd05f17f012b 100644 --- a/graphx/src/test/resources/log4j.properties +++ b/graphx/src/test/resources/log4j.properties @@ -21,7 +21,7 @@ log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false log4j.appender.file.file=target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout -log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose log4j.logger.org.eclipse.jetty=WARN diff --git a/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala index cc86bafd2d644..42d3f21dbae98 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala @@ -99,4 +99,15 @@ class VertexRDDSuite extends FunSuite with LocalSparkContext { } } + test("mergeFunc") { + // test to see if the mergeFunc is working correctly + withSpark { sc => + val verts = sc.parallelize(List((0L, 0), (1L, 1), (1L, 2), (2L, 3), (2L, 3), (2L, 3))) + val edges = EdgeRDD.fromEdges(sc.parallelize(List.empty[Edge[Int]])) + val rdd = VertexRDD(verts, edges, 0, (a: Int, b: Int) => a + b) + // test merge function + assert(rdd.collect.toSet == Set((0L, 0), (1L, 3), (2L, 9))) + } + } + } diff --git a/graphx/src/test/scala/org/apache/spark/graphx/util/GraphGeneratorsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/util/GraphGeneratorsSuite.scala index b346d4db2ef96..3abefbe52fa8a 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/util/GraphGeneratorsSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/util/GraphGeneratorsSuite.scala @@ -64,8 +64,11 @@ class GraphGeneratorsSuite extends FunSuite with LocalSparkContext { val sigma = 1.3 val maxVal = 100 - val dstId = GraphGenerators.sampleLogNormal(mu, sigma, maxVal) - assert(dstId < maxVal) + val trials = 1000 + for (i <- 1 to trials) { + val dstId = GraphGenerators.sampleLogNormal(mu, sigma, maxVal) + assert(dstId < maxVal) + } val dstId_round1 = GraphGenerators.sampleLogNormal(mu, sigma, maxVal, 12345) val dstId_round2 = GraphGenerators.sampleLogNormal(mu, sigma, maxVal, 12345) diff --git a/make-distribution.sh b/make-distribution.sh index 884659954a491..0bc839e1dbe4d 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -201,7 +201,6 @@ fi # Copy other things mkdir "$DISTDIR"/conf cp "$FWDIR"/conf/*.template "$DISTDIR"/conf -cp "$FWDIR"/conf/slaves "$DISTDIR"/conf cp "$FWDIR/README.md" "$DISTDIR" cp -r "$FWDIR/bin" "$DISTDIR" cp -r "$FWDIR/python" "$DISTDIR" diff --git a/mllib/pom.xml b/mllib/pom.xml index a5eeef88e9d62..696e9396f627c 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -57,7 +57,7 @@ org.scalanlp breeze_${scala.binary.version} - 0.9 + 0.10 diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index fa0fa69f38634..f7251e65e04f1 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -17,16 +17,22 @@ package org.apache.spark.mllib.api.python -import java.nio.{ByteBuffer, ByteOrder} +import java.io.OutputStream import scala.collection.JavaConverters._ +import scala.language.existentials +import scala.reflect.ClassTag + +import net.razorvine.pickle._ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} import org.apache.spark.mllib.classification._ import org.apache.spark.mllib.clustering._ +import org.apache.spark.mllib.feature.Word2Vec +import org.apache.spark.mllib.feature.Word2VecModel import org.apache.spark.mllib.optimization._ -import org.apache.spark.mllib.linalg.{Matrix, SparseVector, Vector, Vectors} +import org.apache.spark.mllib.linalg._ import org.apache.spark.mllib.random.{RandomRDDs => RG} import org.apache.spark.mllib.recommendation._ import org.apache.spark.mllib.regression._ @@ -38,13 +44,12 @@ import org.apache.spark.mllib.stat.{MultivariateStatisticalSummary, Statistics} import org.apache.spark.mllib.stat.correlation.CorrelationNames import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel import org.apache.spark.util.Utils /** * :: DeveloperApi :: * The Java stubs necessary for the Python mllib bindings. - * - * See python/pyspark/mllib/_common.py for the mutually agreed upon data format. */ @DeveloperApi class PythonMLLibAPI extends Serializable { @@ -60,18 +65,19 @@ class PythonMLLibAPI extends Serializable { def loadLabeledPoints( jsc: JavaSparkContext, path: String, - minPartitions: Int): JavaRDD[Array[Byte]] = - MLUtils.loadLabeledPoints(jsc.sc, path, minPartitions).map(SerDe.serializeLabeledPoint) + minPartitions: Int): JavaRDD[LabeledPoint] = + MLUtils.loadLabeledPoints(jsc.sc, path, minPartitions) private def trainRegressionModel( - trainFunc: (RDD[LabeledPoint], Vector) => GeneralizedLinearModel, - dataBytesJRDD: JavaRDD[Array[Byte]], + learner: GeneralizedLinearAlgorithm[_ <: GeneralizedLinearModel], + data: JavaRDD[LabeledPoint], initialWeightsBA: Array[Byte]): java.util.LinkedList[java.lang.Object] = { - val data = dataBytesJRDD.rdd.map(SerDe.deserializeLabeledPoint) - val initialWeights = SerDe.deserializeDoubleVector(initialWeightsBA) - val model = trainFunc(data, initialWeights) + val initialWeights = SerDe.loads(initialWeightsBA).asInstanceOf[Vector] + // Disable the uncached input warning because 'data' is a deliberately uncached MappedRDD. + learner.disableUncachedWarning() + val model = learner.run(data.rdd, initialWeights) val ret = new java.util.LinkedList[java.lang.Object]() - ret.add(SerDe.serializeDoubleVector(model.weights)) + ret.add(SerDe.dumps(model.weights)) ret.add(model.intercept: java.lang.Double) ret } @@ -80,7 +86,7 @@ class PythonMLLibAPI extends Serializable { * Java stub for Python mllib LinearRegressionWithSGD.train() */ def trainLinearRegressionModelWithSGD( - dataBytesJRDD: JavaRDD[Array[Byte]], + data: JavaRDD[LabeledPoint], numIterations: Int, stepSize: Double, miniBatchFraction: Double, @@ -104,9 +110,8 @@ class PythonMLLibAPI extends Serializable { + " Can only be initialized using the following string values: [l1, l2, none].") } trainRegressionModel( - (data, initialWeights) => - lrAlg.run(data, initialWeights), - dataBytesJRDD, + lrAlg, + data, initialWeightsBA) } @@ -114,22 +119,21 @@ class PythonMLLibAPI extends Serializable { * Java stub for Python mllib LassoWithSGD.train() */ def trainLassoModelWithSGD( - dataBytesJRDD: JavaRDD[Array[Byte]], + data: JavaRDD[LabeledPoint], numIterations: Int, stepSize: Double, regParam: Double, miniBatchFraction: Double, initialWeightsBA: Array[Byte]): java.util.List[java.lang.Object] = { + val lassoAlg = new LassoWithSGD() + lassoAlg.optimizer + .setNumIterations(numIterations) + .setRegParam(regParam) + .setStepSize(stepSize) + .setMiniBatchFraction(miniBatchFraction) trainRegressionModel( - (data, initialWeights) => - LassoWithSGD.train( - data, - numIterations, - stepSize, - regParam, - miniBatchFraction, - initialWeights), - dataBytesJRDD, + lassoAlg, + data, initialWeightsBA) } @@ -137,22 +141,21 @@ class PythonMLLibAPI extends Serializable { * Java stub for Python mllib RidgeRegressionWithSGD.train() */ def trainRidgeModelWithSGD( - dataBytesJRDD: JavaRDD[Array[Byte]], + data: JavaRDD[LabeledPoint], numIterations: Int, stepSize: Double, regParam: Double, miniBatchFraction: Double, initialWeightsBA: Array[Byte]): java.util.List[java.lang.Object] = { + val ridgeAlg = new RidgeRegressionWithSGD() + ridgeAlg.optimizer + .setNumIterations(numIterations) + .setRegParam(regParam) + .setStepSize(stepSize) + .setMiniBatchFraction(miniBatchFraction) trainRegressionModel( - (data, initialWeights) => - RidgeRegressionWithSGD.train( - data, - numIterations, - stepSize, - regParam, - miniBatchFraction, - initialWeights), - dataBytesJRDD, + ridgeAlg, + data, initialWeightsBA) } @@ -160,7 +163,7 @@ class PythonMLLibAPI extends Serializable { * Java stub for Python mllib SVMWithSGD.train() */ def trainSVMModelWithSGD( - dataBytesJRDD: JavaRDD[Array[Byte]], + data: JavaRDD[LabeledPoint], numIterations: Int, stepSize: Double, regParam: Double, @@ -184,9 +187,8 @@ class PythonMLLibAPI extends Serializable { + " Can only be initialized using the following string values: [l1, l2, none].") } trainRegressionModel( - (data, initialWeights) => - SVMAlg.run(data, initialWeights), - dataBytesJRDD, + SVMAlg, + data, initialWeightsBA) } @@ -194,7 +196,7 @@ class PythonMLLibAPI extends Serializable { * Java stub for Python mllib LogisticRegressionWithSGD.train() */ def trainLogisticRegressionModelWithSGD( - dataBytesJRDD: JavaRDD[Array[Byte]], + data: JavaRDD[LabeledPoint], numIterations: Int, stepSize: Double, miniBatchFraction: Double, @@ -218,9 +220,8 @@ class PythonMLLibAPI extends Serializable { + " Can only be initialized using the following string values: [l1, l2, none].") } trainRegressionModel( - (data, initialWeights) => - LogRegAlg.run(data, initialWeights), - dataBytesJRDD, + LogRegAlg, + data, initialWeightsBA) } @@ -228,14 +229,13 @@ class PythonMLLibAPI extends Serializable { * Java stub for NaiveBayes.train() */ def trainNaiveBayes( - dataBytesJRDD: JavaRDD[Array[Byte]], + data: JavaRDD[LabeledPoint], lambda: Double): java.util.List[java.lang.Object] = { - val data = dataBytesJRDD.rdd.map(SerDe.deserializeLabeledPoint) - val model = NaiveBayes.train(data, lambda) + val model = NaiveBayes.train(data.rdd, lambda) val ret = new java.util.LinkedList[java.lang.Object]() - ret.add(SerDe.serializeDoubleVector(Vectors.dense(model.labels))) - ret.add(SerDe.serializeDoubleVector(Vectors.dense(model.pi))) - ret.add(SerDe.serializeDoubleMatrix(model.theta)) + ret.add(Vectors.dense(model.labels)) + ret.add(Vectors.dense(model.pi)) + ret.add(model.theta) ret } @@ -243,16 +243,19 @@ class PythonMLLibAPI extends Serializable { * Java stub for Python mllib KMeans.train() */ def trainKMeansModel( - dataBytesJRDD: JavaRDD[Array[Byte]], + data: JavaRDD[Vector], k: Int, maxIterations: Int, runs: Int, - initializationMode: String): java.util.List[java.lang.Object] = { - val data = dataBytesJRDD.rdd.map(bytes => SerDe.deserializeDoubleVector(bytes)) - val model = KMeans.train(data, k, maxIterations, runs, initializationMode) - val ret = new java.util.LinkedList[java.lang.Object]() - ret.add(SerDe.serializeDoubleMatrix(model.clusterCenters.map(_.toArray))) - ret + initializationMode: String): KMeansModel = { + val kMeansAlg = new KMeans() + .setK(k) + .setMaxIterations(maxIterations) + .setRuns(runs) + .setInitializationMode(initializationMode) + // Disable the uncached input warning because 'data' is a deliberately uncached MappedRDD. + .disableUncachedWarning() + return kMeansAlg.run(data.rdd) } /** @@ -262,13 +265,12 @@ class PythonMLLibAPI extends Serializable { * the Py4J documentation. */ def trainALSModel( - ratingsBytesJRDD: JavaRDD[Array[Byte]], + ratings: JavaRDD[Rating], rank: Int, iterations: Int, lambda: Double, blocks: Int): MatrixFactorizationModel = { - val ratings = ratingsBytesJRDD.rdd.map(SerDe.unpackRating) - ALS.train(ratings, rank, iterations, lambda, blocks) + ALS.train(ratings.rdd, rank, iterations, lambda, blocks) } /** @@ -278,14 +280,66 @@ class PythonMLLibAPI extends Serializable { * exit; see the Py4J documentation. */ def trainImplicitALSModel( - ratingsBytesJRDD: JavaRDD[Array[Byte]], + ratingsJRDD: JavaRDD[Rating], rank: Int, iterations: Int, lambda: Double, blocks: Int, alpha: Double): MatrixFactorizationModel = { - val ratings = ratingsBytesJRDD.rdd.map(SerDe.unpackRating) - ALS.trainImplicit(ratings, rank, iterations, lambda, blocks, alpha) + ALS.trainImplicit(ratingsJRDD.rdd, rank, iterations, lambda, blocks, alpha) + } + + /** + * Java stub for Python mllib Word2Vec fit(). This stub returns a + * handle to the Java object instead of the content of the Java object. + * Extra care needs to be taken in the Python code to ensure it gets freed on + * exit; see the Py4J documentation. + * @param dataJRDD input JavaRDD + * @param vectorSize size of vector + * @param learningRate initial learning rate + * @param numPartitions number of partitions + * @param numIterations number of iterations + * @param seed initial seed for random generator + * @return A handle to java Word2VecModelWrapper instance at python side + */ + def trainWord2Vec( + dataJRDD: JavaRDD[java.util.ArrayList[String]], + vectorSize: Int, + learningRate: Double, + numPartitions: Int, + numIterations: Int, + seed: Long): Word2VecModelWrapper = { + val data = dataJRDD.rdd.persist(StorageLevel.MEMORY_AND_DISK_SER) + val word2vec = new Word2Vec() + .setVectorSize(vectorSize) + .setLearningRate(learningRate) + .setNumPartitions(numPartitions) + .setNumIterations(numIterations) + .setSeed(seed) + val model = word2vec.fit(data) + data.unpersist() + new Word2VecModelWrapper(model) + } + + private[python] class Word2VecModelWrapper(model: Word2VecModel) { + def transform(word: String): Vector = { + model.transform(word) + } + + def findSynonyms(word: String, num: Int): java.util.List[java.lang.Object] = { + val vec = transform(word) + findSynonyms(vec, num) + } + + def findSynonyms(vector: Vector, num: Int): java.util.List[java.lang.Object] = { + val result = model.findSynonyms(vector, num) + val similarity = Vectors.dense(result.map(_._2)) + val words = result.map(_._1) + val ret = new java.util.LinkedList[java.lang.Object]() + ret.add(words) + ret.add(similarity) + ret + } } /** @@ -293,11 +347,11 @@ class PythonMLLibAPI extends Serializable { * This stub returns a handle to the Java object instead of the content of the Java object. * Extra care needs to be taken in the Python code to ensure it gets freed on exit; * see the Py4J documentation. - * @param dataBytesJRDD Training data + * @param data Training data * @param categoricalFeaturesInfoJMap Categorical features info, as Java map */ def trainDecisionTreeModel( - dataBytesJRDD: JavaRDD[Array[Byte]], + data: JavaRDD[LabeledPoint], algoStr: String, numClasses: Int, categoricalFeaturesInfoJMap: java.util.Map[Int, Int], @@ -307,8 +361,6 @@ class PythonMLLibAPI extends Serializable { minInstancesPerNode: Int, minInfoGain: Double): DecisionTreeModel = { - val data = dataBytesJRDD.rdd.map(SerDe.deserializeLabeledPoint) - val algo = Algo.fromString(algoStr) val impurity = Impurities.fromString(impurityStr) @@ -322,44 +374,15 @@ class PythonMLLibAPI extends Serializable { minInstancesPerNode = minInstancesPerNode, minInfoGain = minInfoGain) - DecisionTree.train(data, strategy) - } - - /** - * Predict the label of the given data point. - * This is a Java stub for python DecisionTreeModel.predict() - * - * @param featuresBytes Serialized feature vector for data point - * @return predicted label - */ - def predictDecisionTreeModel( - model: DecisionTreeModel, - featuresBytes: Array[Byte]): Double = { - val features: Vector = SerDe.deserializeDoubleVector(featuresBytes) - model.predict(features) - } - - /** - * Predict the labels of the given data points. - * This is a Java stub for python DecisionTreeModel.predict() - * - * @param dataJRDD A JavaRDD with serialized feature vectors - * @return JavaRDD of serialized predictions - */ - def predictDecisionTreeModel( - model: DecisionTreeModel, - dataJRDD: JavaRDD[Array[Byte]]): JavaRDD[Array[Byte]] = { - val data = dataJRDD.rdd.map(xBytes => SerDe.deserializeDoubleVector(xBytes)) - model.predict(data).map(SerDe.serializeDouble) + DecisionTree.train(data.rdd, strategy) } /** * Java stub for mllib Statistics.colStats(X: RDD[Vector]). * TODO figure out return type. */ - def colStats(X: JavaRDD[Array[Byte]]): MultivariateStatisticalSummarySerialized = { - val cStats = Statistics.colStats(X.rdd.map(SerDe.deserializeDoubleVector(_))) - new MultivariateStatisticalSummarySerialized(cStats) + def colStats(rdd: JavaRDD[Vector]): MultivariateStatisticalSummary = { + Statistics.colStats(rdd.rdd) } /** @@ -367,19 +390,15 @@ class PythonMLLibAPI extends Serializable { * Returns the correlation matrix serialized into a byte array understood by deserializers in * pyspark. */ - def corr(X: JavaRDD[Array[Byte]], method: String): Array[Byte] = { - val inputMatrix = X.rdd.map(SerDe.deserializeDoubleVector(_)) - val result = Statistics.corr(inputMatrix, getCorrNameOrDefault(method)) - SerDe.serializeDoubleMatrix(SerDe.to2dArray(result)) + def corr(x: JavaRDD[Vector], method: String): Matrix = { + Statistics.corr(x.rdd, getCorrNameOrDefault(method)) } /** * Java stub for mllib Statistics.corr(x: RDD[Double], y: RDD[Double], method: String). */ - def corr(x: JavaRDD[Array[Byte]], y: JavaRDD[Array[Byte]], method: String): Double = { - val xDeser = x.rdd.map(SerDe.deserializeDouble(_)) - val yDeser = y.rdd.map(SerDe.deserializeDouble(_)) - Statistics.corr(xDeser, yDeser, getCorrNameOrDefault(method)) + def corr(x: JavaRDD[Double], y: JavaRDD[Double], method: String): Double = { + Statistics.corr(x.rdd, y.rdd, getCorrNameOrDefault(method)) } // used by the corr methods to retrieve the name of the correlation method passed in via pyspark @@ -411,10 +430,10 @@ class PythonMLLibAPI extends Serializable { def uniformRDD(jsc: JavaSparkContext, size: Long, numPartitions: java.lang.Integer, - seed: java.lang.Long): JavaRDD[Array[Byte]] = { + seed: java.lang.Long): JavaRDD[Double] = { val parts = getNumPartitionsOrDefault(numPartitions, jsc) val s = getSeedOrDefault(seed) - RG.uniformRDD(jsc.sc, size, parts, s).map(SerDe.serializeDouble) + RG.uniformRDD(jsc.sc, size, parts, s) } /** @@ -423,10 +442,10 @@ class PythonMLLibAPI extends Serializable { def normalRDD(jsc: JavaSparkContext, size: Long, numPartitions: java.lang.Integer, - seed: java.lang.Long): JavaRDD[Array[Byte]] = { + seed: java.lang.Long): JavaRDD[Double] = { val parts = getNumPartitionsOrDefault(numPartitions, jsc) val s = getSeedOrDefault(seed) - RG.normalRDD(jsc.sc, size, parts, s).map(SerDe.serializeDouble) + RG.normalRDD(jsc.sc, size, parts, s) } /** @@ -436,10 +455,10 @@ class PythonMLLibAPI extends Serializable { mean: Double, size: Long, numPartitions: java.lang.Integer, - seed: java.lang.Long): JavaRDD[Array[Byte]] = { + seed: java.lang.Long): JavaRDD[Double] = { val parts = getNumPartitionsOrDefault(numPartitions, jsc) val s = getSeedOrDefault(seed) - RG.poissonRDD(jsc.sc, mean, size, parts, s).map(SerDe.serializeDouble) + RG.poissonRDD(jsc.sc, mean, size, parts, s) } /** @@ -449,10 +468,10 @@ class PythonMLLibAPI extends Serializable { numRows: Long, numCols: Int, numPartitions: java.lang.Integer, - seed: java.lang.Long): JavaRDD[Array[Byte]] = { + seed: java.lang.Long): JavaRDD[Vector] = { val parts = getNumPartitionsOrDefault(numPartitions, jsc) val s = getSeedOrDefault(seed) - RG.uniformVectorRDD(jsc.sc, numRows, numCols, parts, s).map(SerDe.serializeDoubleVector) + RG.uniformVectorRDD(jsc.sc, numRows, numCols, parts, s) } /** @@ -462,10 +481,10 @@ class PythonMLLibAPI extends Serializable { numRows: Long, numCols: Int, numPartitions: java.lang.Integer, - seed: java.lang.Long): JavaRDD[Array[Byte]] = { + seed: java.lang.Long): JavaRDD[Vector] = { val parts = getNumPartitionsOrDefault(numPartitions, jsc) val s = getSeedOrDefault(seed) - RG.normalVectorRDD(jsc.sc, numRows, numCols, parts, s).map(SerDe.serializeDoubleVector) + RG.normalVectorRDD(jsc.sc, numRows, numCols, parts, s) } /** @@ -476,259 +495,168 @@ class PythonMLLibAPI extends Serializable { numRows: Long, numCols: Int, numPartitions: java.lang.Integer, - seed: java.lang.Long): JavaRDD[Array[Byte]] = { + seed: java.lang.Long): JavaRDD[Vector] = { val parts = getNumPartitionsOrDefault(numPartitions, jsc) val s = getSeedOrDefault(seed) - RG.poissonVectorRDD(jsc.sc, mean, numRows, numCols, parts, s).map(SerDe.serializeDoubleVector) + RG.poissonVectorRDD(jsc.sc, mean, numRows, numCols, parts, s) } } /** - * :: DeveloperApi :: - * MultivariateStatisticalSummary with Vector fields serialized. + * SerDe utility functions for PythonMLLibAPI. */ -@DeveloperApi -class MultivariateStatisticalSummarySerialized(val summary: MultivariateStatisticalSummary) - extends Serializable { +private[spark] object SerDe extends Serializable { - def mean: Array[Byte] = SerDe.serializeDoubleVector(summary.mean) + val PYSPARK_PACKAGE = "pyspark.mllib" - def variance: Array[Byte] = SerDe.serializeDoubleVector(summary.variance) + /** + * Base class used for pickle + */ + private[python] abstract class BasePickler[T: ClassTag] + extends IObjectPickler with IObjectConstructor { + + private val cls = implicitly[ClassTag[T]].runtimeClass + private val module = PYSPARK_PACKAGE + "." + cls.getName.split('.')(4) + private val name = cls.getSimpleName + + // register this to Pickler and Unpickler + def register(): Unit = { + Pickler.registerCustomPickler(this.getClass, this) + Pickler.registerCustomPickler(cls, this) + Unpickler.registerConstructor(module, name, this) + } - def count: Long = summary.count + def pickle(obj: Object, out: OutputStream, pickler: Pickler): Unit = { + if (obj == this) { + out.write(Opcodes.GLOBAL) + out.write((module + "\n" + name + "\n").getBytes()) + } else { + pickler.save(this) // it will be memorized by Pickler + saveState(obj, out, pickler) + out.write(Opcodes.REDUCE) + } + } - def numNonzeros: Array[Byte] = SerDe.serializeDoubleVector(summary.numNonzeros) + private[python] def saveObjects(out: OutputStream, pickler: Pickler, objects: Any*) = { + if (objects.length == 0 || objects.length > 3) { + out.write(Opcodes.MARK) + } + objects.foreach(pickler.save(_)) + val code = objects.length match { + case 1 => Opcodes.TUPLE1 + case 2 => Opcodes.TUPLE2 + case 3 => Opcodes.TUPLE3 + case _ => Opcodes.TUPLE + } + out.write(code) + } - def max: Array[Byte] = SerDe.serializeDoubleVector(summary.max) + private[python] def saveState(obj: Object, out: OutputStream, pickler: Pickler) + } - def min: Array[Byte] = SerDe.serializeDoubleVector(summary.min) -} + // Pickler for DenseVector + private[python] class DenseVectorPickler extends BasePickler[DenseVector] { -/** - * SerDe utility functions for PythonMLLibAPI. - */ -private[spark] object SerDe extends Serializable { - private val DENSE_VECTOR_MAGIC: Byte = 1 - private val SPARSE_VECTOR_MAGIC: Byte = 2 - private val DENSE_MATRIX_MAGIC: Byte = 3 - private val LABELED_POINT_MAGIC: Byte = 4 - - private[python] def deserializeDoubleVector(bytes: Array[Byte], offset: Int = 0): Vector = { - require(bytes.length - offset >= 5, "Byte array too short") - val magic = bytes(offset) - if (magic == DENSE_VECTOR_MAGIC) { - deserializeDenseVector(bytes, offset) - } else if (magic == SPARSE_VECTOR_MAGIC) { - deserializeSparseVector(bytes, offset) - } else { - throw new IllegalArgumentException("Magic " + magic + " is wrong.") + def saveState(obj: Object, out: OutputStream, pickler: Pickler) = { + val vector: DenseVector = obj.asInstanceOf[DenseVector] + saveObjects(out, pickler, vector.toArray) } - } - private[python] def deserializeDouble(bytes: Array[Byte], offset: Int = 0): Double = { - require(bytes.length - offset == 8, "Wrong size byte array for Double") - val bb = ByteBuffer.wrap(bytes, offset, bytes.length - offset) - bb.order(ByteOrder.nativeOrder()) - bb.getDouble - } - - private[python] def deserializeDenseVector(bytes: Array[Byte], offset: Int = 0): Vector = { - val packetLength = bytes.length - offset - require(packetLength >= 5, "Byte array too short") - val bb = ByteBuffer.wrap(bytes, offset, bytes.length - offset) - bb.order(ByteOrder.nativeOrder()) - val magic = bb.get() - require(magic == DENSE_VECTOR_MAGIC, "Invalid magic: " + magic) - val length = bb.getInt() - require (packetLength == 5 + 8 * length, "Invalid packet length: " + packetLength) - val db = bb.asDoubleBuffer() - val ans = new Array[Double](length.toInt) - db.get(ans) - Vectors.dense(ans) - } - - private[python] def deserializeSparseVector(bytes: Array[Byte], offset: Int = 0): Vector = { - val packetLength = bytes.length - offset - require(packetLength >= 9, "Byte array too short") - val bb = ByteBuffer.wrap(bytes, offset, bytes.length - offset) - bb.order(ByteOrder.nativeOrder()) - val magic = bb.get() - require(magic == SPARSE_VECTOR_MAGIC, "Invalid magic: " + magic) - val size = bb.getInt() - val nonZeros = bb.getInt() - require (packetLength == 9 + 12 * nonZeros, "Invalid packet length: " + packetLength) - val ib = bb.asIntBuffer() - val indices = new Array[Int](nonZeros) - ib.get(indices) - bb.position(bb.position() + 4 * nonZeros) - val db = bb.asDoubleBuffer() - val values = new Array[Double](nonZeros) - db.get(values) - Vectors.sparse(size, indices, values) + def construct(args: Array[Object]): Object = { + require(args.length == 1) + if (args.length != 1) { + throw new PickleException("should be 1") + } + new DenseVector(args(0).asInstanceOf[Array[Double]]) + } } - /** - * Returns an 8-byte array for the input Double. - * - * Note: we currently do not use a magic byte for double for storage efficiency. - * This should be reconsidered when we add Ser/De for other 8-byte types (e.g. Long), for safety. - * The corresponding deserializer, deserializeDouble, needs to be modified as well if the - * serialization scheme changes. - */ - private[python] def serializeDouble(double: Double): Array[Byte] = { - val bytes = new Array[Byte](8) - val bb = ByteBuffer.wrap(bytes) - bb.order(ByteOrder.nativeOrder()) - bb.putDouble(double) - bytes - } - - private[python] def serializeDenseVector(doubles: Array[Double]): Array[Byte] = { - val len = doubles.length - val bytes = new Array[Byte](5 + 8 * len) - val bb = ByteBuffer.wrap(bytes) - bb.order(ByteOrder.nativeOrder()) - bb.put(DENSE_VECTOR_MAGIC) - bb.putInt(len) - val db = bb.asDoubleBuffer() - db.put(doubles) - bytes - } - - private[python] def serializeSparseVector(vector: SparseVector): Array[Byte] = { - val nonZeros = vector.indices.length - val bytes = new Array[Byte](9 + 12 * nonZeros) - val bb = ByteBuffer.wrap(bytes) - bb.order(ByteOrder.nativeOrder()) - bb.put(SPARSE_VECTOR_MAGIC) - bb.putInt(vector.size) - bb.putInt(nonZeros) - val ib = bb.asIntBuffer() - ib.put(vector.indices) - bb.position(bb.position() + 4 * nonZeros) - val db = bb.asDoubleBuffer() - db.put(vector.values) - bytes - } - - private[python] def serializeDoubleVector(vector: Vector): Array[Byte] = vector match { - case s: SparseVector => - serializeSparseVector(s) - case _ => - serializeDenseVector(vector.toArray) - } - - private[python] def deserializeDoubleMatrix(bytes: Array[Byte]): Array[Array[Double]] = { - val packetLength = bytes.length - if (packetLength < 9) { - throw new IllegalArgumentException("Byte array too short.") + // Pickler for DenseMatrix + private[python] class DenseMatrixPickler extends BasePickler[DenseMatrix] { + + def saveState(obj: Object, out: OutputStream, pickler: Pickler) = { + val m: DenseMatrix = obj.asInstanceOf[DenseMatrix] + saveObjects(out, pickler, m.numRows, m.numCols, m.values) } - val bb = ByteBuffer.wrap(bytes) - bb.order(ByteOrder.nativeOrder()) - val magic = bb.get() - if (magic != DENSE_MATRIX_MAGIC) { - throw new IllegalArgumentException("Magic " + magic + " is wrong.") + + def construct(args: Array[Object]): Object = { + if (args.length != 3) { + throw new PickleException("should be 3") + } + new DenseMatrix(args(0).asInstanceOf[Int], args(1).asInstanceOf[Int], + args(2).asInstanceOf[Array[Double]]) } - val rows = bb.getInt() - val cols = bb.getInt() - if (packetLength != 9 + 8 * rows * cols) { - throw new IllegalArgumentException("Size " + rows + "x" + cols + " is wrong.") + } + + // Pickler for SparseVector + private[python] class SparseVectorPickler extends BasePickler[SparseVector] { + + def saveState(obj: Object, out: OutputStream, pickler: Pickler) = { + val v: SparseVector = obj.asInstanceOf[SparseVector] + saveObjects(out, pickler, v.size, v.indices, v.values) } - val db = bb.asDoubleBuffer() - val ans = new Array[Array[Double]](rows.toInt) - for (i <- 0 until rows.toInt) { - ans(i) = new Array[Double](cols.toInt) - db.get(ans(i)) + + def construct(args: Array[Object]): Object = { + if (args.length != 3) { + throw new PickleException("should be 3") + } + new SparseVector(args(0).asInstanceOf[Int], args(1).asInstanceOf[Array[Int]], + args(2).asInstanceOf[Array[Double]]) } - ans } - private[python] def serializeDoubleMatrix(doubles: Array[Array[Double]]): Array[Byte] = { - val rows = doubles.length - var cols = 0 - if (rows > 0) { - cols = doubles(0).length + // Pickler for LabeledPoint + private[python] class LabeledPointPickler extends BasePickler[LabeledPoint] { + + def saveState(obj: Object, out: OutputStream, pickler: Pickler) = { + val point: LabeledPoint = obj.asInstanceOf[LabeledPoint] + saveObjects(out, pickler, point.label, point.features) } - val bytes = new Array[Byte](9 + 8 * rows * cols) - val bb = ByteBuffer.wrap(bytes) - bb.order(ByteOrder.nativeOrder()) - bb.put(DENSE_MATRIX_MAGIC) - bb.putInt(rows) - bb.putInt(cols) - val db = bb.asDoubleBuffer() - for (i <- 0 until rows) { - db.put(doubles(i)) + + def construct(args: Array[Object]): Object = { + if (args.length != 2) { + throw new PickleException("should be 2") + } + new LabeledPoint(args(0).asInstanceOf[Double], args(1).asInstanceOf[Vector]) } - bytes } - private[python] def serializeLabeledPoint(p: LabeledPoint): Array[Byte] = { - val fb = serializeDoubleVector(p.features) - val bytes = new Array[Byte](1 + 8 + fb.length) - val bb = ByteBuffer.wrap(bytes) - bb.order(ByteOrder.nativeOrder()) - bb.put(LABELED_POINT_MAGIC) - bb.putDouble(p.label) - bb.put(fb) - bytes - } + // Pickler for Rating + private[python] class RatingPickler extends BasePickler[Rating] { - private[python] def deserializeLabeledPoint(bytes: Array[Byte]): LabeledPoint = { - require(bytes.length >= 9, "Byte array too short") - val magic = bytes(0) - if (magic != LABELED_POINT_MAGIC) { - throw new IllegalArgumentException("Magic " + magic + " is wrong.") + def saveState(obj: Object, out: OutputStream, pickler: Pickler) = { + val rating: Rating = obj.asInstanceOf[Rating] + saveObjects(out, pickler, rating.user, rating.product, rating.rating) } - val labelBytes = ByteBuffer.wrap(bytes, 1, 8) - labelBytes.order(ByteOrder.nativeOrder()) - val label = labelBytes.asDoubleBuffer().get(0) - LabeledPoint(label, deserializeDoubleVector(bytes, 9)) - } - // Reformat a Matrix into Array[Array[Double]] for serialization - private[python] def to2dArray(matrix: Matrix): Array[Array[Double]] = { - val values = matrix.toArray - Array.tabulate(matrix.numRows, matrix.numCols)((i, j) => values(i + j * matrix.numRows)) + def construct(args: Array[Object]): Object = { + if (args.length != 3) { + throw new PickleException("should be 3") + } + new Rating(args(0).asInstanceOf[Int], args(1).asInstanceOf[Int], + args(2).asInstanceOf[Double]) + } } + def initialize(): Unit = { + new DenseVectorPickler().register() + new DenseMatrixPickler().register() + new SparseVectorPickler().register() + new LabeledPointPickler().register() + new RatingPickler().register() + } - /** Unpack a Rating object from an array of bytes */ - private[python] def unpackRating(ratingBytes: Array[Byte]): Rating = { - val bb = ByteBuffer.wrap(ratingBytes) - bb.order(ByteOrder.nativeOrder()) - val user = bb.getInt() - val product = bb.getInt() - val rating = bb.getDouble() - new Rating(user, product, rating) + def dumps(obj: AnyRef): Array[Byte] = { + new Pickler().dumps(obj) } - /** Unpack a tuple of Ints from an array of bytes */ - def unpackTuple(tupleBytes: Array[Byte]): (Int, Int) = { - val bb = ByteBuffer.wrap(tupleBytes) - bb.order(ByteOrder.nativeOrder()) - val v1 = bb.getInt() - val v2 = bb.getInt() - (v1, v2) + def loads(bytes: Array[Byte]): AnyRef = { + new Unpickler().loads(bytes) } - /** - * Serialize a Rating object into an array of bytes. - * It can be deserialized using RatingDeserializer(). - * - * @param rate the Rating object to serialize - * @return - */ - def serializeRating(rate: Rating): Array[Byte] = { - val len = 3 - val bytes = new Array[Byte](4 + 8 * len) - val bb = ByteBuffer.wrap(bytes) - bb.order(ByteOrder.nativeOrder()) - bb.putInt(len) - val db = bb.asDoubleBuffer() - db.put(rate.user.toDouble) - db.put(rate.product.toDouble) - db.put(rate.rating) - bytes + /* convert object into Tuple */ + def asTupleRDD(rdd: RDD[Array[Any]]): RDD[(Int, Int)] = { + rdd.map(x => (x(0).asInstanceOf[Int], x(1).asInstanceOf[Int])) } } 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 fce8fe29f6e40..7443f232ec3e7 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 @@ -27,6 +27,7 @@ import org.apache.spark.SparkContext._ import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel import org.apache.spark.util.random.XORShiftRandom /** @@ -112,11 +113,26 @@ class KMeans private ( this } + /** Whether a warning should be logged if the input RDD is uncached. */ + private var warnOnUncachedInput = true + + /** Disable warnings about uncached input. */ + private[spark] def disableUncachedWarning(): this.type = { + warnOnUncachedInput = false + this + } + /** * Train a K-means model on the given set of points; `data` should be cached for high * performance, because this is an iterative algorithm. */ def run(data: RDD[Vector]): KMeansModel = { + + if (warnOnUncachedInput && data.getStorageLevel == StorageLevel.NONE) { + logWarning("The input data is not directly cached, which may hurt performance if its" + + " parent RDDs are also uncached.") + } + // Compute squared norms and cache them. val norms = data.map(v => breezeNorm(v.toBreeze, 2.0)) norms.persist() @@ -125,6 +141,12 @@ class KMeans private ( } val model = runBreeze(breezeData) norms.unpersist() + + // Warn at the end of the run as well, for increased visibility. + if (warnOnUncachedInput && data.getStorageLevel == StorageLevel.NONE) { + logWarning("The input data was not directly cached, which may hurt performance if its" + + " parent RDDs are also uncached.") + } model } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala index d40d5553c1d21..720bb70b08dbf 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala @@ -30,9 +30,18 @@ import org.apache.spark.rdd.RDD * Inverse document frequency (IDF). * The standard formulation is used: `idf = log((m + 1) / (d(t) + 1))`, where `m` is the total * number of documents and `d(t)` is the number of documents that contain term `t`. + * + * This implementation supports filtering out terms which do not appear in a minimum number + * of documents (controlled by the variable `minDocFreq`). For terms that are not in + * at least `minDocFreq` documents, the IDF is found as 0, resulting in TF-IDFs of 0. + * + * @param minDocFreq minimum of documents in which a term + * should appear for filtering */ @Experimental -class IDF { +class IDF(val minDocFreq: Int) { + + def this() = this(0) // TODO: Allow different IDF formulations. @@ -41,7 +50,8 @@ class IDF { * @param dataset an RDD of term frequency vectors */ def fit(dataset: RDD[Vector]): IDFModel = { - val idf = dataset.treeAggregate(new IDF.DocumentFrequencyAggregator)( + val idf = dataset.treeAggregate(new IDF.DocumentFrequencyAggregator( + minDocFreq = minDocFreq))( seqOp = (df, v) => df.add(v), combOp = (df1, df2) => df1.merge(df2) ).idf() @@ -60,13 +70,16 @@ class IDF { private object IDF { /** Document frequency aggregator. */ - class DocumentFrequencyAggregator extends Serializable { + class DocumentFrequencyAggregator(val minDocFreq: Int) extends Serializable { /** number of documents */ private var m = 0L /** document frequency vector */ private var df: BDV[Long] = _ + + def this() = this(0) + /** Adds a new document. */ def add(doc: Vector): this.type = { if (isEmpty) { @@ -123,7 +136,18 @@ private object IDF { val inv = new Array[Double](n) var j = 0 while (j < n) { - inv(j) = math.log((m + 1.0)/ (df(j) + 1.0)) + /* + * If the term is not present in the minimum + * number of documents, set IDF to 0. This + * will cause multiplication in IDFModel to + * set TF-IDF to 0. + * + * Since arrays are initialized to 0 by default, + * we just omit changing those entries. + */ + if(df(j) >= minDocFreq) { + inv(j) = math.log((m + 1.0) / (df(j) + 1.0)) + } j += 1 } Vectors.dense(inv) @@ -140,6 +164,11 @@ class IDFModel private[mllib] (val idf: Vector) extends Serializable { /** * Transforms term frequency (TF) vectors to TF-IDF vectors. + * + * If `minDocFreq` was set for the IDF calculation, + * the terms which occur in fewer than `minDocFreq` + * documents will have an entry of 0. + * * @param dataset an RDD of term frequency vectors * @return an RDD of TF-IDF vectors */ diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala index fc1444705364a..d321994c2a651 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala @@ -67,7 +67,7 @@ private case class VocabWord( class Word2Vec extends Serializable with Logging { private var vectorSize = 100 - private var startingAlpha = 0.025 + private var learningRate = 0.025 private var numPartitions = 1 private var numIterations = 1 private var seed = Utils.random.nextLong() @@ -84,7 +84,7 @@ class Word2Vec extends Serializable with Logging { * Sets initial learning rate (default: 0.025). */ def setLearningRate(learningRate: Double): this.type = { - this.startingAlpha = learningRate + this.learningRate = learningRate this } @@ -286,7 +286,7 @@ class Word2Vec extends Serializable with Logging { val syn0Global = Array.fill[Float](vocabSize * vectorSize)((initRandom.nextFloat() - 0.5f) / vectorSize) val syn1Global = new Array[Float](vocabSize * vectorSize) - var alpha = startingAlpha + var alpha = learningRate for (k <- 1 to numIterations) { val partial = newSentences.mapPartitionsWithIndex { case (idx, iter) => val random = new XORShiftRandom(seed ^ ((idx + 1) << 16) ^ ((-k - 1) << 8)) @@ -300,8 +300,8 @@ class Word2Vec extends Serializable with Logging { lwc = wordCount // TODO: discount by iteration? alpha = - startingAlpha * (1 - numPartitions * wordCount.toDouble / (trainWordsCount + 1)) - if (alpha < startingAlpha * 0.0001) alpha = startingAlpha * 0.0001 + learningRate * (1 - numPartitions * wordCount.toDouble / (trainWordsCount + 1)) + if (alpha < learningRate * 0.0001) alpha = learningRate * 0.0001 logInfo("wordCount = " + wordCount + ", alpha = " + alpha) } wc += sentence.size @@ -437,7 +437,7 @@ class Word2VecModel private[mllib] ( * Find synonyms of a word * @param word a word * @param num number of synonyms to find - * @return array of (word, similarity) + * @return array of (word, cosineSimilarity) */ def findSynonyms(word: String, num: Int): Array[(String, Double)] = { val vector = transform(word) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala index 70e23033c8754..54ee930d61003 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala @@ -18,13 +18,17 @@ package org.apache.spark.mllib.linalg import com.github.fommil.netlib.{BLAS => NetlibBLAS, F2jBLAS} +import com.github.fommil.netlib.BLAS.{getInstance => NativeBLAS} + +import org.apache.spark.Logging /** * BLAS routines for MLlib's vectors and matrices. */ -private[mllib] object BLAS extends Serializable { +private[mllib] object BLAS extends Serializable with Logging { @transient private var _f2jBLAS: NetlibBLAS = _ + @transient private var _nativeBLAS: NetlibBLAS = _ // For level-1 routines, we use Java implementation. private def f2jBLAS: NetlibBLAS = { @@ -197,4 +201,328 @@ private[mllib] object BLAS extends Serializable { throw new IllegalArgumentException(s"scal doesn't support vector type ${x.getClass}.") } } + + // For level-3 routines, we use the native BLAS. + private def nativeBLAS: NetlibBLAS = { + if (_nativeBLAS == null) { + _nativeBLAS = NativeBLAS + } + _nativeBLAS + } + + /** + * C := alpha * A * B + beta * C + * @param transA whether to use the transpose of matrix A (true), or A itself (false). + * @param transB whether to use the transpose of matrix B (true), or B itself (false). + * @param alpha a scalar to scale the multiplication A * B. + * @param A the matrix A that will be left multiplied to B. Size of m x k. + * @param B the matrix B that will be left multiplied by A. Size of k x n. + * @param beta a scalar that can be used to scale matrix C. + * @param C the resulting matrix C. Size of m x n. + */ + def gemm( + transA: Boolean, + transB: Boolean, + alpha: Double, + A: Matrix, + B: DenseMatrix, + beta: Double, + C: DenseMatrix): Unit = { + if (alpha == 0.0) { + logDebug("gemm: alpha is equal to 0. Returning C.") + } else { + A match { + case sparse: SparseMatrix => + gemm(transA, transB, alpha, sparse, B, beta, C) + case dense: DenseMatrix => + gemm(transA, transB, alpha, dense, B, beta, C) + case _ => + throw new IllegalArgumentException(s"gemm doesn't support matrix type ${A.getClass}.") + } + } + } + + /** + * C := alpha * A * B + beta * C + * + * @param alpha a scalar to scale the multiplication A * B. + * @param A the matrix A that will be left multiplied to B. Size of m x k. + * @param B the matrix B that will be left multiplied by A. Size of k x n. + * @param beta a scalar that can be used to scale matrix C. + * @param C the resulting matrix C. Size of m x n. + */ + def gemm( + alpha: Double, + A: Matrix, + B: DenseMatrix, + beta: Double, + C: DenseMatrix): Unit = { + gemm(false, false, alpha, A, B, beta, C) + } + + /** + * C := alpha * A * B + beta * C + * For `DenseMatrix` A. + */ + private def gemm( + transA: Boolean, + transB: Boolean, + alpha: Double, + A: DenseMatrix, + B: DenseMatrix, + beta: Double, + C: DenseMatrix): Unit = { + val mA: Int = if (!transA) A.numRows else A.numCols + val nB: Int = if (!transB) B.numCols else B.numRows + val kA: Int = if (!transA) A.numCols else A.numRows + val kB: Int = if (!transB) B.numRows else B.numCols + val tAstr = if (!transA) "N" else "T" + val tBstr = if (!transB) "N" else "T" + + require(kA == kB, s"The columns of A don't match the rows of B. A: $kA, B: $kB") + require(mA == C.numRows, s"The rows of C don't match the rows of A. C: ${C.numRows}, A: $mA") + require(nB == C.numCols, + s"The columns of C don't match the columns of B. C: ${C.numCols}, A: $nB") + + nativeBLAS.dgemm(tAstr, tBstr, mA, nB, kA, alpha, A.values, A.numRows, B.values, B.numRows, + beta, C.values, C.numRows) + } + + /** + * C := alpha * A * B + beta * C + * For `SparseMatrix` A. + */ + private def gemm( + transA: Boolean, + transB: Boolean, + alpha: Double, + A: SparseMatrix, + B: DenseMatrix, + beta: Double, + C: DenseMatrix): Unit = { + val mA: Int = if (!transA) A.numRows else A.numCols + val nB: Int = if (!transB) B.numCols else B.numRows + val kA: Int = if (!transA) A.numCols else A.numRows + val kB: Int = if (!transB) B.numRows else B.numCols + + require(kA == kB, s"The columns of A don't match the rows of B. A: $kA, B: $kB") + require(mA == C.numRows, s"The rows of C don't match the rows of A. C: ${C.numRows}, A: $mA") + require(nB == C.numCols, + s"The columns of C don't match the columns of B. C: ${C.numCols}, A: $nB") + + val Avals = A.values + val Arows = if (!transA) A.rowIndices else A.colPtrs + val Acols = if (!transA) A.colPtrs else A.rowIndices + + // Slicing is easy in this case. This is the optimal multiplication setting for sparse matrices + if (transA){ + var colCounterForB = 0 + if (!transB) { // Expensive to put the check inside the loop + while (colCounterForB < nB) { + var rowCounterForA = 0 + val Cstart = colCounterForB * mA + val Bstart = colCounterForB * kA + while (rowCounterForA < mA) { + var i = Arows(rowCounterForA) + val indEnd = Arows(rowCounterForA + 1) + var sum = 0.0 + while (i < indEnd) { + sum += Avals(i) * B.values(Bstart + Acols(i)) + i += 1 + } + val Cindex = Cstart + rowCounterForA + C.values(Cindex) = beta * C.values(Cindex) + sum * alpha + rowCounterForA += 1 + } + colCounterForB += 1 + } + } else { + while (colCounterForB < nB) { + var rowCounter = 0 + val Cstart = colCounterForB * mA + while (rowCounter < mA) { + var i = Arows(rowCounter) + val indEnd = Arows(rowCounter + 1) + var sum = 0.0 + while (i < indEnd) { + sum += Avals(i) * B(colCounterForB, Acols(i)) + i += 1 + } + val Cindex = Cstart + rowCounter + C.values(Cindex) = beta * C.values(Cindex) + sum * alpha + rowCounter += 1 + } + colCounterForB += 1 + } + } + } else { + // Scale matrix first if `beta` is not equal to 0.0 + if (beta != 0.0){ + f2jBLAS.dscal(C.values.length, beta, C.values, 1) + } + // Perform matrix multiplication and add to C. The rows of A are multiplied by the columns of + // B, and added to C. + var colCounterForB = 0 // the column to be updated in C + if (!transB) { // Expensive to put the check inside the loop + while (colCounterForB < nB) { + var colCounterForA = 0 // The column of A to multiply with the row of B + val Bstart = colCounterForB * kB + val Cstart = colCounterForB * mA + while (colCounterForA < kA) { + var i = Acols(colCounterForA) + val indEnd = Acols(colCounterForA + 1) + val Bval = B.values(Bstart + colCounterForA) * alpha + while (i < indEnd){ + C.values(Cstart + Arows(i)) += Avals(i) * Bval + i += 1 + } + colCounterForA += 1 + } + colCounterForB += 1 + } + } else { + while (colCounterForB < nB) { + var colCounterForA = 0 // The column of A to multiply with the row of B + val Cstart = colCounterForB * mA + while (colCounterForA < kA){ + var i = Acols(colCounterForA) + val indEnd = Acols(colCounterForA + 1) + val Bval = B(colCounterForB, colCounterForA) * alpha + while (i < indEnd){ + C.values(Cstart + Arows(i)) += Avals(i) * Bval + i += 1 + } + colCounterForA += 1 + } + colCounterForB += 1 + } + } + } + } + + /** + * y := alpha * A * x + beta * y + * @param trans whether to use the transpose of matrix A (true), or A itself (false). + * @param alpha a scalar to scale the multiplication A * x. + * @param A the matrix A that will be left multiplied to x. Size of m x n. + * @param x the vector x that will be left multiplied by A. Size of n x 1. + * @param beta a scalar that can be used to scale vector y. + * @param y the resulting vector y. Size of m x 1. + */ + def gemv( + trans: Boolean, + alpha: Double, + A: Matrix, + x: DenseVector, + beta: Double, + y: DenseVector): Unit = { + + val mA: Int = if (!trans) A.numRows else A.numCols + val nx: Int = x.size + val nA: Int = if (!trans) A.numCols else A.numRows + + require(nA == nx, s"The columns of A don't match the number of elements of x. A: $nA, x: $nx") + require(mA == y.size, + s"The rows of A don't match the number of elements of y. A: $mA, y:${y.size}}") + if (alpha == 0.0) { + logDebug("gemv: alpha is equal to 0. Returning y.") + } else { + A match { + case sparse: SparseMatrix => + gemv(trans, alpha, sparse, x, beta, y) + case dense: DenseMatrix => + gemv(trans, alpha, dense, x, beta, y) + case _ => + throw new IllegalArgumentException(s"gemv doesn't support matrix type ${A.getClass}.") + } + } + } + + /** + * y := alpha * A * x + beta * y + * + * @param alpha a scalar to scale the multiplication A * x. + * @param A the matrix A that will be left multiplied to x. Size of m x n. + * @param x the vector x that will be left multiplied by A. Size of n x 1. + * @param beta a scalar that can be used to scale vector y. + * @param y the resulting vector y. Size of m x 1. + */ + def gemv( + alpha: Double, + A: Matrix, + x: DenseVector, + beta: Double, + y: DenseVector): Unit = { + gemv(false, alpha, A, x, beta, y) + } + + /** + * y := alpha * A * x + beta * y + * For `DenseMatrix` A. + */ + private def gemv( + trans: Boolean, + alpha: Double, + A: DenseMatrix, + x: DenseVector, + beta: Double, + y: DenseVector): Unit = { + val tStrA = if (!trans) "N" else "T" + nativeBLAS.dgemv(tStrA, A.numRows, A.numCols, alpha, A.values, A.numRows, x.values, 1, beta, + y.values, 1) + } + + /** + * y := alpha * A * x + beta * y + * For `SparseMatrix` A. + */ + private def gemv( + trans: Boolean, + alpha: Double, + A: SparseMatrix, + x: DenseVector, + beta: Double, + y: DenseVector): Unit = { + + val mA: Int = if(!trans) A.numRows else A.numCols + val nA: Int = if(!trans) A.numCols else A.numRows + + val Avals = A.values + val Arows = if (!trans) A.rowIndices else A.colPtrs + val Acols = if (!trans) A.colPtrs else A.rowIndices + + // Slicing is easy in this case. This is the optimal multiplication setting for sparse matrices + if (trans){ + var rowCounter = 0 + while (rowCounter < mA){ + var i = Arows(rowCounter) + val indEnd = Arows(rowCounter + 1) + var sum = 0.0 + while(i < indEnd){ + sum += Avals(i) * x.values(Acols(i)) + i += 1 + } + y.values(rowCounter) = beta * y.values(rowCounter) + sum * alpha + rowCounter += 1 + } + } else { + // Scale vector first if `beta` is not equal to 0.0 + if (beta != 0.0){ + scal(beta, y) + } + // Perform matrix-vector multiplication and add to y + var colCounterForA = 0 + while (colCounterForA < nA){ + var i = Acols(colCounterForA) + val indEnd = Acols(colCounterForA + 1) + val xVal = x.values(colCounterForA) * alpha + while (i < indEnd){ + val rowIndex = Arows(i) + y.values(rowIndex) += Avals(i) * xVal + i += 1 + } + colCounterForA += 1 + } + } + } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala index b11ba5d30fbd3..2cc52e94282ba 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala @@ -17,12 +17,16 @@ package org.apache.spark.mllib.linalg -import breeze.linalg.{Matrix => BM, DenseMatrix => BDM} +import java.util.Arrays + +import breeze.linalg.{Matrix => BM, DenseMatrix => BDM, CSCMatrix => BSM} + +import org.apache.spark.util.random.XORShiftRandom /** * Trait for a local matrix. */ -trait Matrix extends Serializable { +sealed trait Matrix extends Serializable { /** Number of rows. */ def numRows: Int @@ -37,13 +41,51 @@ trait Matrix extends Serializable { private[mllib] def toBreeze: BM[Double] /** Gets the (i, j)-th element. */ - private[mllib] def apply(i: Int, j: Int): Double = toBreeze(i, j) + private[mllib] def apply(i: Int, j: Int): Double + + /** Return the index for the (i, j)-th element in the backing array. */ + private[mllib] def index(i: Int, j: Int): Int + + /** Update element at (i, j) */ + private[mllib] def update(i: Int, j: Int, v: Double): Unit + + /** Get a deep copy of the matrix. */ + def copy: Matrix + /** Convenience method for `Matrix`-`DenseMatrix` multiplication. */ + def multiply(y: DenseMatrix): DenseMatrix = { + val C: DenseMatrix = Matrices.zeros(numRows, y.numCols).asInstanceOf[DenseMatrix] + BLAS.gemm(false, false, 1.0, this, y, 0.0, C) + C + } + + /** Convenience method for `Matrix`-`DenseVector` multiplication. */ + def multiply(y: DenseVector): DenseVector = { + val output = new DenseVector(new Array[Double](numRows)) + BLAS.gemv(1.0, this, y, 0.0, output) + output + } + + /** Convenience method for `Matrix`^T^-`DenseMatrix` multiplication. */ + def transposeMultiply(y: DenseMatrix): DenseMatrix = { + val C: DenseMatrix = Matrices.zeros(numCols, y.numCols).asInstanceOf[DenseMatrix] + BLAS.gemm(true, false, 1.0, this, y, 0.0, C) + C + } + + /** Convenience method for `Matrix`^T^-`DenseVector` multiplication. */ + def transposeMultiply(y: DenseVector): DenseVector = { + val output = new DenseVector(new Array[Double](numCols)) + BLAS.gemv(true, 1.0, this, y, 0.0, output) + output + } + + /** A human readable representation of the matrix */ override def toString: String = toBreeze.toString() } /** - * Column-majored dense matrix. + * Column-major dense matrix. * The entry values are stored in a single array of doubles with columns listed in sequence. * For example, the following matrix * {{{ @@ -59,11 +101,104 @@ trait Matrix extends Serializable { */ class DenseMatrix(val numRows: Int, val numCols: Int, val values: Array[Double]) extends Matrix { - require(values.length == numRows * numCols) + require(values.length == numRows * numCols, "The number of values supplied doesn't match the " + + s"size of the matrix! values.length: ${values.length}, numRows * numCols: ${numRows * numCols}") override def toArray: Array[Double] = values - private[mllib] override def toBreeze: BM[Double] = new BDM[Double](numRows, numCols, values) + override def equals(o: Any) = o match { + case m: DenseMatrix => + m.numRows == numRows && m.numCols == numCols && Arrays.equals(toArray, m.toArray) + case _ => false + } + + private[mllib] def toBreeze: BM[Double] = new BDM[Double](numRows, numCols, values) + + private[mllib] def apply(i: Int): Double = values(i) + + private[mllib] def apply(i: Int, j: Int): Double = values(index(i, j)) + + private[mllib] def index(i: Int, j: Int): Int = i + numRows * j + + private[mllib] def update(i: Int, j: Int, v: Double): Unit = { + values(index(i, j)) = v + } + + override def copy = new DenseMatrix(numRows, numCols, values.clone()) +} + +/** + * Column-major sparse matrix. + * The entry values are stored in Compressed Sparse Column (CSC) format. + * For example, the following matrix + * {{{ + * 1.0 0.0 4.0 + * 0.0 3.0 5.0 + * 2.0 0.0 6.0 + * }}} + * is stored as `values: [1.0, 2.0, 3.0, 4.0, 5.0, 6.0]`, + * `rowIndices=[0, 2, 1, 0, 1, 2]`, `colPointers=[0, 2, 3, 6]`. + * + * @param numRows number of rows + * @param numCols number of columns + * @param colPtrs the index corresponding to the start of a new column + * @param rowIndices the row index of the entry. They must be in strictly increasing order for each + * column + * @param values non-zero matrix entries in column major + */ +class SparseMatrix( + val numRows: Int, + val numCols: Int, + val colPtrs: Array[Int], + val rowIndices: Array[Int], + val values: Array[Double]) extends Matrix { + + require(values.length == rowIndices.length, "The number of row indices and values don't match! " + + s"values.length: ${values.length}, rowIndices.length: ${rowIndices.length}") + require(colPtrs.length == numCols + 1, "The length of the column indices should be the " + + s"number of columns + 1. Currently, colPointers.length: ${colPtrs.length}, " + + s"numCols: $numCols") + + override def toArray: Array[Double] = { + val arr = new Array[Double](numRows * numCols) + var j = 0 + while (j < numCols) { + var i = colPtrs(j) + val indEnd = colPtrs(j + 1) + val offset = j * numRows + while (i < indEnd) { + val rowIndex = rowIndices(i) + arr(offset + rowIndex) = values(i) + i += 1 + } + j += 1 + } + arr + } + + private[mllib] def toBreeze: BM[Double] = + new BSM[Double](values, numRows, numCols, colPtrs, rowIndices) + + private[mllib] def apply(i: Int, j: Int): Double = { + val ind = index(i, j) + if (ind < 0) 0.0 else values(ind) + } + + private[mllib] def index(i: Int, j: Int): Int = { + Arrays.binarySearch(rowIndices, colPtrs(j), colPtrs(j + 1), i) + } + + private[mllib] def update(i: Int, j: Int, v: Double): Unit = { + val ind = index(i, j) + if (ind == -1){ + throw new NoSuchElementException("The given row and column indices correspond to a zero " + + "value. Only non-zero elements in Sparse Matrices can be updated.") + } else { + values(index(i, j)) = v + } + } + + override def copy = new SparseMatrix(numRows, numCols, colPtrs, rowIndices, values.clone()) } /** @@ -72,7 +207,7 @@ class DenseMatrix(val numRows: Int, val numCols: Int, val values: Array[Double]) object Matrices { /** - * Creates a column-majored dense matrix. + * Creates a column-major dense matrix. * * @param numRows number of rows * @param numCols number of columns @@ -82,6 +217,24 @@ object Matrices { new DenseMatrix(numRows, numCols, values) } + /** + * Creates a column-major sparse matrix in Compressed Sparse Column (CSC) format. + * + * @param numRows number of rows + * @param numCols number of columns + * @param colPtrs the index corresponding to the start of a new column + * @param rowIndices the row index of the entry + * @param values non-zero matrix entries in column major + */ + def sparse( + numRows: Int, + numCols: Int, + colPtrs: Array[Int], + rowIndices: Array[Int], + values: Array[Double]): Matrix = { + new SparseMatrix(numRows, numCols, colPtrs, rowIndices, values) + } + /** * Creates a Matrix instance from a breeze matrix. * @param breeze a breeze matrix @@ -93,9 +246,84 @@ object Matrices { require(dm.majorStride == dm.rows, "Do not support stride size different from the number of rows.") new DenseMatrix(dm.rows, dm.cols, dm.data) + case sm: BSM[Double] => + new SparseMatrix(sm.rows, sm.cols, sm.colPtrs, sm.rowIndices, sm.data) case _ => throw new UnsupportedOperationException( s"Do not support conversion from type ${breeze.getClass.getName}.") } } + + /** + * Generate a `DenseMatrix` consisting of zeros. + * @param numRows number of rows of the matrix + * @param numCols number of columns of the matrix + * @return `DenseMatrix` with size `numRows` x `numCols` and values of zeros + */ + def zeros(numRows: Int, numCols: Int): Matrix = + new DenseMatrix(numRows, numCols, new Array[Double](numRows * numCols)) + + /** + * Generate a `DenseMatrix` consisting of ones. + * @param numRows number of rows of the matrix + * @param numCols number of columns of the matrix + * @return `DenseMatrix` with size `numRows` x `numCols` and values of ones + */ + def ones(numRows: Int, numCols: Int): Matrix = + new DenseMatrix(numRows, numCols, Array.fill(numRows * numCols)(1.0)) + + /** + * Generate an Identity Matrix in `DenseMatrix` format. + * @param n number of rows and columns of the matrix + * @return `DenseMatrix` with size `n` x `n` and values of ones on the diagonal + */ + def eye(n: Int): Matrix = { + val identity = Matrices.zeros(n, n) + var i = 0 + while (i < n){ + identity.update(i, i, 1.0) + i += 1 + } + identity + } + + /** + * Generate a `DenseMatrix` consisting of i.i.d. uniform random numbers. + * @param numRows number of rows of the matrix + * @param numCols number of columns of the matrix + * @return `DenseMatrix` with size `numRows` x `numCols` and values in U(0, 1) + */ + def rand(numRows: Int, numCols: Int): Matrix = { + val rand = new XORShiftRandom + new DenseMatrix(numRows, numCols, Array.fill(numRows * numCols)(rand.nextDouble())) + } + + /** + * Generate a `DenseMatrix` consisting of i.i.d. gaussian random numbers. + * @param numRows number of rows of the matrix + * @param numCols number of columns of the matrix + * @return `DenseMatrix` with size `numRows` x `numCols` and values in N(0, 1) + */ + def randn(numRows: Int, numCols: Int): Matrix = { + val rand = new XORShiftRandom + new DenseMatrix(numRows, numCols, Array.fill(numRows * numCols)(rand.nextGaussian())) + } + + /** + * Generate a diagonal matrix in `DenseMatrix` format from the supplied values. + * @param vector a `Vector` tat will form the values on the diagonal of the matrix + * @return Square `DenseMatrix` with size `values.length` x `values.length` and `values` + * on the diagonal + */ + def diag(vector: Vector): Matrix = { + val n = vector.size + val matrix = Matrices.eye(n) + val values = vector.toArray + var i = 0 + while (i < n) { + matrix.update(i, i, values(i)) + i += 1 + } + matrix + } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala index a45781d12e41e..6af225b7f49f7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala @@ -33,7 +33,7 @@ import org.apache.spark.SparkException * * Note: Users should not implement this interface. */ -trait Vector extends Serializable { +sealed trait Vector extends Serializable { /** * Size of the vector. 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 2e414a73be8e0..8380058cf9b41 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 @@ -19,17 +19,22 @@ package org.apache.spark.mllib.linalg.distributed import java.util.Arrays -import breeze.linalg.{DenseMatrix => BDM, DenseVector => BDV, SparseVector => BSV} -import breeze.linalg.{svd => brzSvd, axpy => brzAxpy} +import scala.collection.mutable.ListBuffer + +import breeze.linalg.{DenseMatrix => BDM, DenseVector => BDV, SparseVector => BSV, axpy => brzAxpy, + svd => brzSvd} import breeze.numerics.{sqrt => brzSqrt} import com.github.fommil.netlib.BLAS.{getInstance => blas} +import org.apache.spark.Logging +import org.apache.spark.SparkContext._ import org.apache.spark.annotation.Experimental import org.apache.spark.mllib.linalg._ -import org.apache.spark.rdd.RDD -import org.apache.spark.Logging import org.apache.spark.mllib.rdd.RDDFunctions._ import org.apache.spark.mllib.stat.{MultivariateOnlineSummarizer, MultivariateStatisticalSummary} +import org.apache.spark.rdd.RDD +import org.apache.spark.util.random.XORShiftRandom +import org.apache.spark.storage.StorageLevel /** * :: Experimental :: @@ -231,6 +236,10 @@ class RowMatrix( val brzSvd.SVD(uFull: BDM[Double], sigmaSquaresFull: BDV[Double], _) = brzSvd(G) (sigmaSquaresFull, uFull) case SVDMode.DistARPACK => + if (rows.getStorageLevel == StorageLevel.NONE) { + logWarning("The input data is not directly cached, which may hurt performance if its" + + " parent RDDs are also uncached.") + } require(k < n, s"k must be smaller than n in dist-eigs mode but got k=$k and n=$n.") EigenValueDecomposition.symmetricEigs(multiplyGramianMatrixBy, n, k, tol, maxIter) } @@ -256,6 +265,12 @@ class RowMatrix( logWarning(s"Requested $k singular values but only found $sk nonzeros.") } + // Warn at the end of the run as well, for increased visibility. + if (computeMode == SVDMode.DistARPACK && rows.getStorageLevel == StorageLevel.NONE) { + logWarning("The input data was not directly cached, which may hurt performance if its" + + " parent RDDs are also uncached.") + } + val s = Vectors.dense(Arrays.copyOfRange(sigmas.data, 0, sk)) val V = Matrices.dense(n, sk, Arrays.copyOfRange(u.data, 0, n * sk)) @@ -400,6 +415,165 @@ class RowMatrix( new RowMatrix(AB, nRows, B.numCols) } + /** + * Compute all cosine similarities between columns of this matrix using the brute-force + * approach of computing normalized dot products. + * + * @return An n x n sparse upper-triangular matrix of cosine similarities between + * columns of this matrix. + */ + def columnSimilarities(): CoordinateMatrix = { + columnSimilarities(0.0) + } + + /** + * Compute similarities between columns of this matrix using a sampling approach. + * + * The threshold parameter is a trade-off knob between estimate quality and computational cost. + * + * Setting a threshold of 0 guarantees deterministic correct results, but comes at exactly + * the same cost as the brute-force approach. Setting the threshold to positive values + * incurs strictly less computational cost than the brute-force approach, however the + * similarities computed will be estimates. + * + * The sampling guarantees relative-error correctness for those pairs of columns that have + * similarity greater than the given similarity threshold. + * + * To describe the guarantee, we set some notation: + * Let A be the smallest in magnitude non-zero element of this matrix. + * Let B be the largest in magnitude non-zero element of this matrix. + * Let L be the maximum number of non-zeros per row. + * + * For example, for {0,1} matrices: A=B=1. + * Another example, for the Netflix matrix: A=1, B=5 + * + * For those column pairs that are above the threshold, + * the computed similarity is correct to within 20% relative error with probability + * at least 1 - (0.981)^10/B^ + * + * The shuffle size is bounded by the *smaller* of the following two expressions: + * + * O(n log(n) L / (threshold * A)) + * O(m L^2^) + * + * The latter is the cost of the brute-force approach, so for non-zero thresholds, + * the cost is always cheaper than the brute-force approach. + * + * @param threshold Set to 0 for deterministic guaranteed correctness. + * Similarities above this threshold are estimated + * with the cost vs estimate quality trade-off described above. + * @return An n x n sparse upper-triangular matrix of cosine similarities + * between columns of this matrix. + */ + def columnSimilarities(threshold: Double): CoordinateMatrix = { + require(threshold >= 0, s"Threshold cannot be negative: $threshold") + + if (threshold > 1) { + logWarning(s"Threshold is greater than 1: $threshold " + + "Computation will be more efficient with promoted sparsity, " + + " however there is no correctness guarantee.") + } + + val gamma = if (threshold < 1e-6) { + Double.PositiveInfinity + } else { + 10 * math.log(numCols()) / threshold + } + + columnSimilaritiesDIMSUM(computeColumnSummaryStatistics().normL2.toArray, gamma) + } + + /** + * Find all similar columns using the DIMSUM sampling algorithm, described in two papers + * + * http://arxiv.org/abs/1206.2082 + * http://arxiv.org/abs/1304.1467 + * + * @param colMags A vector of column magnitudes + * @param gamma The oversampling parameter. For provable results, set to 10 * log(n) / s, + * where s is the smallest similarity score to be estimated, + * and n is the number of columns + * @return An n x n sparse upper-triangular matrix of cosine similarities + * between columns of this matrix. + */ + private[mllib] def columnSimilaritiesDIMSUM( + colMags: Array[Double], + gamma: Double): CoordinateMatrix = { + require(gamma > 1.0, s"Oversampling should be greater than 1: $gamma") + require(colMags.size == this.numCols(), "Number of magnitudes didn't match column dimension") + val sg = math.sqrt(gamma) // sqrt(gamma) used many times + + // Don't divide by zero for those columns with zero magnitude + val colMagsCorrected = colMags.map(x => if (x == 0) 1.0 else x) + + val sc = rows.context + val pBV = sc.broadcast(colMagsCorrected.map(c => sg / c)) + val qBV = sc.broadcast(colMagsCorrected.map(c => math.min(sg, c))) + + val sims = rows.mapPartitionsWithIndex { (indx, iter) => + val p = pBV.value + val q = qBV.value + + val rand = new XORShiftRandom(indx) + val scaled = new Array[Double](p.size) + iter.flatMap { row => + val buf = new ListBuffer[((Int, Int), Double)]() + row match { + case sv: SparseVector => + val nnz = sv.indices.size + var k = 0 + while (k < nnz) { + scaled(k) = sv.values(k) / q(sv.indices(k)) + k += 1 + } + k = 0 + while (k < nnz) { + val i = sv.indices(k) + val iVal = scaled(k) + if (iVal != 0 && rand.nextDouble() < p(i)) { + var l = k + 1 + while (l < nnz) { + val j = sv.indices(l) + val jVal = scaled(l) + if (jVal != 0 && rand.nextDouble() < p(j)) { + buf += (((i, j), iVal * jVal)) + } + l += 1 + } + } + k += 1 + } + case dv: DenseVector => + val n = dv.values.size + var i = 0 + while (i < n) { + scaled(i) = dv.values(i) / q(i) + i += 1 + } + i = 0 + while (i < n) { + val iVal = scaled(i) + if (iVal != 0 && rand.nextDouble() < p(i)) { + var j = i + 1 + while (j < n) { + val jVal = scaled(j) + if (jVal != 0 && rand.nextDouble() < p(j)) { + buf += (((i, j), iVal * jVal)) + } + j += 1 + } + } + i += 1 + } + } + buf + } + }.reduceByKey(_ + _).map { case ((i, j), sim) => + MatrixEntry(i.toLong, j.toLong, sim) + } + new CoordinateMatrix(sims, numCols(), numCols()) + } + private[mllib] override def toBreeze(): BDM[Double] = { val m = numRows().toInt val n = numCols().toInt 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 478c6485052b6..66b58ba770160 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 @@ -106,19 +106,4 @@ class MatrixFactorizationModel private[mllib] ( } scored.top(num)(Ordering.by(_._2)) } - - /** - * :: DeveloperApi :: - * Predict the rating of many users for many products. - * This is a Java stub for python predictAll() - * - * @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 usersProducts = usersProductsJRDD.rdd.map(xBytes => SerDe.unpackTuple(xBytes)) - predict(usersProducts).map(rate => SerDe.serializeRating(rate)) - } - } 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 20c1fdd2269ce..00dfc86c9e0bd 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 @@ -24,6 +24,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.linalg.{Vectors, Vector} import org.apache.spark.mllib.util.MLUtils._ +import org.apache.spark.storage.StorageLevel /** * :: DeveloperApi :: @@ -74,6 +75,8 @@ abstract class GeneralizedLinearModel(val weights: Vector, val intercept: Double def predict(testData: Vector): Double = { predictPoint(testData, weights, intercept) } + + override def toString() = "(weights=%s, intercept=%s)".format(weights, intercept) } /** @@ -133,6 +136,15 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] this } + /** Whether a warning should be logged if the input RDD is uncached. */ + private var warnOnUncachedInput = true + + /** Disable warnings about uncached input. */ + private[spark] def disableUncachedWarning(): this.type = { + warnOnUncachedInput = false + this + } + /** * Run the algorithm with the configured parameters on an input * RDD of LabeledPoint entries. @@ -149,6 +161,11 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] */ def run(input: RDD[LabeledPoint], initialWeights: Vector): M = { + if (warnOnUncachedInput && input.getStorageLevel == StorageLevel.NONE) { + logWarning("The input data is not directly cached, which may hurt performance if its" + + " parent RDDs are also uncached.") + } + // Check the data properties before running the optimizer if (validateData && !validators.forall(func => func(input))) { throw new SparkException("Input validation failed.") @@ -223,6 +240,12 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] weights = scaler.transform(weights) } + // Warn at the end of the run as well, for increased visibility. + if (warnOnUncachedInput && input.getStorageLevel == StorageLevel.NONE) { + logWarning("The input data was not directly cached, which may hurt performance if its" + + " parent RDDs are also uncached.") + } + createModel(weights, intercept) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala index 7d845c44365dd..3025d4837cab4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala @@ -42,6 +42,8 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S private var n = 0 private var currMean: BDV[Double] = _ private var currM2n: BDV[Double] = _ + private var currM2: BDV[Double] = _ + private var currL1: BDV[Double] = _ private var totalCnt: Long = 0 private var nnz: BDV[Double] = _ private var currMax: BDV[Double] = _ @@ -60,6 +62,8 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S currMean = BDV.zeros[Double](n) currM2n = BDV.zeros[Double](n) + currM2 = BDV.zeros[Double](n) + currL1 = BDV.zeros[Double](n) nnz = BDV.zeros[Double](n) currMax = BDV.fill(n)(Double.MinValue) currMin = BDV.fill(n)(Double.MaxValue) @@ -81,6 +85,8 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S val tmpPrevMean = currMean(i) currMean(i) = (currMean(i) * nnz(i) + value) / (nnz(i) + 1.0) currM2n(i) += (value - currMean(i)) * (value - tmpPrevMean) + currM2(i) += value * value + currL1(i) += math.abs(value) nnz(i) += 1.0 } @@ -97,7 +103,7 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S * @return This MultivariateOnlineSummarizer object. */ def merge(other: MultivariateOnlineSummarizer): this.type = { - if (this.totalCnt != 0 && other.totalCnt != 0) { + if (this.totalCnt != 0 && other.totalCnt != 0) { require(n == other.n, s"Dimensions mismatch when merging with another summarizer. " + s"Expecting $n but got ${other.n}.") totalCnt += other.totalCnt @@ -114,6 +120,15 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S currM2n(i) += other.currM2n(i) + deltaMean(i) * deltaMean(i) * nnz(i) * other.nnz(i) / (nnz(i) + other.nnz(i)) } + // merge m2 together + if (nnz(i) + other.nnz(i) != 0.0) { + currM2(i) += other.currM2(i) + } + // merge l1 together + if (nnz(i) + other.nnz(i) != 0.0) { + currL1(i) += other.currL1(i) + } + if (currMax(i) < other.currMax(i)) { currMax(i) = other.currMax(i) } @@ -127,6 +142,8 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S this.n = other.n this.currMean = other.currMean.copy this.currM2n = other.currM2n.copy + this.currM2 = other.currM2.copy + this.currL1 = other.currL1.copy this.totalCnt = other.totalCnt this.nnz = other.nnz.copy this.currMax = other.currMax.copy @@ -198,4 +215,23 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S } Vectors.fromBreeze(currMin) } + + override def normL2: Vector = { + require(totalCnt > 0, s"Nothing has been added to this summarizer.") + + val realMagnitude = BDV.zeros[Double](n) + + var i = 0 + while (i < currM2.size) { + realMagnitude(i) = math.sqrt(currM2(i)) + i += 1 + } + + Vectors.fromBreeze(realMagnitude) + } + + override def normL1: Vector = { + require(totalCnt > 0, s"Nothing has been added to this summarizer.") + Vectors.fromBreeze(currL1) + } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalSummary.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalSummary.scala index f9eb343da2b82..6a364c93284af 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalSummary.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalSummary.scala @@ -53,4 +53,14 @@ trait MultivariateStatisticalSummary { * Minimum value of each column. */ def min: Vector + + /** + * Euclidean magnitude of each column + */ + def normL2: Vector + + /** + * L1 norm of each column + */ + def normL1: Vector } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala index c7f2576c822b1..b311d10023894 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala @@ -18,12 +18,13 @@ package org.apache.spark.mllib.tree import scala.collection.JavaConverters._ +import scala.collection.mutable import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.JavaRDD import org.apache.spark.Logging -import org.apache.spark.mllib.rdd.RDDFunctions._ import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.tree.RandomForest.NodeIndexInfo import org.apache.spark.mllib.tree.configuration.Strategy import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.configuration.FeatureType._ @@ -33,8 +34,8 @@ import org.apache.spark.mllib.tree.impurity.{Impurities, Impurity} import org.apache.spark.mllib.tree.impurity._ import org.apache.spark.mllib.tree.model._ import org.apache.spark.rdd.RDD -import org.apache.spark.storage.StorageLevel import org.apache.spark.util.random.XORShiftRandom +import org.apache.spark.SparkContext._ /** @@ -56,99 +57,10 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo * @return DecisionTreeModel that can be used for prediction */ def train(input: RDD[LabeledPoint]): DecisionTreeModel = { - - val timer = new TimeTracker() - - timer.start("total") - - timer.start("init") - - val retaggedInput = input.retag(classOf[LabeledPoint]) - val metadata = DecisionTreeMetadata.buildMetadata(retaggedInput, strategy) - logDebug("algo = " + strategy.algo) - logDebug("maxBins = " + metadata.maxBins) - - // Find the splits and the corresponding bins (interval between the splits) using a sample - // of the input data. - timer.start("findSplitsBins") - val (splits, bins) = DecisionTree.findSplitsBins(retaggedInput, metadata) - timer.stop("findSplitsBins") - logDebug("numBins: feature: number of bins") - logDebug(Range(0, metadata.numFeatures).map { featureIndex => - s"\t$featureIndex\t${metadata.numBins(featureIndex)}" - }.mkString("\n")) - - // Bin feature values (TreePoint representation). - // Cache input RDD for speedup during multiple passes. - val treeInput = TreePoint.convertToTreeRDD(retaggedInput, bins, metadata) - .persist(StorageLevel.MEMORY_AND_DISK) - - // depth of the decision tree - val maxDepth = strategy.maxDepth - require(maxDepth <= 30, - s"DecisionTree currently only supports maxDepth <= 30, but was given maxDepth = $maxDepth.") - - // Calculate level for single group construction - - // Max memory usage for aggregates - val maxMemoryUsage = strategy.maxMemoryInMB * 1024L * 1024L - logDebug("max memory usage for aggregates = " + maxMemoryUsage + " bytes.") - // TODO: Calculate memory usage more precisely. - val numElementsPerNode = DecisionTree.getElementsPerNode(metadata) - - logDebug("numElementsPerNode = " + numElementsPerNode) - val arraySizePerNode = 8 * numElementsPerNode // approx. memory usage for bin aggregate array - val maxNumberOfNodesPerGroup = math.max(maxMemoryUsage / arraySizePerNode, 1) - logDebug("maxNumberOfNodesPerGroup = " + maxNumberOfNodesPerGroup) - // nodes at a level is 2^level. level is zero indexed. - val maxLevelForSingleGroup = math.max( - (math.log(maxNumberOfNodesPerGroup) / math.log(2)).floor.toInt, 0) - logDebug("max level for single group = " + maxLevelForSingleGroup) - - timer.stop("init") - - /* - * The main idea here is to perform level-wise training of the decision tree nodes thus - * reducing the passes over the data from l to log2(l) where l is the total number of nodes. - * Each data sample is handled by a particular node at that level (or it reaches a leaf - * beforehand and is not used in later levels. - */ - - var topNode: Node = null // set on first iteration - var level = 0 - var break = false - while (level <= maxDepth && !break) { - logDebug("#####################################") - logDebug("level = " + level) - logDebug("#####################################") - - // Find best split for all nodes at a level. - timer.start("findBestSplits") - val (tmpTopNode: Node, doneTraining: Boolean) = DecisionTree.findBestSplits(treeInput, - metadata, level, topNode, splits, bins, maxLevelForSingleGroup, timer) - timer.stop("findBestSplits") - - if (level == 0) { - topNode = tmpTopNode - } - if (doneTraining) { - break = true - logDebug("done training") - } - - level += 1 - } - - logDebug("#####################################") - logDebug("Extracting tree model") - logDebug("#####################################") - - timer.stop("total") - - logInfo("Internal timing for DecisionTree:") - logInfo(s"$timer") - - new DecisionTreeModel(topNode, strategy.algo) + // Note: random seed will not be used since numTrees = 1. + val rf = new RandomForest(strategy, numTrees = 1, featureSubsetStrategy = "all", seed = 0) + val rfModel = rf.train(input) + rfModel.trees(0) } } @@ -352,58 +264,10 @@ object DecisionTree extends Serializable with Logging { impurity, maxDepth, maxBins) } - /** - * Returns an array of optimal splits for all nodes at a given level. Splits the task into - * multiple groups if the level-wise training task could lead to memory overflow. - * - * @param input Training data: RDD of [[org.apache.spark.mllib.tree.impl.TreePoint]] - * @param metadata Learning and dataset metadata - * @param level Level of the tree - * @param topNode Root node of the tree (or invalid node when training first level). - * @param splits possible splits for all features, indexed (numFeatures)(numSplits) - * @param bins possible bins for all features, indexed (numFeatures)(numBins) - * @param maxLevelForSingleGroup the deepest level for single-group level-wise computation. - * @return (root, doneTraining) where: - * root = Root node (which is newly created on the first iteration), - * doneTraining = true if no more internal nodes were created. - */ - private[tree] def findBestSplits( - input: RDD[TreePoint], - metadata: DecisionTreeMetadata, - level: Int, - topNode: Node, - splits: Array[Array[Split]], - bins: Array[Array[Bin]], - maxLevelForSingleGroup: Int, - timer: TimeTracker = new TimeTracker): (Node, Boolean) = { - - // split into groups to avoid memory overflow during aggregation - if (level > maxLevelForSingleGroup) { - // When information for all nodes at a given level cannot be stored in memory, - // the nodes are divided into multiple groups at each level with the number of groups - // increasing exponentially per level. For example, if maxLevelForSingleGroup is 10, - // numGroups is equal to 2 at level 11 and 4 at level 12, respectively. - val numGroups = 1 << level - maxLevelForSingleGroup - logDebug("numGroups = " + numGroups) - // Iterate over each group of nodes at a level. - var groupIndex = 0 - var doneTraining = true - while (groupIndex < numGroups) { - val (_, doneTrainingGroup) = findBestSplitsPerGroup(input, metadata, level, - topNode, splits, bins, timer, numGroups, groupIndex) - doneTraining = doneTraining && doneTrainingGroup - groupIndex += 1 - } - (topNode, doneTraining) // Not first iteration, so topNode was already set. - } else { - findBestSplitsPerGroup(input, metadata, level, topNode, splits, bins, timer) - } - } - /** * Get the node index corresponding to this data point. - * This function mimics prediction, passing an example from the root node down to a node - * at the current level being trained; that node's index is returned. + * This function mimics prediction, passing an example from the root node down to a leaf + * or unsplit node; that node's index is returned. * * @param node Node in tree from which to classify the given data point. * @param binnedFeatures Binned feature vector for data point. @@ -413,14 +277,15 @@ object DecisionTree extends Serializable with Logging { * Otherwise, last node reachable in tree matching this example. * Note: This is the global node index, i.e., the index used in the tree. * This index is different from the index used during training a particular - * set of nodes in a (level, group). + * group of nodes on one call to [[findBestSplits()]]. */ private def predictNodeIndex( node: Node, binnedFeatures: Array[Int], bins: Array[Array[Bin]], unorderedFeatures: Set[Int]): Int = { - if (node.isLeaf) { + if (node.isLeaf || node.split.isEmpty) { + // Node is either leaf, or has not yet been split. node.id } else { val featureIndex = node.split.get.feature @@ -463,45 +328,58 @@ object DecisionTree extends Serializable with Logging { * for each subset is updated. * * @param agg Array storing aggregate calculation, with a set of sufficient statistics for - * each (node, feature, bin). + * each (feature, bin). * @param treePoint Data point being aggregated. - * @param nodeIndex Node corresponding to treePoint. Indexed from 0 at start of (level, group). * @param bins possible bins for all features, indexed (numFeatures)(numBins) * @param unorderedFeatures Set of indices of unordered features. + * @param instanceWeight Weight (importance) of instance in dataset. */ private def mixedBinSeqOp( agg: DTStatsAggregator, treePoint: TreePoint, - nodeIndex: Int, bins: Array[Array[Bin]], - unorderedFeatures: Set[Int]): Unit = { - // Iterate over all features. - val numFeatures = treePoint.binnedFeatures.size - val nodeOffset = agg.getNodeOffset(nodeIndex) - var featureIndex = 0 - while (featureIndex < numFeatures) { + unorderedFeatures: Set[Int], + instanceWeight: Double, + featuresForNode: Option[Array[Int]]): Unit = { + val numFeaturesPerNode = if (featuresForNode.nonEmpty) { + // Use subsampled features + featuresForNode.get.size + } else { + // Use all features + agg.metadata.numFeatures + } + // Iterate over features. + var featureIndexIdx = 0 + while (featureIndexIdx < numFeaturesPerNode) { + val featureIndex = if (featuresForNode.nonEmpty) { + featuresForNode.get.apply(featureIndexIdx) + } else { + featureIndexIdx + } if (unorderedFeatures.contains(featureIndex)) { // Unordered feature val featureValue = treePoint.binnedFeatures(featureIndex) val (leftNodeFeatureOffset, rightNodeFeatureOffset) = - agg.getLeftRightNodeFeatureOffsets(nodeIndex, featureIndex) + agg.getLeftRightFeatureOffsets(featureIndexIdx) // Update the left or right bin for each split. - val numSplits = agg.numSplits(featureIndex) + val numSplits = agg.metadata.numSplits(featureIndex) var splitIndex = 0 while (splitIndex < numSplits) { if (bins(featureIndex)(splitIndex).highSplit.categories.contains(featureValue)) { - agg.nodeFeatureUpdate(leftNodeFeatureOffset, splitIndex, treePoint.label) + agg.featureUpdate(leftNodeFeatureOffset, splitIndex, treePoint.label, + instanceWeight) } else { - agg.nodeFeatureUpdate(rightNodeFeatureOffset, splitIndex, treePoint.label) + agg.featureUpdate(rightNodeFeatureOffset, splitIndex, treePoint.label, + instanceWeight) } splitIndex += 1 } } else { // Ordered feature val binIndex = treePoint.binnedFeatures(featureIndex) - agg.nodeUpdate(nodeOffset, featureIndex, binIndex, treePoint.label) + agg.update(featureIndexIdx, binIndex, treePoint.label, instanceWeight) } - featureIndex += 1 + featureIndexIdx += 1 } } @@ -511,68 +389,77 @@ object DecisionTree extends Serializable with Logging { * For each feature, the sufficient statistics of one bin are updated. * * @param agg Array storing aggregate calculation, with a set of sufficient statistics for - * each (node, feature, bin). + * each (feature, bin). * @param treePoint Data point being aggregated. - * @param nodeIndex Node corresponding to treePoint. Indexed from 0 at start of (level, group). - * @return agg + * @param instanceWeight Weight (importance) of instance in dataset. */ private def orderedBinSeqOp( agg: DTStatsAggregator, treePoint: TreePoint, - nodeIndex: Int): Unit = { + instanceWeight: Double, + featuresForNode: Option[Array[Int]]): Unit = { val label = treePoint.label - val nodeOffset = agg.getNodeOffset(nodeIndex) - // Iterate over all features. - val numFeatures = agg.numFeatures - var featureIndex = 0 - while (featureIndex < numFeatures) { - val binIndex = treePoint.binnedFeatures(featureIndex) - agg.nodeUpdate(nodeOffset, featureIndex, binIndex, label) - featureIndex += 1 + + // Iterate over features. + if (featuresForNode.nonEmpty) { + // Use subsampled features + var featureIndexIdx = 0 + while (featureIndexIdx < featuresForNode.get.size) { + val binIndex = treePoint.binnedFeatures(featuresForNode.get.apply(featureIndexIdx)) + agg.update(featureIndexIdx, binIndex, label, instanceWeight) + featureIndexIdx += 1 + } + } else { + // Use all features + val numFeatures = agg.metadata.numFeatures + var featureIndex = 0 + while (featureIndex < numFeatures) { + val binIndex = treePoint.binnedFeatures(featureIndex) + agg.update(featureIndex, binIndex, label, instanceWeight) + featureIndex += 1 + } } } /** - * Returns an array of optimal splits for a group of nodes at a given level + * Given a group of nodes, this finds the best split for each node. * * @param input Training data: RDD of [[org.apache.spark.mllib.tree.impl.TreePoint]] * @param metadata Learning and dataset metadata - * @param level Level of the tree - * @param topNode Root node of the tree (or invalid node when training first level). + * @param topNodes Root node for each tree. Used for matching instances with nodes. + * @param nodesForGroup Mapping: treeIndex --> nodes to be split in tree + * @param treeToNodeToIndexInfo Mapping: treeIndex --> nodeIndex --> nodeIndexInfo, + * where nodeIndexInfo stores the index in the group and the + * feature subsets (if using feature subsets). * @param splits possible splits for all features, indexed (numFeatures)(numSplits) * @param bins possible bins for all features, indexed (numFeatures)(numBins) - * @param numGroups total number of node groups at the current level. Default value is set to 1. - * @param groupIndex index of the node group being processed. Default value is set to 0. - * @return (root, doneTraining) where: - * root = Root node (which is newly created on the first iteration), - * doneTraining = true if no more internal nodes were created. + * @param nodeQueue Queue of nodes to split, with values (treeIndex, node). + * Updated with new non-leaf nodes which are created. */ - private def findBestSplitsPerGroup( - input: RDD[TreePoint], + private[tree] def findBestSplits( + input: RDD[BaggedPoint[TreePoint]], metadata: DecisionTreeMetadata, - level: Int, - topNode: Node, + topNodes: Array[Node], + nodesForGroup: Map[Int, Array[Node]], + treeToNodeToIndexInfo: Map[Int, Map[Int, NodeIndexInfo]], splits: Array[Array[Split]], bins: Array[Array[Bin]], - timer: TimeTracker, - numGroups: Int = 1, - groupIndex: Int = 0): (Node, Boolean) = { + nodeQueue: mutable.Queue[(Int, Node)], + timer: TimeTracker = new TimeTracker): Unit = { /* * The high-level descriptions of the best split optimizations are noted here. * - * *Level-wise training* - * We perform bin calculations for all nodes at the given level to avoid making multiple - * passes over the data. Thus, for a slightly increased computation and storage cost we save - * several iterations over the data especially at higher levels of the decision tree. + * *Group-wise training* + * We perform bin calculations for groups of nodes to reduce the number of + * passes over the data. Each iteration requires more computation and storage, + * but saves several iterations over the data. * * *Bin-wise computation* * We use a bin-wise best split computation strategy instead of a straightforward best split * computation strategy. Instead of analyzing each sample for contribution to the left/right * child node impurity of every split, we first categorize each feature of a sample into a - * bin. Each bin is an interval between a low and high split. Since each split, and thus bin, - * is ordered (read ordering for categorical variables in the findSplitsBins method), - * we exploit this structure to calculate aggregates for bins and then use these aggregates + * bin. We exploit this structure to calculate aggregates for bins and then use these aggregates * to calculate information gain for each split. * * *Aggregation over partitions* @@ -582,42 +469,15 @@ object DecisionTree extends Serializable with Logging { * drastically reduce the communication overhead. */ - // Common calculations for multiple nested methods: - - // numNodes: Number of nodes in this (level of tree, group), - // where nodes at deeper (larger) levels may be divided into groups. - val numNodes = Node.maxNodesInLevel(level) / numGroups + // numNodes: Number of nodes in this group + val numNodes = nodesForGroup.values.map(_.size).sum logDebug("numNodes = " + numNodes) - logDebug("numFeatures = " + metadata.numFeatures) logDebug("numClasses = " + metadata.numClasses) logDebug("isMulticlass = " + metadata.isMulticlass) logDebug("isMulticlassWithCategoricalFeatures = " + metadata.isMulticlassWithCategoricalFeatures) - // shift when more than one group is used at deep tree level - val groupShift = numNodes * groupIndex - - // Used for treePointToNodeIndex to get an index for this (level, group). - // - Node.startIndexInLevel(level) gives the global index offset for nodes at this level. - // - groupShift corrects for groups in this level before the current group. - val globalNodeIndexOffset = Node.startIndexInLevel(level) + groupShift - - /** - * Find the node index for the given example. - * Nodes are indexed from 0 at the start of this (level, group). - * If the example does not reach this level, returns a value < 0. - */ - def treePointToNodeIndex(treePoint: TreePoint): Int = { - if (level == 0) { - 0 - } else { - val globalNodeIndex = - predictNodeIndex(topNode, treePoint.binnedFeatures, bins, metadata.unorderedFeatures) - globalNodeIndex - globalNodeIndexOffset - } - } - /** * Performs a sequential aggregation over a partition. * @@ -626,94 +486,137 @@ object DecisionTree extends Serializable with Logging { * * @param agg Array storing aggregate calculation, with a set of sufficient statistics for * each (node, feature, bin). - * @param treePoint Data point being aggregated. + * @param baggedPoint Data point being aggregated. * @return agg */ def binSeqOp( - agg: DTStatsAggregator, - treePoint: TreePoint): DTStatsAggregator = { - val nodeIndex = treePointToNodeIndex(treePoint) - // If the example does not reach this level, then nodeIndex < 0. - // If the example reaches this level but is handled in a different group, - // then either nodeIndex < 0 (previous group) or nodeIndex >= numNodes (later group). - if (nodeIndex >= 0 && nodeIndex < numNodes) { - if (metadata.unorderedFeatures.isEmpty) { - orderedBinSeqOp(agg, treePoint, nodeIndex) - } else { - mixedBinSeqOp(agg, treePoint, nodeIndex, bins, metadata.unorderedFeatures) + agg: Array[DTStatsAggregator], + baggedPoint: BaggedPoint[TreePoint]): Array[DTStatsAggregator] = { + treeToNodeToIndexInfo.foreach { case (treeIndex, nodeIndexToInfo) => + val nodeIndex = predictNodeIndex(topNodes(treeIndex), baggedPoint.datum.binnedFeatures, + bins, metadata.unorderedFeatures) + val nodeInfo = nodeIndexToInfo.getOrElse(nodeIndex, null) + // If the example does not reach a node in this group, then nodeIndex = null. + if (nodeInfo != null) { + val aggNodeIndex = nodeInfo.nodeIndexInGroup + val featuresForNode = nodeInfo.featureSubset + val instanceWeight = baggedPoint.subsampleWeights(treeIndex) + if (metadata.unorderedFeatures.isEmpty) { + orderedBinSeqOp(agg(aggNodeIndex), baggedPoint.datum, instanceWeight, featuresForNode) + } else { + mixedBinSeqOp(agg(aggNodeIndex), baggedPoint.datum, bins, metadata.unorderedFeatures, + instanceWeight, featuresForNode) + } } } agg } - // Calculate bin aggregates. - timer.start("aggregation") - val binAggregates: DTStatsAggregator = { - val initAgg = new DTStatsAggregator(metadata, numNodes) - input.treeAggregate(initAgg)(binSeqOp, DTStatsAggregator.binCombOp) + /** + * Get node index in group --> features indices map, + * which is a short cut to find feature indices for a node given node index in group + * @param treeToNodeToIndexInfo + * @return + */ + def getNodeToFeatures(treeToNodeToIndexInfo: Map[Int, Map[Int, NodeIndexInfo]]) + : Option[Map[Int, Array[Int]]] = if (!metadata.subsamplingFeatures) { + None + } else { + val mutableNodeToFeatures = new mutable.HashMap[Int, Array[Int]]() + treeToNodeToIndexInfo.values.foreach { nodeIdToNodeInfo => + nodeIdToNodeInfo.values.foreach { nodeIndexInfo => + assert(nodeIndexInfo.featureSubset.isDefined) + mutableNodeToFeatures(nodeIndexInfo.nodeIndexInGroup) = nodeIndexInfo.featureSubset.get + } + } + Some(mutableNodeToFeatures.toMap) } - timer.stop("aggregation") - // Calculate best splits for all nodes at a given level + // Calculate best splits for all nodes in the group timer.start("chooseSplits") - // On the first iteration, we need to get and return the newly created root node. - var newTopNode: Node = topNode - - // Iterate over all nodes at this level - var nodeIndex = 0 - var internalNodeCount = 0 - while (nodeIndex < numNodes) { - val (split: Split, stats: InformationGainStats, predict: Predict) = - binsToBestSplit(binAggregates, nodeIndex, level, metadata, splits) - logDebug("best split = " + split) - - val globalNodeIndex = globalNodeIndexOffset + nodeIndex - - // Extract info for this node at the current level. - val isLeaf = (stats.gain <= 0) || (level == metadata.maxDepth) - val node = - new Node(globalNodeIndex, predict.predict, isLeaf, Some(split), None, None, Some(stats)) - logDebug("Node = " + node) - - if (!isLeaf) { - internalNodeCount += 1 - } - if (level == 0) { - newTopNode = node - } else { - // Set parent. - val parentNode = Node.getNode(Node.parentIndex(globalNodeIndex), topNode) - if (Node.isLeftChild(globalNodeIndex)) { - parentNode.leftNode = Some(node) - } else { - parentNode.rightNode = Some(node) + + // In each partition, iterate all instances and compute aggregate stats for each node, + // yield an (nodeIndex, nodeAggregateStats) pair for each node. + // After a `reduceByKey` operation, + // stats of a node will be shuffled to a particular partition and be combined together, + // then best splits for nodes are found there. + // Finally, only best Splits for nodes are collected to driver to construct decision tree. + val nodeToFeatures = getNodeToFeatures(treeToNodeToIndexInfo) + val nodeToFeaturesBc = input.sparkContext.broadcast(nodeToFeatures) + val nodeToBestSplits = + input.mapPartitions { points => + // Construct a nodeStatsAggregators array to hold node aggregate stats, + // each node will have a nodeStatsAggregator + val nodeStatsAggregators = Array.tabulate(numNodes) { nodeIndex => + val featuresForNode = nodeToFeaturesBc.value.flatMap { nodeToFeatures => + Some(nodeToFeatures(nodeIndex)) + } + new DTStatsAggregator(metadata, featuresForNode) } - } - if (level < metadata.maxDepth) { - logDebug("leftChildIndex = " + Node.leftChildIndex(globalNodeIndex) + - ", impurity = " + stats.leftImpurity) - logDebug("rightChildIndex = " + Node.rightChildIndex(globalNodeIndex) + - ", impurity = " + stats.rightImpurity) - } - nodeIndex += 1 - } + // iterator all instances in current partition and update aggregate stats + points.foreach(binSeqOp(nodeStatsAggregators, _)) + + // transform nodeStatsAggregators array to (nodeIndex, nodeAggregateStats) pairs, + // which can be combined with other partition using `reduceByKey` + nodeStatsAggregators.view.zipWithIndex.map(_.swap).iterator + }.reduceByKey((a, b) => a.merge(b)) + .map { case (nodeIndex, aggStats) => + val featuresForNode = nodeToFeaturesBc.value.flatMap { nodeToFeatures => + Some(nodeToFeatures(nodeIndex)) + } + + // find best split for each node + val (split: Split, stats: InformationGainStats, predict: Predict) = + binsToBestSplit(aggStats, splits, featuresForNode) + (nodeIndex, (split, stats, predict)) + }.collectAsMap() + timer.stop("chooseSplits") - val doneTraining = internalNodeCount == 0 - (newTopNode, doneTraining) + // Iterate over all nodes in this group. + nodesForGroup.foreach { case (treeIndex, nodesForTree) => + nodesForTree.foreach { node => + val nodeIndex = node.id + val nodeInfo = treeToNodeToIndexInfo(treeIndex)(nodeIndex) + val aggNodeIndex = nodeInfo.nodeIndexInGroup + val (split: Split, stats: InformationGainStats, predict: Predict) = + nodeToBestSplits(aggNodeIndex) + logDebug("best split = " + split) + + // Extract info for this node. Create children if not leaf. + val isLeaf = (stats.gain <= 0) || (Node.indexToLevel(nodeIndex) == metadata.maxDepth) + assert(node.id == nodeIndex) + node.predict = predict.predict + node.isLeaf = isLeaf + node.stats = Some(stats) + logDebug("Node = " + node) + + if (!isLeaf) { + node.split = Some(split) + node.leftNode = Some(Node.emptyNode(Node.leftChildIndex(nodeIndex))) + node.rightNode = Some(Node.emptyNode(Node.rightChildIndex(nodeIndex))) + nodeQueue.enqueue((treeIndex, node.leftNode.get)) + nodeQueue.enqueue((treeIndex, node.rightNode.get)) + logDebug("leftChildIndex = " + node.leftNode.get.id + + ", impurity = " + stats.leftImpurity) + logDebug("rightChildIndex = " + node.rightNode.get.id + + ", impurity = " + stats.rightImpurity) + } + } + } + } /** * Calculate the information gain for a given (feature, split) based upon left/right aggregates. * @param leftImpurityCalculator left node aggregates for this (feature, split) * @param rightImpurityCalculator right node aggregate for this (feature, split) - * @return information gain and statistics for all splits + * @return information gain and statistics for split */ private def calculateGainForSplit( leftImpurityCalculator: ImpurityCalculator, rightImpurityCalculator: ImpurityCalculator, - level: Int, metadata: DecisionTreeMetadata): InformationGainStats = { val leftCount = leftImpurityCalculator.count val rightCount = rightImpurityCalculator.count @@ -753,7 +656,7 @@ object DecisionTree extends Serializable with Logging { * Calculate predict value for current node, given stats of any split. * Note that this function is called only once for each node. * @param leftImpurityCalculator left node aggregates for a split - * @param rightImpurityCalculator right node aggregates for a node + * @param rightImpurityCalculator right node aggregates for a split * @return predict value for current node */ private def calculatePredict( @@ -770,30 +673,33 @@ object DecisionTree extends Serializable with Logging { /** * Find the best split for a node. * @param binAggregates Bin statistics. - * @param nodeIndex Index for node to split in this (level, group). - * @return tuple for best split: (Split, information gain) + * @return tuple for best split: (Split, information gain, prediction at node) */ private def binsToBestSplit( binAggregates: DTStatsAggregator, - nodeIndex: Int, - level: Int, - metadata: DecisionTreeMetadata, - splits: Array[Array[Split]]): (Split, InformationGainStats, Predict) = { + splits: Array[Array[Split]], + featuresForNode: Option[Array[Int]]): (Split, InformationGainStats, Predict) = { // calculate predict only once var predict: Option[Predict] = None // For each (feature, split), calculate the gain, and select the best (feature, split). - val (bestSplit, bestSplitStats) = Range(0, metadata.numFeatures).map { featureIndex => - val numSplits = metadata.numSplits(featureIndex) - if (metadata.isContinuous(featureIndex)) { + val (bestSplit, bestSplitStats) = + Range(0, binAggregates.metadata.numFeaturesPerNode).map { featureIndexIdx => + val featureIndex = if (featuresForNode.nonEmpty) { + featuresForNode.get.apply(featureIndexIdx) + } else { + featureIndexIdx + } + val numSplits = binAggregates.metadata.numSplits(featureIndex) + if (binAggregates.metadata.isContinuous(featureIndex)) { // Cumulative sum (scanLeft) of bin statistics. // Afterwards, binAggregates for a bin is the sum of aggregates for // that bin + all preceding bins. - val nodeFeatureOffset = binAggregates.getNodeFeatureOffset(nodeIndex, featureIndex) + val nodeFeatureOffset = binAggregates.getFeatureOffset(featureIndexIdx) var splitIndex = 0 while (splitIndex < numSplits) { - binAggregates.mergeForNodeFeature(nodeFeatureOffset, splitIndex + 1, splitIndex) + binAggregates.mergeForFeature(nodeFeatureOffset, splitIndex + 1, splitIndex) splitIndex += 1 } // Find best split. @@ -803,27 +709,29 @@ object DecisionTree extends Serializable with Logging { val rightChildStats = binAggregates.getImpurityCalculator(nodeFeatureOffset, numSplits) rightChildStats.subtract(leftChildStats) predict = Some(predict.getOrElse(calculatePredict(leftChildStats, rightChildStats))) - val gainStats = calculateGainForSplit(leftChildStats, rightChildStats, level, metadata) + val gainStats = calculateGainForSplit(leftChildStats, + rightChildStats, binAggregates.metadata) (splitIdx, gainStats) }.maxBy(_._2.gain) (splits(featureIndex)(bestFeatureSplitIndex), bestFeatureGainStats) - } else if (metadata.isUnordered(featureIndex)) { + } else if (binAggregates.metadata.isUnordered(featureIndex)) { // Unordered categorical feature val (leftChildOffset, rightChildOffset) = - binAggregates.getLeftRightNodeFeatureOffsets(nodeIndex, featureIndex) + binAggregates.getLeftRightFeatureOffsets(featureIndexIdx) val (bestFeatureSplitIndex, bestFeatureGainStats) = Range(0, numSplits).map { splitIndex => val leftChildStats = binAggregates.getImpurityCalculator(leftChildOffset, splitIndex) val rightChildStats = binAggregates.getImpurityCalculator(rightChildOffset, splitIndex) predict = Some(predict.getOrElse(calculatePredict(leftChildStats, rightChildStats))) - val gainStats = calculateGainForSplit(leftChildStats, rightChildStats, level, metadata) + val gainStats = calculateGainForSplit(leftChildStats, + rightChildStats, binAggregates.metadata) (splitIndex, gainStats) }.maxBy(_._2.gain) (splits(featureIndex)(bestFeatureSplitIndex), bestFeatureGainStats) } else { // Ordered categorical feature - val nodeFeatureOffset = binAggregates.getNodeFeatureOffset(nodeIndex, featureIndex) - val numBins = metadata.numBins(featureIndex) + val nodeFeatureOffset = binAggregates.getFeatureOffset(featureIndexIdx) + val numBins = binAggregates.metadata.numBins(featureIndex) /* Each bin is one category (feature value). * The bins are ordered based on centroidForCategories, and this ordering determines which @@ -831,7 +739,7 @@ object DecisionTree extends Serializable with Logging { * * centroidForCategories is a list: (category, centroid) */ - val centroidForCategories = if (metadata.isMulticlass) { + val centroidForCategories = if (binAggregates.metadata.isMulticlass) { // For categorical variables in multiclass classification, // the bins are ordered by the impurity of their corresponding labels. Range(0, numBins).map { case featureValue => @@ -872,7 +780,7 @@ object DecisionTree extends Serializable with Logging { while (splitIndex < numSplits) { val currentCategory = categoriesSortedByCentroid(splitIndex)._1 val nextCategory = categoriesSortedByCentroid(splitIndex + 1)._1 - binAggregates.mergeForNodeFeature(nodeFeatureOffset, nextCategory, currentCategory) + binAggregates.mergeForFeature(nodeFeatureOffset, nextCategory, currentCategory) splitIndex += 1 } // lastCategory = index of bin with total aggregates for this (node, feature) @@ -887,7 +795,8 @@ object DecisionTree extends Serializable with Logging { binAggregates.getImpurityCalculator(nodeFeatureOffset, lastCategory) rightChildStats.subtract(leftChildStats) predict = Some(predict.getOrElse(calculatePredict(leftChildStats, rightChildStats))) - val gainStats = calculateGainForSplit(leftChildStats, rightChildStats, level, metadata) + val gainStats = calculateGainForSplit(leftChildStats, + rightChildStats, binAggregates.metadata) (splitIndex, gainStats) }.maxBy(_._2.gain) val categoriesForSplit = @@ -903,18 +812,6 @@ object DecisionTree extends Serializable with Logging { (bestSplit, bestSplitStats, predict.get) } - /** - * Get the number of values to be stored per node in the bin aggregates. - */ - private def getElementsPerNode(metadata: DecisionTreeMetadata): Long = { - val totalBins = metadata.numBins.map(_.toLong).sum - if (metadata.isClassification) { - metadata.numClasses * totalBins - } else { - 3 * totalBins - } - } - /** * Returns splits and bins for decision tree calculation. * Continuous and categorical features are handled differently. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala new file mode 100644 index 0000000000000..fa7a26f17c3ca --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala @@ -0,0 +1,452 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.tree + +import scala.collection.JavaConverters._ +import scala.collection.mutable + +import org.apache.spark.Logging +import org.apache.spark.annotation.Experimental +import org.apache.spark.api.java.JavaRDD +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.tree.configuration.Algo._ +import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ +import org.apache.spark.mllib.tree.configuration.Strategy +import org.apache.spark.mllib.tree.impl.{BaggedPoint, TreePoint, DecisionTreeMetadata, TimeTracker} +import org.apache.spark.mllib.tree.impurity.Impurities +import org.apache.spark.mllib.tree.model._ +import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.Utils + +/** + * :: Experimental :: + * A class which implements a random forest learning algorithm for classification and regression. + * It supports both continuous and categorical features. + * + * The settings for featureSubsetStrategy are based on the following references: + * - log2: tested in Breiman (2001) + * - sqrt: recommended by Breiman manual for random forests + * - The defaults of sqrt (classification) and onethird (regression) match the R randomForest + * package. + * @see [[http://www.stat.berkeley.edu/~breiman/randomforest2001.pdf Breiman (2001)]] + * @see [[http://www.stat.berkeley.edu/~breiman/Using_random_forests_V3.1.pdf Breiman manual for + * random forests]] + * + * @param strategy The configuration parameters for the random forest algorithm which specify + * the type of algorithm (classification, regression, etc.), feature type + * (continuous, categorical), depth of the tree, quantile calculation strategy, + * etc. + * @param numTrees If 1, then no bootstrapping is used. If > 1, then bootstrapping is done. + * @param featureSubsetStrategy Number of features to consider for splits at each node. + * Supported: "auto" (default), "all", "sqrt", "log2", "onethird". + * If "auto" is set, this parameter is set based on numTrees: + * if numTrees == 1, set to "all"; + * if numTrees > 1 (forest) set to "sqrt" for classification and + * to "onethird" for regression. + * @param seed Random seed for bootstrapping and choosing feature subsets. + */ +@Experimental +private class RandomForest ( + private val strategy: Strategy, + private val numTrees: Int, + featureSubsetStrategy: String, + private val seed: Int) + extends Serializable with Logging { + + strategy.assertValid() + require(numTrees > 0, s"RandomForest requires numTrees > 0, but was given numTrees = $numTrees.") + require(RandomForest.supportedFeatureSubsetStrategies.contains(featureSubsetStrategy), + s"RandomForest given invalid featureSubsetStrategy: $featureSubsetStrategy." + + s" Supported values: ${RandomForest.supportedFeatureSubsetStrategies.mkString(", ")}.") + + /** + * Method to train a decision tree model over an RDD + * @param input Training data: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]] + * @return RandomForestModel that can be used for prediction + */ + def train(input: RDD[LabeledPoint]): RandomForestModel = { + + val timer = new TimeTracker() + + timer.start("total") + + timer.start("init") + + val retaggedInput = input.retag(classOf[LabeledPoint]) + val metadata = + DecisionTreeMetadata.buildMetadata(retaggedInput, strategy, numTrees, featureSubsetStrategy) + logDebug("algo = " + strategy.algo) + logDebug("numTrees = " + numTrees) + logDebug("seed = " + seed) + logDebug("maxBins = " + metadata.maxBins) + logDebug("featureSubsetStrategy = " + featureSubsetStrategy) + logDebug("numFeaturesPerNode = " + metadata.numFeaturesPerNode) + + // Find the splits and the corresponding bins (interval between the splits) using a sample + // of the input data. + timer.start("findSplitsBins") + val (splits, bins) = DecisionTree.findSplitsBins(retaggedInput, metadata) + timer.stop("findSplitsBins") + logDebug("numBins: feature: number of bins") + logDebug(Range(0, metadata.numFeatures).map { featureIndex => + s"\t$featureIndex\t${metadata.numBins(featureIndex)}" + }.mkString("\n")) + + // Bin feature values (TreePoint representation). + // Cache input RDD for speedup during multiple passes. + val treeInput = TreePoint.convertToTreeRDD(retaggedInput, bins, metadata) + val baggedInput = if (numTrees > 1) { + BaggedPoint.convertToBaggedRDD(treeInput, numTrees, seed) + } else { + BaggedPoint.convertToBaggedRDDWithoutSampling(treeInput) + }.persist(StorageLevel.MEMORY_AND_DISK) + + // depth of the decision tree + val maxDepth = strategy.maxDepth + require(maxDepth <= 30, + s"DecisionTree currently only supports maxDepth <= 30, but was given maxDepth = $maxDepth.") + + // Max memory usage for aggregates + // TODO: Calculate memory usage more precisely. + val maxMemoryUsage: Long = strategy.maxMemoryInMB * 1024L * 1024L + logDebug("max memory usage for aggregates = " + maxMemoryUsage + " bytes.") + val maxMemoryPerNode = { + val featureSubset: Option[Array[Int]] = if (metadata.subsamplingFeatures) { + // Find numFeaturesPerNode largest bins to get an upper bound on memory usage. + Some(metadata.numBins.zipWithIndex.sortBy(- _._1) + .take(metadata.numFeaturesPerNode).map(_._2)) + } else { + None + } + RandomForest.aggregateSizeForNode(metadata, featureSubset) * 8L + } + require(maxMemoryPerNode <= maxMemoryUsage, + s"RandomForest/DecisionTree given maxMemoryInMB = ${strategy.maxMemoryInMB}," + + " which is too small for the given features." + + s" Minimum value = ${maxMemoryPerNode / (1024L * 1024L)}") + + timer.stop("init") + + /* + * The main idea here is to perform group-wise training of the decision tree nodes thus + * reducing the passes over the data from (# nodes) to (# nodes / maxNumberOfNodesPerGroup). + * Each data sample is handled by a particular node (or it reaches a leaf and is not used + * in lower levels). + */ + + // FIFO queue of nodes to train: (treeIndex, node) + val nodeQueue = new mutable.Queue[(Int, Node)]() + + val rng = new scala.util.Random() + rng.setSeed(seed) + + // Allocate and queue root nodes. + val topNodes: Array[Node] = Array.fill[Node](numTrees)(Node.emptyNode(nodeIndex = 1)) + Range(0, numTrees).foreach(treeIndex => nodeQueue.enqueue((treeIndex, topNodes(treeIndex)))) + + while (nodeQueue.nonEmpty) { + // Collect some nodes to split, and choose features for each node (if subsampling). + // Each group of nodes may come from one or multiple trees, and at multiple levels. + val (nodesForGroup, treeToNodeToIndexInfo) = + RandomForest.selectNodesToSplit(nodeQueue, maxMemoryUsage, metadata, rng) + // Sanity check (should never occur): + assert(nodesForGroup.size > 0, + s"RandomForest selected empty nodesForGroup. Error for unknown reason.") + + // Choose node splits, and enqueue new nodes as needed. + timer.start("findBestSplits") + DecisionTree.findBestSplits(baggedInput, metadata, topNodes, nodesForGroup, + treeToNodeToIndexInfo, splits, bins, nodeQueue, timer) + timer.stop("findBestSplits") + } + + timer.stop("total") + + logInfo("Internal timing for DecisionTree:") + logInfo(s"$timer") + + val trees = topNodes.map(topNode => new DecisionTreeModel(topNode, strategy.algo)) + RandomForestModel.build(trees) + } + +} + +object RandomForest extends Serializable with Logging { + + /** + * Method to train a decision tree model for binary or multiclass classification. + * + * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. + * Labels should take values {0, 1, ..., numClasses-1}. + * @param strategy Parameters for training each tree in the forest. + * @param numTrees Number of trees in the random forest. + * @param featureSubsetStrategy Number of features to consider for splits at each node. + * Supported: "auto" (default), "all", "sqrt", "log2", "onethird". + * If "auto" is set, this parameter is set based on numTrees: + * if numTrees == 1, set to "all"; + * if numTrees > 1 (forest) set to "sqrt" for classification and + * to "onethird" for regression. + * @param seed Random seed for bootstrapping and choosing feature subsets. + * @return RandomForestModel that can be used for prediction + */ + def trainClassifier( + input: RDD[LabeledPoint], + strategy: Strategy, + numTrees: Int, + featureSubsetStrategy: String, + seed: Int): RandomForestModel = { + require(strategy.algo == Classification, + s"RandomForest.trainClassifier given Strategy with invalid algo: ${strategy.algo}") + val rf = new RandomForest(strategy, numTrees, featureSubsetStrategy, seed) + rf.train(input) + } + + /** + * Method to train a decision tree model for binary or multiclass classification. + * + * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. + * Labels should take values {0, 1, ..., numClasses-1}. + * @param numClassesForClassification number of classes for classification. + * @param categoricalFeaturesInfo Map storing arity of categorical features. + * E.g., an entry (n -> k) indicates that feature n is categorical + * with k categories indexed from 0: {0, 1, ..., k-1}. + * @param numTrees Number of trees in the random forest. + * @param featureSubsetStrategy Number of features to consider for splits at each node. + * Supported: "auto" (default), "all", "sqrt", "log2", "onethird". + * If "auto" is set, this parameter is set based on numTrees: + * if numTrees == 1, set to "all"; + * if numTrees > 1 (forest) set to "sqrt" for classification and + * to "onethird" for regression. + * @param impurity Criterion used for information gain calculation. + * Supported values: "gini" (recommended) or "entropy". + * @param maxDepth Maximum depth of the tree. + * E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes. + * (suggested value: 4) + * @param maxBins maximum number of bins used for splitting features + * (suggested value: 100) + * @param seed Random seed for bootstrapping and choosing feature subsets. + * @return RandomForestModel that can be used for prediction + */ + def trainClassifier( + input: RDD[LabeledPoint], + numClassesForClassification: Int, + categoricalFeaturesInfo: Map[Int, Int], + numTrees: Int, + featureSubsetStrategy: String, + impurity: String, + maxDepth: Int, + maxBins: Int, + seed: Int = Utils.random.nextInt()): RandomForestModel = { + val impurityType = Impurities.fromString(impurity) + val strategy = new Strategy(Classification, impurityType, maxDepth, + numClassesForClassification, maxBins, Sort, categoricalFeaturesInfo) + trainClassifier(input, strategy, numTrees, featureSubsetStrategy, seed) + } + + /** + * Java-friendly API for [[org.apache.spark.mllib.tree.RandomForest$#trainClassifier]] + */ + def trainClassifier( + input: JavaRDD[LabeledPoint], + numClassesForClassification: Int, + categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer], + numTrees: Int, + featureSubsetStrategy: String, + impurity: String, + maxDepth: Int, + maxBins: Int, + seed: Int): RandomForestModel = { + trainClassifier(input.rdd, numClassesForClassification, + categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap, + numTrees, featureSubsetStrategy, impurity, maxDepth, maxBins, seed) + } + + /** + * Method to train a decision tree model for regression. + * + * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. + * Labels are real numbers. + * @param strategy Parameters for training each tree in the forest. + * @param numTrees Number of trees in the random forest. + * @param featureSubsetStrategy Number of features to consider for splits at each node. + * Supported: "auto" (default), "all", "sqrt", "log2", "onethird". + * If "auto" is set, this parameter is set based on numTrees: + * if numTrees == 1, set to "all"; + * if numTrees > 1 (forest) set to "sqrt" for classification and + * to "onethird" for regression. + * @param seed Random seed for bootstrapping and choosing feature subsets. + * @return RandomForestModel that can be used for prediction + */ + def trainRegressor( + input: RDD[LabeledPoint], + strategy: Strategy, + numTrees: Int, + featureSubsetStrategy: String, + seed: Int): RandomForestModel = { + require(strategy.algo == Regression, + s"RandomForest.trainRegressor given Strategy with invalid algo: ${strategy.algo}") + val rf = new RandomForest(strategy, numTrees, featureSubsetStrategy, seed) + rf.train(input) + } + + /** + * Method to train a decision tree model for regression. + * + * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. + * Labels are real numbers. + * @param categoricalFeaturesInfo Map storing arity of categorical features. + * E.g., an entry (n -> k) indicates that feature n is categorical + * with k categories indexed from 0: {0, 1, ..., k-1}. + * @param numTrees Number of trees in the random forest. + * @param featureSubsetStrategy Number of features to consider for splits at each node. + * Supported: "auto" (default), "all", "sqrt", "log2", "onethird". + * If "auto" is set, this parameter is set based on numTrees: + * if numTrees == 1, set to "all"; + * if numTrees > 1 (forest) set to "sqrt" for classification and + * to "onethird" for regression. + * @param impurity Criterion used for information gain calculation. + * Supported values: "variance". + * @param maxDepth Maximum depth of the tree. + * E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes. + * (suggested value: 4) + * @param maxBins maximum number of bins used for splitting features + * (suggested value: 100) + * @param seed Random seed for bootstrapping and choosing feature subsets. + * @return RandomForestModel that can be used for prediction + */ + def trainRegressor( + input: RDD[LabeledPoint], + categoricalFeaturesInfo: Map[Int, Int], + numTrees: Int, + featureSubsetStrategy: String, + impurity: String, + maxDepth: Int, + maxBins: Int, + seed: Int = Utils.random.nextInt()): RandomForestModel = { + val impurityType = Impurities.fromString(impurity) + val strategy = new Strategy(Regression, impurityType, maxDepth, + 0, maxBins, Sort, categoricalFeaturesInfo) + trainRegressor(input, strategy, numTrees, featureSubsetStrategy, seed) + } + + /** + * Java-friendly API for [[org.apache.spark.mllib.tree.RandomForest$#trainRegressor]] + */ + def trainRegressor( + input: JavaRDD[LabeledPoint], + categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer], + numTrees: Int, + featureSubsetStrategy: String, + impurity: String, + maxDepth: Int, + maxBins: Int, + seed: Int): RandomForestModel = { + trainRegressor(input.rdd, + categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap, + numTrees, featureSubsetStrategy, impurity, maxDepth, maxBins, seed) + } + + /** + * List of supported feature subset sampling strategies. + */ + val supportedFeatureSubsetStrategies: Array[String] = + Array("auto", "all", "sqrt", "log2", "onethird") + + private[tree] class NodeIndexInfo( + val nodeIndexInGroup: Int, + val featureSubset: Option[Array[Int]]) extends Serializable + + /** + * Pull nodes off of the queue, and collect a group of nodes to be split on this iteration. + * This tracks the memory usage for aggregates and stops adding nodes when too much memory + * will be needed; this allows an adaptive number of nodes since different nodes may require + * different amounts of memory (if featureSubsetStrategy is not "all"). + * + * @param nodeQueue Queue of nodes to split. + * @param maxMemoryUsage Bound on size of aggregate statistics. + * @return (nodesForGroup, treeToNodeToIndexInfo). + * nodesForGroup holds the nodes to split: treeIndex --> nodes in tree. + * + * treeToNodeToIndexInfo holds indices selected features for each node: + * treeIndex --> (global) node index --> (node index in group, feature indices). + * The (global) node index is the index in the tree; the node index in group is the + * index in [0, numNodesInGroup) of the node in this group. + * The feature indices are None if not subsampling features. + */ + private[tree] def selectNodesToSplit( + nodeQueue: mutable.Queue[(Int, Node)], + maxMemoryUsage: Long, + metadata: DecisionTreeMetadata, + rng: scala.util.Random): (Map[Int, Array[Node]], Map[Int, Map[Int, NodeIndexInfo]]) = { + // Collect some nodes to split: + // nodesForGroup(treeIndex) = nodes to split + val mutableNodesForGroup = new mutable.HashMap[Int, mutable.ArrayBuffer[Node]]() + val mutableTreeToNodeToIndexInfo = + new mutable.HashMap[Int, mutable.HashMap[Int, NodeIndexInfo]]() + var memUsage: Long = 0L + var numNodesInGroup = 0 + while (nodeQueue.nonEmpty && memUsage < maxMemoryUsage) { + val (treeIndex, node) = nodeQueue.head + // Choose subset of features for node (if subsampling). + val featureSubset: Option[Array[Int]] = if (metadata.subsamplingFeatures) { + // TODO: Use more efficient subsampling? (use selection-and-rejection or reservoir) + Some(rng.shuffle(Range(0, metadata.numFeatures).toList) + .take(metadata.numFeaturesPerNode).toArray) + } else { + None + } + // Check if enough memory remains to add this node to the group. + val nodeMemUsage = RandomForest.aggregateSizeForNode(metadata, featureSubset) * 8L + if (memUsage + nodeMemUsage <= maxMemoryUsage) { + nodeQueue.dequeue() + mutableNodesForGroup.getOrElseUpdate(treeIndex, new mutable.ArrayBuffer[Node]()) += node + mutableTreeToNodeToIndexInfo + .getOrElseUpdate(treeIndex, new mutable.HashMap[Int, NodeIndexInfo]())(node.id) + = new NodeIndexInfo(numNodesInGroup, featureSubset) + } + numNodesInGroup += 1 + memUsage += nodeMemUsage + } + // Convert mutable maps to immutable ones. + val nodesForGroup: Map[Int, Array[Node]] = mutableNodesForGroup.mapValues(_.toArray).toMap + val treeToNodeToIndexInfo = mutableTreeToNodeToIndexInfo.mapValues(_.toMap).toMap + (nodesForGroup, treeToNodeToIndexInfo) + } + + /** + * Get the number of values to be stored for this node in the bin aggregates. + * @param featureSubset Indices of features which may be split at this node. + * If None, then use all features. + */ + private[tree] def aggregateSizeForNode( + metadata: DecisionTreeMetadata, + featureSubset: Option[Array[Int]]): Long = { + val totalBins = if (featureSubset.nonEmpty) { + featureSubset.get.map(featureIndex => metadata.numBins(featureIndex).toLong).sum + } else { + metadata.numBins.map(_.toLong).sum + } + if (metadata.isClassification) { + metadata.numClasses * totalBins + } else { + 3 * totalBins + } + } + +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/BaggedPoint.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/BaggedPoint.scala new file mode 100644 index 0000000000000..937c8a2ac5836 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/BaggedPoint.scala @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.tree.impl + +import cern.jet.random.Poisson +import cern.jet.random.engine.DRand + +import org.apache.spark.rdd.RDD +import org.apache.spark.util.Utils + +/** + * Internal representation of a datapoint which belongs to several subsamples of the same dataset, + * particularly for bagging (e.g., for random forests). + * + * This holds one instance, as well as an array of weights which represent the (weighted) + * number of times which this instance appears in each subsample. + * E.g., (datum, [1, 0, 4]) indicates that there are 3 subsamples of the dataset and that + * this datum has 1 copy, 0 copies, and 4 copies in the 3 subsamples, respectively. + * + * @param datum Data instance + * @param subsampleWeights Weight of this instance in each subsampled dataset. + * + * TODO: This does not currently support (Double) weighted instances. Once MLlib has weighted + * dataset support, update. (We store subsampleWeights as Double for this future extension.) + */ +private[tree] class BaggedPoint[Datum](val datum: Datum, val subsampleWeights: Array[Double]) + extends Serializable + +private[tree] object BaggedPoint { + + /** + * Convert an input dataset into its BaggedPoint representation, + * choosing subsample counts for each instance. + * Each subsample has the same number of instances as the original dataset, + * and is created by subsampling with replacement. + * @param input Input dataset. + * @param numSubsamples Number of subsamples of this RDD to take. + * @param seed Random seed. + * @return BaggedPoint dataset representation + */ + def convertToBaggedRDD[Datum]( + input: RDD[Datum], + numSubsamples: Int, + seed: Int = Utils.random.nextInt()): RDD[BaggedPoint[Datum]] = { + input.mapPartitionsWithIndex { (partitionIndex, instances) => + // TODO: Support different sampling rates, and sampling without replacement. + // Use random seed = seed + partitionIndex + 1 to make generation reproducible. + val poisson = new Poisson(1.0, new DRand(seed + partitionIndex + 1)) + instances.map { instance => + val subsampleWeights = new Array[Double](numSubsamples) + var subsampleIndex = 0 + while (subsampleIndex < numSubsamples) { + subsampleWeights(subsampleIndex) = poisson.nextInt() + subsampleIndex += 1 + } + new BaggedPoint(instance, subsampleWeights) + } + } + } + + def convertToBaggedRDDWithoutSampling[Datum](input: RDD[Datum]): RDD[BaggedPoint[Datum]] = { + input.map(datum => new BaggedPoint(datum, Array(1.0))) + } + +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DTStatsAggregator.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DTStatsAggregator.scala index 61a94246711bf..55f422dff0d71 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DTStatsAggregator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DTStatsAggregator.scala @@ -19,14 +19,17 @@ package org.apache.spark.mllib.tree.impl import org.apache.spark.mllib.tree.impurity._ + + /** - * DecisionTree statistics aggregator. - * This holds a flat array of statistics for a set of (nodes, features, bins) + * DecisionTree statistics aggregator for a node. + * This holds a flat array of statistics for a set of (features, bins) * and helps with indexing. + * This class is abstract to support learning with and without feature subsampling. */ private[tree] class DTStatsAggregator( val metadata: DecisionTreeMetadata, - val numNodes: Int) extends Serializable { + featureSubset: Option[Array[Int]]) extends Serializable { /** * [[ImpurityAggregator]] instance specifying the impurity type. @@ -41,25 +44,18 @@ private[tree] class DTStatsAggregator( /** * Number of elements (Double values) used for the sufficient statistics of each bin. */ - val statsSize: Int = impurityAggregator.statsSize - - val numFeatures: Int = metadata.numFeatures + private val statsSize: Int = impurityAggregator.statsSize /** * Number of bins for each feature. This is indexed by the feature index. */ - val numBins: Array[Int] = metadata.numBins - - /** - * Number of splits for the given feature. - */ - def numSplits(featureIndex: Int): Int = metadata.numSplits(featureIndex) - - /** - * Indicator for each feature of whether that feature is an unordered feature. - * TODO: Is Array[Boolean] any faster? - */ - def isUnordered(featureIndex: Int): Boolean = metadata.isUnordered(featureIndex) + private val numBins: Array[Int] = { + if (featureSubset.isDefined) { + featureSubset.get.map(metadata.numBins(_)) + } else { + metadata.numBins + } + } /** * Offset for each feature for calculating indices into the [[allStats]] array. @@ -69,108 +65,101 @@ private[tree] class DTStatsAggregator( } /** - * Number of elements for each node, corresponding to stride between nodes in [[allStats]]. + * Indicator for each feature of whether that feature is an unordered feature. + * TODO: Is Array[Boolean] any faster? */ - private val nodeStride: Int = featureOffsets.last + def isUnordered(featureIndex: Int): Boolean = metadata.isUnordered(featureIndex) /** - * Total number of elements stored in this aggregator. + * Total number of elements stored in this aggregator */ - val allStatsSize: Int = numNodes * nodeStride + private val allStatsSize: Int = featureOffsets.last /** * Flat array of elements. - * Index for start of stats for a (node, feature, bin) is: - * index = nodeIndex * nodeStride + featureOffsets(featureIndex) + binIndex * statsSize - * Note: For unordered features, the left child stats have binIndex in [0, numBins(featureIndex)) - * and the right child stats in [numBins(featureIndex), 2 * numBins(featureIndex)) + * Index for start of stats for a (feature, bin) is: + * index = featureOffsets(featureIndex) + binIndex * statsSize + * Note: For unordered features, + * the left child stats have binIndex in [0, numBins(featureIndex) / 2)) + * and the right child stats in [numBins(featureIndex) / 2), numBins(featureIndex)) */ - val allStats: Array[Double] = new Array[Double](allStatsSize) + private val allStats: Array[Double] = new Array[Double](allStatsSize) + /** * Get an [[ImpurityCalculator]] for a given (node, feature, bin). - * @param nodeFeatureOffset For ordered features, this is a pre-computed (node, feature) offset - * from [[getNodeFeatureOffset]]. + * @param featureOffset For ordered features, this is a pre-computed (node, feature) offset + * from [[getFeatureOffset]]. * For unordered features, this is a pre-computed * (node, feature, left/right child) offset from - * [[getLeftRightNodeFeatureOffsets]]. + * [[getLeftRightFeatureOffsets]]. */ - def getImpurityCalculator(nodeFeatureOffset: Int, binIndex: Int): ImpurityCalculator = { - impurityAggregator.getCalculator(allStats, nodeFeatureOffset + binIndex * statsSize) + def getImpurityCalculator(featureOffset: Int, binIndex: Int): ImpurityCalculator = { + impurityAggregator.getCalculator(allStats, featureOffset + binIndex * statsSize) } /** - * Update the stats for a given (node, feature, bin) for ordered features, using the given label. + * Update the stats for a given (feature, bin) for ordered features, using the given label. */ - def update(nodeIndex: Int, featureIndex: Int, binIndex: Int, label: Double): Unit = { - val i = nodeIndex * nodeStride + featureOffsets(featureIndex) + binIndex * statsSize - impurityAggregator.update(allStats, i, label) + def update(featureIndex: Int, binIndex: Int, label: Double, instanceWeight: Double): Unit = { + val i = featureOffsets(featureIndex) + binIndex * statsSize + impurityAggregator.update(allStats, i, label, instanceWeight) } - /** - * Pre-compute node offset for use with [[nodeUpdate]]. - */ - def getNodeOffset(nodeIndex: Int): Int = nodeIndex * nodeStride - /** * Faster version of [[update]]. - * Update the stats for a given (node, feature, bin) for ordered features, using the given label. - * @param nodeOffset Pre-computed node offset from [[getNodeOffset]]. - */ - def nodeUpdate(nodeOffset: Int, featureIndex: Int, binIndex: Int, label: Double): Unit = { - val i = nodeOffset + featureOffsets(featureIndex) + binIndex * statsSize - impurityAggregator.update(allStats, i, label) + * Update the stats for a given (feature, bin), using the given label. + * @param featureOffset For ordered features, this is a pre-computed feature offset + * from [[getFeatureOffset]]. + * For unordered features, this is a pre-computed + * (feature, left/right child) offset from + * [[getLeftRightFeatureOffsets]]. + */ + def featureUpdate( + featureOffset: Int, + binIndex: Int, + label: Double, + instanceWeight: Double): Unit = { + impurityAggregator.update(allStats, featureOffset + binIndex * statsSize, + label, instanceWeight) } /** - * Pre-compute (node, feature) offset for use with [[nodeFeatureUpdate]]. + * Pre-compute feature offset for use with [[featureUpdate]]. * For ordered features only. */ - def getNodeFeatureOffset(nodeIndex: Int, featureIndex: Int): Int = { + def getFeatureOffset(featureIndex: Int): Int = { require(!isUnordered(featureIndex), - s"DTStatsAggregator.getNodeFeatureOffset is for ordered features only, but was called" + - s" for unordered feature $featureIndex.") - nodeIndex * nodeStride + featureOffsets(featureIndex) + s"DTStatsAggregator.getFeatureOffset is for ordered features only, but was called" + + s" for unordered feature $featureIndex.") + featureOffsets(featureIndex) } /** - * Pre-compute (node, feature) offset for use with [[nodeFeatureUpdate]]. + * Pre-compute feature offset for use with [[featureUpdate]]. * For unordered features only. */ - def getLeftRightNodeFeatureOffsets(nodeIndex: Int, featureIndex: Int): (Int, Int) = { + def getLeftRightFeatureOffsets(featureIndex: Int): (Int, Int) = { require(isUnordered(featureIndex), - s"DTStatsAggregator.getLeftRightNodeFeatureOffsets is for unordered features only," + - s" but was called for ordered feature $featureIndex.") - val baseOffset = nodeIndex * nodeStride + featureOffsets(featureIndex) + s"DTStatsAggregator.getLeftRightFeatureOffsets is for unordered features only," + + s" but was called for ordered feature $featureIndex.") + val baseOffset = featureOffsets(featureIndex) (baseOffset, baseOffset + (numBins(featureIndex) >> 1) * statsSize) } /** - * Faster version of [[update]]. - * Update the stats for a given (node, feature, bin), using the given label. - * @param nodeFeatureOffset For ordered features, this is a pre-computed (node, feature) offset - * from [[getNodeFeatureOffset]]. - * For unordered features, this is a pre-computed - * (node, feature, left/right child) offset from - * [[getLeftRightNodeFeatureOffsets]]. - */ - def nodeFeatureUpdate(nodeFeatureOffset: Int, binIndex: Int, label: Double): Unit = { - impurityAggregator.update(allStats, nodeFeatureOffset + binIndex * statsSize, label) - } - - /** - * For a given (node, feature), merge the stats for two bins. - * @param nodeFeatureOffset For ordered features, this is a pre-computed (node, feature) offset - * from [[getNodeFeatureOffset]]. + * For a given feature, merge the stats for two bins. + * @param featureOffset For ordered features, this is a pre-computed feature offset + * from [[getFeatureOffset]]. * For unordered features, this is a pre-computed - * (node, feature, left/right child) offset from - * [[getLeftRightNodeFeatureOffsets]]. + * (feature, left/right child) offset from + * [[getLeftRightFeatureOffsets]]. * @param binIndex The other bin is merged into this bin. * @param otherBinIndex This bin is not modified. */ - def mergeForNodeFeature(nodeFeatureOffset: Int, binIndex: Int, otherBinIndex: Int): Unit = { - impurityAggregator.merge(allStats, nodeFeatureOffset + binIndex * statsSize, - nodeFeatureOffset + otherBinIndex * statsSize) + def mergeForFeature(featureOffset: Int, binIndex: Int, otherBinIndex: Int): Unit = { + impurityAggregator.merge(allStats, featureOffset + binIndex * statsSize, + featureOffset + otherBinIndex * statsSize) } /** @@ -180,7 +169,7 @@ private[tree] class DTStatsAggregator( def merge(other: DTStatsAggregator): DTStatsAggregator = { require(allStatsSize == other.allStatsSize, s"DTStatsAggregator.merge requires that both aggregators have the same length stats vectors." - + s" This aggregator is of length $allStatsSize, but the other is ${other.allStatsSize}.") + + s" This aggregator is of length $allStatsSize, but the other is ${other.allStatsSize}.") var i = 0 // TODO: Test BLAS.axpy while (i < allStatsSize) { @@ -189,18 +178,4 @@ private[tree] class DTStatsAggregator( } this } - -} - -private[tree] object DTStatsAggregator extends Serializable { - - /** - * Combines two aggregates (modifying the first) and returns the combination. - */ - def binCombOp( - agg1: DTStatsAggregator, - agg2: DTStatsAggregator): DTStatsAggregator = { - agg1.merge(agg2) - } - } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala index b6d49e5555b1a..212dce25236e0 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala @@ -48,7 +48,9 @@ private[tree] class DecisionTreeMetadata( val quantileStrategy: QuantileStrategy, val maxDepth: Int, val minInstancesPerNode: Int, - val minInfoGain: Double) extends Serializable { + val minInfoGain: Double, + val numTrees: Int, + val numFeaturesPerNode: Int) extends Serializable { def isUnordered(featureIndex: Int): Boolean = unorderedFeatures.contains(featureIndex) @@ -73,6 +75,11 @@ private[tree] class DecisionTreeMetadata( numBins(featureIndex) - 1 } + /** + * Indicates if feature subsampling is being used. + */ + def subsamplingFeatures: Boolean = numFeatures != numFeaturesPerNode + } private[tree] object DecisionTreeMetadata { @@ -82,7 +89,11 @@ private[tree] object DecisionTreeMetadata { * This computes which categorical features will be ordered vs. unordered, * as well as the number of splits and bins for each feature. */ - def buildMetadata(input: RDD[LabeledPoint], strategy: Strategy): DecisionTreeMetadata = { + def buildMetadata( + input: RDD[LabeledPoint], + strategy: Strategy, + numTrees: Int, + featureSubsetStrategy: String): DecisionTreeMetadata = { val numFeatures = input.take(1)(0).features.size val numExamples = input.count() @@ -128,13 +139,43 @@ private[tree] object DecisionTreeMetadata { } } + // Set number of features to use per node (for random forests). + val _featureSubsetStrategy = featureSubsetStrategy match { + case "auto" => + if (numTrees == 1) { + "all" + } else { + if (strategy.algo == Classification) { + "sqrt" + } else { + "onethird" + } + } + case _ => featureSubsetStrategy + } + val numFeaturesPerNode: Int = _featureSubsetStrategy match { + case "all" => numFeatures + case "sqrt" => math.sqrt(numFeatures).ceil.toInt + case "log2" => math.max(1, (math.log(numFeatures) / math.log(2)).ceil.toInt) + case "onethird" => (numFeatures / 3.0).ceil.toInt + } + new DecisionTreeMetadata(numFeatures, numExamples, numClasses, numBins.max, strategy.categoricalFeaturesInfo, unorderedFeatures.toSet, numBins, strategy.impurity, strategy.quantileCalculationStrategy, strategy.maxDepth, - strategy.minInstancesPerNode, strategy.minInfoGain) + strategy.minInstancesPerNode, strategy.minInfoGain, numTrees, numFeaturesPerNode) } /** + * Version of [[buildMetadata()]] for DecisionTree. + */ + def buildMetadata( + input: RDD[LabeledPoint], + strategy: Strategy): DecisionTreeMetadata = { + buildMetadata(input, strategy, numTrees = 1, featureSubsetStrategy = "all") + } + + /** * Given the arity of a categorical feature (arity = number of categories), * return the number of bins for the feature if it is to be treated as an unordered feature. * There is 1 split for every partitioning of categories into 2 disjoint, non-empty sets; diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala index 1c8afc2d0f4bc..0e02345aa3774 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala @@ -89,12 +89,12 @@ private[tree] class EntropyAggregator(numClasses: Int) * @param allStats Flat stats array, with stats for this (node, feature, bin) contiguous. * @param offset Start index of stats for this (node, feature, bin). */ - def update(allStats: Array[Double], offset: Int, label: Double): Unit = { + def update(allStats: Array[Double], offset: Int, label: Double, instanceWeight: Double): Unit = { if (label >= statsSize) { throw new IllegalArgumentException(s"EntropyAggregator given label $label" + s" but requires label < numClasses (= $statsSize).") } - allStats(offset + label.toInt) += 1 + allStats(offset + label.toInt) += instanceWeight } /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala index 5cfdf345d163c..7c83cd48e16a0 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala @@ -85,12 +85,12 @@ private[tree] class GiniAggregator(numClasses: Int) * @param allStats Flat stats array, with stats for this (node, feature, bin) contiguous. * @param offset Start index of stats for this (node, feature, bin). */ - def update(allStats: Array[Double], offset: Int, label: Double): Unit = { + def update(allStats: Array[Double], offset: Int, label: Double, instanceWeight: Double): Unit = { if (label >= statsSize) { throw new IllegalArgumentException(s"GiniAggregator given label $label" + s" but requires label < numClasses (= $statsSize).") } - allStats(offset + label.toInt) += 1 + allStats(offset + label.toInt) += instanceWeight } /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala index 5a047d6cb5480..60e2ab2bb829e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala @@ -78,7 +78,7 @@ private[tree] abstract class ImpurityAggregator(val statsSize: Int) extends Seri * @param allStats Flat stats array, with stats for this (node, feature, bin) contiguous. * @param offset Start index of stats for this (node, feature, bin). */ - def update(allStats: Array[Double], offset: Int, label: Double): Unit + def update(allStats: Array[Double], offset: Int, label: Double, instanceWeight: Double): Unit /** * Get an [[ImpurityCalculator]] for a (node, feature, bin). diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala index e9ccecb1b8067..df9eafa5da16a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala @@ -75,10 +75,10 @@ private[tree] class VarianceAggregator() * @param allStats Flat stats array, with stats for this (node, feature, bin) contiguous. * @param offset Start index of stats for this (node, feature, bin). */ - def update(allStats: Array[Double], offset: Int, label: Double): Unit = { - allStats(offset) += 1 - allStats(offset + 1) += label - allStats(offset + 2) += label * label + def update(allStats: Array[Double], offset: Int, label: Double, instanceWeight: Double): Unit = { + allStats(offset) += instanceWeight + allStats(offset + 1) += instanceWeight * label + allStats(offset + 2) += instanceWeight * label * label } /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala index 271b2c4ad813e..ec1d99ab26f9c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala @@ -68,15 +68,23 @@ class DecisionTreeModel(val topNode: Node, val algo: Algo) extends Serializable } /** - * Print full model. + * Print a summary of the model. */ override def toString: String = algo match { case Classification => - s"DecisionTreeModel classifier\n" + topNode.subtreeToString(2) + s"DecisionTreeModel classifier of depth $depth with $numNodes nodes" case Regression => - s"DecisionTreeModel regressor\n" + topNode.subtreeToString(2) + s"DecisionTreeModel regressor of depth $depth with $numNodes nodes" case _ => throw new IllegalArgumentException( s"DecisionTreeModel given unknown algo parameter: $algo.") } + /** + * Print the full model to a string. + */ + def toDebugString: String = { + val header = toString + "\n" + header + topNode.subtreeToString(2) + } + } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala index f3e2619bd8ba0..a89e71e115806 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala @@ -38,6 +38,17 @@ class InformationGainStats( "gain = %f, impurity = %f, left impurity = %f, right impurity = %f" .format(gain, impurity, leftImpurity, rightImpurity) } + + override def equals(o: Any) = + o match { + case other: InformationGainStats => { + gain == other.gain && + impurity == other.impurity && + leftImpurity == other.leftImpurity && + rightImpurity == other.rightImpurity + } + case _ => false + } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala index 5f0095d23c7ed..56c3e25d9285f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala @@ -41,12 +41,12 @@ import org.apache.spark.mllib.linalg.Vector @DeveloperApi class Node ( val id: Int, - val predict: Double, - val isLeaf: Boolean, - val split: Option[Split], + var predict: Double, + var isLeaf: Boolean, + var split: Option[Split], var leftNode: Option[Node], var rightNode: Option[Node], - val stats: Option[InformationGainStats]) extends Serializable with Logging { + var stats: Option[InformationGainStats]) extends Serializable with Logging { override def toString = "id = " + id + ", isLeaf = " + isLeaf + ", predict = " + predict + ", " + "split = " + split + ", stats = " + stats @@ -167,6 +167,11 @@ class Node ( private[tree] object Node { + /** + * Return a node with the given node id (but nothing else set). + */ + def emptyNode(nodeIndex: Int): Node = new Node(nodeIndex, 0, false, None, None, None, None) + /** * Return the index of the left child of this node. */ diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/RandomForestModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/RandomForestModel.scala new file mode 100644 index 0000000000000..4d66d6d81caa5 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/RandomForestModel.scala @@ -0,0 +1,115 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.tree.model + +import scala.collection.mutable + +import org.apache.spark.annotation.Experimental +import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.tree.configuration.Algo._ +import org.apache.spark.rdd.RDD + +/** + * :: Experimental :: + * Random forest model for classification or regression. + * This model stores a collection of [[DecisionTreeModel]] instances and uses them to make + * aggregate predictions. + * @param trees Trees which make up this forest. This cannot be empty. + * @param algo algorithm type -- classification or regression + */ +@Experimental +class RandomForestModel(val trees: Array[DecisionTreeModel], val algo: Algo) extends Serializable { + + require(trees.size > 0, s"RandomForestModel cannot be created with empty trees collection.") + + /** + * Predict values for a single data point. + * + * @param features array representing a single data point + * @return Double prediction from the trained model + */ + def predict(features: Vector): Double = { + algo match { + case Classification => + val predictionToCount = new mutable.HashMap[Int, Int]() + trees.foreach { tree => + val prediction = tree.predict(features).toInt + predictionToCount(prediction) = predictionToCount.getOrElse(prediction, 0) + 1 + } + predictionToCount.maxBy(_._2)._1 + case Regression => + trees.map(_.predict(features)).sum / trees.size + } + } + + /** + * Predict values for the given data set. + * + * @param features RDD representing data points to be predicted + * @return RDD[Double] where each entry contains the corresponding prediction + */ + def predict(features: RDD[Vector]): RDD[Double] = { + features.map(x => predict(x)) + } + + /** + * Get number of trees in forest. + */ + def numTrees: Int = trees.size + + /** + * Get total number of nodes, summed over all trees in the forest. + */ + def totalNumNodes: Int = trees.map(tree => tree.numNodes).sum + + /** + * Print a summary of the model. + */ + override def toString: String = algo match { + case Classification => + s"RandomForestModel classifier with $numTrees trees" + case Regression => + s"RandomForestModel regressor with $numTrees trees" + case _ => throw new IllegalArgumentException( + s"RandomForestModel given unknown algo parameter: $algo.") + } + + /** + * Print the full model to a string. + */ + def toDebugString: String = { + val header = toString + "\n" + header + trees.zipWithIndex.map { case (tree, treeIndex) => + s" Tree $treeIndex:\n" + tree.topNode.subtreeToString(4) + }.fold("")(_ + _) + } + +} + +private[tree] object RandomForestModel { + + def build(trees: Array[DecisionTreeModel]): RandomForestModel = { + require(trees.size > 0, s"RandomForestModel cannot be created with empty trees collection.") + val algo: Algo = trees(0).algo + require(trees.forall(_.algo == algo), + "RandomForestModel cannot combine trees which have different output types" + + " (classification/regression).") + new RandomForestModel(trees, algo) + } + +} diff --git a/mllib/src/test/java/org/apache/spark/mllib/feature/JavaTfIdfSuite.java b/mllib/src/test/java/org/apache/spark/mllib/feature/JavaTfIdfSuite.java index e8d99f4ae43ae..064263e02cd11 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/feature/JavaTfIdfSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/feature/JavaTfIdfSuite.java @@ -63,4 +63,24 @@ public void tfIdf() { Assert.assertEquals(0.0, v.apply(indexOfThis), 1e-15); } } + + @Test + public void tfIdfMinimumDocumentFrequency() { + // The tests are to check Java compatibility. + HashingTF tf = new HashingTF(); + JavaRDD> documents = sc.parallelize(Lists.newArrayList( + Lists.newArrayList("this is a sentence".split(" ")), + Lists.newArrayList("this is another sentence".split(" ")), + Lists.newArrayList("this is still a sentence".split(" "))), 2); + JavaRDD termFreqs = tf.transform(documents); + termFreqs.collect(); + IDF idf = new IDF(2); + JavaRDD tfIdfs = idf.fit(termFreqs).transform(termFreqs); + List localTfIdfs = tfIdfs.collect(); + int indexOfThis = tf.indexOf("this"); + for (Vector v: localTfIdfs) { + Assert.assertEquals(0.0, v.apply(indexOfThis), 1e-15); + } + } + } diff --git a/mllib/src/test/resources/log4j.properties b/mllib/src/test/resources/log4j.properties index ddfc4ac6b23ed..a469badf603c6 100644 --- a/mllib/src/test/resources/log4j.properties +++ b/mllib/src/test/resources/log4j.properties @@ -21,7 +21,7 @@ log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false log4j.appender.file.file=target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout -log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose log4j.logger.org.eclipse.jetty=WARN diff --git a/mllib/src/test/scala/org/apache/spark/mllib/api/python/PythonMLLibAPISuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/api/python/PythonMLLibAPISuite.scala index 092d67bbc5238..db8ed62fa46ce 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/api/python/PythonMLLibAPISuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/api/python/PythonMLLibAPISuite.scala @@ -19,12 +19,15 @@ package org.apache.spark.mllib.api.python import org.scalatest.FunSuite -import org.apache.spark.mllib.linalg.{Matrices, Vectors} +import org.apache.spark.mllib.linalg.{DenseMatrix, Matrices, Vectors} import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.recommendation.Rating class PythonMLLibAPISuite extends FunSuite { - test("vector serialization") { + SerDe.initialize() + + test("pickle vector") { val vectors = Seq( Vectors.dense(Array.empty[Double]), Vectors.dense(0.0), @@ -33,14 +36,13 @@ class PythonMLLibAPISuite extends FunSuite { Vectors.sparse(1, Array.empty[Int], Array.empty[Double]), Vectors.sparse(2, Array(1), Array(-2.0))) vectors.foreach { v => - val bytes = SerDe.serializeDoubleVector(v) - val u = SerDe.deserializeDoubleVector(bytes) + val u = SerDe.loads(SerDe.dumps(v)) assert(u.getClass === v.getClass) assert(u === v) } } - test("labeled point serialization") { + test("pickle labeled point") { val points = Seq( LabeledPoint(0.0, Vectors.dense(Array.empty[Double])), LabeledPoint(1.0, Vectors.dense(0.0)), @@ -49,34 +51,44 @@ class PythonMLLibAPISuite extends FunSuite { LabeledPoint(1.0, Vectors.sparse(1, Array.empty[Int], Array.empty[Double])), LabeledPoint(-0.5, Vectors.sparse(2, Array(1), Array(-2.0)))) points.foreach { p => - val bytes = SerDe.serializeLabeledPoint(p) - val q = SerDe.deserializeLabeledPoint(bytes) + val q = SerDe.loads(SerDe.dumps(p)).asInstanceOf[LabeledPoint] assert(q.label === p.label) assert(q.features.getClass === p.features.getClass) assert(q.features === p.features) } } - test("double serialization") { + test("pickle double") { for (x <- List(123.0, -10.0, 0.0, Double.MaxValue, Double.MinValue, Double.NaN)) { - val bytes = SerDe.serializeDouble(x) - val deser = SerDe.deserializeDouble(bytes) + val deser = SerDe.loads(SerDe.dumps(x.asInstanceOf[AnyRef])).asInstanceOf[Double] // We use `equals` here for comparison because we cannot use `==` for NaN assert(x.equals(deser)) } } - test("matrix to 2D array") { + test("pickle matrix") { val values = Array[Double](0, 1.2, 3, 4.56, 7, 8) val matrix = Matrices.dense(2, 3, values) - val arr = SerDe.to2dArray(matrix) - val expected = Array(Array[Double](0, 3, 7), Array[Double](1.2, 4.56, 8)) - assert(arr === expected) + val nm = SerDe.loads(SerDe.dumps(matrix)).asInstanceOf[DenseMatrix] + assert(matrix === nm) // Test conversion for empty matrix val empty = Array[Double]() val emptyMatrix = Matrices.dense(0, 0, empty) - val empty2D = SerDe.to2dArray(emptyMatrix) - assert(empty2D === Array[Array[Double]]()) + val ne = SerDe.loads(SerDe.dumps(emptyMatrix)).asInstanceOf[DenseMatrix] + assert(emptyMatrix == ne) + } + + test("pickle rating") { + val rat = new Rating(1, 2, 3.0) + val rat2 = SerDe.loads(SerDe.dumps(rat)).asInstanceOf[Rating] + assert(rat == rat2) + + // Test name of class only occur once + val rats = (1 to 10).map(x => new Rating(x, x + 1, x + 3.0)).toArray + val bytes = SerDe.dumps(rats) + assert(bytes.toString.split("Rating").length == 1) + assert(bytes.length / 10 < 25) // 25 bytes per rating + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/feature/IDFSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/feature/IDFSuite.scala index 53d9c0c640b98..43974f84e3ca8 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/feature/IDFSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/feature/IDFSuite.scala @@ -38,7 +38,7 @@ class IDFSuite extends FunSuite with LocalSparkContext { val idf = new IDF val model = idf.fit(termFrequencies) val expected = Vectors.dense(Array(0, 3, 1, 2).map { x => - math.log((m.toDouble + 1.0) / (x + 1.0)) + math.log((m + 1.0) / (x + 1.0)) }) assert(model.idf ~== expected absTol 1e-12) val tfidf = model.transform(termFrequencies).cache().zipWithIndex().map(_.swap).collectAsMap() @@ -54,4 +54,38 @@ class IDFSuite extends FunSuite with LocalSparkContext { assert(tfidf2.indices === Array(1)) assert(tfidf2.values(0) ~== (1.0 * expected(1)) absTol 1e-12) } + + test("idf minimum document frequency filtering") { + val n = 4 + val localTermFrequencies = Seq( + Vectors.sparse(n, Array(1, 3), Array(1.0, 2.0)), + Vectors.dense(0.0, 1.0, 2.0, 3.0), + Vectors.sparse(n, Array(1), Array(1.0)) + ) + val m = localTermFrequencies.size + val termFrequencies = sc.parallelize(localTermFrequencies, 2) + val idf = new IDF(minDocFreq = 1) + val model = idf.fit(termFrequencies) + val expected = Vectors.dense(Array(0, 3, 1, 2).map { x => + if (x > 0) { + math.log((m + 1.0) / (x + 1.0)) + } else { + 0 + } + }) + assert(model.idf ~== expected absTol 1e-12) + val tfidf = model.transform(termFrequencies).cache().zipWithIndex().map(_.swap).collectAsMap() + assert(tfidf.size === 3) + val tfidf0 = tfidf(0L).asInstanceOf[SparseVector] + assert(tfidf0.indices === Array(1, 3)) + assert(Vectors.dense(tfidf0.values) ~== + Vectors.dense(1.0 * expected(1), 2.0 * expected(3)) absTol 1e-12) + val tfidf1 = tfidf(1L).asInstanceOf[DenseVector] + assert(Vectors.dense(tfidf1.values) ~== + Vectors.dense(0.0, 1.0 * expected(1), 2.0 * expected(2), 3.0 * expected(3)) absTol 1e-12) + val tfidf2 = tfidf(2L).asInstanceOf[SparseVector] + assert(tfidf2.indices === Array(1)) + assert(tfidf2.values(0) ~== (1.0 * expected(1)) absTol 1e-12) + } + } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/BLASSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/BLASSuite.scala index 1952e6734ecf7..5d70c914f14b0 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/BLASSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/BLASSuite.scala @@ -126,4 +126,115 @@ class BLASSuite extends FunSuite { } } } + + test("gemm") { + + val dA = + new DenseMatrix(4, 3, Array(0.0, 1.0, 0.0, 0.0, 2.0, 0.0, 1.0, 0.0, 0.0, 0.0, 0.0, 3.0)) + val sA = new SparseMatrix(4, 3, Array(0, 1, 3, 4), Array(1, 0, 2, 3), Array(1.0, 2.0, 1.0, 3.0)) + + val B = new DenseMatrix(3, 2, Array(1.0, 0.0, 0.0, 0.0, 2.0, 1.0)) + val expected = new DenseMatrix(4, 2, Array(0.0, 1.0, 0.0, 0.0, 4.0, 0.0, 2.0, 3.0)) + + assert(dA multiply B ~== expected absTol 1e-15) + assert(sA multiply B ~== expected absTol 1e-15) + + val C1 = new DenseMatrix(4, 2, Array(1.0, 0.0, 2.0, 1.0, 0.0, 0.0, 1.0, 0.0)) + val C2 = C1.copy + val C3 = C1.copy + val C4 = C1.copy + val C5 = C1.copy + val C6 = C1.copy + val C7 = C1.copy + val C8 = C1.copy + val expected2 = new DenseMatrix(4, 2, Array(2.0, 1.0, 4.0, 2.0, 4.0, 0.0, 4.0, 3.0)) + val expected3 = new DenseMatrix(4, 2, Array(2.0, 2.0, 4.0, 2.0, 8.0, 0.0, 6.0, 6.0)) + + gemm(1.0, dA, B, 2.0, C1) + gemm(1.0, sA, B, 2.0, C2) + gemm(2.0, dA, B, 2.0, C3) + gemm(2.0, sA, B, 2.0, C4) + assert(C1 ~== expected2 absTol 1e-15) + assert(C2 ~== expected2 absTol 1e-15) + assert(C3 ~== expected3 absTol 1e-15) + assert(C4 ~== expected3 absTol 1e-15) + + withClue("columns of A don't match the rows of B") { + intercept[Exception] { + gemm(true, false, 1.0, dA, B, 2.0, C1) + } + } + + val dAT = + new DenseMatrix(3, 4, Array(0.0, 2.0, 0.0, 1.0, 0.0, 0.0, 0.0, 1.0, 0.0, 0.0, 0.0, 3.0)) + val sAT = + new SparseMatrix(3, 4, Array(0, 1, 2, 3, 4), Array(1, 0, 1, 2), Array(2.0, 1.0, 1.0, 3.0)) + + assert(dAT transposeMultiply B ~== expected absTol 1e-15) + assert(sAT transposeMultiply B ~== expected absTol 1e-15) + + gemm(true, false, 1.0, dAT, B, 2.0, C5) + gemm(true, false, 1.0, sAT, B, 2.0, C6) + gemm(true, false, 2.0, dAT, B, 2.0, C7) + gemm(true, false, 2.0, sAT, B, 2.0, C8) + assert(C5 ~== expected2 absTol 1e-15) + assert(C6 ~== expected2 absTol 1e-15) + assert(C7 ~== expected3 absTol 1e-15) + assert(C8 ~== expected3 absTol 1e-15) + } + + test("gemv") { + + val dA = + new DenseMatrix(4, 3, Array(0.0, 1.0, 0.0, 0.0, 2.0, 0.0, 1.0, 0.0, 0.0, 0.0, 0.0, 3.0)) + val sA = new SparseMatrix(4, 3, Array(0, 1, 3, 4), Array(1, 0, 2, 3), Array(1.0, 2.0, 1.0, 3.0)) + + val x = new DenseVector(Array(1.0, 2.0, 3.0)) + val expected = new DenseVector(Array(4.0, 1.0, 2.0, 9.0)) + + assert(dA multiply x ~== expected absTol 1e-15) + assert(sA multiply x ~== expected absTol 1e-15) + + val y1 = new DenseVector(Array(1.0, 3.0, 1.0, 0.0)) + val y2 = y1.copy + val y3 = y1.copy + val y4 = y1.copy + val y5 = y1.copy + val y6 = y1.copy + val y7 = y1.copy + val y8 = y1.copy + val expected2 = new DenseVector(Array(6.0, 7.0, 4.0, 9.0)) + val expected3 = new DenseVector(Array(10.0, 8.0, 6.0, 18.0)) + + gemv(1.0, dA, x, 2.0, y1) + gemv(1.0, sA, x, 2.0, y2) + gemv(2.0, dA, x, 2.0, y3) + gemv(2.0, sA, x, 2.0, y4) + assert(y1 ~== expected2 absTol 1e-15) + assert(y2 ~== expected2 absTol 1e-15) + assert(y3 ~== expected3 absTol 1e-15) + assert(y4 ~== expected3 absTol 1e-15) + withClue("columns of A don't match the rows of B") { + intercept[Exception] { + gemv(true, 1.0, dA, x, 2.0, y1) + } + } + + val dAT = + new DenseMatrix(3, 4, Array(0.0, 2.0, 0.0, 1.0, 0.0, 0.0, 0.0, 1.0, 0.0, 0.0, 0.0, 3.0)) + val sAT = + new SparseMatrix(3, 4, Array(0, 1, 2, 3, 4), Array(1, 0, 1, 2), Array(2.0, 1.0, 1.0, 3.0)) + + assert(dAT transposeMultiply x ~== expected absTol 1e-15) + assert(sAT transposeMultiply x ~== expected absTol 1e-15) + + gemv(true, 1.0, dAT, x, 2.0, y5) + gemv(true, 1.0, sAT, x, 2.0, y6) + gemv(true, 2.0, dAT, x, 2.0, y7) + gemv(true, 2.0, sAT, x, 2.0, y8) + assert(y5 ~== expected2 absTol 1e-15) + assert(y6 ~== expected2 absTol 1e-15) + assert(y7 ~== expected3 absTol 1e-15) + assert(y8 ~== expected3 absTol 1e-15) + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/BreezeMatrixConversionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/BreezeMatrixConversionSuite.scala index 82d49c76ed02b..73a6d3a27d868 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/BreezeMatrixConversionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/BreezeMatrixConversionSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.mllib.linalg import org.scalatest.FunSuite -import breeze.linalg.{DenseMatrix => BDM} +import breeze.linalg.{DenseMatrix => BDM, CSCMatrix => BSM} class BreezeMatrixConversionSuite extends FunSuite { test("dense matrix to breeze") { @@ -37,4 +37,26 @@ class BreezeMatrixConversionSuite extends FunSuite { assert(mat.numCols === breeze.cols) assert(mat.values.eq(breeze.data), "should not copy data") } + + test("sparse matrix to breeze") { + val values = Array(1.0, 2.0, 4.0, 5.0) + val colPtrs = Array(0, 2, 4) + val rowIndices = Array(1, 2, 1, 2) + val mat = Matrices.sparse(3, 2, colPtrs, rowIndices, values) + val breeze = mat.toBreeze.asInstanceOf[BSM[Double]] + assert(breeze.rows === mat.numRows) + assert(breeze.cols === mat.numCols) + assert(breeze.data.eq(mat.asInstanceOf[SparseMatrix].values), "should not copy data") + } + + test("sparse breeze matrix to sparse matrix") { + val values = Array(1.0, 2.0, 4.0, 5.0) + val colPtrs = Array(0, 2, 4) + val rowIndices = Array(1, 2, 1, 2) + val breeze = new BSM[Double](values, 3, 2, colPtrs, rowIndices) + val mat = Matrices.fromBreeze(breeze).asInstanceOf[SparseMatrix] + assert(mat.numRows === breeze.rows) + assert(mat.numCols === breeze.cols) + assert(mat.values.eq(breeze.data), "should not copy data") + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala index 9c66b4db9f16b..5f8b8c4b72697 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala @@ -36,4 +36,80 @@ class MatricesSuite extends FunSuite { Matrices.dense(3, 2, Array(0.0, 1.0, 2.0)) } } + + test("sparse matrix construction") { + val m = 3 + val n = 2 + val values = Array(1.0, 2.0, 4.0, 5.0) + val colPtrs = Array(0, 2, 4) + val rowIndices = Array(1, 2, 1, 2) + val mat = Matrices.sparse(m, n, colPtrs, rowIndices, values).asInstanceOf[SparseMatrix] + assert(mat.numRows === m) + assert(mat.numCols === n) + assert(mat.values.eq(values), "should not copy data") + assert(mat.colPtrs.eq(colPtrs), "should not copy data") + assert(mat.rowIndices.eq(rowIndices), "should not copy data") + } + + test("sparse matrix construction with wrong number of elements") { + intercept[IllegalArgumentException] { + Matrices.sparse(3, 2, Array(0, 1), Array(1, 2, 1), Array(0.0, 1.0, 2.0)) + } + + intercept[IllegalArgumentException] { + Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(0.0, 1.0, 2.0)) + } + } + + test("matrix copies are deep copies") { + val m = 3 + val n = 2 + + val denseMat = Matrices.dense(m, n, Array(0.0, 1.0, 2.0, 3.0, 4.0, 5.0)) + val denseCopy = denseMat.copy + + assert(!denseMat.toArray.eq(denseCopy.toArray)) + + val values = Array(1.0, 2.0, 4.0, 5.0) + val colPtrs = Array(0, 2, 4) + val rowIndices = Array(1, 2, 1, 2) + val sparseMat = Matrices.sparse(m, n, colPtrs, rowIndices, values) + val sparseCopy = sparseMat.copy + + assert(!sparseMat.toArray.eq(sparseCopy.toArray)) + } + + test("matrix indexing and updating") { + val m = 3 + val n = 2 + val allValues = Array(0.0, 1.0, 2.0, 3.0, 4.0, 0.0) + + val denseMat = new DenseMatrix(m, n, allValues) + + assert(denseMat(0, 1) === 3.0) + assert(denseMat(0, 1) === denseMat.values(3)) + assert(denseMat(0, 1) === denseMat(3)) + assert(denseMat(0, 0) === 0.0) + + denseMat.update(0, 0, 10.0) + assert(denseMat(0, 0) === 10.0) + assert(denseMat.values(0) === 10.0) + + val sparseValues = Array(1.0, 2.0, 3.0, 4.0) + val colPtrs = Array(0, 2, 4) + val rowIndices = Array(1, 2, 0, 1) + val sparseMat = new SparseMatrix(m, n, colPtrs, rowIndices, sparseValues) + + assert(sparseMat(0, 1) === 3.0) + assert(sparseMat(0, 1) === sparseMat.values(2)) + assert(sparseMat(0, 0) === 0.0) + + intercept[NoSuchElementException] { + sparseMat.update(0, 0, 10.0) + } + + sparseMat.update(0, 1, 10.0) + assert(sparseMat(0, 1) === 10.0) + assert(sparseMat.values(2) === 10.0) + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala index 1d3a3221365cc..63f3ed58c0d4d 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala @@ -95,6 +95,40 @@ class RowMatrixSuite extends FunSuite with LocalSparkContext { } } + test("similar columns") { + val colMags = Vectors.dense(Math.sqrt(126), Math.sqrt(66), Math.sqrt(94)) + val expected = BDM( + (0.0, 54.0, 72.0), + (0.0, 0.0, 78.0), + (0.0, 0.0, 0.0)) + + for (i <- 0 until n; j <- 0 until n) { + expected(i, j) /= (colMags(i) * colMags(j)) + } + + for (mat <- Seq(denseMat, sparseMat)) { + val G = mat.columnSimilarities(0.11).toBreeze() + for (i <- 0 until n; j <- 0 until n) { + if (expected(i, j) > 0) { + val actual = expected(i, j) + val estimate = G(i, j) + assert(math.abs(actual - estimate) / actual < 0.2, + s"Similarities not close enough: $actual vs $estimate") + } + } + } + + for (mat <- Seq(denseMat, sparseMat)) { + val G = mat.columnSimilarities() + assert(closeToZero(G.toBreeze() - expected)) + } + + for (mat <- Seq(denseMat, sparseMat)) { + val G = mat.columnSimilaritiesDIMSUM(colMags.toArray, 150.0) + assert(closeToZero(G.toBreeze() - expected)) + } + } + test("svd of a full-rank matrix") { for (mat <- Seq(denseMat, sparseMat)) { for (mode <- Seq("auto", "local-svd", "local-eigs", "dist-eigs")) { @@ -190,6 +224,9 @@ class RowMatrixSuite extends FunSuite with LocalSparkContext { assert(summary.numNonzeros === Vectors.dense(3.0, 3.0, 4.0), "nnz mismatch") assert(summary.max === Vectors.dense(9.0, 7.0, 8.0), "max mismatch") assert(summary.min === Vectors.dense(0.0, 0.0, 1.0), "column mismatch.") + assert(summary.normL2 === Vectors.dense(Math.sqrt(126), Math.sqrt(66), Math.sqrt(94)), + "magnitude mismatch.") + assert(summary.normL1 === Vectors.dense(18.0, 12.0, 16.0), "L1 norm mismatch") } } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala index 2b2e579b992f6..a48ed71a1c5fc 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 @@ -18,6 +18,7 @@ package org.apache.spark.mllib.tree import scala.collection.JavaConverters._ +import scala.collection.mutable import org.scalatest.FunSuite @@ -26,39 +27,13 @@ import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.configuration.FeatureType._ import org.apache.spark.mllib.tree.configuration.Strategy -import org.apache.spark.mllib.tree.impl.{DecisionTreeMetadata, TreePoint} +import org.apache.spark.mllib.tree.impl.{BaggedPoint, DecisionTreeMetadata, TreePoint} import org.apache.spark.mllib.tree.impurity.{Entropy, Gini, Variance} import org.apache.spark.mllib.tree.model.{InformationGainStats, DecisionTreeModel, Node} import org.apache.spark.mllib.util.LocalSparkContext class DecisionTreeSuite extends FunSuite with LocalSparkContext { - def validateClassifier( - model: DecisionTreeModel, - input: Seq[LabeledPoint], - requiredAccuracy: Double) { - val predictions = input.map(x => model.predict(x.features)) - val numOffPredictions = predictions.zip(input).count { case (prediction, expected) => - prediction != expected.label - } - val accuracy = (input.length - numOffPredictions).toDouble / input.length - assert(accuracy >= requiredAccuracy, - s"validateClassifier calculated accuracy $accuracy but required $requiredAccuracy.") - } - - def validateRegressor( - model: DecisionTreeModel, - input: Seq[LabeledPoint], - requiredMSE: Double) { - val predictions = input.map(x => model.predict(x.features)) - val squaredError = predictions.zip(input).map { case (prediction, expected) => - val err = prediction - expected.label - err * err - }.sum - val mse = squaredError / input.length - assert(mse <= requiredMSE, s"validateRegressor calculated MSE $mse but required $requiredMSE.") - } - test("Binary classification with continuous features: split and bin calculation") { val arr = DecisionTreeSuite.generateOrderedLabeledPointsWithLabel1() assert(arr.length === 1000) @@ -233,7 +208,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { numClassesForClassification = 100, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 10, 1-> 10)) - // 2^10 - 1 > 100, so categorical features will be ordered + // 2^(10-1) - 1 > 100, so categorical features will be ordered val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) assert(!metadata.isUnordered(featureIndex = 0)) @@ -269,9 +244,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(splits(0).length === 0) assert(bins(0).length === 0) - val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val (rootNode: Node, doneTraining: Boolean) = - DecisionTree.findBestSplits(treeInput, metadata, 0, null, splits, bins, 10) + val rootNode = DecisionTree.train(rdd, strategy).topNode val split = rootNode.split.get assert(split.categories === List(1.0)) @@ -299,10 +272,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(!metadata.isUnordered(featureIndex = 0)) assert(!metadata.isUnordered(featureIndex = 1)) - val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) - val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val (rootNode, doneTraining) = DecisionTree.findBestSplits(treeInput, metadata, 0, - null, splits, bins, 10) + val rootNode = DecisionTree.train(rdd, strategy).topNode val split = rootNode.split.get assert(split.categories.length === 1) @@ -331,7 +301,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(!metadata.isUnordered(featureIndex = 1)) val model = DecisionTree.train(rdd, strategy) - validateRegressor(model, arr, 0.0) + DecisionTreeSuite.validateRegressor(model, arr, 0.0) assert(model.numNodes === 3) assert(model.depth === 1) } @@ -352,12 +322,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bins.length === 2) assert(bins(0).length === 100) - val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val (rootNode, doneTraining) = DecisionTree.findBestSplits(treeInput, metadata, 0, - null, splits, bins, 10) - - val split = rootNode.split.get - assert(split.feature === 0) + val rootNode = DecisionTree.train(rdd, strategy).topNode val stats = rootNode.stats.get assert(stats.gain === 0) @@ -381,12 +346,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bins.length === 2) assert(bins(0).length === 100) - val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val (rootNode, doneTraining) = DecisionTree.findBestSplits(treeInput, metadata, 0, - null, splits, bins, 10) - - val split = rootNode.split.get - assert(split.feature === 0) + val rootNode = DecisionTree.train(rdd, strategy).topNode val stats = rootNode.stats.get assert(stats.gain === 0) @@ -411,12 +371,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bins.length === 2) assert(bins(0).length === 100) - val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val (rootNode, doneTraining) = DecisionTree.findBestSplits(treeInput, metadata, 0, - null, splits, bins, 10) - - val split = rootNode.split.get - assert(split.feature === 0) + val rootNode = DecisionTree.train(rdd, strategy).topNode val stats = rootNode.stats.get assert(stats.gain === 0) @@ -441,12 +396,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bins.length === 2) assert(bins(0).length === 100) - val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val (rootNode, doneTraining) = DecisionTree.findBestSplits(treeInput, metadata, 0, - null, splits, bins, 10) - - val split = rootNode.split.get - assert(split.feature === 0) + val rootNode = DecisionTree.train(rdd, strategy).topNode val stats = rootNode.stats.get assert(stats.gain === 0) @@ -471,25 +421,39 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val strategyOneNode = new Strategy(Classification, Entropy, maxDepth = 1, numClassesForClassification = 2, maxBins = 100) val modelOneNode = DecisionTree.train(rdd, strategyOneNode) - val rootNodeCopy1 = modelOneNode.topNode.deepCopy() - val rootNodeCopy2 = modelOneNode.topNode.deepCopy() + val rootNode1 = modelOneNode.topNode.deepCopy() + val rootNode2 = modelOneNode.topNode.deepCopy() + assert(rootNode1.leftNode.nonEmpty) + assert(rootNode1.rightNode.nonEmpty) - // Single group second level tree construction. val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val (rootNode, _) = DecisionTree.findBestSplits(treeInput, metadata, 1, - rootNodeCopy1, splits, bins, 10) - assert(rootNode.leftNode.nonEmpty) - assert(rootNode.rightNode.nonEmpty) + val baggedInput = BaggedPoint.convertToBaggedRDDWithoutSampling(treeInput) + + // Single group second level tree construction. + val nodesForGroup = Map((0, Array(rootNode1.leftNode.get, rootNode1.rightNode.get))) + val treeToNodeToIndexInfo = Map((0, Map( + (rootNode1.leftNode.get.id, new RandomForest.NodeIndexInfo(0, None)), + (rootNode1.rightNode.get.id, new RandomForest.NodeIndexInfo(1, None))))) + val nodeQueue = new mutable.Queue[(Int, Node)]() + DecisionTree.findBestSplits(baggedInput, metadata, Array(rootNode1), + nodesForGroup, treeToNodeToIndexInfo, splits, bins, nodeQueue) val children1 = new Array[Node](2) - children1(0) = rootNode.leftNode.get - children1(1) = rootNode.rightNode.get - - // maxLevelForSingleGroup parameter is set to 0 to force splitting into groups for second - // level tree construction. - val (rootNode2, _) = DecisionTree.findBestSplits(treeInput, metadata, 1, - rootNodeCopy2, splits, bins, 0) - assert(rootNode2.leftNode.nonEmpty) - assert(rootNode2.rightNode.nonEmpty) + children1(0) = rootNode1.leftNode.get + children1(1) = rootNode1.rightNode.get + + // Train one second-level node at a time. + val nodesForGroupA = Map((0, Array(rootNode2.leftNode.get))) + val treeToNodeToIndexInfoA = Map((0, Map( + (rootNode2.leftNode.get.id, new RandomForest.NodeIndexInfo(0, None))))) + nodeQueue.clear() + DecisionTree.findBestSplits(baggedInput, metadata, Array(rootNode2), + nodesForGroupA, treeToNodeToIndexInfoA, splits, bins, nodeQueue) + val nodesForGroupB = Map((0, Array(rootNode2.rightNode.get))) + val treeToNodeToIndexInfoB = Map((0, Map( + (rootNode2.rightNode.get.id, new RandomForest.NodeIndexInfo(0, None))))) + nodeQueue.clear() + DecisionTree.findBestSplits(baggedInput, metadata, Array(rootNode2), + nodesForGroupB, treeToNodeToIndexInfoB, splits, bins, nodeQueue) val children2 = new Array[Node](2) children2(0) = rootNode2.leftNode.get children2(1) = rootNode2.rightNode.get @@ -521,10 +485,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(metadata.isUnordered(featureIndex = 0)) assert(metadata.isUnordered(featureIndex = 1)) - val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) - val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val (rootNode, doneTraining) = DecisionTree.findBestSplits(treeInput, metadata, 0, - null, splits, bins, 10) + val rootNode = DecisionTree.train(rdd, strategy).topNode val split = rootNode.split.get assert(split.feature === 0) @@ -544,7 +505,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { numClassesForClassification = 2) val model = DecisionTree.train(rdd, strategy) - validateClassifier(model, arr, 1.0) + DecisionTreeSuite.validateClassifier(model, arr, 1.0) assert(model.numNodes === 3) assert(model.depth === 1) } @@ -561,7 +522,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { numClassesForClassification = 2) val model = DecisionTree.train(rdd, strategy) - validateClassifier(model, arr, 1.0) + DecisionTreeSuite.validateClassifier(model, arr, 1.0) assert(model.numNodes === 3) assert(model.depth === 1) assert(model.topNode.split.get.feature === 1) @@ -581,14 +542,11 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(metadata.isUnordered(featureIndex = 1)) val model = DecisionTree.train(rdd, strategy) - validateClassifier(model, arr, 1.0) + DecisionTreeSuite.validateClassifier(model, arr, 1.0) assert(model.numNodes === 3) assert(model.depth === 1) - val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) - val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val (rootNode, doneTraining) = DecisionTree.findBestSplits(treeInput, metadata, 0, - null, splits, bins, 10) + val rootNode = model.topNode val split = rootNode.split.get assert(split.feature === 0) @@ -610,12 +568,9 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) val model = DecisionTree.train(rdd, strategy) - validateClassifier(model, arr, 0.9) + DecisionTreeSuite.validateClassifier(model, arr, 0.9) - val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) - val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val (rootNode, doneTraining) = DecisionTree.findBestSplits(treeInput, metadata, 0, - null, splits, bins, 10) + val rootNode = model.topNode val split = rootNode.split.get assert(split.feature === 1) @@ -635,12 +590,9 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(metadata.isUnordered(featureIndex = 0)) val model = DecisionTree.train(rdd, strategy) - validateClassifier(model, arr, 0.9) + DecisionTreeSuite.validateClassifier(model, arr, 0.9) - val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) - val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val (rootNode, doneTraining) = DecisionTree.findBestSplits(treeInput, metadata, 0, - null, splits, bins, 10) + val rootNode = model.topNode val split = rootNode.split.get assert(split.feature === 1) @@ -660,10 +612,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(!metadata.isUnordered(featureIndex = 0)) assert(!metadata.isUnordered(featureIndex = 1)) - val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) - val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val (rootNode, doneTraining) = DecisionTree.findBestSplits(treeInput, metadata, 0, - null, splits, bins, 10) + val rootNode = DecisionTree.train(rdd, strategy).topNode val split = rootNode.split.get assert(split.feature === 0) @@ -682,7 +631,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(strategy.isMulticlassClassification) val model = DecisionTree.train(rdd, strategy) - validateClassifier(model, arr, 0.6) + DecisionTreeSuite.validateClassifier(model, arr, 0.6) } test("split must satisfy min instances per node requirements") { @@ -691,24 +640,20 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { arr(1) = new LabeledPoint(1.0, Vectors.sparse(2, Seq((1, 1.0)))) arr(2) = new LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 1.0)))) - val input = sc.parallelize(arr) + val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 2, numClassesForClassification = 2, minInstancesPerNode = 2) - val model = DecisionTree.train(input, strategy) + val model = DecisionTree.train(rdd, strategy) assert(model.topNode.isLeaf) assert(model.topNode.predict == 0.0) - val predicts = input.map(p => model.predict(p.features)).collect() + val predicts = rdd.map(p => model.predict(p.features)).collect() predicts.foreach { predict => assert(predict == 0.0) } - // test for findBestSplits when no valid split can be found - val metadata = DecisionTreeMetadata.buildMetadata(input, strategy) - val (splits, bins) = DecisionTree.findSplitsBins(input, metadata) - val treeInput = TreePoint.convertToTreeRDD(input, bins, metadata) - val (rootNode, doneTraining) = DecisionTree.findBestSplits(treeInput, metadata, 0, - null, splits, bins, 10) + // test when no valid split can be found + val rootNode = model.topNode val gain = rootNode.stats.get assert(gain == InformationGainStats.invalidInformationGainStats) @@ -723,15 +668,12 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { arr(2) = new LabeledPoint(0.0, Vectors.dense(0.0, 0.0)) arr(3) = new LabeledPoint(0.0, Vectors.dense(0.0, 0.0)) - val input = sc.parallelize(arr) + val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxBins = 2, maxDepth = 2, categoricalFeaturesInfo = Map(0 -> 2, 1-> 2), numClassesForClassification = 2, minInstancesPerNode = 2) - val metadata = DecisionTreeMetadata.buildMetadata(input, strategy) - val (splits, bins) = DecisionTree.findSplitsBins(input, metadata) - val treeInput = TreePoint.convertToTreeRDD(input, bins, metadata) - val (rootNode, doneTraining) = DecisionTree.findBestSplits(treeInput, metadata, 0, - null, splits, bins, 10) + + val rootNode = DecisionTree.train(rdd, strategy).topNode val split = rootNode.split.get val gain = rootNode.stats.get @@ -757,12 +699,8 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(predict == 0.0) } - // test for findBestSplits when no valid split can be found - val metadata = DecisionTreeMetadata.buildMetadata(input, strategy) - val (splits, bins) = DecisionTree.findSplitsBins(input, metadata) - val treeInput = TreePoint.convertToTreeRDD(input, bins, metadata) - val (rootNode, doneTraining) = DecisionTree.findBestSplits(treeInput, metadata, 0, - null, splits, bins, 10) + // test when no valid split can be found + val rootNode = model.topNode val gain = rootNode.stats.get assert(gain == InformationGainStats.invalidInformationGainStats) @@ -771,6 +709,32 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { object DecisionTreeSuite { + def validateClassifier( + model: DecisionTreeModel, + input: Seq[LabeledPoint], + requiredAccuracy: Double) { + val predictions = input.map(x => model.predict(x.features)) + val numOffPredictions = predictions.zip(input).count { case (prediction, expected) => + prediction != expected.label + } + val accuracy = (input.length - numOffPredictions).toDouble / input.length + assert(accuracy >= requiredAccuracy, + s"validateClassifier calculated accuracy $accuracy but required $requiredAccuracy.") + } + + def validateRegressor( + model: DecisionTreeModel, + input: Seq[LabeledPoint], + requiredMSE: Double) { + val predictions = input.map(x => model.predict(x.features)) + val squaredError = predictions.zip(input).map { case (prediction, expected) => + val err = prediction - expected.label + err * err + }.sum + val mse = squaredError / input.length + assert(mse <= requiredMSE, s"validateRegressor calculated MSE $mse but required $requiredMSE.") + } + def generateOrderedLabeledPointsWithLabel0(): Array[LabeledPoint] = { val arr = new Array[LabeledPoint](1000) for (i <- 0 until 1000) { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala new file mode 100644 index 0000000000000..20d372dc1d3ca --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala @@ -0,0 +1,246 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.tree + +import scala.collection.mutable + +import org.scalatest.FunSuite + +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.tree.configuration.Algo._ +import org.apache.spark.mllib.tree.configuration.Strategy +import org.apache.spark.mllib.tree.impl.{BaggedPoint, DecisionTreeMetadata} +import org.apache.spark.mllib.tree.impurity.{Gini, Variance} +import org.apache.spark.mllib.tree.model.{Node, RandomForestModel} +import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.util.StatCounter + +/** + * Test suite for [[RandomForest]]. + */ +class RandomForestSuite extends FunSuite with LocalSparkContext { + + test("BaggedPoint RDD: without subsampling") { + val arr = RandomForestSuite.generateOrderedLabeledPoints(numFeatures = 1) + val rdd = sc.parallelize(arr) + val baggedRDD = BaggedPoint.convertToBaggedRDDWithoutSampling(rdd) + baggedRDD.collect().foreach { baggedPoint => + assert(baggedPoint.subsampleWeights.size == 1 && baggedPoint.subsampleWeights(0) == 1) + } + } + + test("BaggedPoint RDD: with subsampling") { + val numSubsamples = 100 + val (expectedMean, expectedStddev) = (1.0, 1.0) + + val seeds = Array(123, 5354, 230, 349867, 23987) + val arr = RandomForestSuite.generateOrderedLabeledPoints(numFeatures = 1) + val rdd = sc.parallelize(arr) + seeds.foreach { seed => + val baggedRDD = BaggedPoint.convertToBaggedRDD(rdd, numSubsamples, seed = seed) + val subsampleCounts: Array[Array[Double]] = baggedRDD.map(_.subsampleWeights).collect() + RandomForestSuite.testRandomArrays(subsampleCounts, numSubsamples, expectedMean, + expectedStddev, epsilon = 0.01) + } + } + + test("Binary classification with continuous features:" + + " comparing DecisionTree vs. RandomForest(numTrees = 1)") { + + val arr = RandomForestSuite.generateOrderedLabeledPoints(numFeatures = 50) + val rdd = sc.parallelize(arr) + val categoricalFeaturesInfo = Map.empty[Int, Int] + val numTrees = 1 + + val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 2, + numClassesForClassification = 2, categoricalFeaturesInfo = categoricalFeaturesInfo) + + val rf = RandomForest.trainClassifier(rdd, strategy, numTrees = numTrees, + featureSubsetStrategy = "auto", seed = 123) + assert(rf.trees.size === 1) + val rfTree = rf.trees(0) + + val dt = DecisionTree.train(rdd, strategy) + + RandomForestSuite.validateClassifier(rf, arr, 0.9) + DecisionTreeSuite.validateClassifier(dt, arr, 0.9) + + // Make sure trees are the same. + assert(rfTree.toString == dt.toString) + } + + test("Regression with continuous features:" + + " comparing DecisionTree vs. RandomForest(numTrees = 1)") { + + val arr = RandomForestSuite.generateOrderedLabeledPoints(numFeatures = 50) + val rdd = sc.parallelize(arr) + val categoricalFeaturesInfo = Map.empty[Int, Int] + val numTrees = 1 + + val strategy = new Strategy(algo = Regression, impurity = Variance, maxDepth = 2, + numClassesForClassification = 2, categoricalFeaturesInfo = categoricalFeaturesInfo) + + val rf = RandomForest.trainRegressor(rdd, strategy, numTrees = numTrees, + featureSubsetStrategy = "auto", seed = 123) + assert(rf.trees.size === 1) + val rfTree = rf.trees(0) + + val dt = DecisionTree.train(rdd, strategy) + + RandomForestSuite.validateRegressor(rf, arr, 0.01) + DecisionTreeSuite.validateRegressor(dt, arr, 0.01) + + // Make sure trees are the same. + assert(rfTree.toString == dt.toString) + } + + test("Binary classification with continuous features: subsampling features") { + val numFeatures = 50 + val arr = RandomForestSuite.generateOrderedLabeledPoints(numFeatures) + val rdd = sc.parallelize(arr) + val categoricalFeaturesInfo = Map.empty[Int, Int] + + val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 2, + numClassesForClassification = 2, categoricalFeaturesInfo = categoricalFeaturesInfo) + + // Select feature subset for top nodes. Return true if OK. + def checkFeatureSubsetStrategy( + numTrees: Int, + featureSubsetStrategy: String, + numFeaturesPerNode: Int): Unit = { + val seeds = Array(123, 5354, 230, 349867, 23987) + val maxMemoryUsage: Long = 128 * 1024L * 1024L + val metadata = + DecisionTreeMetadata.buildMetadata(rdd, strategy, numTrees, featureSubsetStrategy) + seeds.foreach { seed => + val failString = s"Failed on test with:" + + s"numTrees=$numTrees, featureSubsetStrategy=$featureSubsetStrategy," + + s" numFeaturesPerNode=$numFeaturesPerNode, seed=$seed" + val nodeQueue = new mutable.Queue[(Int, Node)]() + val topNodes: Array[Node] = new Array[Node](numTrees) + Range(0, numTrees).foreach { treeIndex => + topNodes(treeIndex) = Node.emptyNode(nodeIndex = 1) + nodeQueue.enqueue((treeIndex, topNodes(treeIndex))) + } + val rng = new scala.util.Random(seed = seed) + val (nodesForGroup: Map[Int, Array[Node]], + treeToNodeToIndexInfo: Map[Int, Map[Int, RandomForest.NodeIndexInfo]]) = + RandomForest.selectNodesToSplit(nodeQueue, maxMemoryUsage, metadata, rng) + + assert(nodesForGroup.size === numTrees, failString) + assert(nodesForGroup.values.forall(_.size == 1), failString) // 1 node per tree + + if (numFeaturesPerNode == numFeatures) { + // featureSubset values should all be None + assert(treeToNodeToIndexInfo.values.forall(_.values.forall(_.featureSubset.isEmpty)), + failString) + } else { + // Check number of features. + assert(treeToNodeToIndexInfo.values.forall(_.values.forall( + _.featureSubset.get.size === numFeaturesPerNode)), failString) + } + } + } + + checkFeatureSubsetStrategy(numTrees = 1, "auto", numFeatures) + checkFeatureSubsetStrategy(numTrees = 1, "all", numFeatures) + checkFeatureSubsetStrategy(numTrees = 1, "sqrt", math.sqrt(numFeatures).ceil.toInt) + checkFeatureSubsetStrategy(numTrees = 1, "log2", + (math.log(numFeatures) / math.log(2)).ceil.toInt) + checkFeatureSubsetStrategy(numTrees = 1, "onethird", (numFeatures / 3.0).ceil.toInt) + + checkFeatureSubsetStrategy(numTrees = 2, "all", numFeatures) + checkFeatureSubsetStrategy(numTrees = 2, "auto", math.sqrt(numFeatures).ceil.toInt) + checkFeatureSubsetStrategy(numTrees = 2, "sqrt", math.sqrt(numFeatures).ceil.toInt) + checkFeatureSubsetStrategy(numTrees = 2, "log2", + (math.log(numFeatures) / math.log(2)).ceil.toInt) + checkFeatureSubsetStrategy(numTrees = 2, "onethird", (numFeatures / 3.0).ceil.toInt) + } + +} + +object RandomForestSuite { + + /** + * Aggregates all values in data, and tests whether the empirical mean and stddev are within + * epsilon of the expected values. + * @param data Every element of the data should be an i.i.d. sample from some distribution. + */ + def testRandomArrays( + data: Array[Array[Double]], + numCols: Int, + expectedMean: Double, + expectedStddev: Double, + epsilon: Double) { + val values = new mutable.ArrayBuffer[Double]() + data.foreach { row => + assert(row.size == numCols) + values ++= row + } + val stats = new StatCounter(values) + assert(math.abs(stats.mean - expectedMean) < epsilon) + assert(math.abs(stats.stdev - expectedStddev) < epsilon) + } + + def validateClassifier( + model: RandomForestModel, + input: Seq[LabeledPoint], + requiredAccuracy: Double) { + val predictions = input.map(x => model.predict(x.features)) + val numOffPredictions = predictions.zip(input).count { case (prediction, expected) => + prediction != expected.label + } + val accuracy = (input.length - numOffPredictions).toDouble / input.length + assert(accuracy >= requiredAccuracy, + s"validateClassifier calculated accuracy $accuracy but required $requiredAccuracy.") + } + + def validateRegressor( + model: RandomForestModel, + input: Seq[LabeledPoint], + requiredMSE: Double) { + val predictions = input.map(x => model.predict(x.features)) + val squaredError = predictions.zip(input).map { case (prediction, expected) => + val err = prediction - expected.label + err * err + }.sum + val mse = squaredError / input.length + assert(mse <= requiredMSE, s"validateRegressor calculated MSE $mse but required $requiredMSE.") + } + + def generateOrderedLabeledPoints(numFeatures: Int): Array[LabeledPoint] = { + val numInstances = 1000 + val arr = new Array[LabeledPoint](numInstances) + for (i <- 0 until numInstances) { + val label = if (i < numInstances / 10) { + 0.0 + } else if (i < numInstances / 2) { + 1.0 + } else if (i < numInstances * 0.9) { + 0.0 + } else { + 1.0 + } + val features = Array.fill[Double](numFeatures)(i.toDouble) + arr(i) = new LabeledPoint(label, Vectors.dense(features)) + } + arr + } + +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtils.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtils.scala index 29cc42d8cbea7..30b906aaa3ba4 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtils.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtils.scala @@ -17,7 +17,7 @@ package org.apache.spark.mllib.util -import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.linalg.{Matrix, Vector} import org.scalatest.exceptions.TestFailedException object TestingUtils { @@ -169,4 +169,67 @@ object TestingUtils { override def toString = x.toString } + case class CompareMatrixRightSide( + fun: (Matrix, Matrix, Double) => Boolean, y: Matrix, eps: Double, method: String) + + /** + * Implicit class for comparing two matrices using relative tolerance or absolute tolerance. + */ + implicit class MatrixWithAlmostEquals(val x: Matrix) { + + /** + * When the difference of two vectors are within eps, returns true; otherwise, returns false. + */ + def ~=(r: CompareMatrixRightSide): Boolean = r.fun(x, r.y, r.eps) + + /** + * When the difference of two vectors are within eps, returns false; otherwise, returns true. + */ + def !~=(r: CompareMatrixRightSide): Boolean = !r.fun(x, r.y, r.eps) + + /** + * Throws exception when the difference of two vectors are NOT within eps; + * otherwise, returns true. + */ + def ~==(r: CompareMatrixRightSide): Boolean = { + if (!r.fun(x, r.y, r.eps)) { + throw new TestFailedException( + s"Expected \n$x\n and \n${r.y}\n to be within ${r.eps}${r.method} for all elements.", 0) + } + true + } + + /** + * Throws exception when the difference of two matrices are within eps; otherwise, returns true. + */ + def !~==(r: CompareMatrixRightSide): Boolean = { + if (r.fun(x, r.y, r.eps)) { + throw new TestFailedException( + s"Did not expect \n$x\n and \n${r.y}\n to be within " + + "${r.eps}${r.method} for all elements.", 0) + } + true + } + + /** + * Comparison using absolute tolerance. + */ + def absTol(eps: Double): CompareMatrixRightSide = CompareMatrixRightSide( + (x: Matrix, y: Matrix, eps: Double) => { + x.toArray.zip(y.toArray).forall(x => x._1 ~= x._2 absTol eps) + }, x, eps, ABS_TOL_MSG) + + /** + * Comparison using relative tolerance. Note that comparing against sparse vector + * with elements having value of zero will raise exception because it involves with + * comparing against zero. + */ + def relTol(eps: Double): CompareMatrixRightSide = CompareMatrixRightSide( + (x: Matrix, y: Matrix, eps: Double) => { + x.toArray.zip(y.toArray).forall(x => x._1 ~= x._2 relTol eps) + }, x, eps, REL_TOL_MSG) + + override def toString = x.toString + } + } diff --git a/pom.xml b/pom.xml index 520aed3806937..7756c89b00cad 100644 --- a/pom.xml +++ b/pom.xml @@ -138,6 +138,7 @@ 0.7.1 1.8.3 1.1.0 + 4.2.6 64m 512m @@ -222,18 +223,6 @@ false - - - spark-staging-1030 - Spark 1.1.0 Staging (1030) - https://repository.apache.org/content/repositories/orgapachespark-1030/ - - true - - - false - - @@ -712,6 +701,35 @@ + + org.apache.hadoop + hadoop-yarn-server-tests + ${yarn.version} + tests + test + + + asm + asm + + + org.ow2.asm + asm + + + org.jboss.netty + netty + + + javax.servlet + servlet-api + + + commons-logging + commons-logging + + + org.apache.hadoop hadoop-yarn-server-web-proxy @@ -1187,7 +1205,7 @@ org.apache.zookeeper zookeeper - 3.4.5-mapr-1406 + 3.4.5-mapr-1406 diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 2f1e05dfcc7b1..d499302124461 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -37,6 +37,20 @@ object MimaExcludes { Seq( MimaBuild.excludeSparkPackage("deploy"), MimaBuild.excludeSparkPackage("graphx") + ) ++ + MimaBuild.excludeSparkClass("mllib.linalg.Matrix") ++ + MimaBuild.excludeSparkClass("mllib.linalg.Vector") ++ + Seq( + ProblemFilters.exclude[IncompatibleTemplateDefProblem]( + "org.apache.spark.scheduler.TaskLocation"), + // Added normL1 and normL2 to trait MultivariateStatisticalSummary + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.mllib.stat.MultivariateStatisticalSummary.normL1"), + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.mllib.stat.MultivariateStatisticalSummary.normL2"), + // MapStatus should be private[spark] + ProblemFilters.exclude[IncompatibleTemplateDefProblem]( + "org.apache.spark.scheduler.MapStatus") ) case v if v.startsWith("1.1") => diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index ab9f8ba120e83..01a5b20e7c51d 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -221,7 +221,8 @@ object SQL { |import org.apache.spark.sql.catalyst.util._ |import org.apache.spark.sql.execution |import org.apache.spark.sql.test.TestSQLContext._ - |import org.apache.spark.sql.parquet.ParquetTestData""".stripMargin + |import org.apache.spark.sql.parquet.ParquetTestData""".stripMargin, + cleanupCommands in console := "sparkContext.stop()" ) } @@ -249,7 +250,8 @@ object Hive { |import org.apache.spark.sql.execution |import org.apache.spark.sql.hive._ |import org.apache.spark.sql.hive.test.TestHive._ - |import org.apache.spark.sql.parquet.ParquetTestData""".stripMargin + |import org.apache.spark.sql.parquet.ParquetTestData""".stripMargin, + cleanupCommands in console := "sparkContext.stop()" ) } @@ -336,7 +338,7 @@ object TestSettings { fork := true, javaOptions in Test += "-Dspark.test.home=" + sparkHome, javaOptions in Test += "-Dspark.testing=1", - javaOptions in Test += "-Dspark.ports.maxRetries=100", + javaOptions in Test += "-Dspark.port.maxRetries=100", javaOptions in Test += "-Dspark.ui.enabled=false", javaOptions in Test += "-Dsun.io.serialization.extendedDebugInfo=true", javaOptions in Test ++= System.getProperties.filter(_._1 startsWith "spark") diff --git a/python/docs/modules.rst b/python/docs/modules.rst deleted file mode 100644 index 183564659fbcf..0000000000000 --- a/python/docs/modules.rst +++ /dev/null @@ -1,7 +0,0 @@ -. -= - -.. toctree:: - :maxdepth: 4 - - pyspark diff --git a/python/docs/pyspark.mllib.rst b/python/docs/pyspark.mllib.rst index e95d19e97f151..4548b8739ed91 100644 --- a/python/docs/pyspark.mllib.rst +++ b/python/docs/pyspark.mllib.rst @@ -20,6 +20,14 @@ pyspark.mllib.clustering module :undoc-members: :show-inheritance: +pyspark.mllib.feature module +------------------------------- + +.. automodule:: pyspark.mllib.feature + :members: + :undoc-members: + :show-inheritance: + pyspark.mllib.linalg module --------------------------- diff --git a/python/epydoc.conf b/python/epydoc.conf index 51c0faf359939..8593e08deda19 100644 --- a/python/epydoc.conf +++ b/python/epydoc.conf @@ -34,5 +34,5 @@ private: no exclude: pyspark.cloudpickle pyspark.worker pyspark.join pyspark.java_gateway pyspark.examples pyspark.shell pyspark.tests - pyspark.rddsampler pyspark.daemon pyspark.mllib._common + pyspark.rddsampler pyspark.daemon pyspark.mllib.tests pyspark.shuffle diff --git a/python/pyspark/accumulators.py b/python/pyspark/accumulators.py index ccbca67656c8d..b8cdbbe3cf2b6 100644 --- a/python/pyspark/accumulators.py +++ b/python/pyspark/accumulators.py @@ -215,6 +215,21 @@ def addInPlace(self, value1, value2): COMPLEX_ACCUMULATOR_PARAM = AddingAccumulatorParam(0.0j) +class PStatsParam(AccumulatorParam): + """PStatsParam is used to merge pstats.Stats""" + + @staticmethod + def zero(value): + return None + + @staticmethod + def addInPlace(value1, value2): + if value1 is None: + return value2 + value1.add(value2) + return value1 + + class _UpdateRequestHandler(SocketServer.StreamRequestHandler): """ diff --git a/python/pyspark/cloudpickle.py b/python/pyspark/cloudpickle.py index 32dda3888c62d..bb0783555aa77 100644 --- a/python/pyspark/cloudpickle.py +++ b/python/pyspark/cloudpickle.py @@ -52,6 +52,7 @@ import itertools from copy_reg import _extension_registry, _inverted_registry, _extension_cache import new +import dis import traceback import platform @@ -61,6 +62,14 @@ import logging cloudLog = logging.getLogger("Cloud.Transport") +#relevant opcodes +STORE_GLOBAL = chr(dis.opname.index('STORE_GLOBAL')) +DELETE_GLOBAL = chr(dis.opname.index('DELETE_GLOBAL')) +LOAD_GLOBAL = chr(dis.opname.index('LOAD_GLOBAL')) +GLOBAL_OPS = [STORE_GLOBAL, DELETE_GLOBAL, LOAD_GLOBAL] + +HAVE_ARGUMENT = chr(dis.HAVE_ARGUMENT) +EXTENDED_ARG = chr(dis.EXTENDED_ARG) if PyImp == "PyPy": # register builtin type in `new` @@ -304,16 +313,37 @@ def save_function_tuple(self, func, forced_imports): write(pickle.REDUCE) # applies _fill_function on the tuple @staticmethod - def extract_code_globals(code): + def extract_code_globals(co): """ Find all globals names read or written to by codeblock co """ - names = set(code.co_names) - if code.co_consts: # see if nested function have any global refs - for const in code.co_consts: + code = co.co_code + names = co.co_names + out_names = set() + + n = len(code) + i = 0 + extended_arg = 0 + while i < n: + op = code[i] + + i = i+1 + if op >= HAVE_ARGUMENT: + oparg = ord(code[i]) + ord(code[i+1])*256 + extended_arg + extended_arg = 0 + i = i+2 + if op == EXTENDED_ARG: + extended_arg = oparg*65536L + if op in GLOBAL_OPS: + out_names.add(names[oparg]) + #print 'extracted', out_names, ' from ', names + + if co.co_consts: # see if nested function have any global refs + for const in co.co_consts: if type(const) is types.CodeType: - names |= CloudPickler.extract_code_globals(const) - return names + out_names |= CloudPickler.extract_code_globals(const) + + return out_names def extract_func_data(self, func): """ diff --git a/python/pyspark/context.py b/python/pyspark/context.py index a17f2c1203d36..a45d79d6424c7 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -20,6 +20,7 @@ import sys from threading import Lock from tempfile import NamedTemporaryFile +import atexit from pyspark import accumulators from pyspark.accumulators import Accumulator @@ -30,7 +31,6 @@ from pyspark.serializers import PickleSerializer, BatchedSerializer, UTF8Deserializer, \ PairDeserializer, CompressedSerializer from pyspark.storagelevel import StorageLevel -from pyspark import rdd from pyspark.rdd import RDD from pyspark.traceback_utils import CallSite, first_spark_call @@ -171,7 +171,7 @@ def _do_init(self, master, appName, sparkHome, pyFiles, environment, batchSize, SparkFiles._sc = self root_dir = SparkFiles.getRootDirectory() - sys.path.append(root_dir) + sys.path.insert(1, root_dir) # Deploy any code dependencies specified in the constructor self._python_includes = list() @@ -183,16 +183,18 @@ def _do_init(self, master, appName, sparkHome, pyFiles, environment, batchSize, for path in self._conf.get("spark.submit.pyFiles", "").split(","): if path != "": (dirname, filename) = os.path.split(path) - self._python_includes.append(filename) - sys.path.append(path) - if dirname not in sys.path: - sys.path.append(dirname) + if filename.lower().endswith("zip") or filename.lower().endswith("egg"): + self._python_includes.append(filename) + sys.path.insert(1, os.path.join(SparkFiles.getRootDirectory(), filename)) # 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 = \ self._jvm.org.apache.spark.util.Utils.createTempDir(local_dir).getAbsolutePath() + # profiling stats collected for each PythonRDD + self._profile_stats = [] + def _initialize_context(self, jconf): """ Initialize SparkContext in function to allow subclass specific initialization @@ -211,6 +213,7 @@ def _ensure_initialized(cls, instance=None, gateway=None): SparkContext._jvm = SparkContext._gateway.jvm SparkContext._writeToFile = SparkContext._jvm.PythonRDD.writeToFile SparkContext._jvm.SerDeUtil.initialize() + SparkContext._jvm.SerDe.initialize() if instance: if (SparkContext._active_spark_context and @@ -407,6 +410,7 @@ def sequenceFile(self, path, keyClass=None, valueClass=None, keyConverter=None, Read a Hadoop SequenceFile with arbitrary key and value Writable class from HDFS, a local file system (available on all nodes), or any Hadoop-supported file system URI. The mechanism is as follows: + 1. A Java RDD is created from the SequenceFile or other InputFormat, and the key and value Writable classes 2. Serialization is attempted via Pyrolite pickling @@ -666,7 +670,7 @@ def addPyFile(self, path): if filename.endswith('.zip') or filename.endswith('.ZIP') or filename.endswith('.egg'): self._python_includes.append(filename) # for tests in local mode - sys.path.append(os.path.join(SparkFiles.getRootDirectory(), filename)) + sys.path.insert(1, os.path.join(SparkFiles.getRootDirectory(), filename)) def setCheckpointDir(self, dirName): """ @@ -792,6 +796,40 @@ def runJob(self, rdd, partitionFunc, partitions=None, allowLocal=False): it = self._jvm.PythonRDD.runJob(self._jsc.sc(), mappedRDD._jrdd, javaPartitions, allowLocal) return list(mappedRDD._collect_iterator_through_file(it)) + def _add_profile(self, id, profileAcc): + if not self._profile_stats: + dump_path = self._conf.get("spark.python.profile.dump") + if dump_path: + atexit.register(self.dump_profiles, dump_path) + else: + atexit.register(self.show_profiles) + + self._profile_stats.append([id, profileAcc, False]) + + def show_profiles(self): + """ Print the profile stats to stdout """ + for i, (id, acc, showed) in enumerate(self._profile_stats): + stats = acc.value + if not showed and stats: + print "=" * 60 + print "Profile of RDD" % id + print "=" * 60 + stats.sort_stats("time", "cumulative").print_stats() + # mark it as showed + self._profile_stats[i][2] = True + + def dump_profiles(self, path): + """ Dump the profile stats into directory `path` + """ + if not os.path.exists(path): + os.makedirs(path) + for id, acc, _ in self._profile_stats: + stats = acc.value + if stats: + p = os.path.join(path, "rdd_%d.pstats" % id) + stats.dump_stats(p) + self._profile_stats = [] + def _test(): import atexit diff --git a/python/pyspark/join.py b/python/pyspark/join.py index b0f1cc1927066..b4a844713745a 100644 --- a/python/pyspark/join.py +++ b/python/pyspark/join.py @@ -80,6 +80,22 @@ def dispatch(seq): return _do_python_join(rdd, other, numPartitions, dispatch) +def python_full_outer_join(rdd, other, numPartitions): + def dispatch(seq): + vbuf, wbuf = [], [] + for (n, v) in seq: + if n == 1: + vbuf.append(v) + elif n == 2: + wbuf.append(v) + if not vbuf: + vbuf.append(None) + if not wbuf: + wbuf.append(None) + return [(v, w) for v in vbuf for w in wbuf] + return _do_python_join(rdd, other, numPartitions, dispatch) + + def python_cogroup(rdds, numPartitions): def make_mapper(i): return lambda (k, v): (k, (i, v)) diff --git a/python/pyspark/mllib/_common.py b/python/pyspark/mllib/_common.py deleted file mode 100644 index 68f6033616726..0000000000000 --- a/python/pyspark/mllib/_common.py +++ /dev/null @@ -1,562 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -import struct -import sys -import numpy -from numpy import ndarray, float64, int64, int32, array_equal, array -from pyspark import SparkContext, RDD -from pyspark.mllib.linalg import SparseVector -from pyspark.serializers import FramedSerializer - - -""" -Common utilities shared throughout MLlib, primarily for dealing with -different data types. These include: -- Serialization utilities to / from byte arrays that Java can handle -- Serializers for other data types, like ALS Rating objects -- Common methods for linear models -- Methods to deal with the different vector types we support, such as - SparseVector and scipy.sparse matrices. -""" - - -# Check whether we have SciPy. MLlib works without it too, but if we have it, some methods, -# such as _dot and _serialize_double_vector, start to support scipy.sparse matrices. - -_have_scipy = False -_scipy_issparse = None -try: - import scipy.sparse - _have_scipy = True - _scipy_issparse = scipy.sparse.issparse -except: - # No SciPy in environment, but that's okay - pass - - -# Serialization functions to and from Scala. These use the following formats, understood -# by the PythonMLLibAPI class in Scala: -# -# Dense double vector format: -# -# [1-byte 1] [4-byte length] [length*8 bytes of data] -# -# Sparse double vector format: -# -# [1-byte 2] [4-byte length] [4-byte nonzeros] [nonzeros*4 bytes of indices] \ -# [nonzeros*8 bytes of values] -# -# Double matrix format: -# -# [1-byte 3] [4-byte rows] [4-byte cols] [rows*cols*8 bytes of data] -# -# LabeledPoint format: -# -# [1-byte 4] [8-byte label] [dense or sparse vector] -# -# This is all in machine-endian. That means that the Java interpreter and the -# Python interpreter must agree on what endian the machine is. - - -DENSE_VECTOR_MAGIC = 1 -SPARSE_VECTOR_MAGIC = 2 -DENSE_MATRIX_MAGIC = 3 -LABELED_POINT_MAGIC = 4 - - -# Workaround for SPARK-2954: before Python 2.7, struct.unpack couldn't unpack bytearray()s. -if sys.version_info[:2] <= (2, 6): - def _unpack(fmt, string): - return struct.unpack(fmt, buffer(string)) -else: - _unpack = struct.unpack - - -def _deserialize_numpy_array(shape, ba, offset, dtype=float64): - """ - Deserialize a numpy array of the given type from an offset in - bytearray ba, assigning it the given shape. - - >>> x = array([1.0, 2.0, 3.0, 4.0, 5.0]) - >>> array_equal(x, _deserialize_numpy_array(x.shape, x.data, 0)) - True - >>> x = array([1.0, 2.0, 3.0, 4.0]).reshape(2,2) - >>> array_equal(x, _deserialize_numpy_array(x.shape, x.data, 0)) - True - >>> x = array([1, 2, 3], dtype=int32) - >>> array_equal(x, _deserialize_numpy_array(x.shape, x.data, 0, dtype=int32)) - True - """ - ar = ndarray(shape=shape, buffer=ba, offset=offset, dtype=dtype, order='C') - return ar.copy() - - -def _serialize_double(d): - """ - Serialize a double (float or numpy.float64) into a mutually understood format. - """ - if type(d) == float or type(d) == float64 or type(d) == int or type(d) == long: - d = float64(d) - ba = bytearray(8) - _copyto(d, buffer=ba, offset=0, shape=[1], dtype=float64) - return ba - else: - raise TypeError("_serialize_double called on non-float input") - - -def _serialize_double_vector(v): - """ - Serialize a double vector into a mutually understood format. - - Note: we currently do not use a magic byte for double for storage - efficiency. This should be reconsidered when we add Ser/De for other - 8-byte types (e.g. Long), for safety. The corresponding deserializer, - _deserialize_double, needs to be modified as well if the serialization - scheme changes. - - >>> x = array([1,2,3]) - >>> y = _deserialize_double_vector(_serialize_double_vector(x)) - >>> array_equal(y, array([1.0, 2.0, 3.0])) - True - """ - v = _convert_vector(v) - if type(v) == ndarray: - return _serialize_dense_vector(v) - elif type(v) == SparseVector: - return _serialize_sparse_vector(v) - else: - raise TypeError("_serialize_double_vector called on a %s; " - "wanted ndarray or SparseVector" % type(v)) - - -def _serialize_dense_vector(v): - """Serialize a dense vector given as a NumPy array.""" - if v.ndim != 1: - raise TypeError("_serialize_double_vector called on a %ddarray; " - "wanted a 1darray" % v.ndim) - if v.dtype != float64: - if numpy.issubdtype(v.dtype, numpy.complex): - raise TypeError("_serialize_double_vector called on an ndarray of %s; " - "wanted ndarray of float64" % v.dtype) - v = v.astype(float64) - length = v.shape[0] - ba = bytearray(5 + 8 * length) - ba[0] = DENSE_VECTOR_MAGIC - length_bytes = ndarray(shape=[1], buffer=ba, offset=1, dtype=int32) - length_bytes[0] = length - _copyto(v, buffer=ba, offset=5, shape=[length], dtype=float64) - return ba - - -def _serialize_sparse_vector(v): - """Serialize a pyspark.mllib.linalg.SparseVector.""" - nonzeros = len(v.indices) - ba = bytearray(9 + 12 * nonzeros) - ba[0] = SPARSE_VECTOR_MAGIC - header = ndarray(shape=[2], buffer=ba, offset=1, dtype=int32) - header[0] = v.size - header[1] = nonzeros - _copyto(v.indices, buffer=ba, offset=9, shape=[nonzeros], dtype=int32) - values_offset = 9 + 4 * nonzeros - _copyto(v.values, buffer=ba, offset=values_offset, shape=[nonzeros], dtype=float64) - return ba - - -def _deserialize_double(ba, offset=0): - """Deserialize a double from a mutually understood format. - - >>> import sys - >>> _deserialize_double(_serialize_double(123.0)) == 123.0 - True - >>> _deserialize_double(_serialize_double(float64(0.0))) == 0.0 - True - >>> _deserialize_double(_serialize_double(1)) == 1.0 - True - >>> _deserialize_double(_serialize_double(1L)) == 1.0 - True - >>> x = sys.float_info.max - >>> _deserialize_double(_serialize_double(sys.float_info.max)) == x - True - >>> y = float64(sys.float_info.max) - >>> _deserialize_double(_serialize_double(sys.float_info.max)) == y - True - """ - if type(ba) != bytearray: - raise TypeError("_deserialize_double called on a %s; wanted bytearray" % type(ba)) - if len(ba) - offset != 8: - raise TypeError("_deserialize_double called on a %d-byte array; wanted 8 bytes." % nb) - return _unpack("d", ba[offset:])[0] - - -def _deserialize_double_vector(ba, offset=0): - """Deserialize a double vector from a mutually understood format. - - >>> x = array([1.0, 2.0, 3.0, 4.0, -1.0, 0.0, -0.0]) - >>> array_equal(x, _deserialize_double_vector(_serialize_double_vector(x))) - True - >>> s = SparseVector(4, [1, 3], [3.0, 5.5]) - >>> s == _deserialize_double_vector(_serialize_double_vector(s)) - True - """ - if type(ba) != bytearray: - raise TypeError("_deserialize_double_vector called on a %s; " - "wanted bytearray" % type(ba)) - nb = len(ba) - offset - if nb < 5: - raise TypeError("_deserialize_double_vector called on a %d-byte array, " - "which is too short" % nb) - if ba[offset] == DENSE_VECTOR_MAGIC: - return _deserialize_dense_vector(ba, offset) - elif ba[offset] == SPARSE_VECTOR_MAGIC: - return _deserialize_sparse_vector(ba, offset) - else: - raise TypeError("_deserialize_double_vector called on bytearray " - "with wrong magic") - - -def _deserialize_dense_vector(ba, offset=0): - """Deserialize a dense vector into a numpy array.""" - nb = len(ba) - offset - if nb < 5: - raise TypeError("_deserialize_dense_vector called on a %d-byte array, " - "which is too short" % nb) - length = ndarray(shape=[1], buffer=ba, offset=offset + 1, dtype=int32)[0] - if nb < 8 * length + 5: - raise TypeError("_deserialize_dense_vector called on bytearray " - "with wrong length") - return _deserialize_numpy_array([length], ba, offset + 5) - - -def _deserialize_sparse_vector(ba, offset=0): - """Deserialize a sparse vector into a MLlib SparseVector object.""" - nb = len(ba) - offset - if nb < 9: - raise TypeError("_deserialize_sparse_vector called on a %d-byte array, " - "which is too short" % nb) - header = ndarray(shape=[2], buffer=ba, offset=offset + 1, dtype=int32) - size = header[0] - nonzeros = header[1] - if nb < 9 + 12 * nonzeros: - raise TypeError("_deserialize_sparse_vector called on bytearray " - "with wrong length") - indices = _deserialize_numpy_array([nonzeros], ba, offset + 9, dtype=int32) - values = _deserialize_numpy_array([nonzeros], ba, offset + 9 + 4 * nonzeros, dtype=float64) - return SparseVector(int(size), indices, values) - - -def _serialize_double_matrix(m): - """Serialize a double matrix into a mutually understood format.""" - if (type(m) == ndarray and m.ndim == 2): - if m.dtype != float64: - if numpy.issubdtype(m.dtype, numpy.complex): - raise TypeError("_serialize_double_matrix called on an ndarray of %s; " - "wanted ndarray of float64" % m.dtype) - m = m.astype(float64) - rows = m.shape[0] - cols = m.shape[1] - ba = bytearray(9 + 8 * rows * cols) - ba[0] = DENSE_MATRIX_MAGIC - lengths = ndarray(shape=[3], buffer=ba, offset=1, dtype=int32) - lengths[0] = rows - lengths[1] = cols - _copyto(m, buffer=ba, offset=9, shape=[rows, cols], dtype=float64) - return ba - else: - raise TypeError("_serialize_double_matrix called on a " - "non-double-matrix") - - -def _deserialize_double_matrix(ba): - """Deserialize a double matrix from a mutually understood format.""" - if type(ba) != bytearray: - raise TypeError("_deserialize_double_matrix called on a %s; " - "wanted bytearray" % type(ba)) - if len(ba) < 9: - raise TypeError("_deserialize_double_matrix called on a %d-byte array, " - "which is too short" % len(ba)) - if ba[0] != DENSE_MATRIX_MAGIC: - raise TypeError("_deserialize_double_matrix called on bytearray " - "with wrong magic") - lengths = ndarray(shape=[2], buffer=ba, offset=1, dtype=int32) - rows = lengths[0] - cols = lengths[1] - if (len(ba) != 8 * rows * cols + 9): - raise TypeError("_deserialize_double_matrix called on bytearray " - "with wrong length") - return _deserialize_numpy_array([rows, cols], ba, 9) - - -def _serialize_labeled_point(p): - """ - Serialize a LabeledPoint with a features vector of any type. - - >>> from pyspark.mllib.regression import LabeledPoint - >>> dp0 = LabeledPoint(0.5, array([1.0, 2.0, 3.0, 4.0, -1.0, 0.0, -0.0])) - >>> dp1 = _deserialize_labeled_point(_serialize_labeled_point(dp0)) - >>> dp1.label == dp0.label - True - >>> array_equal(dp1.features, dp0.features) - True - >>> sp0 = LabeledPoint(0.0, SparseVector(4, [1, 3], [3.0, 5.5])) - >>> sp1 = _deserialize_labeled_point(_serialize_labeled_point(sp0)) - >>> sp1.label == sp1.label - True - >>> sp1.features == sp0.features - True - """ - from pyspark.mllib.regression import LabeledPoint - serialized_features = _serialize_double_vector(p.features) - header = bytearray(9) - header[0] = LABELED_POINT_MAGIC - header_float = ndarray(shape=[1], buffer=header, offset=1, dtype=float64) - header_float[0] = p.label - return header + serialized_features - - -def _deserialize_labeled_point(ba, offset=0): - """Deserialize a LabeledPoint from a mutually understood format.""" - from pyspark.mllib.regression import LabeledPoint - if type(ba) != bytearray: - raise TypeError("Expecting a bytearray but got %s" % type(ba)) - if ba[offset] != LABELED_POINT_MAGIC: - raise TypeError("Expecting magic number %d but got %d" % (LABELED_POINT_MAGIC, ba[0])) - label = ndarray(shape=[1], buffer=ba, offset=offset + 1, dtype=float64)[0] - features = _deserialize_double_vector(ba, offset + 9) - return LabeledPoint(label, features) - - -def _copyto(array, buffer, offset, shape, dtype): - """ - Copy the contents of a vector to a destination bytearray at the - given offset. - - TODO: In the future this could use numpy.copyto on NumPy 1.7+, but - we should benchmark that to see whether it provides a benefit. - """ - temp_array = ndarray(shape=shape, buffer=buffer, offset=offset, dtype=dtype, order='C') - temp_array[...] = array - - -def _get_unmangled_rdd(data, serializer, cache=True): - """ - :param cache: If True, the serialized RDD is cached. (default = True) - WARNING: Users should unpersist() this later! - """ - dataBytes = data.map(serializer) - dataBytes._bypass_serializer = True - if cache: - dataBytes.cache() - return dataBytes - - -def _get_unmangled_double_vector_rdd(data, cache=True): - """ - Map a pickled Python RDD of Python dense or sparse vectors to a Java RDD of - _serialized_double_vectors. - :param cache: If True, the serialized RDD is cached. (default = True) - WARNING: Users should unpersist() this later! - """ - return _get_unmangled_rdd(data, _serialize_double_vector, cache) - - -def _get_unmangled_labeled_point_rdd(data, cache=True): - """ - Map a pickled Python RDD of LabeledPoint to a Java RDD of _serialized_labeled_points. - :param cache: If True, the serialized RDD is cached. (default = True) - WARNING: Users should unpersist() this later! - """ - return _get_unmangled_rdd(data, _serialize_labeled_point, cache) - - -# Common functions for dealing with and training linear models - -def _linear_predictor_typecheck(x, coeffs): - """ - Check that x is a one-dimensional vector of the right shape. - This is a temporary hackaround until we actually implement bulk predict. - """ - x = _convert_vector(x) - if type(x) == ndarray: - if x.ndim == 1: - if x.shape != coeffs.shape: - raise RuntimeError("Got array of %d elements; wanted %d" % ( - numpy.shape(x)[0], coeffs.shape[0])) - else: - raise RuntimeError("Bulk predict not yet supported.") - elif type(x) == SparseVector: - if x.size != coeffs.shape[0]: - raise RuntimeError("Got sparse vector of size %d; wanted %d" % ( - x.size, coeffs.shape[0])) - elif isinstance(x, RDD): - raise RuntimeError("Bulk predict not yet supported.") - else: - raise TypeError("Argument of type " + type(x).__name__ + " unsupported") - - -# If we weren't given initial weights, take a zero vector of the appropriate -# length. -def _get_initial_weights(initial_weights, data): - if initial_weights is None: - initial_weights = _convert_vector(data.first().features) - if type(initial_weights) == ndarray: - if initial_weights.ndim != 1: - raise TypeError("At least one data element has " - + initial_weights.ndim + " dimensions, which is not 1") - initial_weights = numpy.zeros([initial_weights.shape[0]]) - elif type(initial_weights) == SparseVector: - initial_weights = numpy.zeros([initial_weights.size]) - return initial_weights - - -# train_func should take two parameters, namely data and initial_weights, and -# return the result of a call to the appropriate JVM stub. -# _regression_train_wrapper is responsible for setup and error checking. -def _regression_train_wrapper(sc, train_func, klass, data, initial_weights): - initial_weights = _get_initial_weights(initial_weights, data) - dataBytes = _get_unmangled_labeled_point_rdd(data) - ans = train_func(dataBytes, _serialize_double_vector(initial_weights)) - if len(ans) != 2: - raise RuntimeError("JVM call result had unexpected length") - elif type(ans[0]) != bytearray: - raise RuntimeError("JVM call result had first element of type " - + type(ans[0]).__name__ + " which is not bytearray") - elif type(ans[1]) != float: - raise RuntimeError("JVM call result had second element of type " - + type(ans[0]).__name__ + " which is not float") - return klass(_deserialize_double_vector(ans[0]), ans[1]) - - -# Functions for serializing ALS Rating objects and tuples - -def _serialize_rating(r): - ba = bytearray(16) - intpart = ndarray(shape=[2], buffer=ba, dtype=int32) - doublepart = ndarray(shape=[1], buffer=ba, dtype=float64, offset=8) - intpart[0], intpart[1], doublepart[0] = r - return ba - - -class RatingDeserializer(FramedSerializer): - - def loads(self, string): - res = ndarray(shape=(3, ), buffer=string, dtype=float64, offset=4) - return int(res[0]), int(res[1]), res[2] - - def load_stream(self, stream): - while True: - try: - yield self._read_with_length(stream) - except struct.error: - return - except EOFError: - return - - -def _serialize_tuple(t): - ba = bytearray(8) - intpart = ndarray(shape=[2], buffer=ba, dtype=int32) - intpart[0], intpart[1] = t - return ba - - -# Vector math functions that support all of our vector types - -def _convert_vector(vec): - """ - Convert a vector to a format we support internally. This does - the following: - - * For dense NumPy vectors (ndarray), returns them as is - * For our SparseVector class, returns that as is - * For Python lists, converts them to NumPy vectors - * For scipy.sparse.*_matrix column vectors, converts them to - our own SparseVector type. - - This should be called before passing any data to our algorithms - or attempting to serialize it to Java. - """ - if type(vec) == ndarray or type(vec) == SparseVector: - return vec - elif type(vec) == list: - return array(vec, dtype=float64) - elif _have_scipy: - if _scipy_issparse(vec): - assert vec.shape[1] == 1, "Expected column vector" - csc = vec.tocsc() - return SparseVector(vec.shape[0], csc.indices, csc.data) - raise TypeError("Expected NumPy array, SparseVector, or scipy.sparse matrix") - - -def _squared_distance(v1, v2): - """ - Squared distance of two NumPy or sparse vectors. - - >>> dense1 = array([1., 2.]) - >>> sparse1 = SparseVector(2, [0, 1], [1., 2.]) - >>> dense2 = array([2., 1.]) - >>> sparse2 = SparseVector(2, [0, 1], [2., 1.]) - >>> _squared_distance(dense1, dense2) - 2.0 - >>> _squared_distance(dense1, sparse2) - 2.0 - >>> _squared_distance(sparse1, dense2) - 2.0 - >>> _squared_distance(sparse1, sparse2) - 2.0 - """ - v1 = _convert_vector(v1) - v2 = _convert_vector(v2) - if type(v1) == ndarray and type(v2) == ndarray: - diff = v1 - v2 - return numpy.dot(diff, diff) - elif type(v1) == ndarray: - return v2.squared_distance(v1) - else: - return v1.squared_distance(v2) - - -def _dot(vec, target): - """ - Compute the dot product of a vector of the types we support - (Numpy array, list, SparseVector, or SciPy sparse) and a target - NumPy array that is either 1- or 2-dimensional. Equivalent to - calling numpy.dot of the two vectors, but for SciPy ones, we - have to transpose them because they're column vectors. - """ - if type(vec) == ndarray: - return numpy.dot(vec, target) - elif type(vec) == SparseVector: - return vec.dot(target) - elif type(vec) == list: - return numpy.dot(_convert_vector(vec), target) - else: - return vec.transpose().dot(target)[0] - - -def _test(): - import doctest - globs = globals().copy() - globs['sc'] = SparkContext('local[4]', 'PythonTest', batchSize=2) - (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) - globs['sc'].stop() - if failure_count: - exit(-1) - - -if __name__ == "__main__": - _test() diff --git a/python/pyspark/mllib/classification.py b/python/pyspark/mllib/classification.py index 71ab46b61d7fa..a765b1c4f7d87 100644 --- a/python/pyspark/mllib/classification.py +++ b/python/pyspark/mllib/classification.py @@ -15,19 +15,14 @@ # limitations under the License. # +from math import exp + import numpy +from numpy import array -from numpy import array, shape -from pyspark import SparkContext -from pyspark.mllib._common import \ - _dot, _get_unmangled_rdd, _get_unmangled_double_vector_rdd, \ - _serialize_double_matrix, _deserialize_double_matrix, \ - _serialize_double_vector, _deserialize_double_vector, \ - _get_initial_weights, _serialize_rating, _regression_train_wrapper, \ - _linear_predictor_typecheck, _get_unmangled_labeled_point_rdd -from pyspark.mllib.linalg import SparseVector -from pyspark.mllib.regression import LabeledPoint, LinearModel -from math import exp, log +from pyspark import SparkContext, PickleSerializer +from pyspark.mllib.linalg import SparseVector, _convert_to_vector +from pyspark.mllib.regression import LabeledPoint, LinearModel, _regression_train_wrapper __all__ = ['LogisticRegressionModel', 'LogisticRegressionWithSGD', 'SVMModel', @@ -67,8 +62,7 @@ class LogisticRegressionModel(LinearModel): """ def predict(self, x): - _linear_predictor_typecheck(x, self._coeff) - margin = _dot(x, self._coeff) + self._intercept + margin = self.weights.dot(x) + self._intercept if margin > 0: prob = 1 / (1 + exp(-margin)) else: @@ -81,7 +75,7 @@ class LogisticRegressionWithSGD(object): @classmethod def train(cls, data, iterations=100, step=1.0, miniBatchFraction=1.0, - initialWeights=None, regParam=1.0, regType=None, intercept=False): + initialWeights=None, regParam=1.0, regType="none", intercept=False): """ Train a logistic regression model on the given data. @@ -95,22 +89,26 @@ def train(cls, data, iterations=100, step=1.0, miniBatchFraction=1.0, @param regParam: The regularizer parameter (default: 1.0). @param regType: The type of regularizer used for training our model. - Allowed values: "l1" for using L1Updater, - "l2" for using - SquaredL2Updater, - "none" for no regularizer. - (default: "none") + + :Allowed values: + - "l1" for using L1Updater + - "l2" for using SquaredL2Updater + - "none" for no regularizer + + (default: "none") + @param intercept: Boolean parameter which indicates the use or not of the augmented representation for training data (i.e. whether bias features are activated or not). """ sc = data.context - if regType is None: - regType = "none" - train_func = lambda d, i: sc._jvm.PythonMLLibAPI().trainLogisticRegressionModelWithSGD( - d._jrdd, iterations, step, miniBatchFraction, i, regParam, regType, intercept) - return _regression_train_wrapper(sc, train_func, LogisticRegressionModel, data, + + def train(jdata, i): + return sc._jvm.PythonMLLibAPI().trainLogisticRegressionModelWithSGD( + jdata, iterations, step, miniBatchFraction, i, regParam, regType, intercept) + + return _regression_train_wrapper(sc, train, LogisticRegressionModel, data, initialWeights) @@ -141,8 +139,7 @@ class SVMModel(LinearModel): """ def predict(self, x): - _linear_predictor_typecheck(x, self._coeff) - margin = _dot(x, self._coeff) + self._intercept + margin = self.weights.dot(x) + self.intercept return 1 if margin >= 0 else 0 @@ -150,7 +147,7 @@ class SVMWithSGD(object): @classmethod def train(cls, data, iterations=100, step=1.0, regParam=1.0, - miniBatchFraction=1.0, initialWeights=None, regType=None, intercept=False): + miniBatchFraction=1.0, initialWeights=None, regType="none", intercept=False): """ Train a support vector machine on the given data. @@ -164,22 +161,26 @@ def train(cls, data, iterations=100, step=1.0, regParam=1.0, @param initialWeights: The initial weights (default: None). @param regType: The type of regularizer used for training our model. - Allowed values: "l1" for using L1Updater, - "l2" for using - SquaredL2Updater, - "none" for no regularizer. - (default: "none") + + :Allowed values: + - "l1" for using L1Updater + - "l2" for using SquaredL2Updater, + - "none" for no regularizer. + + (default: "none") + @param intercept: Boolean parameter which indicates the use or not of the augmented representation for training data (i.e. whether bias features are activated or not). """ sc = data.context - if regType is None: - regType = "none" - train_func = lambda d, i: sc._jvm.PythonMLLibAPI().trainSVMModelWithSGD( - d._jrdd, iterations, step, regParam, miniBatchFraction, i, regType, intercept) - return _regression_train_wrapper(sc, train_func, SVMModel, data, initialWeights) + + def train(jrdd, i): + return sc._jvm.PythonMLLibAPI().trainSVMModelWithSGD( + jrdd, iterations, step, regParam, miniBatchFraction, i, regType, intercept) + + return _regression_train_wrapper(sc, train, SVMModel, data, initialWeights) class NaiveBayesModel(object): @@ -220,7 +221,8 @@ def __init__(self, labels, pi, theta): def predict(self, x): """Return the most likely class for a data vector x""" - return self.labels[numpy.argmax(self.pi + _dot(x, self.theta.transpose()))] + x = _convert_to_vector(x) + return self.labels[numpy.argmax(self.pi + x.dot(self.theta.transpose()))] class NaiveBayes(object): @@ -242,12 +244,9 @@ def train(cls, data, lambda_=1.0): @param lambda_: The smoothing parameter """ sc = data.context - dataBytes = _get_unmangled_labeled_point_rdd(data) - ans = sc._jvm.PythonMLLibAPI().trainNaiveBayes(dataBytes._jrdd, lambda_) - return NaiveBayesModel( - _deserialize_double_vector(ans[0]), - _deserialize_double_vector(ans[1]), - _deserialize_double_matrix(ans[2])) + jlist = sc._jvm.PythonMLLibAPI().trainNaiveBayes(data._to_java_object_rdd(), lambda_) + labels, pi, theta = PickleSerializer().loads(str(sc._jvm.SerDe.dumps(jlist))) + return NaiveBayesModel(labels.toArray(), pi.toArray(), numpy.array(theta)) def _test(): diff --git a/python/pyspark/mllib/clustering.py b/python/pyspark/mllib/clustering.py index f3e952a1d842a..12c56022717a5 100644 --- a/python/pyspark/mllib/clustering.py +++ b/python/pyspark/mllib/clustering.py @@ -15,15 +15,9 @@ # limitations under the License. # -from numpy import array, dot -from math import sqrt from pyspark import SparkContext -from pyspark.mllib._common import \ - _get_unmangled_rdd, _get_unmangled_double_vector_rdd, _squared_distance, \ - _serialize_double_matrix, _deserialize_double_matrix, \ - _serialize_double_vector, _deserialize_double_vector, \ - _get_initial_weights, _serialize_rating, _regression_train_wrapper -from pyspark.mllib.linalg import SparseVector +from pyspark.serializers import PickleSerializer, AutoBatchedSerializer +from pyspark.mllib.linalg import SparseVector, _convert_to_vector __all__ = ['KMeansModel', 'KMeans'] @@ -32,6 +26,7 @@ class KMeansModel(object): """A clustering model derived from the k-means method. + >>> from numpy import array >>> data = array([0.0,0.0, 1.0,1.0, 9.0,8.0, 8.0,9.0]).reshape(4,2) >>> model = KMeans.train( ... sc.parallelize(data), 2, maxIterations=10, runs=30, initializationMode="random") @@ -71,8 +66,9 @@ def predict(self, x): """Find the cluster to which x belongs in this model.""" best = 0 best_distance = float("inf") - for i in range(0, len(self.centers)): - distance = _squared_distance(x, self.centers[i]) + x = _convert_to_vector(x) + for i in xrange(len(self.centers)): + distance = x.squared_distance(self.centers[i]) if distance < best_distance: best = i best_distance = distance @@ -82,19 +78,17 @@ def predict(self, x): class KMeans(object): @classmethod - def train(cls, data, k, maxIterations=100, runs=1, initializationMode="k-means||"): + def train(cls, rdd, k, maxIterations=100, runs=1, initializationMode="k-means||"): """Train a k-means clustering model.""" - sc = data.context - dataBytes = _get_unmangled_double_vector_rdd(data) - ans = sc._jvm.PythonMLLibAPI().trainKMeansModel( - dataBytes._jrdd, k, maxIterations, runs, initializationMode) - if len(ans) != 1: - raise RuntimeError("JVM call result had unexpected length") - elif type(ans[0]) != bytearray: - raise RuntimeError("JVM call result had first element of type " - + type(ans[0]) + " which is not bytearray") - matrix = _deserialize_double_matrix(ans[0]) - return KMeansModel([row for row in matrix]) + sc = rdd.context + ser = PickleSerializer() + # cache serialized data to avoid objects over head in JVM + cached = rdd.map(_convert_to_vector)._reserialize(AutoBatchedSerializer(ser)).cache() + model = sc._jvm.PythonMLLibAPI().trainKMeansModel( + cached._to_java_object_rdd(), k, maxIterations, runs, initializationMode) + bytes = sc._jvm.SerDe.dumps(model.clusterCenters()) + centers = ser.loads(str(bytes)) + return KMeansModel([c.toArray() for c in centers]) def _test(): diff --git a/python/pyspark/mllib/feature.py b/python/pyspark/mllib/feature.py new file mode 100644 index 0000000000000..a44a27fd3b6a6 --- /dev/null +++ b/python/pyspark/mllib/feature.py @@ -0,0 +1,193 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +""" +Python package for feature in MLlib. +""" +from pyspark.serializers import PickleSerializer, AutoBatchedSerializer + +from pyspark.mllib.linalg import _convert_to_vector + +__all__ = ['Word2Vec', 'Word2VecModel'] + + +class Word2VecModel(object): + """ + class for Word2Vec model + """ + def __init__(self, sc, java_model): + """ + :param sc: Spark context + :param java_model: Handle to Java model object + """ + self._sc = sc + self._java_model = java_model + + def __del__(self): + self._sc._gateway.detach(self._java_model) + + def transform(self, word): + """ + :param word: a word + :return: vector representation of word + Transforms a word to its vector representation + + Note: local use only + """ + # TODO: make transform usable in RDD operations from python side + result = self._java_model.transform(word) + return PickleSerializer().loads(str(self._sc._jvm.SerDe.dumps(result))) + + def findSynonyms(self, x, num): + """ + :param x: a word or a vector representation of word + :param num: number of synonyms to find + :return: array of (word, cosineSimilarity) + Find synonyms of a word + + Note: local use only + """ + # TODO: make findSynonyms usable in RDD operations from python side + ser = PickleSerializer() + if type(x) == str: + jlist = self._java_model.findSynonyms(x, num) + else: + bytes = bytearray(ser.dumps(_convert_to_vector(x))) + vec = self._sc._jvm.SerDe.loads(bytes) + jlist = self._java_model.findSynonyms(vec, num) + words, similarity = ser.loads(str(self._sc._jvm.SerDe.dumps(jlist))) + return zip(words, similarity) + + +class Word2Vec(object): + """ + Word2Vec creates vector representation of words in a text corpus. + The algorithm first constructs a vocabulary from the corpus + and then learns vector representation of words in the vocabulary. + The vector representation can be used as features in + natural language processing and machine learning algorithms. + + We used skip-gram model in our implementation and hierarchical softmax + method to train the model. The variable names in the implementation + matches the original C implementation. + For original C implementation, see https://code.google.com/p/word2vec/ + For research papers, see + Efficient Estimation of Word Representations in Vector Space + and + Distributed Representations of Words and Phrases and their Compositionality. + + >>> sentence = "a b " * 100 + "a c " * 10 + >>> localDoc = [sentence, sentence] + >>> doc = sc.parallelize(localDoc).map(lambda line: line.split(" ")) + >>> model = Word2Vec().setVectorSize(10).setSeed(42L).fit(doc) + >>> syms = model.findSynonyms("a", 2) + >>> str(syms[0][0]) + 'b' + >>> str(syms[1][0]) + 'c' + >>> len(syms) + 2 + >>> vec = model.transform("a") + >>> len(vec) + 10 + >>> syms = model.findSynonyms(vec, 2) + >>> str(syms[0][0]) + 'b' + >>> str(syms[1][0]) + 'c' + >>> len(syms) + 2 + """ + def __init__(self): + """ + Construct Word2Vec instance + """ + self.vectorSize = 100 + self.learningRate = 0.025 + self.numPartitions = 1 + self.numIterations = 1 + self.seed = 42L + + def setVectorSize(self, vectorSize): + """ + Sets vector size (default: 100). + """ + self.vectorSize = vectorSize + return self + + def setLearningRate(self, learningRate): + """ + Sets initial learning rate (default: 0.025). + """ + self.learningRate = learningRate + return self + + def setNumPartitions(self, numPartitions): + """ + Sets number of partitions (default: 1). Use a small number for accuracy. + """ + self.numPartitions = numPartitions + return self + + def setNumIterations(self, numIterations): + """ + Sets number of iterations (default: 1), which should be smaller than or equal to number of + partitions. + """ + self.numIterations = numIterations + return self + + def setSeed(self, seed): + """ + Sets random seed. + """ + self.seed = seed + return self + + def fit(self, data): + """ + Computes the vector representation of each word in vocabulary. + + :param data: training data. RDD of subtype of Iterable[String] + :return: python Word2VecModel instance + """ + sc = data.context + ser = PickleSerializer() + vectorSize = self.vectorSize + learningRate = self.learningRate + numPartitions = self.numPartitions + numIterations = self.numIterations + seed = self.seed + + model = sc._jvm.PythonMLLibAPI().trainWord2Vec( + data._to_java_object_rdd(), vectorSize, + learningRate, numPartitions, numIterations, seed) + return Word2VecModel(sc, model) + + +def _test(): + import doctest + from pyspark import SparkContext + globs = globals().copy() + globs['sc'] = SparkContext('local[4]', 'PythonTest', batchSize=2) + (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) + globs['sc'].stop() + if failure_count: + exit(-1) + +if __name__ == "__main__": + _test() diff --git a/python/pyspark/mllib/linalg.py b/python/pyspark/mllib/linalg.py index e69051c104e37..51014a8ceb785 100644 --- a/python/pyspark/mllib/linalg.py +++ b/python/pyspark/mllib/linalg.py @@ -23,20 +23,215 @@ SciPy is available in their environment. """ -import numpy -from numpy import array, array_equal, ndarray, float64, int32 +import sys +import array +import copy_reg +import numpy as np -__all__ = ['SparseVector', 'Vectors'] +__all__ = ['Vector', 'DenseVector', 'SparseVector', 'Vectors'] -class SparseVector(object): +if sys.version_info[:2] == (2, 7): + # speed up pickling array in Python 2.7 + def fast_pickle_array(ar): + return array.array, (ar.typecode, ar.tostring()) + copy_reg.pickle(array.array, fast_pickle_array) + +# Check whether we have SciPy. MLlib works without it too, but if we have it, some methods, +# such as _dot and _serialize_double_vector, start to support scipy.sparse matrices. + +try: + import scipy.sparse + _have_scipy = True +except: + # No SciPy in environment, but that's okay + _have_scipy = False + + +def _convert_to_vector(l): + if isinstance(l, Vector): + return l + elif type(l) in (array.array, np.array, np.ndarray, list, tuple): + return DenseVector(l) + elif _have_scipy and scipy.sparse.issparse(l): + assert l.shape[1] == 1, "Expected column vector" + csc = l.tocsc() + return SparseVector(l.shape[0], csc.indices, csc.data) + else: + raise TypeError("Cannot convert type %s into Vector" % type(l)) + + +def _vector_size(v): + """ + Returns the size of the vector. + + >>> _vector_size([1., 2., 3.]) + 3 + >>> _vector_size((1., 2., 3.)) + 3 + >>> _vector_size(array.array('d', [1., 2., 3.])) + 3 + >>> _vector_size(np.zeros(3)) + 3 + >>> _vector_size(np.zeros((3, 1))) + 3 + >>> _vector_size(np.zeros((1, 3))) + Traceback (most recent call last): + ... + ValueError: Cannot treat an ndarray of shape (1, 3) as a vector + """ + if isinstance(v, Vector): + return len(v) + elif type(v) in (array.array, list, tuple): + return len(v) + elif type(v) == np.ndarray: + if v.ndim == 1 or (v.ndim == 2 and v.shape[1] == 1): + return len(v) + else: + raise ValueError("Cannot treat an ndarray of shape %s as a vector" % str(v.shape)) + elif _have_scipy and scipy.sparse.issparse(v): + assert v.shape[1] == 1, "Expected column vector" + return v.shape[0] + else: + raise TypeError("Cannot treat type %s as a vector" % type(v)) + + +class Vector(object): + """ + Abstract class for DenseVector and SparseVector + """ + def toArray(self): + """ + Convert the vector into an numpy.ndarray + :return: numpy.ndarray + """ + raise NotImplementedError + + +class DenseVector(Vector): + """ + A dense vector represented by a value array. + """ + def __init__(self, ar): + if not isinstance(ar, array.array): + ar = array.array('d', ar) + self.array = ar + + def __reduce__(self): + return DenseVector, (self.array,) + + def dot(self, other): + """ + Compute the dot product of two Vectors. We support + (Numpy array, list, SparseVector, or SciPy sparse) + and a target NumPy array that is either 1- or 2-dimensional. + Equivalent to calling numpy.dot of the two vectors. + + >>> dense = DenseVector(array.array('d', [1., 2.])) + >>> dense.dot(dense) + 5.0 + >>> dense.dot(SparseVector(2, [0, 1], [2., 1.])) + 4.0 + >>> dense.dot(range(1, 3)) + 5.0 + >>> dense.dot(np.array(range(1, 3))) + 5.0 + >>> dense.dot([1.,]) + Traceback (most recent call last): + ... + AssertionError: dimension mismatch + >>> dense.dot(np.reshape([1., 2., 3., 4.], (2, 2), order='F')) + array([ 5., 11.]) + >>> dense.dot(np.reshape([1., 2., 3.], (3, 1), order='F')) + Traceback (most recent call last): + ... + AssertionError: dimension mismatch + """ + if type(other) == np.ndarray and other.ndim > 1: + assert len(self) == other.shape[0], "dimension mismatch" + return np.dot(self.toArray(), other) + elif _have_scipy and scipy.sparse.issparse(other): + assert len(self) == other.shape[0], "dimension mismatch" + return other.transpose().dot(self.toArray()) + else: + assert len(self) == _vector_size(other), "dimension mismatch" + if isinstance(other, SparseVector): + return other.dot(self) + elif isinstance(other, Vector): + return np.dot(self.toArray(), other.toArray()) + else: + return np.dot(self.toArray(), other) + + def squared_distance(self, other): + """ + Squared distance of two Vectors. + + >>> dense1 = DenseVector(array.array('d', [1., 2.])) + >>> dense1.squared_distance(dense1) + 0.0 + >>> dense2 = np.array([2., 1.]) + >>> dense1.squared_distance(dense2) + 2.0 + >>> dense3 = [2., 1.] + >>> dense1.squared_distance(dense3) + 2.0 + >>> sparse1 = SparseVector(2, [0, 1], [2., 1.]) + >>> dense1.squared_distance(sparse1) + 2.0 + >>> dense1.squared_distance([1.,]) + Traceback (most recent call last): + ... + AssertionError: dimension mismatch + >>> dense1.squared_distance(SparseVector(1, [0,], [1.,])) + Traceback (most recent call last): + ... + AssertionError: dimension mismatch + """ + assert len(self) == _vector_size(other), "dimension mismatch" + if isinstance(other, SparseVector): + return other.squared_distance(self) + elif _have_scipy and scipy.sparse.issparse(other): + return _convert_to_vector(other).squared_distance(self) + + if isinstance(other, Vector): + other = other.toArray() + elif not isinstance(other, np.ndarray): + other = np.array(other) + diff = self.toArray() - other + return np.dot(diff, diff) + + def toArray(self): + return np.array(self.array) + + def __getitem__(self, item): + return self.array[item] + + def __len__(self): + return len(self.array) + + def __str__(self): + return "[" + ",".join([str(v) for v in self.array]) + "]" + + def __repr__(self): + return "DenseVector(%r)" % self.array + + def __eq__(self, other): + return isinstance(other, DenseVector) and self.array == other.array + + def __ne__(self, other): + return not self == other + + def __getattr__(self, item): + return getattr(self.array, item) + + +class SparseVector(Vector): """ A simple sparse vector class for passing data to MLlib. Users may alternatively pass SciPy's {scipy.sparse} data types. """ - def __init__(self, size, *args): """ Create a sparse vector, using either a dictionary, a list of @@ -61,16 +256,19 @@ def __init__(self, size, *args): if type(pairs) == dict: pairs = pairs.items() pairs = sorted(pairs) - self.indices = array([p[0] for p in pairs], dtype=int32) - self.values = array([p[1] for p in pairs], dtype=float64) + self.indices = array.array('i', [p[0] for p in pairs]) + self.values = array.array('d', [p[1] for p in pairs]) else: assert len(args[0]) == len(args[1]), "index and value arrays not same length" - self.indices = array(args[0], dtype=int32) - self.values = array(args[1], dtype=float64) + self.indices = array.array('i', args[0]) + self.values = array.array('d', args[1]) for i in xrange(len(self.indices) - 1): if self.indices[i] >= self.indices[i + 1]: raise TypeError("indices array must be sorted") + def __reduce__(self): + return (SparseVector, (self.size, self.indices, self.values)) + def dot(self, other): """ Dot product with a SparseVector or 1- or 2-dimensional Numpy array. @@ -78,26 +276,46 @@ def dot(self, other): >>> a = SparseVector(4, [1, 3], [3.0, 4.0]) >>> a.dot(a) 25.0 - >>> a.dot(array([1., 2., 3., 4.])) + >>> a.dot(array.array('d', [1., 2., 3., 4.])) 22.0 >>> b = SparseVector(4, [2, 4], [1.0, 2.0]) >>> a.dot(b) 0.0 - >>> a.dot(array([[1, 1], [2, 2], [3, 3], [4, 4]])) + >>> a.dot(np.array([[1, 1], [2, 2], [3, 3], [4, 4]])) array([ 22., 22.]) + >>> a.dot([1., 2., 3.]) + Traceback (most recent call last): + ... + AssertionError: dimension mismatch + >>> a.dot(np.array([1., 2.])) + Traceback (most recent call last): + ... + AssertionError: dimension mismatch + >>> a.dot(DenseVector([1., 2.])) + Traceback (most recent call last): + ... + AssertionError: dimension mismatch + >>> a.dot(np.zeros((3, 2))) + Traceback (most recent call last): + ... + AssertionError: dimension mismatch """ - if type(other) == ndarray: - if other.ndim == 1: - result = 0.0 - for i in xrange(len(self.indices)): - result += self.values[i] * other[self.indices[i]] - return result - elif other.ndim == 2: + if type(other) == np.ndarray: + if other.ndim == 2: results = [self.dot(other[:, i]) for i in xrange(other.shape[1])] - return array(results) - else: - raise Exception("Cannot call dot with %d-dimensional array" % other.ndim) - else: + return np.array(results) + elif other.ndim > 2: + raise ValueError("Cannot call dot with %d-dimensional array" % other.ndim) + + assert len(self) == _vector_size(other), "dimension mismatch" + + if type(other) in (np.ndarray, array.array, DenseVector): + result = 0.0 + for i in xrange(len(self.indices)): + result += self.values[i] * other[self.indices[i]] + return result + + elif type(other) is SparseVector: result = 0.0 i, j = 0, 0 while i < len(self.indices) and j < len(other.indices): @@ -111,6 +329,9 @@ def dot(self, other): j += 1 return result + else: + return self.dot(_convert_to_vector(other)) + def squared_distance(self, other): """ Squared distance from a SparseVector or 1-dimensional NumPy array. @@ -118,30 +339,41 @@ def squared_distance(self, other): >>> a = SparseVector(4, [1, 3], [3.0, 4.0]) >>> a.squared_distance(a) 0.0 - >>> a.squared_distance(array([1., 2., 3., 4.])) + >>> a.squared_distance(array.array('d', [1., 2., 3., 4.])) + 11.0 + >>> a.squared_distance(np.array([1., 2., 3., 4.])) 11.0 >>> b = SparseVector(4, [2, 4], [1.0, 2.0]) >>> a.squared_distance(b) 30.0 >>> b.squared_distance(a) 30.0 + >>> b.squared_distance([1., 2.]) + Traceback (most recent call last): + ... + AssertionError: dimension mismatch + >>> b.squared_distance(SparseVector(3, [1,], [1.0,])) + Traceback (most recent call last): + ... + AssertionError: dimension mismatch """ - if type(other) == ndarray: - if other.ndim == 1: - result = 0.0 - j = 0 # index into our own array - for i in xrange(other.shape[0]): - if j < len(self.indices) and self.indices[j] == i: - diff = self.values[j] - other[i] - result += diff * diff - j += 1 - else: - result += other[i] * other[i] - return result - else: + assert len(self) == _vector_size(other), "dimension mismatch" + if type(other) in (list, array.array, DenseVector, np.array, np.ndarray): + if type(other) is np.array and other.ndim != 1: raise Exception("Cannot call squared_distance with %d-dimensional array" % other.ndim) - else: + result = 0.0 + j = 0 # index into our own array + for i in xrange(len(other)): + if j < len(self.indices) and self.indices[j] == i: + diff = self.values[j] - other[i] + result += diff * diff + j += 1 + else: + result += other[i] * other[i] + return result + + elif type(other) is SparseVector: result = 0.0 i, j = 0, 0 while i < len(self.indices) and j < len(other.indices): @@ -163,16 +395,21 @@ def squared_distance(self, other): result += other.values[j] * other.values[j] j += 1 return result + else: + return self.squared_distance(_convert_to_vector(other)) def toArray(self): """ Returns a copy of this SparseVector as a 1-dimensional NumPy array. """ - arr = numpy.zeros(self.size) + arr = np.zeros((self.size,), dtype=np.float64) for i in xrange(self.indices.size): arr[self.indices[i]] = self.values[i] return arr + def __len__(self): + return self.size + def __str__(self): inds = "[" + ",".join([str(i) for i in self.indices]) + "]" vals = "[" + ",".join([str(v) for v in self.values]) + "]" @@ -195,11 +432,10 @@ def __eq__(self, other): >>> v1 != v2 False """ - return (isinstance(other, self.__class__) and other.size == self.size - and array_equal(other.indices, self.indices) - and array_equal(other.values, self.values)) + and other.indices == self.indices + and other.values == self.values) def __ne__(self, other): return not self.__eq__(other) @@ -242,9 +478,9 @@ def dense(elements): returns a NumPy array. >>> Vectors.dense([1, 2, 3]) - array([ 1., 2., 3.]) + DenseVector(array('d', [1.0, 2.0, 3.0])) """ - return array(elements, dtype=float64) + return DenseVector(elements) @staticmethod def stringify(vector): @@ -257,10 +493,48 @@ def stringify(vector): >>> Vectors.stringify(Vectors.dense([0.0, 1.0])) '[0.0,1.0]' """ - if type(vector) == SparseVector: - return str(vector) - else: - return "[" + ",".join([str(v) for v in vector]) + "]" + return str(vector) + + +class Matrix(object): + """ + Represents a local matrix. + """ + + def __init__(self, numRows, numCols): + self.numRows = numRows + self.numCols = numCols + + def toArray(self): + """ + Returns its elements in a NumPy ndarray. + """ + raise NotImplementedError + + +class DenseMatrix(Matrix): + """ + Column-major dense matrix. + """ + def __init__(self, numRows, numCols, values): + Matrix.__init__(self, numRows, numCols) + assert len(values) == numRows * numCols + self.values = values + + def __reduce__(self): + return DenseMatrix, (self.numRows, self.numCols, self.values) + + def toArray(self): + """ + Return an numpy.ndarray + + >>> arr = array.array('d', [float(i) for i in range(4)]) + >>> m = DenseMatrix(2, 2, arr) + >>> m.toArray() + array([[ 0., 2.], + [ 1., 3.]]) + """ + return np.reshape(self.values, (self.numRows, self.numCols), order='F') def _test(): diff --git a/python/pyspark/mllib/random.py b/python/pyspark/mllib/random.py index d53c95fd59c25..a787e4dea2c55 100644 --- a/python/pyspark/mllib/random.py +++ b/python/pyspark/mllib/random.py @@ -19,15 +19,32 @@ Python package for random data generation. """ +from functools import wraps from pyspark.rdd import RDD -from pyspark.mllib._common import _deserialize_double, _deserialize_double_vector -from pyspark.serializers import NoOpSerializer +from pyspark.serializers import BatchedSerializer, PickleSerializer __all__ = ['RandomRDDs', ] +def serialize(f): + @wraps(f) + def func(sc, *a, **kw): + jrdd = f(sc, *a, **kw) + return RDD(sc._jvm.PythonRDD.javaToPython(jrdd), sc, + BatchedSerializer(PickleSerializer(), 1024)) + return func + + +def toArray(f): + @wraps(f) + def func(sc, *a, **kw): + rdd = f(sc, *a, **kw) + return rdd.map(lambda vec: vec.toArray()) + return func + + class RandomRDDs(object): """ Generator methods for creating RDDs comprised of i.i.d samples from @@ -35,6 +52,7 @@ class RandomRDDs(object): """ @staticmethod + @serialize def uniformRDD(sc, size, numPartitions=None, seed=None): """ Generates an RDD comprised of i.i.d. samples from the @@ -56,11 +74,10 @@ def uniformRDD(sc, size, numPartitions=None, seed=None): >>> parts == sc.defaultParallelism True """ - jrdd = sc._jvm.PythonMLLibAPI().uniformRDD(sc._jsc, size, numPartitions, seed) - uniform = RDD(jrdd, sc, NoOpSerializer()) - return uniform.map(lambda bytes: _deserialize_double(bytearray(bytes))) + return sc._jvm.PythonMLLibAPI().uniformRDD(sc._jsc, size, numPartitions, seed) @staticmethod + @serialize def normalRDD(sc, size, numPartitions=None, seed=None): """ Generates an RDD comprised of i.i.d. samples from the standard normal @@ -80,11 +97,10 @@ def normalRDD(sc, size, numPartitions=None, seed=None): >>> abs(stats.stdev() - 1.0) < 0.1 True """ - jrdd = sc._jvm.PythonMLLibAPI().normalRDD(sc._jsc, size, numPartitions, seed) - normal = RDD(jrdd, sc, NoOpSerializer()) - return normal.map(lambda bytes: _deserialize_double(bytearray(bytes))) + return sc._jvm.PythonMLLibAPI().normalRDD(sc._jsc, size, numPartitions, seed) @staticmethod + @serialize def poissonRDD(sc, mean, size, numPartitions=None, seed=None): """ Generates an RDD comprised of i.i.d. samples from the Poisson @@ -101,11 +117,11 @@ def poissonRDD(sc, mean, size, numPartitions=None, seed=None): >>> abs(stats.stdev() - sqrt(mean)) < 0.5 True """ - jrdd = sc._jvm.PythonMLLibAPI().poissonRDD(sc._jsc, mean, size, numPartitions, seed) - poisson = RDD(jrdd, sc, NoOpSerializer()) - return poisson.map(lambda bytes: _deserialize_double(bytearray(bytes))) + return sc._jvm.PythonMLLibAPI().poissonRDD(sc._jsc, mean, size, numPartitions, seed) @staticmethod + @toArray + @serialize def uniformVectorRDD(sc, numRows, numCols, numPartitions=None, seed=None): """ Generates an RDD comprised of vectors containing i.i.d. samples drawn @@ -120,12 +136,12 @@ def uniformVectorRDD(sc, numRows, numCols, numPartitions=None, seed=None): >>> RandomRDDs.uniformVectorRDD(sc, 10, 10, 4).getNumPartitions() 4 """ - jrdd = sc._jvm.PythonMLLibAPI() \ + return sc._jvm.PythonMLLibAPI() \ .uniformVectorRDD(sc._jsc, numRows, numCols, numPartitions, seed) - uniform = RDD(jrdd, sc, NoOpSerializer()) - return uniform.map(lambda bytes: _deserialize_double_vector(bytearray(bytes))) @staticmethod + @toArray + @serialize def normalVectorRDD(sc, numRows, numCols, numPartitions=None, seed=None): """ Generates an RDD comprised of vectors containing i.i.d. samples drawn @@ -140,12 +156,12 @@ def normalVectorRDD(sc, numRows, numCols, numPartitions=None, seed=None): >>> abs(mat.std() - 1.0) < 0.1 True """ - jrdd = sc._jvm.PythonMLLibAPI() \ + return sc._jvm.PythonMLLibAPI() \ .normalVectorRDD(sc._jsc, numRows, numCols, numPartitions, seed) - normal = RDD(jrdd, sc, NoOpSerializer()) - return normal.map(lambda bytes: _deserialize_double_vector(bytearray(bytes))) @staticmethod + @toArray + @serialize def poissonVectorRDD(sc, mean, numRows, numCols, numPartitions=None, seed=None): """ Generates an RDD comprised of vectors containing i.i.d. samples drawn @@ -163,10 +179,8 @@ def poissonVectorRDD(sc, mean, numRows, numCols, numPartitions=None, seed=None): >>> abs(mat.std() - sqrt(mean)) < 0.5 True """ - jrdd = sc._jvm.PythonMLLibAPI() \ + return sc._jvm.PythonMLLibAPI() \ .poissonVectorRDD(sc._jsc, mean, numRows, numCols, numPartitions, seed) - poisson = RDD(jrdd, sc, NoOpSerializer()) - return poisson.map(lambda bytes: _deserialize_double_vector(bytearray(bytes))) def _test(): diff --git a/python/pyspark/mllib/recommendation.py b/python/pyspark/mllib/recommendation.py index 2df23394da6f8..59c1c5ff0ced0 100644 --- a/python/pyspark/mllib/recommendation.py +++ b/python/pyspark/mllib/recommendation.py @@ -16,17 +16,25 @@ # from pyspark import SparkContext -from pyspark.mllib._common import \ - _get_unmangled_rdd, _get_unmangled_double_vector_rdd, \ - _serialize_double_matrix, _deserialize_double_matrix, \ - _serialize_double_vector, _deserialize_double_vector, \ - _get_initial_weights, _serialize_rating, _regression_train_wrapper, \ - _serialize_tuple, RatingDeserializer +from pyspark.serializers import PickleSerializer, AutoBatchedSerializer from pyspark.rdd import RDD __all__ = ['MatrixFactorizationModel', 'ALS'] +class Rating(object): + def __init__(self, user, product, rating): + self.user = int(user) + self.product = int(product) + self.rating = float(rating) + + def __reduce__(self): + return Rating, (self.user, self.product, self.rating) + + def __repr__(self): + return "Rating(%d, %d, %d)" % (self.user, self.product, self.rating) + + class MatrixFactorizationModel(object): """A matrix factorisation model trained by regularized alternating @@ -39,7 +47,9 @@ class MatrixFactorizationModel(object): >>> model = ALS.trainImplicit(ratings, 1) >>> model.predict(2,2) is not None True + >>> testset = sc.parallelize([(1, 2), (1, 1)]) + >>> model = ALS.train(ratings, 1) >>> model.predictAll(testset).count() == 2 True """ @@ -54,34 +64,61 @@ def __del__(self): def predict(self, user, product): return self._java_model.predict(user, product) - def predictAll(self, usersProducts): - usersProductsJRDD = _get_unmangled_rdd(usersProducts, _serialize_tuple) - return RDD(self._java_model.predict(usersProductsJRDD._jrdd), - self._context, RatingDeserializer()) + def predictAll(self, user_product): + assert isinstance(user_product, RDD), "user_product should be RDD of (user, product)" + first = user_product.first() + if isinstance(first, list): + user_product = user_product.map(tuple) + first = tuple(first) + assert type(first) is tuple and len(first) == 2, \ + "user_product should be RDD of (user, product)" + if any(isinstance(x, str) for x in first): + user_product = user_product.map(lambda (u, p): (int(x), int(p))) + first = tuple(map(int, first)) + assert all(type(x) is int for x in first), "user and product in user_product shoul be int" + sc = self._context + tuplerdd = sc._jvm.SerDe.asTupleRDD(user_product._to_java_object_rdd().rdd()) + jresult = self._java_model.predict(tuplerdd).toJavaRDD() + return RDD(sc._jvm.PythonRDD.javaToPython(jresult), sc, + AutoBatchedSerializer(PickleSerializer())) class ALS(object): + @classmethod + def _prepare(cls, ratings): + assert isinstance(ratings, RDD), "ratings should be RDD" + first = ratings.first() + if not isinstance(first, Rating): + if isinstance(first, (tuple, list)): + ratings = ratings.map(lambda x: Rating(*x)) + else: + raise ValueError("rating should be RDD of Rating or tuple/list") + # serialize them by AutoBatchedSerializer before cache to reduce the + # objects overhead in JVM + cached = ratings._reserialize(AutoBatchedSerializer(PickleSerializer())).cache() + return cached._to_java_object_rdd() + @classmethod def train(cls, ratings, rank, iterations=5, lambda_=0.01, blocks=-1): sc = ratings.context - ratingBytes = _get_unmangled_rdd(ratings, _serialize_rating) - mod = sc._jvm.PythonMLLibAPI().trainALSModel( - ratingBytes._jrdd, rank, iterations, lambda_, blocks) + jrating = cls._prepare(ratings) + mod = sc._jvm.PythonMLLibAPI().trainALSModel(jrating, rank, iterations, lambda_, blocks) return MatrixFactorizationModel(sc, mod) @classmethod def trainImplicit(cls, ratings, rank, iterations=5, lambda_=0.01, blocks=-1, alpha=0.01): sc = ratings.context - ratingBytes = _get_unmangled_rdd(ratings, _serialize_rating) + jrating = cls._prepare(ratings) mod = sc._jvm.PythonMLLibAPI().trainImplicitALSModel( - ratingBytes._jrdd, rank, iterations, lambda_, blocks, alpha) + jrating, rank, iterations, lambda_, blocks, alpha) return MatrixFactorizationModel(sc, mod) def _test(): import doctest - globs = globals().copy() + import pyspark.mllib.recommendation + globs = pyspark.mllib.recommendation.__dict__.copy() globs['sc'] = SparkContext('local[4]', 'PythonTest', batchSize=2) (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) globs['sc'].stop() diff --git a/python/pyspark/mllib/regression.py b/python/pyspark/mllib/regression.py index f572dcfb840b6..54f34a98337ca 100644 --- a/python/pyspark/mllib/regression.py +++ b/python/pyspark/mllib/regression.py @@ -15,14 +15,14 @@ # limitations under the License. # -from numpy import array, ndarray -from pyspark import SparkContext -from pyspark.mllib._common import _dot, _regression_train_wrapper, \ - _linear_predictor_typecheck, _have_scipy, _scipy_issparse -from pyspark.mllib.linalg import SparseVector, Vectors +import numpy as np +from numpy import array +from pyspark import SparkContext +from pyspark.mllib.linalg import SparseVector, _convert_to_vector +from pyspark.serializers import PickleSerializer, AutoBatchedSerializer -__all__ = ['LabeledPoint', 'LinearModel', 'LinearRegressionModel', 'RidgeRegressionModel' +__all__ = ['LabeledPoint', 'LinearModel', 'LinearRegressionModel', 'RidgeRegressionModel', 'LinearRegressionWithSGD', 'LassoWithSGD', 'RidgeRegressionWithSGD'] @@ -38,16 +38,16 @@ class LabeledPoint(object): def __init__(self, label, features): self.label = label - if (type(features) == ndarray or type(features) == SparseVector - or (_have_scipy and _scipy_issparse(features))): - self.features = features - elif type(features) == list: - self.features = array(features) - else: - raise TypeError("Expected NumPy array, list, SparseVector, or scipy.sparse matrix") + self.features = _convert_to_vector(features) + + def __reduce__(self): + return (LabeledPoint, (self.label, self.features)) def __str__(self): - return "(" + ",".join((str(self.label), Vectors.stringify(self.features))) + ")" + return "(" + ",".join((str(self.label), str(self.features))) + ")" + + def __repr__(self): + return "LabeledPoint(" + ",".join((repr(self.label), repr(self.features))) + ")" class LinearModel(object): @@ -55,7 +55,7 @@ class LinearModel(object): """A linear model that has a vector of coefficients and an intercept.""" def __init__(self, weights, intercept): - self._coeff = weights + self._coeff = _convert_to_vector(weights) self._intercept = intercept @property @@ -66,23 +66,27 @@ def weights(self): def intercept(self): return self._intercept + def __repr__(self): + return "(weights=%s, intercept=%s)" % (self._coeff, self._intercept) + class LinearRegressionModelBase(LinearModel): """A linear regression model. - >>> lrmb = LinearRegressionModelBase(array([1.0, 2.0]), 0.1) - >>> abs(lrmb.predict(array([-1.03, 7.777])) - 14.624) < 1e-6 + >>> lrmb = LinearRegressionModelBase(np.array([1.0, 2.0]), 0.1) + >>> abs(lrmb.predict(np.array([-1.03, 7.777])) - 14.624) < 1e-6 True >>> abs(lrmb.predict(SparseVector(2, {0: -1.03, 1: 7.777})) - 14.624) < 1e-6 True """ def predict(self, x): - """Predict the value of the dependent variable given a vector x""" - """containing values for the independent variables.""" - _linear_predictor_typecheck(x, self._coeff) - return _dot(x, self._coeff) + self._intercept + """ + Predict the value of the dependent variable given a vector x + containing values for the independent variables. + """ + return self.weights.dot(x) + self.intercept class LinearRegressionModel(LinearRegressionModelBase): @@ -96,10 +100,10 @@ class LinearRegressionModel(LinearRegressionModelBase): ... LabeledPoint(3.0, [2.0]), ... LabeledPoint(2.0, [3.0]) ... ] - >>> lrm = LinearRegressionWithSGD.train(sc.parallelize(data), initialWeights=array([1.0])) - >>> abs(lrm.predict(array([0.0])) - 0) < 0.5 + >>> lrm = LinearRegressionWithSGD.train(sc.parallelize(data), initialWeights=np.array([1.0])) + >>> abs(lrm.predict(np.array([0.0])) - 0) < 0.5 True - >>> abs(lrm.predict(array([1.0])) - 1) < 0.5 + >>> abs(lrm.predict(np.array([1.0])) - 1) < 0.5 True >>> abs(lrm.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5 True @@ -117,11 +121,27 @@ class LinearRegressionModel(LinearRegressionModelBase): """ +# train_func should take two parameters, namely data and initial_weights, and +# return the result of a call to the appropriate JVM stub. +# _regression_train_wrapper is responsible for setup and error checking. +def _regression_train_wrapper(sc, train_func, modelClass, data, initial_weights): + initial_weights = initial_weights or [0.0] * len(data.first().features) + ser = PickleSerializer() + initial_bytes = bytearray(ser.dumps(_convert_to_vector(initial_weights))) + # use AutoBatchedSerializer before cache to reduce the memory + # overhead in JVM + cached = data._reserialize(AutoBatchedSerializer(ser)).cache() + ans = train_func(cached._to_java_object_rdd(), initial_bytes) + assert len(ans) == 2, "JVM call result had unexpected length" + weights = ser.loads(str(ans[0])) + return modelClass(weights, ans[1]) + + class LinearRegressionWithSGD(object): @classmethod def train(cls, data, iterations=100, step=1.0, miniBatchFraction=1.0, - initialWeights=None, regParam=1.0, regType=None, intercept=False): + initialWeights=None, regParam=1.0, regType="none", intercept=False): """ Train a linear regression model on the given data. @@ -135,22 +155,26 @@ def train(cls, data, iterations=100, step=1.0, miniBatchFraction=1.0, @param regParam: The regularizer parameter (default: 1.0). @param regType: The type of regularizer used for training our model. - Allowed values: "l1" for using L1Updater, - "l2" for using - SquaredL2Updater, - "none" for no regularizer. - (default: "none") + + :Allowed values: + - "l1" for using L1Updater, + - "l2" for using SquaredL2Updater, + - "none" for no regularizer. + + (default: "none") + @param intercept: Boolean parameter which indicates the use or not of the augmented representation for training data (i.e. whether bias features are activated or not). """ sc = data.context - if regType is None: - regType = "none" - train_f = lambda d, i: sc._jvm.PythonMLLibAPI().trainLinearRegressionModelWithSGD( - d._jrdd, iterations, step, miniBatchFraction, i, regParam, regType, intercept) - return _regression_train_wrapper(sc, train_f, LinearRegressionModel, data, initialWeights) + + def train(jrdd, i): + return sc._jvm.PythonMLLibAPI().trainLinearRegressionModelWithSGD( + jrdd, iterations, step, miniBatchFraction, i, regParam, regType, intercept) + + return _regression_train_wrapper(sc, train, LinearRegressionModel, data, initialWeights) class LassoModel(LinearRegressionModelBase): @@ -166,9 +190,9 @@ class LassoModel(LinearRegressionModelBase): ... LabeledPoint(2.0, [3.0]) ... ] >>> lrm = LassoWithSGD.train(sc.parallelize(data), initialWeights=array([1.0])) - >>> abs(lrm.predict(array([0.0])) - 0) < 0.5 + >>> abs(lrm.predict(np.array([0.0])) - 0) < 0.5 True - >>> abs(lrm.predict(array([1.0])) - 1) < 0.5 + >>> abs(lrm.predict(np.array([1.0])) - 1) < 0.5 True >>> abs(lrm.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5 True @@ -179,7 +203,7 @@ class LassoModel(LinearRegressionModelBase): ... LabeledPoint(2.0, SparseVector(1, {0: 3.0})) ... ] >>> lrm = LinearRegressionWithSGD.train(sc.parallelize(data), initialWeights=array([1.0])) - >>> abs(lrm.predict(array([0.0])) - 0) < 0.5 + >>> abs(lrm.predict(np.array([0.0])) - 0) < 0.5 True >>> abs(lrm.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5 True @@ -193,9 +217,11 @@ def train(cls, data, iterations=100, step=1.0, regParam=1.0, miniBatchFraction=1.0, initialWeights=None): """Train a Lasso regression model on the given data.""" sc = data.context - train_f = lambda d, i: sc._jvm.PythonMLLibAPI().trainLassoModelWithSGD( - d._jrdd, iterations, step, regParam, miniBatchFraction, i) - return _regression_train_wrapper(sc, train_f, LassoModel, data, initialWeights) + + def train(jrdd, i): + return sc._jvm.PythonMLLibAPI().trainLassoModelWithSGD( + jrdd, iterations, step, regParam, miniBatchFraction, i) + return _regression_train_wrapper(sc, train, LassoModel, data, initialWeights) class RidgeRegressionModel(LinearRegressionModelBase): @@ -211,9 +237,9 @@ class RidgeRegressionModel(LinearRegressionModelBase): ... LabeledPoint(2.0, [3.0]) ... ] >>> lrm = RidgeRegressionWithSGD.train(sc.parallelize(data), initialWeights=array([1.0])) - >>> abs(lrm.predict(array([0.0])) - 0) < 0.5 + >>> abs(lrm.predict(np.array([0.0])) - 0) < 0.5 True - >>> abs(lrm.predict(array([1.0])) - 1) < 0.5 + >>> abs(lrm.predict(np.array([1.0])) - 1) < 0.5 True >>> abs(lrm.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5 True @@ -224,7 +250,7 @@ class RidgeRegressionModel(LinearRegressionModelBase): ... LabeledPoint(2.0, SparseVector(1, {0: 3.0})) ... ] >>> lrm = LinearRegressionWithSGD.train(sc.parallelize(data), initialWeights=array([1.0])) - >>> abs(lrm.predict(array([0.0])) - 0) < 0.5 + >>> abs(lrm.predict(np.array([0.0])) - 0) < 0.5 True >>> abs(lrm.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5 True @@ -238,9 +264,12 @@ def train(cls, data, iterations=100, step=1.0, regParam=1.0, miniBatchFraction=1.0, initialWeights=None): """Train a ridge regression model on the given data.""" sc = data.context - train_func = lambda d, i: sc._jvm.PythonMLLibAPI().trainRidgeModelWithSGD( - d._jrdd, iterations, step, regParam, miniBatchFraction, i) - return _regression_train_wrapper(sc, train_func, RidgeRegressionModel, data, initialWeights) + + def train(jrdd, i): + return sc._jvm.PythonMLLibAPI().trainRidgeModelWithSGD( + jrdd, iterations, step, regParam, miniBatchFraction, i) + + return _regression_train_wrapper(sc, train, RidgeRegressionModel, data, initialWeights) def _test(): diff --git a/python/pyspark/mllib/stat.py b/python/pyspark/mllib/stat.py index 8c726f171c978..b9de0909a6fb1 100644 --- a/python/pyspark/mllib/stat.py +++ b/python/pyspark/mllib/stat.py @@ -19,14 +19,26 @@ Python package for statistical functions in MLlib. """ -from pyspark.mllib._common import \ - _get_unmangled_double_vector_rdd, _get_unmangled_rdd, \ - _serialize_double, _deserialize_double_matrix, _deserialize_double_vector +from functools import wraps + +from pyspark import PickleSerializer __all__ = ['MultivariateStatisticalSummary', 'Statistics'] +def serialize(f): + ser = PickleSerializer() + + @wraps(f) + def func(self): + jvec = f(self) + bytes = self._sc._jvm.SerDe.dumps(jvec) + return ser.loads(str(bytes)).toArray() + + return func + + class MultivariateStatisticalSummary(object): """ @@ -44,33 +56,38 @@ def __init__(self, sc, java_summary): def __del__(self): self._sc._gateway.detach(self._java_summary) + @serialize def mean(self): - return _deserialize_double_vector(self._java_summary.mean()) + return self._java_summary.mean() + @serialize def variance(self): - return _deserialize_double_vector(self._java_summary.variance()) + return self._java_summary.variance() def count(self): return self._java_summary.count() + @serialize def numNonzeros(self): - return _deserialize_double_vector(self._java_summary.numNonzeros()) + return self._java_summary.numNonzeros() + @serialize def max(self): - return _deserialize_double_vector(self._java_summary.max()) + return self._java_summary.max() + @serialize def min(self): - return _deserialize_double_vector(self._java_summary.min()) + return self._java_summary.min() class Statistics(object): @staticmethod - def colStats(X): + def colStats(rdd): """ Computes column-wise summary statistics for the input RDD[Vector]. - >>> from linalg import Vectors + >>> from pyspark.mllib.linalg import Vectors >>> rdd = sc.parallelize([Vectors.dense([2, 0, 0, -2]), ... Vectors.dense([4, 5, 0, 3]), ... Vectors.dense([6, 7, 0, 8])]) @@ -88,9 +105,9 @@ def colStats(X): >>> cStats.min() array([ 2., 0., 0., -2.]) """ - sc = X.ctx - Xser = _get_unmangled_double_vector_rdd(X) - cStats = sc._jvm.PythonMLLibAPI().colStats(Xser._jrdd) + sc = rdd.ctx + jrdd = rdd._to_java_object_rdd() + cStats = sc._jvm.PythonMLLibAPI().colStats(jrdd) return MultivariateStatisticalSummary(sc, cStats) @staticmethod @@ -117,7 +134,7 @@ def corr(x, y=None, method=None): >>> from math import isnan >>> isnan(Statistics.corr(x, zeros)) True - >>> from linalg import Vectors + >>> from pyspark.mllib.linalg import Vectors >>> rdd = sc.parallelize([Vectors.dense([1, 0, 0, -2]), Vectors.dense([4, 5, 0, 3]), ... Vectors.dense([6, 7, 0, 8]), Vectors.dense([9, 0, 0, 1])]) >>> pearsonCorr = Statistics.corr(rdd) @@ -144,18 +161,16 @@ def corr(x, y=None, method=None): # check if y is used to specify the method name instead. if type(y) == str: raise TypeError("Use 'method=' to specify method name.") + + jx = x._to_java_object_rdd() if not y: - try: - Xser = _get_unmangled_double_vector_rdd(x) - except TypeError: - raise TypeError("corr called on a single RDD not consisted of Vectors.") - resultMat = sc._jvm.PythonMLLibAPI().corr(Xser._jrdd, method) - return _deserialize_double_matrix(resultMat) + resultMat = sc._jvm.PythonMLLibAPI().corr(jx, method) + bytes = sc._jvm.SerDe.dumps(resultMat) + ser = PickleSerializer() + return ser.loads(str(bytes)).toArray() else: - xSer = _get_unmangled_rdd(x, _serialize_double) - ySer = _get_unmangled_rdd(y, _serialize_double) - result = sc._jvm.PythonMLLibAPI().corr(xSer._jrdd, ySer._jrdd, method) - return result + jy = y._to_java_object_rdd() + return sc._jvm.PythonMLLibAPI().corr(jx, jy, method) def _test(): diff --git a/python/pyspark/mllib/tests.py b/python/pyspark/mllib/tests.py index 8a851bd35c0e8..5c20e100e144f 100644 --- a/python/pyspark/mllib/tests.py +++ b/python/pyspark/mllib/tests.py @@ -20,6 +20,8 @@ """ import sys +import array as pyarray + from numpy import array, array_equal if sys.version_info[:2] <= (2, 6): @@ -27,11 +29,10 @@ else: import unittest -from pyspark.mllib._common import _convert_vector, _serialize_double_vector, \ - _deserialize_double_vector, _dot, _squared_distance -from pyspark.mllib.linalg import SparseVector +from pyspark.serializers import PickleSerializer +from pyspark.mllib.linalg import Vector, SparseVector, DenseVector, _convert_to_vector from pyspark.mllib.regression import LabeledPoint -from pyspark.tests import PySparkTestCase +from pyspark.tests import ReusedPySparkTestCase as PySparkTestCase _have_scipy = False @@ -42,39 +43,52 @@ # No SciPy, but that's okay, we'll skip those tests pass +ser = PickleSerializer() + + +def _squared_distance(a, b): + if isinstance(a, Vector): + return a.squared_distance(b) + else: + return b.squared_distance(a) -class VectorTests(unittest.TestCase): + +class VectorTests(PySparkTestCase): + + def _test_serialize(self, v): + jvec = self.sc._jvm.SerDe.loads(bytearray(ser.dumps(v))) + nv = ser.loads(str(self.sc._jvm.SerDe.dumps(jvec))) + self.assertEqual(v, nv) + vs = [v] * 100 + jvecs = self.sc._jvm.SerDe.loads(bytearray(ser.dumps(vs))) + nvs = ser.loads(str(self.sc._jvm.SerDe.dumps(jvecs))) + self.assertEqual(vs, nvs) def test_serialize(self): - sv = SparseVector(4, {1: 1, 3: 2}) - dv = array([1., 2., 3., 4.]) - lst = [1, 2, 3, 4] - self.assertTrue(sv is _convert_vector(sv)) - self.assertTrue(dv is _convert_vector(dv)) - self.assertTrue(array_equal(dv, _convert_vector(lst))) - self.assertEquals(sv, _deserialize_double_vector(_serialize_double_vector(sv))) - self.assertTrue(array_equal(dv, _deserialize_double_vector(_serialize_double_vector(dv)))) - self.assertTrue(array_equal(dv, _deserialize_double_vector(_serialize_double_vector(lst)))) + self._test_serialize(DenseVector(range(10))) + self._test_serialize(DenseVector(array([1., 2., 3., 4.]))) + self._test_serialize(DenseVector(pyarray.array('d', range(10)))) + self._test_serialize(SparseVector(4, {1: 1, 3: 2})) def test_dot(self): sv = SparseVector(4, {1: 1, 3: 2}) - dv = array([1., 2., 3., 4.]) - lst = [1, 2, 3, 4] + dv = DenseVector(array([1., 2., 3., 4.])) + lst = DenseVector([1, 2, 3, 4]) mat = array([[1., 2., 3., 4.], [1., 2., 3., 4.], [1., 2., 3., 4.], [1., 2., 3., 4.]]) - self.assertEquals(10.0, _dot(sv, dv)) - self.assertTrue(array_equal(array([3., 6., 9., 12.]), _dot(sv, mat))) - self.assertEquals(30.0, _dot(dv, dv)) - self.assertTrue(array_equal(array([10., 20., 30., 40.]), _dot(dv, mat))) - self.assertEquals(30.0, _dot(lst, dv)) - self.assertTrue(array_equal(array([10., 20., 30., 40.]), _dot(lst, mat))) + self.assertEquals(10.0, sv.dot(dv)) + self.assertTrue(array_equal(array([3., 6., 9., 12.]), sv.dot(mat))) + self.assertEquals(30.0, dv.dot(dv)) + self.assertTrue(array_equal(array([10., 20., 30., 40.]), dv.dot(mat))) + self.assertEquals(30.0, lst.dot(dv)) + self.assertTrue(array_equal(array([10., 20., 30., 40.]), lst.dot(mat))) def test_squared_distance(self): sv = SparseVector(4, {1: 1, 3: 2}) - dv = array([1., 2., 3., 4.]) - lst = [4, 3, 2, 1] + dv = DenseVector(array([1., 2., 3., 4.])) + lst = DenseVector([4, 3, 2, 1]) self.assertEquals(15.0, _squared_distance(sv, dv)) self.assertEquals(25.0, _squared_distance(sv, lst)) self.assertEquals(20.0, _squared_distance(dv, lst)) @@ -198,41 +212,36 @@ def test_serialize(self): lil[1, 0] = 1 lil[3, 0] = 2 sv = SparseVector(4, {1: 1, 3: 2}) - self.assertEquals(sv, _convert_vector(lil)) - self.assertEquals(sv, _convert_vector(lil.tocsc())) - self.assertEquals(sv, _convert_vector(lil.tocoo())) - self.assertEquals(sv, _convert_vector(lil.tocsr())) - self.assertEquals(sv, _convert_vector(lil.todok())) - self.assertEquals(sv, _deserialize_double_vector(_serialize_double_vector(lil))) - self.assertEquals(sv, _deserialize_double_vector(_serialize_double_vector(lil.tocsc()))) - self.assertEquals(sv, _deserialize_double_vector(_serialize_double_vector(lil.tocsr()))) - self.assertEquals(sv, _deserialize_double_vector(_serialize_double_vector(lil.todok()))) + self.assertEquals(sv, _convert_to_vector(lil)) + self.assertEquals(sv, _convert_to_vector(lil.tocsc())) + self.assertEquals(sv, _convert_to_vector(lil.tocoo())) + self.assertEquals(sv, _convert_to_vector(lil.tocsr())) + self.assertEquals(sv, _convert_to_vector(lil.todok())) + + def serialize(l): + return ser.loads(ser.dumps(_convert_to_vector(l))) + self.assertEquals(sv, serialize(lil)) + self.assertEquals(sv, serialize(lil.tocsc())) + self.assertEquals(sv, serialize(lil.tocsr())) + self.assertEquals(sv, serialize(lil.todok())) def test_dot(self): from scipy.sparse import lil_matrix lil = lil_matrix((4, 1)) lil[1, 0] = 1 lil[3, 0] = 2 - dv = array([1., 2., 3., 4.]) - sv = SparseVector(4, {0: 1, 1: 2, 2: 3, 3: 4}) - mat = array([[1., 2., 3., 4.], - [1., 2., 3., 4.], - [1., 2., 3., 4.], - [1., 2., 3., 4.]]) - self.assertEquals(10.0, _dot(lil, dv)) - self.assertTrue(array_equal(array([3., 6., 9., 12.]), _dot(lil, mat))) + dv = DenseVector(array([1., 2., 3., 4.])) + self.assertEquals(10.0, dv.dot(lil)) def test_squared_distance(self): from scipy.sparse import lil_matrix lil = lil_matrix((4, 1)) lil[1, 0] = 3 lil[3, 0] = 2 - dv = array([1., 2., 3., 4.]) + dv = DenseVector(array([1., 2., 3., 4.])) sv = SparseVector(4, {0: 1, 1: 2, 2: 3, 3: 4}) - self.assertEquals(15.0, _squared_distance(lil, dv)) - self.assertEquals(15.0, _squared_distance(lil, sv)) - self.assertEquals(15.0, _squared_distance(dv, lil)) - self.assertEquals(15.0, _squared_distance(sv, lil)) + self.assertEquals(15.0, dv.squared_distance(lil)) + self.assertEquals(15.0, sv.squared_distance(lil)) def scipy_matrix(self, size, values): """Create a column SciPy matrix from a dictionary of values""" diff --git a/python/pyspark/mllib/tree.py b/python/pyspark/mllib/tree.py index 5b13ab682bbfc..5d7abfb96b7fe 100644 --- a/python/pyspark/mllib/tree.py +++ b/python/pyspark/mllib/tree.py @@ -18,13 +18,9 @@ from py4j.java_collections import MapConverter from pyspark import SparkContext, RDD -from pyspark.mllib._common import \ - _get_unmangled_rdd, _get_unmangled_double_vector_rdd, _serialize_double_vector, \ - _deserialize_labeled_point, _get_unmangled_labeled_point_rdd, \ - _deserialize_double +from pyspark.serializers import BatchedSerializer, PickleSerializer +from pyspark.mllib.linalg import Vector, _convert_to_vector from pyspark.mllib.regression import LabeledPoint -from pyspark.serializers import NoOpSerializer - __all__ = ['DecisionTreeModel', 'DecisionTree'] @@ -52,24 +48,28 @@ def __del__(self): def predict(self, x): """ Predict the label of one or more examples. + :param x: Data point (feature vector), or an RDD of data points (feature vectors). """ - pythonAPI = self._sc._jvm.PythonMLLibAPI() + SerDe = self._sc._jvm.SerDe + ser = PickleSerializer() if isinstance(x, RDD): # Bulk prediction - if x.count() == 0: + first = x.take(1) + if not first: return self._sc.parallelize([]) - dataBytes = _get_unmangled_double_vector_rdd(x, cache=False) - jSerializedPreds = \ - pythonAPI.predictDecisionTreeModel(self._java_model, - dataBytes._jrdd) - serializedPreds = RDD(jSerializedPreds, self._sc, NoOpSerializer()) - return serializedPreds.map(lambda bytes: _deserialize_double(bytearray(bytes))) + if not isinstance(first[0], Vector): + x = x.map(_convert_to_vector) + jPred = self._java_model.predict(x._to_java_object_rdd()).toJavaRDD() + jpyrdd = self._sc._jvm.PythonRDD.javaToPython(jPred) + return RDD(jpyrdd, self._sc, BatchedSerializer(ser, 1024)) + else: # Assume x is a single data point. - x_ = _serialize_double_vector(x) - return pythonAPI.predictDecisionTreeModel(self._java_model, x_) + bytes = bytearray(ser.dumps(_convert_to_vector(x))) + vec = self._sc._jvm.SerDe.loads(bytes) + return self._java_model.predict(vec) def numNodes(self): return self._java_model.numNodes() @@ -77,9 +77,14 @@ def numNodes(self): def depth(self): return self._java_model.depth() - def __str__(self): + def __repr__(self): + """ Print summary of model. """ return self._java_model.toString() + def toDebugString(self): + """ Print full model. """ + return self._java_model.toDebugString() + class DecisionTree(object): @@ -90,52 +95,23 @@ class DecisionTree(object): EXPERIMENTAL: This is an experimental API. It will probably be modified for Spark v1.2. - Example usage: - - >>> from numpy import array - >>> import sys - >>> from pyspark.mllib.regression import LabeledPoint - >>> from pyspark.mllib.tree import DecisionTree - >>> from pyspark.mllib.linalg import SparseVector - >>> - >>> data = [ - ... LabeledPoint(0.0, [0.0]), - ... LabeledPoint(1.0, [1.0]), - ... LabeledPoint(1.0, [2.0]), - ... LabeledPoint(1.0, [3.0]) - ... ] - >>> categoricalFeaturesInfo = {} # no categorical features - >>> model = DecisionTree.trainClassifier(sc.parallelize(data), numClasses=2, - ... categoricalFeaturesInfo=categoricalFeaturesInfo) - >>> sys.stdout.write(model) - DecisionTreeModel classifier - If (feature 0 <= 0.5) - Predict: 0.0 - Else (feature 0 > 0.5) - Predict: 1.0 - >>> model.predict(array([1.0])) > 0 - True - >>> model.predict(array([0.0])) == 0 - True - >>> sparse_data = [ - ... LabeledPoint(0.0, SparseVector(2, {0: 0.0})), - ... LabeledPoint(1.0, SparseVector(2, {1: 1.0})), - ... LabeledPoint(0.0, SparseVector(2, {0: 0.0})), - ... LabeledPoint(1.0, SparseVector(2, {1: 2.0})) - ... ] - >>> - >>> model = DecisionTree.trainRegressor(sc.parallelize(sparse_data), - ... categoricalFeaturesInfo=categoricalFeaturesInfo) - >>> model.predict(array([0.0, 1.0])) == 1 - True - >>> model.predict(array([0.0, 0.0])) == 0 - True - >>> model.predict(SparseVector(2, {1: 1.0})) == 1 - True - >>> model.predict(SparseVector(2, {1: 0.0})) == 0 - True """ + @staticmethod + def _train(data, type, numClasses, categoricalFeaturesInfo, + impurity="gini", maxDepth=5, maxBins=32, minInstancesPerNode=1, + minInfoGain=0.0): + first = data.first() + assert isinstance(first, LabeledPoint), "the data should be RDD of LabeledPoint" + sc = data.context + jrdd = data._to_java_object_rdd() + cfiMap = MapConverter().convert(categoricalFeaturesInfo, + sc._gateway._gateway_client) + model = sc._jvm.PythonMLLibAPI().trainDecisionTreeModel( + jrdd, type, numClasses, cfiMap, + impurity, maxDepth, maxBins, minInstancesPerNode, minInfoGain) + return DecisionTreeModel(sc, model) + @staticmethod def trainClassifier(data, numClasses, categoricalFeaturesInfo, impurity="gini", maxDepth=5, maxBins=32, minInstancesPerNode=1, @@ -159,18 +135,35 @@ def trainClassifier(data, numClasses, categoricalFeaturesInfo, the parent split :param minInfoGain: Min info gain required to create a split :return: DecisionTreeModel + + Example usage: + + >>> from numpy import array + >>> from pyspark.mllib.regression import LabeledPoint + >>> from pyspark.mllib.tree import DecisionTree + >>> + >>> data = [ + ... LabeledPoint(0.0, [0.0]), + ... LabeledPoint(1.0, [1.0]), + ... LabeledPoint(1.0, [2.0]), + ... LabeledPoint(1.0, [3.0]) + ... ] + >>> model = DecisionTree.trainClassifier(sc.parallelize(data), 2, {}) + >>> print model, # it already has newline + DecisionTreeModel classifier of depth 1 with 3 nodes + >>> print model.toDebugString(), # it already has newline + DecisionTreeModel classifier of depth 1 with 3 nodes + If (feature 0 <= 0.5) + Predict: 0.0 + Else (feature 0 > 0.5) + Predict: 1.0 + >>> model.predict(array([1.0])) > 0 + True + >>> model.predict(array([0.0])) == 0 + True """ - sc = data.context - dataBytes = _get_unmangled_labeled_point_rdd(data) - categoricalFeaturesInfoJMap = \ - MapConverter().convert(categoricalFeaturesInfo, - sc._gateway._gateway_client) - model = sc._jvm.PythonMLLibAPI().trainDecisionTreeModel( - dataBytes._jrdd, "classification", - numClasses, categoricalFeaturesInfoJMap, - impurity, maxDepth, maxBins, minInstancesPerNode, minInfoGain) - dataBytes.unpersist() - return DecisionTreeModel(sc, model) + return DecisionTree._train(data, "classification", numClasses, categoricalFeaturesInfo, + impurity, maxDepth, maxBins, minInstancesPerNode, minInfoGain) @staticmethod def trainRegressor(data, categoricalFeaturesInfo, @@ -194,18 +187,33 @@ def trainRegressor(data, categoricalFeaturesInfo, the parent split :param minInfoGain: Min info gain required to create a split :return: DecisionTreeModel + + Example usage: + + >>> from numpy import array + >>> from pyspark.mllib.regression import LabeledPoint + >>> from pyspark.mllib.tree import DecisionTree + >>> from pyspark.mllib.linalg import SparseVector + >>> + >>> sparse_data = [ + ... LabeledPoint(0.0, SparseVector(2, {0: 0.0})), + ... LabeledPoint(1.0, SparseVector(2, {1: 1.0})), + ... LabeledPoint(0.0, SparseVector(2, {0: 0.0})), + ... LabeledPoint(1.0, SparseVector(2, {1: 2.0})) + ... ] + >>> + >>> model = DecisionTree.trainRegressor(sc.parallelize(sparse_data), {}) + >>> model.predict(array([0.0, 1.0])) == 1 + True + >>> model.predict(array([0.0, 0.0])) == 0 + True + >>> model.predict(SparseVector(2, {1: 1.0})) == 1 + True + >>> model.predict(SparseVector(2, {1: 0.0})) == 0 + True """ - sc = data.context - dataBytes = _get_unmangled_labeled_point_rdd(data) - categoricalFeaturesInfoJMap = \ - MapConverter().convert(categoricalFeaturesInfo, - sc._gateway._gateway_client) - model = sc._jvm.PythonMLLibAPI().trainDecisionTreeModel( - dataBytes._jrdd, "regression", - 0, categoricalFeaturesInfoJMap, - impurity, maxDepth, maxBins, minInstancesPerNode, minInfoGain) - dataBytes.unpersist() - return DecisionTreeModel(sc, model) + return DecisionTree._train(data, "regression", 0, categoricalFeaturesInfo, + impurity, maxDepth, maxBins, minInstancesPerNode, minInfoGain) def _test(): diff --git a/python/pyspark/mllib/util.py b/python/pyspark/mllib/util.py index 1c7b8c809ab5b..8233d4e81f1ca 100644 --- a/python/pyspark/mllib/util.py +++ b/python/pyspark/mllib/util.py @@ -18,11 +18,10 @@ import numpy as np import warnings -from pyspark.mllib.linalg import Vectors, SparseVector -from pyspark.mllib.regression import LabeledPoint -from pyspark.mllib._common import _convert_vector, _deserialize_labeled_point from pyspark.rdd import RDD -from pyspark.serializers import NoOpSerializer +from pyspark.serializers import BatchedSerializer, PickleSerializer +from pyspark.mllib.linalg import Vectors, SparseVector, _convert_to_vector +from pyspark.mllib.regression import LabeledPoint class MLUtils(object): @@ -32,15 +31,12 @@ class MLUtils(object): """ @staticmethod - def _parse_libsvm_line(line, multiclass): - warnings.warn("deprecated", DeprecationWarning) - return _parse_libsvm_line(line) - - @staticmethod - def _parse_libsvm_line(line): + def _parse_libsvm_line(line, multiclass=None): """ Parses a line in LIBSVM format into (label, indices, values). """ + if multiclass is not None: + warnings.warn("deprecated", DeprecationWarning) items = line.split(None) label = float(items[0]) nnz = len(items) - 1 @@ -55,27 +51,20 @@ def _parse_libsvm_line(line): @staticmethod def _convert_labeled_point_to_libsvm(p): """Converts a LabeledPoint to a string in LIBSVM format.""" + assert isinstance(p, LabeledPoint) items = [str(p.label)] - v = _convert_vector(p.features) - if type(v) == np.ndarray: - for i in xrange(len(v)): - items.append(str(i + 1) + ":" + str(v[i])) - elif type(v) == SparseVector: + v = _convert_to_vector(p.features) + if isinstance(v, SparseVector): nnz = len(v.indices) for i in xrange(nnz): items.append(str(v.indices[i] + 1) + ":" + str(v.values[i])) else: - raise TypeError("_convert_labeled_point_to_libsvm needs either ndarray or SparseVector" - " but got " % type(v)) + for i in xrange(len(v)): + items.append(str(i + 1) + ":" + str(v[i])) return " ".join(items) @staticmethod - def loadLibSVMFile(sc, path, multiclass=False, numFeatures=-1, minPartitions=None): - warnings.warn("deprecated", DeprecationWarning) - return loadLibSVMFile(sc, path, numFeatures, minPartitions) - - @staticmethod - def loadLibSVMFile(sc, path, numFeatures=-1, minPartitions=None): + def loadLibSVMFile(sc, path, numFeatures=-1, minPartitions=None, multiclass=None): """ Loads labeled data in the LIBSVM format into an RDD of LabeledPoint. The LIBSVM format is a text-based format used by @@ -122,6 +111,8 @@ def loadLibSVMFile(sc, path, numFeatures=-1, minPartitions=None): >>> print examples[2] (-1.0,(6,[1,3,5],[4.0,5.0,6.0])) """ + if multiclass is not None: + warnings.warn("deprecated", DeprecationWarning) lines = sc.textFile(path, minPartitions) parsed = lines.map(lambda l: MLUtils._parse_libsvm_line(l)) @@ -182,9 +173,9 @@ def loadLabeledPoints(sc, path, minPartitions=None): (0.0,[1.01,2.02,3.03]) """ minPartitions = minPartitions or min(sc.defaultParallelism, 2) - jSerialized = sc._jvm.PythonMLLibAPI().loadLabeledPoints(sc._jsc, path, minPartitions) - serialized = RDD(jSerialized, sc, NoOpSerializer()) - return serialized.map(lambda bytes: _deserialize_labeled_point(bytearray(bytes))) + jrdd = sc._jvm.PythonMLLibAPI().loadLabeledPoints(sc._jsc, path, minPartitions) + jpyrdd = sc._jvm.PythonRDD.javaToPython(jrdd) + return RDD(jpyrdd, sc, BatchedSerializer(PickleSerializer())) def _test(): diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index cb09c191bed71..e77669aad76b6 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -15,7 +15,6 @@ # limitations under the License. # -from base64 import standard_b64encode as b64enc import copy from collections import defaultdict from itertools import chain, ifilter, imap @@ -32,11 +31,12 @@ from random import Random from math import sqrt, log, isinf, isnan +from pyspark.accumulators import PStatsParam from pyspark.serializers import NoOpSerializer, CartesianDeserializer, \ BatchedSerializer, CloudPickleSerializer, PairDeserializer, \ - PickleSerializer, pack_long, CompressedSerializer + PickleSerializer, pack_long, AutoBatchedSerializer from pyspark.join import python_join, python_left_outer_join, \ - python_right_outer_join, python_cogroup + python_right_outer_join, python_full_outer_join, python_cogroup from pyspark.statcounter import StatCounter from pyspark.rddsampler import RDDSampler, RDDStratifiedSampler from pyspark.storagelevel import StorageLevel @@ -1208,6 +1208,7 @@ def saveAsSequenceFile(self, path, compressionCodecClass=None): Output a Python RDD of key-value pairs (of form C{RDD[(K, V)]}) to any Hadoop file system, using the L{org.apache.hadoop.io.Writable} types that we convert from the RDD's key and value types. The mechanism is as follows: + 1. Pyrolite is used to convert pickled Python RDD into RDD of Java objects. 2. Keys and values of this Java RDD are converted to Writables and written out. @@ -1375,7 +1376,7 @@ def leftOuterJoin(self, other, numPartitions=None): For each element (k, v) in C{self}, the resulting RDD will either contain all pairs (k, (v, w)) for w in C{other}, or the pair - (k, (v, None)) if no elements in other have key k. + (k, (v, None)) if no elements in C{other} have key k. Hash-partitions the resulting RDD into the given number of partitions. @@ -1403,6 +1404,27 @@ def rightOuterJoin(self, other, numPartitions=None): """ return python_right_outer_join(self, other, numPartitions) + def fullOuterJoin(self, other, numPartitions=None): + """ + Perform a right outer join of C{self} and C{other}. + + For each element (k, v) in C{self}, the resulting RDD will either + contain all pairs (k, (v, w)) for w in C{other}, or the pair + (k, (v, None)) if no elements in C{other} have key k. + + Similarly, for each element (k, w) in C{other}, the resulting RDD will + either contain all pairs (k, (v, w)) for v in C{self}, or the pair + (k, (None, w)) if no elements in C{self} have key k. + + Hash-partitions the resulting RDD into the given number of partitions. + + >>> x = sc.parallelize([("a", 1), ("b", 4)]) + >>> y = sc.parallelize([("a", 2), ("c", 8)]) + >>> sorted(x.fullOuterJoin(y).collect()) + [('a', (1, 2)), ('b', (4, None)), ('c', (None, 8))] + """ + return python_full_outer_join(self, other, numPartitions) + # TODO: add option to control map-side combining # portable_hash is used as default, because builtin hash of None is different # cross machines. @@ -1927,10 +1949,10 @@ def _to_java_object_rdd(self): It will convert each Python object into Java object by Pyrolite, whenever the RDD is serialized in batch or not. """ - if not self._is_pickled(): - self = self._reserialize(BatchedSerializer(PickleSerializer(), 1024)) - batched = isinstance(self._jrdd_deserializer, BatchedSerializer) - return self.ctx._jvm.PythonRDD.pythonToJava(self._jrdd, batched) + rdd = self._reserialize(AutoBatchedSerializer(PickleSerializer())) \ + if not self._is_pickled() else self + is_batch = isinstance(rdd._jrdd_deserializer, BatchedSerializer) + return self.ctx._jvm.PythonRDD.pythonToJava(rdd._jrdd, is_batch) def countApprox(self, timeout, confidence=0.95): """ @@ -2052,6 +2074,12 @@ def pipeline_func(split, iterator): self._jrdd_deserializer = self.ctx.serializer self._bypass_serializer = False self._partitionFunc = prev._partitionFunc if self.preservesPartitioning else None + self._broadcast = None + + def __del__(self): + if self._broadcast: + self._broadcast.unpersist() + self._broadcast = None @property def _jrdd(self): @@ -2059,10 +2087,16 @@ def _jrdd(self): return self._jrdd_val if self._bypass_serializer: self._jrdd_deserializer = NoOpSerializer() - command = (self.func, self._prev_jrdd_deserializer, + enable_profile = self.ctx._conf.get("spark.python.profile", "false") == "true" + profileStats = self.ctx.accumulator(None, PStatsParam) if enable_profile else None + command = (self.func, profileStats, self._prev_jrdd_deserializer, self._jrdd_deserializer) + # the serialized command will be compressed by broadcast ser = CloudPickleSerializer() pickled_command = ser.dumps(command) + if len(pickled_command) > (1 << 20): # 1M + self._broadcast = self.ctx.broadcast(pickled_command) + pickled_command = ser.dumps(self._broadcast) broadcast_vars = ListConverter().convert( [x._jbroadcast for x in self.ctx._pickled_broadcast_vars], self.ctx._gateway._gateway_client) @@ -2077,6 +2111,10 @@ def _jrdd(self): self.ctx.pythonExec, broadcast_vars, self.ctx._javaAccumulator) self._jrdd_val = python_rdd.asJavaRDD() + + if enable_profile: + self._id = self._jrdd_val.id() + self.ctx._add_profile(self._id, profileStats) return self._jrdd_val def id(self): diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index 44ac5642836e0..099fa54cf2bd7 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -68,6 +68,7 @@ import types import collections import zlib +import itertools from pyspark import cloudpickle @@ -210,6 +211,41 @@ def __eq__(self, other): return (isinstance(other, BatchedSerializer) and other.serializer == self.serializer) + def __repr__(self): + return "BatchedSerializer<%s>" % str(self.serializer) + + +class AutoBatchedSerializer(BatchedSerializer): + """ + Choose the size of batch automatically based on the size of object + """ + + def __init__(self, serializer, bestSize=1 << 20): + BatchedSerializer.__init__(self, serializer, -1) + self.bestSize = bestSize + + def dump_stream(self, iterator, stream): + batch, best = 1, self.bestSize + iterator = iter(iterator) + while True: + vs = list(itertools.islice(iterator, batch)) + if not vs: + break + + bytes = self.serializer.dumps(vs) + write_int(len(bytes), stream) + stream.write(bytes) + + size = len(bytes) + if size < best: + batch *= 2 + elif size > best * 10 and batch > 1: + batch /= 2 + + def __eq__(self, other): + return (isinstance(other, AutoBatchedSerializer) and + other.serializer == self.serializer) + def __str__(self): return "BatchedSerializer<%s>" % str(self.serializer) @@ -243,7 +279,7 @@ def __eq__(self, other): return (isinstance(other, CartesianDeserializer) and self.key_ser == other.key_ser and self.val_ser == other.val_ser) - def __str__(self): + def __repr__(self): return "CartesianDeserializer<%s, %s>" % \ (str(self.key_ser), str(self.val_ser)) @@ -270,7 +306,7 @@ def __eq__(self, other): return (isinstance(other, PairDeserializer) and self.key_ser == other.key_ser and self.val_ser == other.val_ser) - def __str__(self): + def __repr__(self): return "PairDeserializer<%s, %s>" % (str(self.key_ser), str(self.val_ser)) diff --git a/python/pyspark/shuffle.py b/python/pyspark/shuffle.py index ce597cbe91e15..d57a802e4734a 100644 --- a/python/pyspark/shuffle.py +++ b/python/pyspark/shuffle.py @@ -396,7 +396,6 @@ def _external_items(self): for v in self.data.iteritems(): yield v self.data.clear() - gc.collect() # remove the merged partition for j in range(self.spills): @@ -428,7 +427,7 @@ def _recursive_merged_items(self, start): subdirs = [os.path.join(d, "parts", str(i)) for d in self.localdirs] m = ExternalMerger(self.agg, self.memory_limit, self.serializer, - subdirs, self.scale * self.partitions) + subdirs, self.scale * self.partitions, self.partitions) m.pdata = [{} for _ in range(self.partitions)] limit = self._next_limit() @@ -486,7 +485,7 @@ def sorted(self, iterator, key=None, reverse=False): goes above the limit. """ global MemoryBytesSpilled, DiskBytesSpilled - batch = 10 + batch = 100 chunks, current_chunk = [], [] iterator = iter(iterator) while True: diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 8f6dbab240c7b..114644ab8b79d 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -27,7 +27,7 @@ from array import array from operator import itemgetter -from pyspark.rdd import RDD, PipelinedRDD +from pyspark.rdd import RDD from pyspark.serializers import BatchedSerializer, PickleSerializer, CloudPickleSerializer from pyspark.storagelevel import StorageLevel from pyspark.traceback_utils import SCCallSiteSync @@ -201,7 +201,7 @@ def __init__(self, elementType, containsNull=True): self.elementType = elementType self.containsNull = containsNull - def __str__(self): + def __repr__(self): return "ArrayType(%s,%s)" % (self.elementType, str(self.containsNull).lower()) @@ -440,6 +440,7 @@ def _parse_datatype_string(datatype_string): float: DoubleType, str: StringType, unicode: StringType, + bytearray: BinaryType, decimal.Decimal: DecimalType, datetime.datetime: TimestampType, datetime.date: TimestampType, @@ -690,11 +691,12 @@ def _infer_schema_type(obj, dataType): ByteType: (int, long), ShortType: (int, long), IntegerType: (int, long), - LongType: (long,), + LongType: (int, long), FloatType: (float,), DoubleType: (float,), DecimalType: (decimal.Decimal,), StringType: (str, unicode), + BinaryType: (bytearray,), TimestampType: (datetime.datetime,), ArrayType: (list, tuple, array), MapType: (dict,), @@ -728,9 +730,9 @@ def _verify_type(obj, dataType): return _type = type(dataType) - if _type not in _acceptable_types: - return + assert _type in _acceptable_types, "unkown datatype: %s" % dataType + # subclass of them can not be deserialized in JVM if type(obj) not in _acceptable_types[_type]: raise TypeError("%s can not accept abject in type %s" % (dataType, type(obj))) @@ -836,43 +838,29 @@ def _create_cls(dataType): >>> obj = _create_cls(schema)(row) >>> pickle.loads(pickle.dumps(obj)) Row(a=[1], b={'key': Row(c=1, d=2.0)}) + >>> pickle.loads(pickle.dumps(obj.a)) + [1] + >>> pickle.loads(pickle.dumps(obj.b)) + {'key': Row(c=1, d=2.0)} """ if isinstance(dataType, ArrayType): cls = _create_cls(dataType.elementType) - class List(list): - - def __getitem__(self, i): - # create object with datetype - return _create_object(cls, list.__getitem__(self, i)) - - def __repr__(self): - # call collect __repr__ for nested objects - return "[%s]" % (", ".join(repr(self[i]) - for i in range(len(self)))) - - def __reduce__(self): - return list.__reduce__(self) + def List(l): + if l is None: + return + return [_create_object(cls, v) for v in l] return List elif isinstance(dataType, MapType): - vcls = _create_cls(dataType.valueType) + cls = _create_cls(dataType.valueType) - class Dict(dict): - - def __getitem__(self, k): - # create object with datetype - return _create_object(vcls, dict.__getitem__(self, k)) - - def __repr__(self): - # call collect __repr__ for nested objects - return "{%s}" % (", ".join("%r: %r" % (k, self[k]) - for k in self)) - - def __reduce__(self): - return dict.__reduce__(self) + def Dict(d): + if d is None: + return + return dict((k, _create_object(cls, v)) for k, v in d.items()) return Dict @@ -972,10 +960,14 @@ def registerFunction(self, name, f, returnType=StringType()): [Row(c0=4)] """ func = lambda _, it: imap(lambda x: f(*x), it) - command = (func, + command = (func, None, BatchedSerializer(PickleSerializer(), 1024), BatchedSerializer(PickleSerializer(), 1024)) - pickled_command = CloudPickleSerializer().dumps(command) + ser = CloudPickleSerializer() + pickled_command = ser.dumps(command) + if len(pickled_command) > (1 << 20): # 1M + broadcast = self._sc.broadcast(pickled_command) + pickled_command = ser.dumps(broadcast) broadcast_vars = ListConverter().convert( [x._jbroadcast for x in self._sc._pickled_broadcast_vars], self._sc._gateway._gateway_client) @@ -1117,6 +1109,11 @@ def applySchema(self, rdd, schema): # take the first few rows to verify schema rows = rdd.take(10) + # Row() cannot been deserialized by Pyrolite + if rows and isinstance(rows[0], tuple) and rows[0].__class__.__name__ == 'Row': + rdd = rdd.map(tuple) + rows = rdd.take(10) + for row in rows: _verify_type(row, schema) diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 0b3854347ad2e..7f05d48ade2b3 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -45,7 +45,7 @@ from pyspark.serializers import read_int, BatchedSerializer, MarshalSerializer, PickleSerializer, \ CloudPickleSerializer from pyspark.shuffle import Aggregator, InMemoryMerger, ExternalMerger, ExternalSorter -from pyspark.sql import SQLContext, IntegerType +from pyspark.sql import SQLContext, IntegerType, Row from pyspark import shuffle _have_scipy = False @@ -67,10 +67,10 @@ SPARK_HOME = os.environ["SPARK_HOME"] -class TestMerger(unittest.TestCase): +class MergerTests(unittest.TestCase): def setUp(self): - self.N = 1 << 16 + self.N = 1 << 14 self.l = [i for i in xrange(self.N)] self.data = zip(self.l, self.l) self.agg = Aggregator(lambda x: [x], @@ -115,7 +115,7 @@ def test_medium_dataset(self): sum(xrange(self.N)) * 3) def test_huge_dataset(self): - m = ExternalMerger(self.agg, 10) + m = ExternalMerger(self.agg, 10, partitions=3) m.mergeCombiners(map(lambda (k, v): (k, [str(v)]), self.data * 10)) self.assertTrue(m.spills >= 1) self.assertEqual(sum(len(v) for k, v in m._recursive_merged_items(0)), @@ -123,7 +123,7 @@ def test_huge_dataset(self): m._cleanup() -class TestSorter(unittest.TestCase): +class SorterTests(unittest.TestCase): def test_in_memory_sort(self): l = range(1024) random.shuffle(l) @@ -213,6 +213,24 @@ def test_pickling_file_handles(self): out2 = ser.loads(ser.dumps(out1)) self.assertEquals(out1, out2) + def test_func_globals(self): + + class Unpicklable(object): + def __reduce__(self): + raise Exception("not picklable") + + global exit + exit = Unpicklable() + + ser = CloudPickleSerializer() + self.assertRaises(Exception, lambda: ser.dumps(exit)) + + def foo(): + sys.exit(0) + + self.assertTrue("exit" in foo.func_code.co_names) + ser.dumps(foo) + class PySparkTestCase(unittest.TestCase): @@ -226,16 +244,25 @@ def tearDown(self): sys.path = self._old_sys_path -class TestCheckpoint(PySparkTestCase): +class ReusedPySparkTestCase(unittest.TestCase): + + @classmethod + def setUpClass(cls): + cls.sc = SparkContext('local[4]', cls.__name__, batchSize=2) + + @classmethod + def tearDownClass(cls): + cls.sc.stop() + + +class CheckpointTests(ReusedPySparkTestCase): def setUp(self): - PySparkTestCase.setUp(self) self.checkpointDir = tempfile.NamedTemporaryFile(delete=False) os.unlink(self.checkpointDir.name) self.sc.setCheckpointDir(self.checkpointDir.name) def tearDown(self): - PySparkTestCase.tearDown(self) shutil.rmtree(self.checkpointDir.name) def test_basic_checkpointing(self): @@ -270,7 +297,7 @@ def test_checkpoint_and_restore(self): self.assertEquals([1, 2, 3, 4], recovered.collect()) -class TestAddFile(PySparkTestCase): +class AddFileTests(PySparkTestCase): def test_add_py_file(self): # To ensure that we're actually testing addPyFile's effects, check that @@ -323,8 +350,20 @@ def func(): from userlib import UserClass self.assertEqual("Hello World from inside a package!", UserClass().hello()) + def test_overwrite_system_module(self): + self.sc.addPyFile(os.path.join(SPARK_HOME, "python/test_support/SimpleHTTPServer.py")) -class TestRDDFunctions(PySparkTestCase): + import SimpleHTTPServer + self.assertEqual("My Server", SimpleHTTPServer.__name__) + + def func(x): + import SimpleHTTPServer + return SimpleHTTPServer.__name__ + + self.assertEqual(["My Server"], self.sc.parallelize(range(1)).map(func).collect()) + + +class RDDTests(ReusedPySparkTestCase): def test_id(self): rdd = self.sc.parallelize(range(10)) @@ -335,12 +374,6 @@ def test_id(self): self.assertEqual(id + 1, id2) self.assertEqual(id2, rdd2.id()) - def test_failed_sparkcontext_creation(self): - # Regression test for SPARK-1550 - self.sc.stop() - self.assertRaises(Exception, lambda: SparkContext("an-invalid-master-name")) - self.sc = SparkContext("local") - def test_save_as_textfile_with_unicode(self): # Regression test for SPARK-970 x = u"\u00A1Hola, mundo!" @@ -434,6 +467,16 @@ def test_large_broadcast(self): m = self.sc.parallelize(range(1), 1).map(lambda x: len(bdata.value)).sum() self.assertEquals(N, m) + def test_large_closure(self): + N = 1000000 + data = [float(i) for i in xrange(N)] + rdd = self.sc.parallelize(range(1), 1).map(lambda x: len(data)) + self.assertEquals(N, rdd.first()) + self.assertTrue(rdd._broadcast is not None) + rdd = self.sc.parallelize(range(1), 1).map(lambda x: 1) + self.assertEqual(1, rdd.first()) + self.assertTrue(rdd._broadcast is None) + def test_zip_with_different_serializers(self): a = self.sc.parallelize(range(5)) b = self.sc.parallelize(range(100, 105)) @@ -596,10 +639,39 @@ def test_distinct(self): self.assertEquals(result.count(), 3) -class TestSQL(PySparkTestCase): +class ProfilerTests(PySparkTestCase): + + def setUp(self): + self._old_sys_path = list(sys.path) + class_name = self.__class__.__name__ + conf = SparkConf().set("spark.python.profile", "true") + self.sc = SparkContext('local[4]', class_name, batchSize=2, conf=conf) + + def test_profiler(self): + + def heavy_foo(x): + for i in range(1 << 20): + x = 1 + rdd = self.sc.parallelize(range(100)) + rdd.foreach(heavy_foo) + profiles = self.sc._profile_stats + self.assertEqual(1, len(profiles)) + id, acc, _ = profiles[0] + stats = acc.value + self.assertTrue(stats is not None) + width, stat_list = stats.get_print_list([]) + func_names = [func_name for fname, n, func_name in stat_list] + self.assertTrue("heavy_foo" in func_names) + + self.sc.show_profiles() + d = tempfile.gettempdir() + self.sc.dump_profiles(d) + self.assertTrue("rdd_%d.pstats" % id in os.listdir(d)) + + +class SQLTests(ReusedPySparkTestCase): def setUp(self): - PySparkTestCase.setUp(self) self.sqlCtx = SQLContext(self.sc) def test_udf(self): @@ -653,28 +725,50 @@ def test_distinct(self): self.assertEquals(result.getNumPartitions(), 5) self.assertEquals(result.count(), 3) + def test_apply_schema_to_row(self): + srdd = self.sqlCtx.jsonRDD(self.sc.parallelize(["""{"a":2}"""])) + srdd2 = self.sqlCtx.applySchema(srdd.map(lambda x: x), srdd.schema()) + self.assertEqual(srdd.collect(), srdd2.collect()) -class TestIO(PySparkTestCase): + rdd = self.sc.parallelize(range(10)).map(lambda x: Row(a=x)) + srdd3 = self.sqlCtx.applySchema(rdd, srdd.schema()) + self.assertEqual(10, srdd3.count()) - def test_stdout_redirection(self): - import subprocess + def test_serialize_nested_array_and_map(self): + d = [Row(l=[Row(a=1, b='s')], d={"key": Row(c=1.0, d="2")})] + rdd = self.sc.parallelize(d) + srdd = self.sqlCtx.inferSchema(rdd) + row = srdd.first() + self.assertEqual(1, len(row.l)) + self.assertEqual(1, row.l[0].a) + self.assertEqual("2", row.d["key"].d) - def func(x): - subprocess.check_call('ls', shell=True) - self.sc.parallelize([1]).foreach(func) + l = srdd.map(lambda x: x.l).first() + self.assertEqual(1, len(l)) + self.assertEqual('s', l[0].b) + d = srdd.map(lambda x: x.d).first() + self.assertEqual(1, len(d)) + self.assertEqual(1.0, d["key"].c) -class TestInputFormat(PySparkTestCase): + row = srdd.map(lambda x: x.d["key"]).first() + self.assertEqual(1.0, row.c) + self.assertEqual("2", row.d) - def setUp(self): - PySparkTestCase.setUp(self) - self.tempdir = tempfile.NamedTemporaryFile(delete=False) - os.unlink(self.tempdir.name) - self.sc._jvm.WriteInputFormatTestDataGenerator.generateData(self.tempdir.name, self.sc._jsc) - def tearDown(self): - PySparkTestCase.tearDown(self) - shutil.rmtree(self.tempdir.name) +class InputFormatTests(ReusedPySparkTestCase): + + @classmethod + def setUpClass(cls): + ReusedPySparkTestCase.setUpClass() + cls.tempdir = tempfile.NamedTemporaryFile(delete=False) + os.unlink(cls.tempdir.name) + cls.sc._jvm.WriteInputFormatTestDataGenerator.generateData(cls.tempdir.name, cls.sc._jsc) + + @classmethod + def tearDownClass(cls): + ReusedPySparkTestCase.tearDownClass() + shutil.rmtree(cls.tempdir.name) def test_sequencefiles(self): basepath = self.tempdir.name @@ -854,15 +948,13 @@ def test_converters(self): self.assertEqual(maps, em) -class TestOutputFormat(PySparkTestCase): +class OutputFormatTests(ReusedPySparkTestCase): def setUp(self): - PySparkTestCase.setUp(self) self.tempdir = tempfile.NamedTemporaryFile(delete=False) os.unlink(self.tempdir.name) def tearDown(self): - PySparkTestCase.tearDown(self) shutil.rmtree(self.tempdir.name, ignore_errors=True) def test_sequencefiles(self): @@ -1092,7 +1184,7 @@ def test_reserialization(self): def test_unbatched_save_and_read(self): basepath = self.tempdir.name ei = [(1, u'aa'), (1, u'aa'), (2, u'aa'), (2, u'bb'), (2, u'bb'), (3, u'cc')] - self.sc.parallelize(ei, numSlices=len(ei)).saveAsSequenceFile( + self.sc.parallelize(ei, len(ei)).saveAsSequenceFile( basepath + "/unbatched/") unbatched_sequence = sorted(self.sc.sequenceFile( @@ -1138,13 +1230,12 @@ def test_malformed_RDD(self): basepath = self.tempdir.name # non-batch-serialized RDD[[(K, V)]] should be rejected data = [[(1, "a")], [(2, "aa")], [(3, "aaa")]] - rdd = self.sc.parallelize(data, numSlices=len(data)) + rdd = self.sc.parallelize(data, len(data)) self.assertRaises(Exception, lambda: rdd.saveAsSequenceFile( basepath + "/malformed/sequence")) -class TestDaemon(unittest.TestCase): - +class DaemonTests(unittest.TestCase): def connect(self, port): from socket import socket, AF_INET, SOCK_STREAM sock = socket(AF_INET, SOCK_STREAM) @@ -1190,7 +1281,7 @@ def test_termination_sigterm(self): self.do_termination_test(lambda daemon: os.kill(daemon.pid, SIGTERM)) -class TestWorker(PySparkTestCase): +class WorkerTests(PySparkTestCase): def test_cancel_task(self): temp = tempfile.NamedTemporaryFile(delete=True) @@ -1242,11 +1333,6 @@ def run(): rdd = self.sc.parallelize(range(100), 1) self.assertEqual(100, rdd.map(str).count()) - def test_fd_leak(self): - N = 1100 # fd limit is 1024 by default - rdd = self.sc.parallelize(range(N), N) - self.assertEquals(N, rdd.count()) - def test_after_exception(self): def raise_exception(_): raise Exception() @@ -1279,7 +1365,7 @@ def test_accumulator_when_reuse_worker(self): self.assertEqual(sum(range(100)), acc1.value) -class TestSparkSubmit(unittest.TestCase): +class SparkSubmitTests(unittest.TestCase): def setUp(self): self.programDir = tempfile.mkdtemp() @@ -1392,6 +1478,8 @@ def test_single_script_on_cluster(self): |sc = SparkContext() |print sc.parallelize([1, 2, 3]).map(foo).collect() """) + # this will fail if you have different spark.executor.memory + # in conf/spark-defaults.conf proc = subprocess.Popen( [self.sparkSubmit, "--master", "local-cluster[1,1,512]", script], stdout=subprocess.PIPE) @@ -1400,7 +1488,11 @@ def test_single_script_on_cluster(self): self.assertIn("[2, 4, 6]", out) -class ContextStopTests(unittest.TestCase): +class ContextTests(unittest.TestCase): + + def test_failed_sparkcontext_creation(self): + # Regression test for SPARK-1550 + self.assertRaises(Exception, lambda: SparkContext("an-invalid-master-name")) def test_stop(self): sc = SparkContext() diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 252176ac65fec..8257dddfee1c3 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -23,6 +23,8 @@ import time import socket import traceback +import cProfile +import pstats from pyspark.accumulators import _accumulatorRegistry from pyspark.broadcast import Broadcast, _broadcastRegistry @@ -43,6 +45,13 @@ def report_times(outfile, boot, init, finish): write_long(1000 * finish, outfile) +def add_path(path): + # worker can be used, so donot add path multiple times + if path not in sys.path: + # overwrite system packages + sys.path.insert(1, path) + + def main(infile, outfile): try: boot_time = time.time() @@ -61,11 +70,11 @@ def main(infile, outfile): SparkFiles._is_running_on_worker = True # fetch names of includes (*.zip and *.egg files) and construct PYTHONPATH - sys.path.append(spark_files_dir) # *.py files that were added will be copied here + add_path(spark_files_dir) # *.py files that were added will be copied here num_python_includes = read_int(infile) for _ in range(num_python_includes): filename = utf8_deserializer.loads(infile) - sys.path.append(os.path.join(spark_files_dir, filename)) + add_path(os.path.join(spark_files_dir, filename)) # fetch names and values of broadcast variables num_broadcast_variables = read_int(infile) @@ -77,14 +86,27 @@ def main(infile, outfile): _broadcastRegistry[bid] = Broadcast(bid, value) else: bid = - bid - 1 - _broadcastRegistry.remove(bid) + _broadcastRegistry.pop(bid) _accumulatorRegistry.clear() command = pickleSer._read_with_length(infile) - (func, deserializer, serializer) = command + if isinstance(command, Broadcast): + command = pickleSer.loads(command.value) + (func, stats, deserializer, serializer) = command init_time = time.time() - iterator = deserializer.load_stream(infile) - serializer.dump_stream(func(split_index, iterator), outfile) + + def process(): + iterator = deserializer.load_stream(infile) + serializer.dump_stream(func(split_index, iterator), outfile) + + if stats: + p = cProfile.Profile() + p.runcall(process) + st = pstats.Stats(p) + st.stream = None # make it picklable + stats.add(st.strip_dirs()) + else: + process() except Exception: try: write_int(SpecialLengths.PYTHON_EXCEPTION_THROWN, outfile) diff --git a/python/run-tests b/python/run-tests index a67e5a99fbdcc..63395f72788f9 100755 --- a/python/run-tests +++ b/python/run-tests @@ -34,7 +34,7 @@ rm -rf metastore warehouse function run_test() { echo "Running test: $1" - SPARK_TESTING=1 "$FWDIR"/bin/pyspark $1 2>&1 | tee -a unit-tests.log + SPARK_TESTING=1 time "$FWDIR"/bin/pyspark $1 2>&1 | tee -a unit-tests.log FAILED=$((PIPESTATUS[0]||$FAILED)) @@ -48,6 +48,38 @@ function run_test() { fi } +function run_core_tests() { + echo "Run core tests ..." + run_test "pyspark/rdd.py" + run_test "pyspark/context.py" + run_test "pyspark/conf.py" + PYSPARK_DOC_TEST=1 run_test "pyspark/broadcast.py" + PYSPARK_DOC_TEST=1 run_test "pyspark/accumulators.py" + PYSPARK_DOC_TEST=1 run_test "pyspark/serializers.py" + run_test "pyspark/shuffle.py" + run_test "pyspark/tests.py" +} + +function run_sql_tests() { + echo "Run sql tests ..." + run_test "pyspark/sql.py" +} + +function run_mllib_tests() { + echo "Run mllib tests ..." + run_test "pyspark/mllib/classification.py" + run_test "pyspark/mllib/clustering.py" + run_test "pyspark/mllib/feature.py" + run_test "pyspark/mllib/linalg.py" + run_test "pyspark/mllib/random.py" + run_test "pyspark/mllib/recommendation.py" + run_test "pyspark/mllib/regression.py" + run_test "pyspark/mllib/stat.py" + run_test "pyspark/mllib/tree.py" + run_test "pyspark/mllib/util.py" + run_test "pyspark/mllib/tests.py" +} + echo "Running PySpark tests. Output is in python/unit-tests.log." export PYSPARK_PYTHON="python" @@ -60,30 +92,9 @@ fi echo "Testing with Python version:" $PYSPARK_PYTHON --version -run_test "pyspark/rdd.py" -run_test "pyspark/context.py" -run_test "pyspark/conf.py" -run_test "pyspark/sql.py" -# These tests are included in the module-level docs, and so must -# be handled on a higher level rather than within the python file. -export PYSPARK_DOC_TEST=1 -run_test "pyspark/broadcast.py" -run_test "pyspark/accumulators.py" -run_test "pyspark/serializers.py" -unset PYSPARK_DOC_TEST -run_test "pyspark/shuffle.py" -run_test "pyspark/tests.py" -run_test "pyspark/mllib/_common.py" -run_test "pyspark/mllib/classification.py" -run_test "pyspark/mllib/clustering.py" -run_test "pyspark/mllib/linalg.py" -run_test "pyspark/mllib/random.py" -run_test "pyspark/mllib/recommendation.py" -run_test "pyspark/mllib/regression.py" -run_test "pyspark/mllib/stat.py" -run_test "pyspark/mllib/tests.py" -run_test "pyspark/mllib/tree.py" -run_test "pyspark/mllib/util.py" +run_core_tests +run_sql_tests +run_mllib_tests # Try to test with PyPy if [ $(which pypy) ]; then @@ -91,19 +102,8 @@ if [ $(which pypy) ]; then echo "Testing with PyPy version:" $PYSPARK_PYTHON --version - run_test "pyspark/rdd.py" - run_test "pyspark/context.py" - run_test "pyspark/conf.py" - run_test "pyspark/sql.py" - # These tests are included in the module-level docs, and so must - # be handled on a higher level rather than within the python file. - export PYSPARK_DOC_TEST=1 - run_test "pyspark/broadcast.py" - run_test "pyspark/accumulators.py" - run_test "pyspark/serializers.py" - unset PYSPARK_DOC_TEST - run_test "pyspark/shuffle.py" - run_test "pyspark/tests.py" + run_core_tests + run_sql_tests fi if [[ $FAILED == 0 ]]; then diff --git a/python/test_support/SimpleHTTPServer.py b/python/test_support/SimpleHTTPServer.py new file mode 100644 index 0000000000000..eddbd588e02dc --- /dev/null +++ b/python/test_support/SimpleHTTPServer.py @@ -0,0 +1,22 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +""" +Used to test override standard SimpleHTTPServer module. +""" + +__name__ = "My Server" diff --git a/repl/src/test/resources/log4j.properties b/repl/src/test/resources/log4j.properties index 9c4896e49698c..52098993f5c3c 100644 --- a/repl/src/test/resources/log4j.properties +++ b/repl/src/test/resources/log4j.properties @@ -21,7 +21,7 @@ log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false log4j.appender.file.file=target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout -log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose log4j.logger.org.eclipse.jetty=WARN diff --git a/sbin/slaves.sh b/sbin/slaves.sh index 1d4dc5edf9858..cdad47ee2e594 100755 --- a/sbin/slaves.sh +++ b/sbin/slaves.sh @@ -44,7 +44,9 @@ sbin="`cd "$sbin"; pwd`" # If the slaves file is specified in the command line, # then it takes precedence over the definition in # spark-env.sh. Save it here. -HOSTLIST="$SPARK_SLAVES" +if [ -f "$SPARK_SLAVES" ]; then + HOSTLIST=`cat "$SPARK_SLAVES"` +fi # Check if --config is passed as an argument. It is an optional parameter. # Exit if the argument is not a directory. @@ -67,23 +69,34 @@ fi if [ "$HOSTLIST" = "" ]; then if [ "$SPARK_SLAVES" = "" ]; then - export HOSTLIST="${SPARK_CONF_DIR}/slaves" + if [ -f "${SPARK_CONF_DIR}/slaves" ]; then + HOSTLIST=`cat "${SPARK_CONF_DIR}/slaves"` + else + HOSTLIST=localhost + fi else - export HOSTLIST="${SPARK_SLAVES}" + HOSTLIST=`cat "${SPARK_SLAVES}"` fi fi + + # By default disable strict host key checking if [ "$SPARK_SSH_OPTS" = "" ]; then SPARK_SSH_OPTS="-o StrictHostKeyChecking=no" fi -for slave in `cat "$HOSTLIST"|sed "s/#.*$//;/^$/d"`; do - ssh $SPARK_SSH_OPTS "$slave" $"${@// /\\ }" \ - 2>&1 | sed "s/^/$slave: /" & - if [ "$SPARK_SLAVE_SLEEP" != "" ]; then - sleep $SPARK_SLAVE_SLEEP - fi +for slave in `echo "$HOSTLIST"|sed "s/#.*$//;/^$/d"`; do + if [ -n "${SPARK_SSH_FOREGROUND}" ]; then + ssh $SPARK_SSH_OPTS "$slave" $"${@// /\\ }" \ + 2>&1 | sed "s/^/$slave: /" + else + ssh $SPARK_SSH_OPTS "$slave" $"${@// /\\ }" \ + 2>&1 | sed "s/^/$slave: /" & + fi + if [ "$SPARK_SLAVE_SLEEP" != "" ]; then + sleep $SPARK_SLAVE_SLEEP + fi done wait diff --git a/sbin/spark-config.sh b/sbin/spark-config.sh index 2718d6cba1c9a..1d154e62ed5b6 100755 --- a/sbin/spark-config.sh +++ b/sbin/spark-config.sh @@ -33,7 +33,7 @@ this="$config_bin/$script" export SPARK_PREFIX="`dirname "$this"`"/.. export SPARK_HOME="${SPARK_PREFIX}" -export SPARK_CONF_DIR="$SPARK_HOME/conf" +export SPARK_CONF_DIR="${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.2.1-src.zip:$PYTHONPATH" diff --git a/sbin/spark-daemon.sh b/sbin/spark-daemon.sh index bd476b400e1c3..cba475e2dd8c8 100755 --- a/sbin/spark-daemon.sh +++ b/sbin/spark-daemon.sh @@ -62,7 +62,7 @@ then shift fi -startStop=$1 +option=$1 shift command=$1 shift @@ -122,9 +122,9 @@ if [ "$SPARK_NICENESS" = "" ]; then fi -case $startStop in +case $option in - (start) + (start|spark-submit) mkdir -p "$SPARK_PID_DIR" @@ -142,8 +142,14 @@ case $startStop in spark_rotate_log "$log" echo starting $command, logging to $log - cd "$SPARK_PREFIX" - nohup nice -n $SPARK_NICENESS "$SPARK_PREFIX"/bin/spark-class $command "$@" >> "$log" 2>&1 < /dev/null & + if [ $option == spark-submit ]; then + source "$SPARK_HOME"/bin/utils.sh + gatherSparkSubmitOpts "$@" + nohup nice -n $SPARK_NICENESS "$SPARK_PREFIX"/bin/spark-submit --class $command \ + "${SUBMISSION_OPTS[@]}" spark-internal "${APPLICATION_OPTS[@]}" >> "$log" 2>&1 < /dev/null & + else + nohup nice -n $SPARK_NICENESS "$SPARK_PREFIX"/bin/spark-class $command "$@" >> "$log" 2>&1 < /dev/null & + fi newpid=$! echo $newpid > $pid sleep 2 diff --git a/sbin/start-thriftserver.sh b/sbin/start-thriftserver.sh index 4ce40fe750384..50e8e06418b07 100755 --- a/sbin/start-thriftserver.sh +++ b/sbin/start-thriftserver.sh @@ -27,7 +27,6 @@ set -o posix FWDIR="$(cd "`dirname "$0"`"/..; pwd)" CLASS="org.apache.spark.sql.hive.thriftserver.HiveThriftServer2" -CLASS_NOT_FOUND_EXIT_STATUS=1 function usage { echo "Usage: ./sbin/start-thriftserver [options] [thrift server options]" @@ -49,17 +48,6 @@ if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then exit 0 fi -source "$FWDIR"/bin/utils.sh -SUBMIT_USAGE_FUNCTION=usage -gatherSparkSubmitOpts "$@" +export SUBMIT_USAGE_FUNCTION=usage -"$FWDIR"/bin/spark-submit --class $CLASS "${SUBMISSION_OPTS[@]}" spark-internal "${APPLICATION_OPTS[@]}" -exit_status=$? - -if [[ exit_status -eq CLASS_NOT_FOUND_EXIT_STATUS ]]; then - echo - echo "Failed to load Hive Thrift server main class $CLASS." - echo "You need to build Spark with -Phive." -fi - -exit $exit_status +exec "$FWDIR"/sbin/spark-daemon.sh spark-submit $CLASS 1 "$@" diff --git a/sbin/stop-thriftserver.sh b/sbin/stop-thriftserver.sh new file mode 100755 index 0000000000000..4031a00d4a689 --- /dev/null +++ b/sbin/stop-thriftserver.sh @@ -0,0 +1,25 @@ +#!/usr/bin/env bash + +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Stops the thrift server on the machine this script is executed on. + +sbin="`dirname "$0"`" +sbin="`cd "$sbin"; pwd`" + +"$sbin"/spark-daemon.sh stop org.apache.spark.sql.hive.thriftserver.HiveThriftServer2 1 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 18e10c95754ff..a6769a4e62ab6 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 @@ -33,7 +33,7 @@ object ScalaReflection { /** Converts Scala objects to catalyst rows / types */ def convertToCatalyst(a: Any): Any = a match { - case o: Option[_] => o.orNull + case o: Option[_] => o.map(convertToCatalyst).orNull case s: Seq[_] => s.map(convertToCatalyst) case m: Map[_, _] => m.map { case (k, v) => convertToCatalyst(k) -> convertToCatalyst(v) } case p: Product => new GenericRow(p.productIterator.map(convertToCatalyst).toArray) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index ca69531c69a77..854b5b461bdc8 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -67,11 +67,12 @@ class SqlParser extends StandardTokenParsers with PackratParsers { protected implicit def asParser(k: Keyword): Parser[String] = lexical.allCaseVersions(k.str).map(x => x : Parser[String]).reduce(_ | _) + protected val ABS = Keyword("ABS") protected val ALL = Keyword("ALL") protected val AND = Keyword("AND") + protected val APPROXIMATE = Keyword("APPROXIMATE") protected val AS = Keyword("AS") protected val ASC = Keyword("ASC") - protected val APPROXIMATE = Keyword("APPROXIMATE") protected val AVG = Keyword("AVG") protected val BETWEEN = Keyword("BETWEEN") protected val BY = Keyword("BY") @@ -80,9 +81,9 @@ class SqlParser extends StandardTokenParsers with PackratParsers { protected val COUNT = Keyword("COUNT") protected val DESC = Keyword("DESC") protected val DISTINCT = Keyword("DISTINCT") + protected val EXCEPT = Keyword("EXCEPT") protected val FALSE = Keyword("FALSE") protected val FIRST = Keyword("FIRST") - protected val LAST = Keyword("LAST") protected val FROM = Keyword("FROM") protected val FULL = Keyword("FULL") protected val GROUP = Keyword("GROUP") @@ -91,42 +92,42 @@ class SqlParser extends StandardTokenParsers with PackratParsers { protected val IN = Keyword("IN") protected val INNER = Keyword("INNER") protected val INSERT = Keyword("INSERT") + protected val INTERSECT = Keyword("INTERSECT") protected val INTO = Keyword("INTO") protected val IS = Keyword("IS") protected val JOIN = Keyword("JOIN") + protected val LAST = Keyword("LAST") + protected val LAZY = Keyword("LAZY") protected val LEFT = Keyword("LEFT") + protected val LIKE = Keyword("LIKE") protected val LIMIT = Keyword("LIMIT") + protected val LOWER = Keyword("LOWER") protected val MAX = Keyword("MAX") protected val MIN = Keyword("MIN") protected val NOT = Keyword("NOT") protected val NULL = Keyword("NULL") protected val ON = Keyword("ON") protected val OR = Keyword("OR") - protected val OVERWRITE = Keyword("OVERWRITE") - protected val LIKE = Keyword("LIKE") - protected val RLIKE = Keyword("RLIKE") - protected val UPPER = Keyword("UPPER") - protected val LOWER = Keyword("LOWER") - protected val REGEXP = Keyword("REGEXP") protected val ORDER = Keyword("ORDER") protected val OUTER = Keyword("OUTER") + protected val OVERWRITE = Keyword("OVERWRITE") + protected val REGEXP = Keyword("REGEXP") protected val RIGHT = Keyword("RIGHT") + protected val RLIKE = Keyword("RLIKE") protected val SELECT = Keyword("SELECT") protected val SEMI = Keyword("SEMI") + protected val SQRT = Keyword("SQRT") protected val STRING = Keyword("STRING") + protected val SUBSTR = Keyword("SUBSTR") + protected val SUBSTRING = Keyword("SUBSTRING") protected val SUM = Keyword("SUM") protected val TABLE = Keyword("TABLE") protected val TIMESTAMP = Keyword("TIMESTAMP") protected val TRUE = Keyword("TRUE") protected val UNCACHE = Keyword("UNCACHE") protected val UNION = Keyword("UNION") + protected val UPPER = Keyword("UPPER") protected val WHERE = Keyword("WHERE") - protected val INTERSECT = Keyword("INTERSECT") - protected val EXCEPT = Keyword("EXCEPT") - protected val SUBSTR = Keyword("SUBSTR") - protected val SUBSTRING = Keyword("SUBSTRING") - protected val SQRT = Keyword("SQRT") - protected val ABS = Keyword("ABS") // Use reflection to find the reserved words defined in this class. protected val reservedWords = @@ -151,7 +152,7 @@ class SqlParser extends StandardTokenParsers with PackratParsers { EXCEPT ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Except(q1, q2)} | UNION ~ opt(DISTINCT) ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Distinct(Union(q1, q2)) } ) - | insert | cache + | insert | cache | unCache ) protected lazy val select: Parser[LogicalPlan] = @@ -166,7 +167,7 @@ class SqlParser extends StandardTokenParsers with PackratParsers { val withFilter = f.map(f => Filter(f, base)).getOrElse(base) val withProjection = g.map {g => - Aggregate(assignAliases(g), assignAliases(p), withFilter) + Aggregate(g, assignAliases(p), withFilter) }.getOrElse(Project(assignAliases(p), withFilter)) val withDistinct = d.map(_ => Distinct(withProjection)).getOrElse(withProjection) val withHaving = h.map(h => Filter(h, withDistinct)).getOrElse(withDistinct) @@ -183,8 +184,14 @@ class SqlParser extends StandardTokenParsers with PackratParsers { } protected lazy val cache: Parser[LogicalPlan] = - (CACHE ^^^ true | UNCACHE ^^^ false) ~ TABLE ~ ident ^^ { - case doCache ~ _ ~ tableName => CacheCommand(tableName, doCache) + CACHE ~> opt(LAZY) ~ (TABLE ~> ident) ~ opt(AS ~> select) <~ opt(";") ^^ { + case isLazy ~ tableName ~ plan => + CacheTableCommand(tableName, plan, isLazy.isDefined) + } + + protected lazy val unCache: Parser[LogicalPlan] = + UNCACHE ~ TABLE ~> ident <~ opt(";") ^^ { + case tableName => UncacheTableCommand(tableName) } protected lazy val projections: Parser[Seq[Expression]] = repsep(projection, ",") @@ -275,7 +282,7 @@ class SqlParser extends StandardTokenParsers with PackratParsers { termExpression ~ ">=" ~ termExpression ^^ { case e1 ~ _ ~ e2 => GreaterThanOrEqual(e1, e2) } | termExpression ~ "!=" ~ termExpression ^^ { case e1 ~ _ ~ e2 => Not(EqualTo(e1, e2)) } | termExpression ~ "<>" ~ termExpression ^^ { case e1 ~ _ ~ e2 => Not(EqualTo(e1, e2)) } | - termExpression ~ BETWEEN ~ termExpression ~ AND ~ termExpression ^^ { + termExpression ~ BETWEEN ~ termExpression ~ AND ~ termExpression ^^ { case e ~ _ ~ el ~ _ ~ eu => And(GreaterThanOrEqual(e, el), LessThanOrEqual(e, eu)) } | termExpression ~ RLIKE ~ termExpression ^^ { case e1 ~ _ ~ e2 => RLike(e1, e2) } | diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 574d96d92942b..fe83eb12502dc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -37,6 +37,8 @@ object SimpleAnalyzer extends Analyzer(EmptyCatalog, EmptyFunctionRegistry, true class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Boolean) extends RuleExecutor[LogicalPlan] with HiveTypeCoercion { + val resolver = if (caseSensitive) caseSensitiveResolution else caseInsensitiveResolution + // TODO: pass this in as a parameter. val fixedPoint = FixedPoint(100) @@ -48,8 +50,6 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool lazy val batches: Seq[Batch] = Seq( Batch("MultiInstanceRelations", Once, NewRelationInstances), - Batch("CaseInsensitiveAttributeReferences", Once, - (if (caseSensitive) Nil else LowercaseAttributeReferences :: Nil) : _*), Batch("Resolution", fixedPoint, ResolveReferences :: ResolveRelations :: @@ -93,28 +93,14 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool */ object ResolveRelations extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case i @ InsertIntoTable(UnresolvedRelation(databaseName, name, alias), _, _, _) => + i.copy( + table = EliminateAnalysisOperators(catalog.lookupRelation(databaseName, name, alias))) case UnresolvedRelation(databaseName, name, alias) => catalog.lookupRelation(databaseName, name, alias) } } - /** - * Makes attribute naming case insensitive by turning all UnresolvedAttributes to lowercase. - */ - object LowercaseAttributeReferences extends Rule[LogicalPlan] { - def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case UnresolvedRelation(databaseName, name, alias) => - UnresolvedRelation(databaseName, name, alias.map(_.toLowerCase)) - case Subquery(alias, child) => Subquery(alias.toLowerCase, child) - case q: LogicalPlan => q transformExpressions { - case s: Star => s.copy(table = s.table.map(_.toLowerCase)) - case UnresolvedAttribute(name) => UnresolvedAttribute(name.toLowerCase) - case Alias(c, name) => Alias(c, name.toLowerCase)() - case GetField(c, name) => GetField(c, name.toLowerCase) - } - } - } - /** * Replaces [[UnresolvedAttribute]]s with concrete * [[catalyst.expressions.AttributeReference AttributeReferences]] from a logical plan node's @@ -127,7 +113,7 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool q transformExpressions { case u @ UnresolvedAttribute(name) => // Leave unchanged if resolution fails. Hopefully will be resolved next round. - val result = q.resolveChildren(name).getOrElse(u) + val result = q.resolveChildren(name, resolver).getOrElse(u) logDebug(s"Resolving $u to $result") result } @@ -144,7 +130,7 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { case s @ Sort(ordering, p @ Project(projectList, child)) if !s.resolved && p.resolved => val unresolved = ordering.flatMap(_.collect { case UnresolvedAttribute(name) => name }) - val resolved = unresolved.flatMap(child.resolveChildren) + val resolved = unresolved.flatMap(child.resolve(_, resolver)) val requiredAttributes = AttributeSet(resolved.collect { case a: Attribute => a }) val missingInProject = requiredAttributes -- p.output @@ -154,6 +140,7 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool Sort(ordering, Project(projectList ++ missingInProject, child))) } else { + logDebug(s"Failed to find $missingInProject in ${p.output.mkString(", ")}") s // Nothing we can do here. Return original plan. } case s @ Sort(ordering, a @ Aggregate(grouping, aggs, child)) if !s.resolved && a.resolved => @@ -165,7 +152,7 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool ) logDebug(s"Grouping expressions: $groupingRelation") - val resolved = unresolved.flatMap(groupingRelation.resolve) + val resolved = unresolved.flatMap(groupingRelation.resolve(_, resolver)) val missingInAggs = resolved.filterNot(a.outputSet.contains) logDebug(s"Resolved: $resolved Missing in aggs: $missingInAggs") if (missingInAggs.nonEmpty) { @@ -258,14 +245,14 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool case p @ Project(projectList, child) if containsStar(projectList) => Project( projectList.flatMap { - case s: Star => s.expand(child.output) + case s: Star => s.expand(child.output, resolver) case o => o :: Nil }, child) case t: ScriptTransformation if containsStar(t.input) => t.copy( input = t.input.flatMap { - case s: Star => s.expand(t.child.output) + case s: Star => s.expand(t.child.output, resolver) case o => o :: Nil } ) @@ -273,7 +260,7 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool case a: Aggregate if containsStar(a.aggregateExpressions) => a.copy( aggregateExpressions = a.aggregateExpressions.flatMap { - case s: Star => s.expand(a.child.output) + case s: Star => s.expand(a.child.output, resolver) case o => o :: Nil } ) @@ -290,13 +277,11 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool /** * Removes [[catalyst.plans.logical.Subquery Subquery]] operators from the plan. Subqueries are * only required to provide scoping information for attributes and can be removed once analysis is - * complete. Similarly, this node also removes - * [[catalyst.plans.logical.LowerCaseSchema LowerCaseSchema]] operators. + * complete. */ object EliminateAnalysisOperators extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { case Subquery(_, child) => child - case LowerCaseSchema(child) => child } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala index 616f1e2ecb60f..2059a91ba0612 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala @@ -87,7 +87,7 @@ class SimpleCatalog(val caseSensitive: Boolean) extends Catalog { tableName: String, alias: Option[String] = None): LogicalPlan = { val (dbName, tblName) = processDatabaseAndTableName(databaseName, tableName) - val table = tables.get(tblName).getOrElse(sys.error(s"Table Not Found: $tableName")) + val table = tables.getOrElse(tblName, sys.error(s"Table Not Found: $tableName")) val tableWithQualifiers = Subquery(tblName, table) // If an alias was specified by the lookup, wrap the plan in a subquery so that attributes are diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala index 9f37ca904ffeb..3f672a3e0fd91 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala @@ -22,4 +22,14 @@ package org.apache.spark.sql.catalyst * Analysis consists of translating [[UnresolvedAttribute]]s and [[UnresolvedRelation]]s * into fully typed objects using information in a schema [[Catalog]]. */ -package object analysis +package object analysis { + + /** + * Responsible for resolving which identifiers refer to the same entity. For example, by using + * case insensitive equality. + */ + type Resolver = (String, String) => Boolean + + val caseInsensitiveResolution = (a: String, b: String) => a.equalsIgnoreCase(b) + val caseSensitiveResolution = (a: String, b: String) => a == b +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index a2c61c65487cb..67570a6f73c36 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -54,6 +54,7 @@ case class UnresolvedAttribute(name: String) extends Attribute with trees.LeafNo override def newInstance = this override def withNullability(newNullability: Boolean) = this override def withQualifiers(newQualifiers: Seq[String]) = this + override def withName(newName: String) = UnresolvedAttribute(name) // Unresolved attributes are transient at compile time and don't get evaluated during execution. override def eval(input: Row = null): EvaluatedType = @@ -97,13 +98,14 @@ case class Star( override def newInstance = this override def withNullability(newNullability: Boolean) = this override def withQualifiers(newQualifiers: Seq[String]) = this + override def withName(newName: String) = this - def expand(input: Seq[Attribute]): Seq[NamedExpression] = { + def expand(input: Seq[Attribute], resolver: Resolver): Seq[NamedExpression] = { val expandedAttributes: Seq[Attribute] = table match { // If there is no table specified, use all input attributes. case None => input // If there is a table, pick out attributes that are part of this table. - case Some(t) => input.filter(_.qualifiers contains t) + case Some(t) => input.filter(_.qualifiers.filter(resolver(_, t)).nonEmpty) } val mappedAttributes = expandedAttributes.map(mapFunction).zip(input).map { case (n: NamedExpression, _) => n 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 0ad2b30cf9c1f..f626d09f037bc 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 @@ -86,15 +86,15 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { try Timestamp.valueOf(n) catch { case _: java.lang.IllegalArgumentException => null } }) case BooleanType => - buildCast[Boolean](_, b => new Timestamp((if (b) 1 else 0) * 1000)) + buildCast[Boolean](_, b => new Timestamp((if (b) 1 else 0))) case LongType => - buildCast[Long](_, l => new Timestamp(l * 1000)) + buildCast[Long](_, l => new Timestamp(l)) case IntegerType => - buildCast[Int](_, i => new Timestamp(i * 1000)) + buildCast[Int](_, i => new Timestamp(i)) case ShortType => - buildCast[Short](_, s => new Timestamp(s * 1000)) + buildCast[Short](_, s => new Timestamp(s)) case ByteType => - buildCast[Byte](_, b => new Timestamp(b * 1000)) + buildCast[Byte](_, b => new Timestamp(b)) // TimestampWritable.decimalToTimestamp case DecimalType => buildCast[BigDecimal](_, d => decimalToTimestamp(d)) @@ -107,11 +107,10 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { } private[this] def decimalToTimestamp(d: BigDecimal) = { - val seconds = d.longValue() + val seconds = Math.floor(d.toDouble).toLong val bd = (d - seconds) * 1000000000 val nanos = bd.intValue() - // Convert to millis val millis = seconds * 1000 val t = new Timestamp(millis) @@ -121,11 +120,11 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { } // Timestamp to long, converting milliseconds to seconds - private[this] def timestampToLong(ts: Timestamp) = ts.getTime / 1000 + private[this] def timestampToLong(ts: Timestamp) = Math.floor(ts.getTime / 1000.0).toLong private[this] def timestampToDouble(ts: Timestamp) = { // First part is the seconds since the beginning of time, followed by nanosecs. - ts.getTime / 1000 + ts.getNanos.toDouble / 1000000000 + Math.floor(ts.getTime / 1000.0).toLong + ts.getNanos.toDouble / 1000000000 } // Converts Timestamp to string according to Hive TimestampWritable convention @@ -245,6 +244,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { } private[this] lazy val cast: Any => Any = dataType match { + case dt if dt == child.dataType => identity[Any] 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 6371582ddc6c9..5e3ea7dee1262 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 @@ -182,6 +182,9 @@ abstract class Expression extends TreeNode[Expression] { case i: IntegralType => f.asInstanceOf[(Integral[i.JvmType], i.JvmType, i.JvmType) => i.JvmType]( i.integral, evalE1.asInstanceOf[i.JvmType], evalE2.asInstanceOf[i.JvmType]) + case i: FractionalType => + f.asInstanceOf[(Integral[i.JvmType], i.JvmType, i.JvmType) => i.JvmType]( + i.asIntegral, evalE1.asInstanceOf[i.JvmType], evalE2.asInstanceOf[i.JvmType]) case other => sys.error(s"Type $other does not support numeric operations") } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index 3167e4ba49a87..d04071375bb21 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -59,9 +59,10 @@ abstract class Attribute extends NamedExpression { def withNullability(newNullability: Boolean): Attribute def withQualifiers(newQualifiers: Seq[String]): Attribute + def withName(newName: String): Attribute def toAttribute = this - def newInstance: Attribute + def newInstance(): Attribute } @@ -136,7 +137,7 @@ case class AttributeReference( h } - override def newInstance = + override def newInstance() = AttributeReference(name, dataType, nullable, metadata)(qualifiers = qualifiers) /** @@ -150,6 +151,14 @@ case class AttributeReference( } } + override def withName(newName: String): AttributeReference = { + if (name == newName) { + this + } else { + AttributeReference(newName, dataType, nullable)(exprId, qualifiers) + } + } + /** * Returns a copy of this [[AttributeReference]] with new qualifiers. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index ede431ad4ab27..4f8ad8a7e0223 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -17,13 +17,16 @@ package org.apache.spark.sql.catalyst.plans.logical +import org.apache.spark.Logging +import org.apache.spark.sql.catalyst.analysis.Resolver import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.QueryPlan +import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.catalyst.types.StructType import org.apache.spark.sql.catalyst.trees -abstract class LogicalPlan extends QueryPlan[LogicalPlan] { +abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { self: Product => /** @@ -70,25 +73,71 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] { */ def childrenResolved: Boolean = !children.exists(!_.resolved) + /** + * Returns true when the given logical plan will return the same results as this logical plan. + * + * Since its likely undecideable to generally determine if two given plans will produce the same + * results, it is okay for this function to return false, even if the results are actually + * the same. Such behavior will not affect correctness, only the application of performance + * enhancements like caching. However, it is not acceptable to return true if the results could + * possibly be different. + * + * By default this function performs a modified version of equality that is tolerant of cosmetic + * differences like attribute naming and or expression id differences. Logical operators that + * can do better should override this function. + */ + def sameResult(plan: LogicalPlan): Boolean = { + plan.getClass == this.getClass && + plan.children.size == children.size && { + logDebug(s"[${cleanArgs.mkString(", ")}] == [${plan.cleanArgs.mkString(", ")}]") + cleanArgs == plan.cleanArgs + } && + (plan.children, children).zipped.forall(_ sameResult _) + } + + /** Args that have cleaned such that differences in expression id should not affect equality */ + protected lazy val cleanArgs: Seq[Any] = { + val input = children.flatMap(_.output) + productIterator.map { + // Children are checked using sameResult above. + case tn: TreeNode[_] if children contains tn => null + case e: Expression => BindReferences.bindReference(e, input, allowFailures = true) + case s: Option[_] => s.map { + case e: Expression => BindReferences.bindReference(e, input, allowFailures = true) + case other => other + } + case s: Seq[_] => s.map { + case e: Expression => BindReferences.bindReference(e, input, allowFailures = true) + case other => other + } + case other => other + }.toSeq + } + /** * Optionally resolves the given string to a [[NamedExpression]] using the input from all child * nodes of this LogicalPlan. The attribute is expressed as * as string in the following form: `[scope].AttributeName.[nested].[fields]...`. */ - def resolveChildren(name: String): Option[NamedExpression] = - resolve(name, children.flatMap(_.output)) + def resolveChildren(name: String, resolver: Resolver): Option[NamedExpression] = + resolve(name, children.flatMap(_.output), resolver) /** * Optionally resolves the given string to a [[NamedExpression]] based on the output of this * LogicalPlan. The attribute is expressed as string in the following form: * `[scope].AttributeName.[nested].[fields]...`. */ - def resolve(name: String): Option[NamedExpression] = - resolve(name, output) + def resolve(name: String, resolver: Resolver): Option[NamedExpression] = + resolve(name, output, resolver) /** Performs attribute resolution given a name and a sequence of possible attributes. */ - protected def resolve(name: String, input: Seq[Attribute]): Option[NamedExpression] = { + protected def resolve( + name: String, + input: Seq[Attribute], + resolver: Resolver): Option[NamedExpression] = { + val parts = name.split("\\.") + // Collect all attributes that are output by this nodes children where either the first part // matches the name or where the first part matches the scope and the second part matches the // name. Return these matches along with any remaining parts, which represent dotted access to @@ -96,21 +145,69 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] { val options = input.flatMap { option => // If the first part of the desired name matches a qualifier for this possible match, drop it. val remainingParts = - if (option.qualifiers.contains(parts.head) && parts.size > 1) parts.drop(1) else parts - if (option.name == remainingParts.head) (option, remainingParts.tail.toList) :: Nil else Nil + if (option.qualifiers.find(resolver(_, parts.head)).nonEmpty && parts.size > 1) { + parts.drop(1) + } else { + parts + } + + if (resolver(option.name, remainingParts.head)) { + // Preserve the case of the user's attribute reference. + (option.withName(remainingParts.head), remainingParts.tail.toList) :: Nil + } else { + Nil + } } options.distinct match { - case Seq((a, Nil)) => Some(a) // One match, no nested fields, use it. + // One match, no nested fields, use it. + case Seq((a, Nil)) => Some(a) + // One match, but we also need to extract the requested nested field. case Seq((a, nestedFields)) => - Some(Alias(nestedFields.foldLeft(a: Expression)(GetField), nestedFields.last)()) - case Seq() => None // No matches. + val aliased = + Alias( + resolveNesting(nestedFields, a, resolver), + nestedFields.last)() // Preserve the case of the user's field access. + Some(aliased) + + // No matches. + case Seq() => + logTrace(s"Could not find $name in ${input.mkString(", ")}") + None + + // More than one match. case ambiguousReferences => throw new TreeNodeException( this, s"Ambiguous references to $name: ${ambiguousReferences.mkString(",")}") } } + + /** + * Given a list of successive nested field accesses, and a based expression, attempt to resolve + * the actual field lookups on this expression. + */ + private def resolveNesting( + nestedFields: List[String], + expression: Expression, + resolver: Resolver): Expression = { + + (nestedFields, expression.dataType) match { + case (Nil, _) => expression + case (requestedField :: rest, StructType(fields)) => + val actualField = fields.filter(f => resolver(f.name, requestedField)) + actualField match { + case Seq() => + sys.error( + s"No such struct field $requestedField in ${fields.map(_.name).mkString(", ")}") + case Seq(singleMatch) => + resolveNesting(rest, GetField(expression, singleMatch.name), resolver) + case multipleMatches => + sys.error(s"Ambiguous reference to fields ${multipleMatches.mkString(", ")}") + } + case (_, dt) => sys.error(s"Can't access nested field in type $dt") + } + } } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TestRelation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TestRelation.scala index f8fe558511bfd..19769986ef58c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TestRelation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TestRelation.scala @@ -41,4 +41,10 @@ case class LocalRelation(output: Seq[Attribute], data: Seq[Product] = Nil) } override protected def stringArgs = Iterator(output) + + override def sameResult(plan: LogicalPlan): Boolean = plan match { + case LocalRelation(otherOutput, otherData) => + otherOutput.map(_.dataType) == output.map(_.dataType) && otherData == data + case _ => false + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index c10e751ee3917..f8e9930ac270d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -105,8 +105,8 @@ case class InsertIntoTable( child: LogicalPlan, overwrite: Boolean) extends LogicalPlan { - // The table being inserted into is a child for the purposes of transformations. - override def children = table :: child :: Nil + + override def children = child :: Nil override def output = child.output override lazy val resolved = childrenResolved && child.output.zip(table.output).forall { @@ -148,39 +148,23 @@ case class Aggregate( case class Limit(limitExpr: Expression, child: LogicalPlan) extends UnaryNode { override def output = child.output + + override lazy val statistics: Statistics = + if (output.forall(_.dataType.isInstanceOf[NativeType])) { + val limit = limitExpr.eval(null).asInstanceOf[Int] + val sizeInBytes = (limit: Long) * output.map { a => + NativeType.defaultSizeOf(a.dataType.asInstanceOf[NativeType]) + }.sum + Statistics(sizeInBytes = sizeInBytes) + } else { + Statistics(sizeInBytes = children.map(_.statistics).map(_.sizeInBytes).product) + } } case class Subquery(alias: String, child: LogicalPlan) extends UnaryNode { override def output = child.output.map(_.withQualifiers(alias :: Nil)) } -/** - * Converts the schema of `child` to all lowercase, together with LowercaseAttributeReferences - * this allows for optional case insensitive attribute resolution. This node can be elided after - * analysis. - */ -case class LowerCaseSchema(child: LogicalPlan) extends UnaryNode { - protected def lowerCaseSchema(dataType: DataType): DataType = dataType match { - case StructType(fields) => - StructType(fields.map(f => - StructField(f.name.toLowerCase, lowerCaseSchema(f.dataType), f.nullable, f.metadata))) - case ArrayType(elemType, containsNull) => ArrayType(lowerCaseSchema(elemType), containsNull) - case otherType => otherType - } - - override val output = child.output.map { - case a: AttributeReference => - AttributeReference( - a.name.toLowerCase, - lowerCaseSchema(a.dataType), - a.nullable, - a.metadata)( - a.exprId, - a.qualifiers) - case other => other - } -} - case class Sample(fraction: Double, withReplacement: Boolean, seed: Long, child: LogicalPlan) extends UnaryNode { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala index a01809c1fc5e2..9a3848cfc6b62 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala @@ -56,9 +56,15 @@ case class ExplainCommand(plan: LogicalPlan, extended: Boolean = false) extends } /** - * Returned for the "CACHE TABLE tableName" and "UNCACHE TABLE tableName" command. + * Returned for the "CACHE TABLE tableName [AS SELECT ...]" command. */ -case class CacheCommand(tableName: String, doCache: Boolean) extends Command +case class CacheTableCommand(tableName: String, plan: Option[LogicalPlan], isLazy: Boolean) + extends Command + +/** + * Returned for the "UNCACHE TABLE tableName" command. + */ +case class UncacheTableCommand(tableName: String) extends Command /** * Returned for the "DESCRIBE [EXTENDED] [dbName.]tableName" command. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala index 05ac9d23b0284..5d9deab4da184 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.types import java.sql.Timestamp +import scala.math.Numeric.{FloatAsIfIntegral, BigDecimalAsIfIntegral, DoubleAsIfIntegral} import scala.reflect.ClassTag import scala.reflect.runtime.universe.{typeTag, TypeTag, runtimeMirror} import scala.util.parsing.combinator.RegexParsers @@ -123,6 +124,16 @@ object NativeType { IntegerType, BooleanType, LongType, DoubleType, FloatType, ShortType, ByteType, StringType) def unapply(dt: DataType): Boolean = all.contains(dt) + + val defaultSizeOf: Map[NativeType, Int] = Map( + IntegerType -> 4, + BooleanType -> 1, + LongType -> 8, + DoubleType -> 8, + FloatType -> 4, + ShortType -> 2, + ByteType -> 1, + StringType -> 4096) } trait PrimitiveType extends DataType { @@ -147,8 +158,18 @@ case object StringType extends NativeType with PrimitiveType { def simpleString: String = "string" } -case object BinaryType extends DataType with PrimitiveType { +case object BinaryType extends NativeType with PrimitiveType { private[sql] type JvmType = Array[Byte] + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } + private[sql] val ordering = new Ordering[JvmType] { + def compare(x: Array[Byte], y: Array[Byte]): Int = { + for (i <- 0 until x.length; if i < y.length) { + val res = x(i).compareTo(y(i)) + if (res != 0) return res + } + return x.length - y.length + } + } def simpleString: String = "binary" } @@ -241,6 +262,7 @@ object FractionalType { } abstract class FractionalType extends NumericType { private[sql] val fractional: Fractional[JvmType] + private[sql] val asIntegral: Integral[JvmType] } case object DecimalType extends FractionalType { @@ -249,6 +271,7 @@ case object DecimalType extends FractionalType { private[sql] val numeric = implicitly[Numeric[BigDecimal]] private[sql] val fractional = implicitly[Fractional[BigDecimal]] private[sql] val ordering = implicitly[Ordering[JvmType]] + private[sql] val asIntegral = BigDecimalAsIfIntegral def simpleString: String = "decimal" } @@ -258,6 +281,7 @@ case object DoubleType extends FractionalType { private[sql] val numeric = implicitly[Numeric[Double]] private[sql] val fractional = implicitly[Fractional[Double]] private[sql] val ordering = implicitly[Ordering[JvmType]] + private[sql] val asIntegral = DoubleAsIfIntegral def simpleString: String = "double" } @@ -267,6 +291,7 @@ case object FloatType extends FractionalType { private[sql] val numeric = implicitly[Numeric[Float]] private[sql] val fractional = implicitly[Fractional[Float]] private[sql] val ordering = implicitly[Ordering[JvmType]] + private[sql] val asIntegral = FloatAsIfIntegral def simpleString: String = "float" } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala index 428607d8c8253..488e373854bb3 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala @@ -53,7 +53,8 @@ case class OptionalData( floatField: Option[Float], shortField: Option[Short], byteField: Option[Byte], - booleanField: Option[Boolean]) + booleanField: Option[Boolean], + structField: Option[PrimitiveData]) case class ComplexData( arrayField: Seq[Int], @@ -100,7 +101,7 @@ class ScalaReflectionSuite extends FunSuite { nullable = true)) } - test("optinal data") { + test("optional data") { val schema = schemaFor[OptionalData] assert(schema === Schema( StructType(Seq( @@ -110,7 +111,8 @@ class ScalaReflectionSuite extends FunSuite { StructField("floatField", FloatType, nullable = true), StructField("shortField", ShortType, nullable = true), StructField("byteField", ByteType, nullable = true), - StructField("booleanField", BooleanType, nullable = true))), + StructField("booleanField", BooleanType, nullable = true), + StructField("structField", schemaFor[PrimitiveData].dataType, nullable = true))), nullable = true)) } @@ -228,4 +230,17 @@ class ScalaReflectionSuite extends FunSuite { assert(ArrayType(IntegerType) === typeOfObject3(Seq(1, 2, 3))) assert(ArrayType(ArrayType(IntegerType)) === typeOfObject3(Seq(Seq(1,2,3)))) } + + test("convert PrimitiveData to catalyst") { + val data = PrimitiveData(1, 1, 1, 1, 1, 1, true) + val convertedData = Seq(1, 1.toLong, 1.toDouble, 1.toFloat, 1.toShort, 1.toByte, true) + assert(convertToCatalyst(data) === convertedData) + } + + test("convert Option[Product] to catalyst") { + val primitiveData = PrimitiveData(1, 1, 1, 1, 1, 1, true) + val data = OptionalData(Some(1), Some(1), Some(1), Some(1), Some(1), Some(1), Some(true), Some(primitiveData)) + val convertedData = Seq(1, 1.toLong, 1.toDouble, 1.toFloat, 1.toShort, 1.toByte, true, convertToCatalyst(primitiveData)) + assert(convertToCatalyst(data) === convertedData) + } } 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 b961346dfc995..63931af4bac3d 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 @@ -20,6 +20,8 @@ package org.apache.spark.sql.catalyst.expressions import java.sql.Timestamp import org.scalatest.FunSuite +import org.scalatest.Matchers._ +import org.scalautils.TripleEqualsSupport.Spread import org.apache.spark.sql.catalyst.types._ @@ -129,6 +131,13 @@ class ExpressionEvaluationSuite extends FunSuite { } } + def checkDoubleEvaluation(expression: Expression, expected: Spread[Double], inputRow: Row = EmptyRow): Unit = { + val actual = try evaluate(expression, inputRow) catch { + case e: Exception => fail(s"Exception evaluating $expression", e) + } + actual.asInstanceOf[Double] shouldBe expected + } + test("IN") { checkEvaluation(In(Literal(1), Seq(Literal(1), Literal(2))), true) checkEvaluation(In(Literal(2), Seq(Literal(1), Literal(2))), true) @@ -231,7 +240,9 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation("12.65" cast DecimalType, BigDecimal(12.65)) checkEvaluation(Literal(1) cast LongType, 1) - checkEvaluation(Cast(Literal(1) cast TimestampType, LongType), 1) + checkEvaluation(Cast(Literal(1000) cast TimestampType, LongType), 1.toLong) + checkEvaluation(Cast(Literal(-1200) cast TimestampType, LongType), -2.toLong) + checkEvaluation(Cast(Literal(1.toDouble) cast TimestampType, DoubleType), 1.toDouble) checkEvaluation(Cast(Literal(1.toDouble) cast TimestampType, DoubleType), 1.toDouble) checkEvaluation(Cast(Literal(sts) cast TimestampType, StringType), sts) @@ -242,11 +253,11 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(Cast(Cast(Cast(Cast( Cast("5" cast ByteType, ShortType), IntegerType), FloatType), DoubleType), LongType), 5) checkEvaluation(Cast(Cast(Cast(Cast( - Cast("5" cast ByteType, TimestampType), DecimalType), LongType), StringType), ShortType), 5) + Cast("5" cast ByteType, TimestampType), DecimalType), LongType), StringType), ShortType), 0) checkEvaluation(Cast(Cast(Cast(Cast( Cast("5" cast TimestampType, ByteType), DecimalType), LongType), StringType), ShortType), null) checkEvaluation(Cast(Cast(Cast(Cast( - Cast("5" cast DecimalType, ByteType), TimestampType), LongType), StringType), ShortType), 5) + Cast("5" cast DecimalType, ByteType), TimestampType), LongType), StringType), ShortType), 0) checkEvaluation(Literal(true) cast IntegerType, 1) checkEvaluation(Literal(false) cast IntegerType, 0) checkEvaluation(Cast(Literal(1) cast BooleanType, IntegerType), 1) @@ -293,16 +304,18 @@ class ExpressionEvaluationSuite extends FunSuite { test("timestamp casting") { val millis = 15 * 1000 + 2 + val seconds = millis * 1000 + 2 val ts = new Timestamp(millis) val ts1 = new Timestamp(15 * 1000) // a timestamp without the milliseconds part + val tss = new Timestamp(seconds) checkEvaluation(Cast(ts, ShortType), 15) checkEvaluation(Cast(ts, IntegerType), 15) checkEvaluation(Cast(ts, LongType), 15) checkEvaluation(Cast(ts, FloatType), 15.002f) checkEvaluation(Cast(ts, DoubleType), 15.002) - checkEvaluation(Cast(Cast(ts, ShortType), TimestampType), ts1) - checkEvaluation(Cast(Cast(ts, IntegerType), TimestampType), ts1) - checkEvaluation(Cast(Cast(ts, LongType), TimestampType), ts1) + checkEvaluation(Cast(Cast(tss, ShortType), TimestampType), ts) + checkEvaluation(Cast(Cast(tss, IntegerType), TimestampType), ts) + checkEvaluation(Cast(Cast(tss, LongType), TimestampType), ts) checkEvaluation(Cast(Cast(millis.toFloat / 1000, TimestampType), FloatType), millis.toFloat / 1000) checkEvaluation(Cast(Cast(millis.toDouble / 1000, TimestampType), DoubleType), @@ -467,6 +480,29 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(c1 % c2, 1, row) } + test("fractional arithmetic") { + val row = new GenericRow(Array[Any](1.1, 2.0, 3.1, null)) + val c1 = 'a.double.at(0) + val c2 = 'a.double.at(1) + val c3 = 'a.double.at(2) + val c4 = 'a.double.at(3) + + checkEvaluation(UnaryMinus(c1), -1.1, row) + checkEvaluation(UnaryMinus(Literal(100.0, DoubleType)), -100.0) + checkEvaluation(Add(c1, c4), null, row) + checkEvaluation(Add(c1, c2), 3.1, row) + checkEvaluation(Add(c1, Literal(null, DoubleType)), null, row) + checkEvaluation(Add(Literal(null, DoubleType), c2), null, row) + checkEvaluation(Add(Literal(null, DoubleType), Literal(null, DoubleType)), null, row) + + checkEvaluation(-c1, -1.1, row) + checkEvaluation(c1 + c2, 3.1, row) + checkDoubleEvaluation(c1 - c2, (-0.9 +- 0.001), row) + checkDoubleEvaluation(c1 * c2, (2.2 +- 0.001), row) + checkDoubleEvaluation(c1 / c2, (0.55 +- 0.001), row) + checkDoubleEvaluation(c3 % c2, (1.1 +- 0.001), row) + } + test("BinaryComparison") { val row = new GenericRow(Array[Any](1, 2, 3, null, 3, null)) val c1 = 'a.int.at(0) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/SameResultSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/SameResultSuite.scala new file mode 100644 index 0000000000000..e8a793d107451 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/SameResultSuite.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.sql.catalyst.plans + +import org.scalatest.FunSuite + +import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.expressions.{ExprId, AttributeReference} +import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} +import org.apache.spark.sql.catalyst.util._ + +/** + * Provides helper methods for comparing plans. + */ +class SameResultSuite extends FunSuite { + val testRelation = LocalRelation('a.int, 'b.int, 'c.int) + val testRelation2 = LocalRelation('a.int, 'b.int, 'c.int) + + def assertSameResult(a: LogicalPlan, b: LogicalPlan, result: Boolean = true) = { + val aAnalyzed = a.analyze + val bAnalyzed = b.analyze + + if (aAnalyzed.sameResult(bAnalyzed) != result) { + val comparison = sideBySide(aAnalyzed.toString, bAnalyzed.toString).mkString("\n") + fail(s"Plans should return sameResult = $result\n$comparison") + } + } + + test("relations") { + assertSameResult(testRelation, testRelation2) + } + + test("projections") { + assertSameResult(testRelation.select('a), testRelation2.select('a)) + assertSameResult(testRelation.select('b), testRelation2.select('b)) + assertSameResult(testRelation.select('a, 'b), testRelation2.select('a, 'b)) + assertSameResult(testRelation.select('b, 'a), testRelation2.select('b, 'a)) + + assertSameResult(testRelation, testRelation2.select('a), false) + assertSameResult(testRelation.select('b, 'a), testRelation2.select('a, 'b), false) + } + + test("filters") { + assertSameResult(testRelation.where('a === 'b), testRelation2.where('a === 'b)) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/CacheManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/CacheManager.scala new file mode 100644 index 0000000000000..3bf7382ac67a6 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/CacheManager.scala @@ -0,0 +1,134 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import java.util.concurrent.locks.ReentrantReadWriteLock + +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.columnar.InMemoryRelation +import org.apache.spark.storage.StorageLevel +import org.apache.spark.storage.StorageLevel.MEMORY_ONLY + +/** Holds a cached logical plan and its data */ +private case class CachedData(plan: LogicalPlan, cachedRepresentation: InMemoryRelation) + +/** + * Provides support in a SQLContext for caching query results and automatically using these cached + * results when subsequent queries are executed. Data is cached using byte buffers stored in an + * InMemoryRelation. This relation is automatically substituted query plans that return the + * `sameResult` as the originally cached query. + */ +private[sql] trait CacheManager { + self: SQLContext => + + @transient + private val cachedData = new scala.collection.mutable.ArrayBuffer[CachedData] + + @transient + private val cacheLock = new ReentrantReadWriteLock + + /** Returns true if the table is currently cached in-memory. */ + def isCached(tableName: String): Boolean = lookupCachedData(table(tableName)).nonEmpty + + /** Caches the specified table in-memory. */ + def cacheTable(tableName: String): Unit = cacheQuery(table(tableName)) + + /** Removes the specified table from the in-memory cache. */ + def uncacheTable(tableName: String): Unit = uncacheQuery(table(tableName)) + + /** Acquires a read lock on the cache for the duration of `f`. */ + private def readLock[A](f: => A): A = { + val lock = cacheLock.readLock() + lock.lock() + try f finally { + lock.unlock() + } + } + + /** Acquires a write lock on the cache for the duration of `f`. */ + private def writeLock[A](f: => A): A = { + val lock = cacheLock.writeLock() + lock.lock() + try f finally { + lock.unlock() + } + } + + private[sql] def clearCache(): Unit = writeLock { + cachedData.foreach(_.cachedRepresentation.cachedColumnBuffers.unpersist()) + cachedData.clear() + } + + /** Caches the data produced by the logical representation of the given schema rdd. */ + private[sql] def cacheQuery( + query: SchemaRDD, + storageLevel: StorageLevel = MEMORY_ONLY): Unit = writeLock { + val planToCache = query.queryExecution.optimizedPlan + if (lookupCachedData(planToCache).nonEmpty) { + logWarning("Asked to cache already cached data.") + } else { + cachedData += + CachedData( + planToCache, + InMemoryRelation( + useCompression, columnBatchSize, storageLevel, query.queryExecution.executedPlan)) + } + } + + /** Removes the data for the given SchemaRDD from the cache */ + private[sql] def uncacheQuery(query: SchemaRDD, blocking: Boolean = true): Unit = writeLock { + val planToCache = query.queryExecution.optimizedPlan + val dataIndex = cachedData.indexWhere(_.plan.sameResult(planToCache)) + require(dataIndex >= 0, s"Table $query is not cached.") + cachedData(dataIndex).cachedRepresentation.cachedColumnBuffers.unpersist(blocking) + cachedData.remove(dataIndex) + } + + + /** Optionally returns cached data for the given SchemaRDD */ + private[sql] def lookupCachedData(query: SchemaRDD): Option[CachedData] = readLock { + lookupCachedData(query.queryExecution.optimizedPlan) + } + + /** Optionally returns cached data for the given LogicalPlan. */ + private[sql] def lookupCachedData(plan: LogicalPlan): Option[CachedData] = readLock { + cachedData.find(_.plan.sameResult(plan)) + } + + /** Replaces segments of the given logical plan with cached versions where possible. */ + private[sql] def useCachedData(plan: LogicalPlan): LogicalPlan = { + plan transformDown { + case currentFragment => + lookupCachedData(currentFragment) + .map(_.cachedRepresentation.withOutput(currentFragment.output)) + .getOrElse(currentFragment) + } + } + + /** + * Invalidates the cache of any data that contains `plan`. Note that it is possible that this + * function will over invalidate. + */ + private[sql] def invalidateCache(plan: LogicalPlan): Unit = writeLock { + cachedData.foreach { + case data if data.plan.collect { case p if p.sameResult(plan) => p }.nonEmpty => + data.cachedRepresentation.recache() + case _ => + } + } +} 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 7dbaf7faff0c0..7a55c5bf97a71 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 @@ -50,6 +50,7 @@ import org.apache.spark.{Logging, SparkContext} class SQLContext(@transient val sparkContext: SparkContext) extends org.apache.spark.Logging with SQLConf + with CacheManager with ExpressionConversions with UDFRegistration with Serializable { @@ -75,6 +76,11 @@ class SQLContext(@transient val sparkContext: SparkContext) protected[sql] def executePlan(plan: LogicalPlan): this.QueryExecution = new this.QueryExecution { val logical = plan } + sparkContext.getConf.getAll.foreach { + case (key, value) if key.startsWith("spark.sql") => setConf(key, value) + case _ => + } + /** * :: DeveloperApi :: * Allows catalyst LogicalPlans to be executed as a SchemaRDD. Note that the LogicalPlan @@ -91,7 +97,8 @@ class SQLContext(@transient val sparkContext: SparkContext) */ implicit def createSchemaRDD[A <: Product: TypeTag](rdd: RDD[A]) = { SparkPlan.currentContext.set(self) - new SchemaRDD(this, SparkLogicalPlan(ExistingRdd.fromProductRdd(rdd))(self)) + new SchemaRDD(this, + LogicalRDD(ScalaReflection.attributesFor[A], RDDConversions.productToRowRdd(rdd))(self)) } /** @@ -128,7 +135,7 @@ class SQLContext(@transient val sparkContext: SparkContext) def applySchema(rowRDD: RDD[Row], schema: StructType): SchemaRDD = { // TODO: use MutableProjection when rowRDD is another SchemaRDD and the applied // schema differs from the existing schema on any field data type. - val logicalPlan = SparkLogicalPlan(ExistingRdd(schema.toAttributes, rowRDD))(self) + val logicalPlan = LogicalRDD(schema.toAttributes, rowRDD)(self) new SchemaRDD(this, logicalPlan) } @@ -246,7 +253,7 @@ class SQLContext(@transient val sparkContext: SparkContext) * @group userf */ def registerRDDAsTable(rdd: SchemaRDD, tableName: String): Unit = { - catalog.registerTable(None, tableName, rdd.queryExecution.analyzed) + catalog.registerTable(None, tableName, rdd.queryExecution.logical) } /** @@ -267,45 +274,6 @@ class SQLContext(@transient val sparkContext: SparkContext) def table(tableName: String): SchemaRDD = new SchemaRDD(this, catalog.lookupRelation(None, tableName)) - /** Caches the specified table in-memory. */ - def cacheTable(tableName: String): Unit = { - val currentTable = table(tableName).queryExecution.analyzed - val asInMemoryRelation = currentTable match { - case _: InMemoryRelation => - currentTable - - case _ => - InMemoryRelation(useCompression, columnBatchSize, executePlan(currentTable).executedPlan) - } - - catalog.registerTable(None, tableName, asInMemoryRelation) - } - - /** Removes the specified table from the in-memory cache. */ - def uncacheTable(tableName: String): Unit = { - table(tableName).queryExecution.analyzed match { - // This is kind of a hack to make sure that if this was just an RDD registered as a table, - // we reregister the RDD as a table. - case inMem @ InMemoryRelation(_, _, _, e: ExistingRdd) => - inMem.cachedColumnBuffers.unpersist() - catalog.unregisterTable(None, tableName) - catalog.registerTable(None, tableName, SparkLogicalPlan(e)(self)) - case inMem: InMemoryRelation => - inMem.cachedColumnBuffers.unpersist() - catalog.unregisterTable(None, tableName) - case plan => throw new IllegalArgumentException(s"Table $tableName is not cached: $plan") - } - } - - /** Returns true if the table is currently cached in-memory. */ - def isCached(tableName: String): Boolean = { - val relation = table(tableName).queryExecution.analyzed - relation match { - case _: InMemoryRelation => true - case _ => false - } - } - protected[sql] class SparkPlanner extends SparkStrategies { val sparkContext: SparkContext = self.sparkContext @@ -396,10 +364,12 @@ class SQLContext(@transient val sparkContext: SparkContext) lazy val analyzed = ExtractPythonUdfs(analyzer(logical)) lazy val optimizedPlan = optimizer(analyzed) + lazy val withCachedData = useCachedData(optimizedPlan) + // TODO: Don't just pick the first one... lazy val sparkPlan = { SparkPlan.currentContext.set(self) - planner(optimizedPlan).next() + planner(withCachedData).next() } // executedPlan should not be used to initialize any SparkPlan. It should be // only used for execution. @@ -521,6 +491,6 @@ class SQLContext(@transient val sparkContext: SparkContext) iter.map { m => new GenericRow(m): Row} } - new SchemaRDD(this, SparkLogicalPlan(ExistingRdd(schema.toAttributes, rowRdd))(self)) + new SchemaRDD(this, LogicalRDD(schema.toAttributes, rowRdd)(self)) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index 3bc5dce095511..594bf8ffc20e1 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,6 +19,8 @@ package org.apache.spark.sql import java.util.{Map => JMap, List => JList} +import org.apache.spark.storage.StorageLevel + import scala.collection.JavaConversions._ import scala.collection.JavaConverters._ @@ -32,7 +34,7 @@ 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.execution.{ExistingRdd, SparkLogicalPlan} +import org.apache.spark.sql.execution.LogicalRDD import org.apache.spark.api.java.JavaRDD /** @@ -45,9 +47,8 @@ import org.apache.spark.api.java.JavaRDD * explicitly using the `createSchemaRDD` function on a [[SQLContext]]. * * A `SchemaRDD` can also be created by loading data in from external sources. - * Examples are loading data from Parquet files by using by using the - * `parquetFile` method on [[SQLContext]], and loading JSON datasets - * by using `jsonFile` and `jsonRDD` methods on [[SQLContext]]. + * Examples are loading data from Parquet files by using the `parquetFile` method on [[SQLContext]] + * and loading JSON datasets by using `jsonFile` and `jsonRDD` methods on [[SQLContext]]. * * == SQL Queries == * A SchemaRDD can be registered as a table in the [[SQLContext]] that was used to create it. Once @@ -443,8 +444,7 @@ class SchemaRDD( */ private def applySchema(rdd: RDD[Row]): SchemaRDD = { new SchemaRDD(sqlContext, - SparkLogicalPlan( - ExistingRdd(queryExecution.analyzed.output.map(_.newInstance), rdd))(sqlContext)) + LogicalRDD(queryExecution.analyzed.output.map(_.newInstance()), rdd)(sqlContext)) } // ======================================================================= @@ -498,4 +498,20 @@ class SchemaRDD( override def subtract(other: RDD[Row], p: Partitioner) (implicit ord: Ordering[Row] = null): SchemaRDD = applySchema(super.subtract(other, p)(ord)) + + /** Overridden cache function will always use the in-memory columnar caching. */ + override def cache(): this.type = { + sqlContext.cacheQuery(this) + this + } + + override def persist(newLevel: StorageLevel): this.type = { + sqlContext.cacheQuery(this, newLevel) + this + } + + override def unpersist(blocking: Boolean): this.type = { + sqlContext.uncacheQuery(this, blocking) + this + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala index e52eeb3e1c47e..25ba7d88ba538 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.execution.SparkLogicalPlan +import org.apache.spark.sql.execution.LogicalRDD /** * Contains functions that are shared between all SchemaRDD types (i.e., Scala, Java) @@ -55,8 +55,7 @@ private[sql] trait SchemaRDDLike { // For various commands (like DDL) and queries with side effects, we force query optimization to // happen right away to let these side effects take place eagerly. case _: Command | _: InsertIntoTable | _: CreateTableAsSelect |_: WriteToFile => - queryExecution.toRdd - SparkLogicalPlan(queryExecution.executedPlan)(sqlContext) + LogicalRDD(queryExecution.analyzed.output, queryExecution.toRdd)(sqlContext) case _ => baseLogicalPlan } 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 150ff8a42063d..c006c4330ff66 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 @@ -27,7 +27,7 @@ import org.apache.spark.sql.json.JsonRDD import org.apache.spark.sql.{SQLContext, StructType => SStructType} import org.apache.spark.sql.catalyst.expressions.{AttributeReference, GenericRow, Row => ScalaRow} import org.apache.spark.sql.parquet.ParquetRelation -import org.apache.spark.sql.execution.{ExistingRdd, SparkLogicalPlan} +import org.apache.spark.sql.execution.LogicalRDD import org.apache.spark.sql.types.util.DataTypeConversions.asScalaDataType import org.apache.spark.util.Utils @@ -100,7 +100,7 @@ class JavaSQLContext(val sqlContext: SQLContext) extends UDFRegistration { new GenericRow(extractors.map(e => e.invoke(row)).toArray[Any]): ScalaRow } } - new JavaSchemaRDD(sqlContext, SparkLogicalPlan(ExistingRdd(schema, rowRdd))(sqlContext)) + new JavaSchemaRDD(sqlContext, LogicalRDD(schema, rowRdd)(sqlContext)) } /** @@ -114,7 +114,7 @@ class JavaSQLContext(val sqlContext: SQLContext) extends UDFRegistration { val scalaRowRDD = rowRDD.rdd.map(r => r.row) val scalaSchema = asScalaDataType(schema).asInstanceOf[SStructType] val logicalPlan = - SparkLogicalPlan(ExistingRdd(scalaSchema.toAttributes, scalaRowRDD))(sqlContext) + LogicalRDD(scalaSchema.toAttributes, scalaRowRDD)(sqlContext) new JavaSchemaRDD(sqlContext, logicalPlan) } @@ -151,7 +151,7 @@ class JavaSQLContext(val sqlContext: SQLContext) extends UDFRegistration { val appliedScalaSchema = JsonRDD.nullTypeToStringType(JsonRDD.inferSchema(json.rdd, 1.0)) val scalaRowRDD = JsonRDD.jsonStringToRow(json.rdd, appliedScalaSchema) val logicalPlan = - SparkLogicalPlan(ExistingRdd(appliedScalaSchema.toAttributes, scalaRowRDD))(sqlContext) + LogicalRDD(appliedScalaSchema.toAttributes, scalaRowRDD)(sqlContext) new JavaSchemaRDD(sqlContext, logicalPlan) } @@ -167,7 +167,7 @@ class JavaSQLContext(val sqlContext: SQLContext) extends UDFRegistration { JsonRDD.nullTypeToStringType(JsonRDD.inferSchema(json.rdd, 1.0))).asInstanceOf[SStructType] val scalaRowRDD = JsonRDD.jsonStringToRow(json.rdd, appliedScalaSchema) val logicalPlan = - SparkLogicalPlan(ExistingRdd(appliedScalaSchema.toAttributes, scalaRowRDD))(sqlContext) + LogicalRDD(appliedScalaSchema.toAttributes, scalaRowRDD)(sqlContext) new JavaSchemaRDD(sqlContext, logicalPlan) } 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 8a3612cdf19be..4f79173a26f88 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 @@ -27,10 +27,15 @@ import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.{LeafNode, SparkPlan} +import org.apache.spark.storage.StorageLevel private[sql] object InMemoryRelation { - def apply(useCompression: Boolean, batchSize: Int, child: SparkPlan): InMemoryRelation = - new InMemoryRelation(child.output, useCompression, batchSize, child)() + def apply( + useCompression: Boolean, + batchSize: Int, + storageLevel: StorageLevel, + child: SparkPlan): InMemoryRelation = + new InMemoryRelation(child.output, useCompression, batchSize, storageLevel, child)() } private[sql] case class CachedBatch(buffers: Array[ByteBuffer], stats: Row) @@ -39,6 +44,7 @@ private[sql] case class InMemoryRelation( output: Seq[Attribute], useCompression: Boolean, batchSize: Int, + storageLevel: StorageLevel, child: SparkPlan) (private var _cachedColumnBuffers: RDD[CachedBatch] = null) extends LogicalPlan with MultiInstanceRelation { @@ -51,6 +57,16 @@ private[sql] case class InMemoryRelation( // If the cached column buffers were not passed in, we calculate them in the constructor. // As in Spark, the actual work of caching is lazy. if (_cachedColumnBuffers == null) { + buildBuffers() + } + + def recache() = { + _cachedColumnBuffers.unpersist() + _cachedColumnBuffers = null + buildBuffers() + } + + private def buildBuffers(): Unit = { val output = child.output val cached = child.execute().mapPartitions { rowIterator => new Iterator[CachedBatch] { @@ -80,19 +96,25 @@ private[sql] case class InMemoryRelation( def hasNext = rowIterator.hasNext } - }.cache() + }.persist(storageLevel) cached.setName(child.toString) _cachedColumnBuffers = cached } + def withOutput(newOutput: Seq[Attribute]): InMemoryRelation = { + InMemoryRelation( + newOutput, useCompression, batchSize, storageLevel, child)(_cachedColumnBuffers) + } + override def children = Seq.empty override def newInstance() = { new InMemoryRelation( - output.map(_.newInstance), + output.map(_.newInstance()), useCompression, batchSize, + storageLevel, child)( _cachedColumnBuffers).asInstanceOf[this.type] } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala new file mode 100644 index 0000000000000..2ddf513b6fc98 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala @@ -0,0 +1,119 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution + +import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan + +import scala.reflect.runtime.universe.TypeTag + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{SQLContext, Row} +import org.apache.spark.sql.catalyst.ScalaReflection +import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericMutableRow} + +/** + * :: DeveloperApi :: + */ +@DeveloperApi +object RDDConversions { + def productToRowRdd[A <: Product](data: RDD[A]): RDD[Row] = { + data.mapPartitions { iterator => + if (iterator.isEmpty) { + Iterator.empty + } else { + val bufferedIterator = iterator.buffered + val mutableRow = new GenericMutableRow(bufferedIterator.head.productArity) + + bufferedIterator.map { r => + var i = 0 + while (i < mutableRow.length) { + mutableRow(i) = ScalaReflection.convertToCatalyst(r.productElement(i)) + i += 1 + } + + mutableRow + } + } + } + } + + /* + def toLogicalPlan[A <: Product : TypeTag](productRdd: RDD[A]): LogicalPlan = { + LogicalRDD(ScalaReflection.attributesFor[A], productToRowRdd(productRdd)) + } + */ +} + +case class LogicalRDD(output: Seq[Attribute], rdd: RDD[Row])(sqlContext: SQLContext) + extends LogicalPlan with MultiInstanceRelation { + + def children = Nil + + def newInstance() = + LogicalRDD(output.map(_.newInstance()), rdd)(sqlContext).asInstanceOf[this.type] + + override def sameResult(plan: LogicalPlan) = plan match { + case LogicalRDD(_, otherRDD) => rdd.id == otherRDD.id + case _ => false + } + + @transient override lazy val statistics = Statistics( + // TODO: Instead of returning a default value here, find a way to return a meaningful size + // estimate for RDDs. See PR 1238 for more discussions. + sizeInBytes = BigInt(sqlContext.defaultSizeInBytes) + ) +} + +case class PhysicalRDD(output: Seq[Attribute], rdd: RDD[Row]) extends LeafNode { + override def execute() = rdd +} + +@deprecated("Use LogicalRDD", "1.2.0") +case class ExistingRdd(output: Seq[Attribute], rdd: RDD[Row]) extends LeafNode { + override def execute() = rdd +} + +@deprecated("Use LogicalRDD", "1.2.0") +case class SparkLogicalPlan(alreadyPlanned: SparkPlan)(@transient sqlContext: SQLContext) + extends LogicalPlan with MultiInstanceRelation { + + def output = alreadyPlanned.output + override def children = Nil + + override final def newInstance(): this.type = { + SparkLogicalPlan( + alreadyPlanned match { + case ExistingRdd(output, rdd) => ExistingRdd(output.map(_.newInstance), rdd) + case _ => sys.error("Multiple instance of the same relation detected.") + })(sqlContext).asInstanceOf[this.type] + } + + override def sameResult(plan: LogicalPlan) = plan match { + case SparkLogicalPlan(ExistingRdd(_, rdd)) => + rdd.id == alreadyPlanned.asInstanceOf[ExistingRdd].rdd.id + case _ => false + } + + @transient override lazy val statistics = Statistics( + // TODO: Instead of returning a default value here, find a way to return a meaningful size + // estimate for RDDs. See PR 1238 for more discussions. + sizeInBytes = BigInt(sqlContext.defaultSizeInBytes) + ) +} 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 2b8913985b028..b1a7948b66cb6 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 @@ -126,39 +126,6 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ } } -/** - * :: DeveloperApi :: - * Allows already planned SparkQueries to be linked into logical query plans. - * - * Note that in general it is not valid to use this class to link multiple copies of the same - * physical operator into the same query plan as this violates the uniqueness of expression ids. - * Special handling exists for ExistingRdd as these are already leaf operators and thus we can just - * replace the output attributes with new copies of themselves without breaking any attribute - * linking. - */ -@DeveloperApi -case class SparkLogicalPlan(alreadyPlanned: SparkPlan)(@transient sqlContext: SQLContext) - extends LogicalPlan with MultiInstanceRelation { - - def output = alreadyPlanned.output - override def children = Nil - - override final def newInstance(): this.type = { - SparkLogicalPlan( - alreadyPlanned match { - case ExistingRdd(output, rdd) => ExistingRdd(output.map(_.newInstance), rdd) - case _ => sys.error("Multiple instance of the same relation detected.") - })(sqlContext).asInstanceOf[this.type] - } - - @transient override lazy val statistics = Statistics( - // TODO: Instead of returning a default value here, find a way to return a meaningful size - // estimate for RDDs. See PR 1238 for more discussions. - sizeInBytes = BigInt(sqlContext.defaultSizeInBytes) - ) - -} - private[sql] trait LeafNode extends SparkPlan with trees.LeafNode[SparkPlan] { self: Product => } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 7943d6e1b6fb5..5c16d0c624128 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -272,10 +272,11 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { execution.Aggregate(partial = false, group, agg, planLater(child)) :: Nil case logical.Sample(fraction, withReplacement, seed, child) => execution.Sample(fraction, withReplacement, seed, planLater(child)) :: Nil + case SparkLogicalPlan(alreadyPlanned) => alreadyPlanned :: Nil case logical.LocalRelation(output, data) => - ExistingRdd( + PhysicalRDD( output, - ExistingRdd.productToRowRdd(sparkContext.parallelize(data, numPartitions))) :: Nil + RDDConversions.productToRowRdd(sparkContext.parallelize(data, numPartitions))) :: Nil case logical.Limit(IntegerLiteral(limit), child) => execution.Limit(limit, planLater(child)) :: Nil case Unions(unionChildren) => @@ -287,12 +288,12 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case logical.Generate(generator, join, outer, _, child) => execution.Generate(generator, join = join, outer = outer, planLater(child)) :: Nil case logical.NoRelation => - execution.ExistingRdd(Nil, singleRowRdd) :: Nil + execution.PhysicalRDD(Nil, singleRowRdd) :: Nil case logical.Repartition(expressions, child) => execution.Exchange(HashPartitioning(expressions, numPartitions), planLater(child)) :: Nil case e @ EvaluatePython(udf, child) => BatchPythonEvaluation(udf, e.output, planLater(child)) :: Nil - case SparkLogicalPlan(existingPlan) => existingPlan :: Nil + case LogicalRDD(output, rdd) => PhysicalRDD(output, rdd) :: Nil case _ => Nil } } @@ -303,8 +304,10 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { Seq(execution.SetCommand(key, value, plan.output)(context)) case logical.ExplainCommand(logicalPlan, extended) => Seq(execution.ExplainCommand(logicalPlan, plan.output, extended)(context)) - case logical.CacheCommand(tableName, cache) => - Seq(execution.CacheCommand(tableName, cache)(context)) + case logical.CacheTableCommand(tableName, optPlan, isLazy) => + Seq(execution.CacheTableCommand(tableName, optPlan, isLazy)) + case logical.UncacheTableCommand(tableName) => + Seq(execution.UncacheTableCommand(tableName)) case _ => Nil } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index cac376608be29..977f3c9f32096 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 @@ -210,45 +210,6 @@ case class Sort( override def output = child.output } -/** - * :: DeveloperApi :: - */ -@DeveloperApi -object ExistingRdd { - def productToRowRdd[A <: Product](data: RDD[A]): RDD[Row] = { - data.mapPartitions { iterator => - if (iterator.isEmpty) { - Iterator.empty - } else { - val bufferedIterator = iterator.buffered - val mutableRow = new GenericMutableRow(bufferedIterator.head.productArity) - - bufferedIterator.map { r => - var i = 0 - while (i < mutableRow.length) { - mutableRow(i) = ScalaReflection.convertToCatalyst(r.productElement(i)) - i += 1 - } - - mutableRow - } - } - } - } - - def fromProductRdd[A <: Product : TypeTag](productRdd: RDD[A]) = { - ExistingRdd(ScalaReflection.attributesFor[A], productToRowRdd(productRdd)) - } -} - -/** - * :: DeveloperApi :: - */ -@DeveloperApi -case class ExistingRdd(output: Seq[Attribute], rdd: RDD[Row]) extends LeafNode { - override def execute() = rdd -} - /** * :: DeveloperApi :: * Computes the set of distinct input rows using a HashSet. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala index 94543fc95b470..d49633c24ad4d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala @@ -37,7 +37,7 @@ trait Command { * The `execute()` method of all the physical command classes should reference `sideEffectResult` * so that the command can be executed eagerly right after the command query is created. */ - protected[sql] lazy val sideEffectResult: Seq[Row] = Seq.empty[Row] + protected lazy val sideEffectResult: Seq[Row] = Seq.empty[Row] override def executeCollect(): Array[Row] = sideEffectResult.toArray @@ -53,7 +53,7 @@ case class SetCommand( @transient context: SQLContext) extends LeafNode with Command with Logging { - override protected[sql] lazy val sideEffectResult: Seq[Row] = (key, value) match { + override protected lazy val sideEffectResult: Seq[Row] = (key, value) match { // Set value for key k. case (Some(k), Some(v)) => if (k == SQLConf.Deprecated.MAPRED_REDUCE_TASKS) { @@ -121,7 +121,7 @@ case class ExplainCommand( extends LeafNode with Command { // Run through the optimizer to generate the physical plan. - override protected[sql] lazy val sideEffectResult: Seq[Row] = try { + override protected lazy val sideEffectResult: Seq[Row] = try { // TODO in Hive, the "extended" ExplainCommand prints the AST as well, and detailed properties. val queryExecution = context.executePlan(logicalPlan) val outputString = if (extended) queryExecution.toString else queryExecution.simpleString @@ -138,15 +138,38 @@ case class ExplainCommand( * :: DeveloperApi :: */ @DeveloperApi -case class CacheCommand(tableName: String, doCache: Boolean)(@transient context: SQLContext) +case class CacheTableCommand( + tableName: String, + plan: Option[LogicalPlan], + isLazy: Boolean) extends LeafNode with Command { - override protected[sql] lazy val sideEffectResult = { - if (doCache) { - context.cacheTable(tableName) - } else { - context.uncacheTable(tableName) + override protected lazy val sideEffectResult = { + import sqlContext._ + + plan.foreach(_.registerTempTable(tableName)) + val schemaRDD = table(tableName) + schemaRDD.cache() + + if (!isLazy) { + // Performs eager caching + schemaRDD.count() } + + Seq.empty[Row] + } + + override def output: Seq[Attribute] = Seq.empty +} + + +/** + * :: DeveloperApi :: + */ +@DeveloperApi +case class UncacheTableCommand(tableName: String) extends LeafNode with Command { + override protected lazy val sideEffectResult: Seq[Row] = { + sqlContext.table(tableName).unpersist() Seq.empty[Row] } @@ -161,7 +184,7 @@ case class DescribeCommand(child: SparkPlan, output: Seq[Attribute])( @transient context: SQLContext) extends LeafNode with Command { - override protected[sql] lazy val sideEffectResult: Seq[Row] = { + override protected lazy val sideEffectResult: Seq[Row] = { Row("# Registered as a temporary table", null, null) +: child.output.map(field => Row(field.name, field.dataType.toString, null)) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala index a5a5d139a65cb..ffb732347d30a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -289,9 +289,9 @@ case class InsertIntoParquetTable( def writeShard(context: TaskContext, iter: Iterator[Row]): Int = { // Hadoop wants a 32-bit task attempt ID, so if ours is bigger than Int.MaxValue, roll it // around by taking a mod. We expect that no task will be attempted 2 billion times. - val attemptNumber = (context.attemptId % Int.MaxValue).toInt + val attemptNumber = (context.getAttemptId % Int.MaxValue).toInt /* "reduce task" */ - val attemptId = newTaskAttemptID(jobtrackerID, stageId, isMap = false, context.partitionId, + val attemptId = newTaskAttemptID(jobtrackerID, stageId, isMap = false, context.getPartitionId, attemptNumber) val hadoopContext = newTaskAttemptContext(wrappedConf.value, attemptId) val format = new AppendingParquetOutputFormat(taskIdOffset) @@ -427,11 +427,15 @@ private[parquet] class FilteringParquetRowInputFormat s"maxSplitSize or minSplitSie should not be negative: maxSplitSize = $maxSplitSize;" + s" minSplitSize = $minSplitSize") } - + val splits = mutable.ArrayBuffer.empty[ParquetInputSplit] val getGlobalMetaData = classOf[ParquetFileWriter].getDeclaredMethod("getGlobalMetaData", classOf[JList[Footer]]) getGlobalMetaData.setAccessible(true) val globalMetaData = getGlobalMetaData.invoke(null, footers).asInstanceOf[GlobalMetaData] + // if parquet file is empty, return empty splits. + if (globalMetaData == null) { + return splits + } val readContext = getReadSupport(configuration).init( new InitContext(configuration, @@ -442,7 +446,6 @@ private[parquet] class FilteringParquetRowInputFormat classOf[ParquetInputFormat[_]].getDeclaredMethods.find(_.getName == "generateSplits").get generateSplits.setAccessible(true) - val splits = mutable.ArrayBuffer.empty[ParquetInputSplit] for (footer <- footers) { val fs = footer.getFile.getFileSystem(configuration) val file = footer.getFile diff --git a/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala index 265b67737c475..6bb81c76ed8bd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala @@ -22,7 +22,11 @@ import org.apache.spark.sql.{SQLConf, SQLContext} /** A SQLContext that can be used for local testing. */ object TestSQLContext - extends SQLContext(new SparkContext("local[2]", "TestSQLContext", new SparkConf())) { + extends SQLContext( + new SparkContext( + "local[2]", + "TestSQLContext", + new SparkConf().set("spark.sql.testkey", "true"))) { /** Fewer partitions to speed up testing. */ override private[spark] def numShufflePartitions: Int = diff --git a/sql/core/src/test/resources/log4j.properties b/sql/core/src/test/resources/log4j.properties index c7e0ff1cf6494..fbed0a782dd3e 100644 --- a/sql/core/src/test/resources/log4j.properties +++ b/sql/core/src/test/resources/log4j.properties @@ -30,7 +30,7 @@ log4j.appender.FA=org.apache.log4j.FileAppender log4j.appender.FA.append=false log4j.appender.FA.file=target/unit-tests.log log4j.appender.FA.layout=org.apache.log4j.PatternLayout -log4j.appender.FA.layout.ConversionPattern=%d{HH:mm:ss.SSS} %p %c{1}: %m%n +log4j.appender.FA.layout.ConversionPattern=%d{HH:mm:ss.SSS} %t %p %c{1}: %m%n # Set the logger level of File Appender to WARN log4j.appender.FA.Threshold = INFO diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index befef46d93973..1e624f97004f5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -18,15 +18,41 @@ package org.apache.spark.sql import org.apache.spark.sql.TestData._ -import org.apache.spark.sql.columnar.{InMemoryRelation, InMemoryColumnarTableScan} -import org.apache.spark.sql.test.TestSQLContext +import org.apache.spark.sql.columnar.{InMemoryColumnarTableScan, InMemoryRelation} import org.apache.spark.sql.test.TestSQLContext._ +import org.apache.spark.storage.RDDBlockId case class BigData(s: String) class CachedTableSuite extends QueryTest { TestData // Load test tables. + def assertCached(query: SchemaRDD, numCachedTables: Int = 1): Unit = { + val planWithCaching = query.queryExecution.withCachedData + val cachedData = planWithCaching collect { + case cached: InMemoryRelation => cached + } + + assert( + cachedData.size == numCachedTables, + s"Expected query to contain $numCachedTables, but it actually had ${cachedData.size}\n" + + planWithCaching) + } + + def rddIdOf(tableName: String): Int = { + val executedPlan = table(tableName).queryExecution.executedPlan + executedPlan.collect { + case InMemoryColumnarTableScan(_, _, relation) => + relation.cachedColumnBuffers.id + case _ => + fail(s"Table $tableName is not cached\n" + executedPlan) + }.head + } + + def isMaterialized(rddId: Int): Boolean = { + sparkContext.env.blockManager.get(RDDBlockId(rddId, 0)).nonEmpty + } + test("too big for memory") { val data = "*" * 10000 sparkContext.parallelize(1 to 1000000, 1).map(_ => BigData(data)).registerTempTable("bigData") @@ -35,88 +61,145 @@ class CachedTableSuite extends QueryTest { uncacheTable("bigData") } + test("calling .cache() should use in-memory columnar caching") { + table("testData").cache() + assertCached(table("testData")) + } + + test("calling .unpersist() should drop in-memory columnar cache") { + table("testData").cache() + table("testData").count() + table("testData").unpersist(true) + assertCached(table("testData"), 0) + } + + test("isCached") { + cacheTable("testData") + + assertCached(table("testData")) + assert(table("testData").queryExecution.withCachedData match { + case _: InMemoryRelation => true + case _ => false + }) + + uncacheTable("testData") + assert(!isCached("testData")) + assert(table("testData").queryExecution.withCachedData match { + case _: InMemoryRelation => false + case _ => true + }) + } + test("SPARK-1669: cacheTable should be idempotent") { assume(!table("testData").logicalPlan.isInstanceOf[InMemoryRelation]) cacheTable("testData") - table("testData").queryExecution.analyzed match { - case _: InMemoryRelation => - case _ => - fail("testData should be cached") + assertCached(table("testData")) + + assertResult(1, "InMemoryRelation not found, testData should have been cached") { + table("testData").queryExecution.withCachedData.collect { + case r: InMemoryRelation => r + }.size } cacheTable("testData") - table("testData").queryExecution.analyzed match { - case InMemoryRelation(_, _, _, _: InMemoryColumnarTableScan) => - fail("cacheTable is not idempotent") - - case _ => + assertResult(0, "Double InMemoryRelations found, cacheTable() is not idempotent") { + table("testData").queryExecution.withCachedData.collect { + case r @ InMemoryRelation(_, _, _, _, _: InMemoryColumnarTableScan) => r + }.size } } test("read from cached table and uncache") { - TestSQLContext.cacheTable("testData") - - checkAnswer( - TestSQLContext.table("testData"), - testData.collect().toSeq - ) - - TestSQLContext.table("testData").queryExecution.analyzed match { - case _ : InMemoryRelation => // Found evidence of caching - case noCache => fail(s"No cache node found in plan $noCache") - } - - TestSQLContext.uncacheTable("testData") + cacheTable("testData") + checkAnswer(table("testData"), testData.collect().toSeq) + assertCached(table("testData")) - checkAnswer( - TestSQLContext.table("testData"), - testData.collect().toSeq - ) - - TestSQLContext.table("testData").queryExecution.analyzed match { - case cachePlan: InMemoryRelation => - fail(s"Table still cached after uncache: $cachePlan") - case noCache => // Table uncached successfully - } + uncacheTable("testData") + checkAnswer(table("testData"), testData.collect().toSeq) + assertCached(table("testData"), 0) } test("correct error on uncache of non-cached table") { intercept[IllegalArgumentException] { - TestSQLContext.uncacheTable("testData") + uncacheTable("testData") } } - test("SELECT Star Cached Table") { - TestSQLContext.sql("SELECT * FROM testData").registerTempTable("selectStar") - TestSQLContext.cacheTable("selectStar") - TestSQLContext.sql("SELECT * FROM selectStar WHERE key = 1").collect() - TestSQLContext.uncacheTable("selectStar") + test("SELECT star from cached table") { + sql("SELECT * FROM testData").registerTempTable("selectStar") + cacheTable("selectStar") + checkAnswer( + sql("SELECT * FROM selectStar WHERE key = 1"), + Seq(Row(1, "1"))) + uncacheTable("selectStar") } test("Self-join cached") { val unCachedAnswer = - TestSQLContext.sql("SELECT * FROM testData a JOIN testData b ON a.key = b.key").collect() - TestSQLContext.cacheTable("testData") + sql("SELECT * FROM testData a JOIN testData b ON a.key = b.key").collect() + cacheTable("testData") checkAnswer( - TestSQLContext.sql("SELECT * FROM testData a JOIN testData b ON a.key = b.key"), + sql("SELECT * FROM testData a JOIN testData b ON a.key = b.key"), unCachedAnswer.toSeq) - TestSQLContext.uncacheTable("testData") + uncacheTable("testData") } test("'CACHE TABLE' and 'UNCACHE TABLE' SQL statement") { - TestSQLContext.sql("CACHE TABLE testData") - TestSQLContext.table("testData").queryExecution.executedPlan match { - case _: InMemoryColumnarTableScan => // Found evidence of caching - case _ => fail(s"Table 'testData' should be cached") - } - assert(TestSQLContext.isCached("testData"), "Table 'testData' should be cached") + sql("CACHE TABLE testData") + assertCached(table("testData")) - TestSQLContext.sql("UNCACHE TABLE testData") - TestSQLContext.table("testData").queryExecution.executedPlan match { - case _: InMemoryColumnarTableScan => fail(s"Table 'testData' should not be cached") - case _ => // Found evidence of uncaching - } - assert(!TestSQLContext.isCached("testData"), "Table 'testData' should not be cached") + val rddId = rddIdOf("testData") + assert( + isMaterialized(rddId), + "Eagerly cached in-memory table should have already been materialized") + + sql("UNCACHE TABLE testData") + assert(!isCached("testData"), "Table 'testData' should not be cached") + assert(!isMaterialized(rddId), "Uncached in-memory table should have been unpersisted") + } + + test("CACHE TABLE tableName AS SELECT * FROM anotherTable") { + sql("CACHE TABLE testCacheTable AS SELECT * FROM testData") + assertCached(table("testCacheTable")) + + val rddId = rddIdOf("testCacheTable") + assert( + isMaterialized(rddId), + "Eagerly cached in-memory table should have already been materialized") + + uncacheTable("testCacheTable") + assert(!isMaterialized(rddId), "Uncached in-memory table should have been unpersisted") + } + + test("CACHE TABLE tableName AS SELECT ...") { + sql("CACHE TABLE testCacheTable AS SELECT key FROM testData LIMIT 10") + assertCached(table("testCacheTable")) + + val rddId = rddIdOf("testCacheTable") + assert( + isMaterialized(rddId), + "Eagerly cached in-memory table should have already been materialized") + + uncacheTable("testCacheTable") + assert(!isMaterialized(rddId), "Uncached in-memory table should have been unpersisted") + } + + test("CACHE LAZY TABLE tableName") { + sql("CACHE LAZY TABLE testData") + assertCached(table("testData")) + + val rddId = rddIdOf("testData") + assert( + !isMaterialized(rddId), + "Lazily cached in-memory table shouldn't be materialized eagerly") + + sql("SELECT COUNT(*) FROM testData").collect() + assert( + isMaterialized(rddId), + "Lazily cached in-memory table should have been materialized") + + uncacheTable("testData") + assert(!isMaterialized(rddId), "Uncached in-memory table should have been unpersisted") } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLConfSuite.scala index 584f71b3c13d5..60701f0e154f8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLConfSuite.scala @@ -17,16 +17,25 @@ package org.apache.spark.sql +import org.scalatest.FunSuiteLike + import org.apache.spark.sql.test._ /* Implicits */ import TestSQLContext._ -class SQLConfSuite extends QueryTest { +class SQLConfSuite extends QueryTest with FunSuiteLike { val testKey = "test.key.0" val testVal = "test.val.0" + test("propagate from spark conf") { + // We create a new context here to avoid order dependence with other tests that might call + // clear(). + val newContext = new SQLContext(TestSQLContext.sparkContext) + assert(newContext.getConf("spark.sql.testkey", "false") == "true") + } + test("programmatic ways of basic setting and getting") { clear() assert(getAllConfs.size === 0) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index e4b8aeff60c63..ccf91c1e6a70f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.execution.{ShuffledHashJoin, BroadcastHashJoin} import org.apache.spark.sql.test._ import org.scalatest.BeforeAndAfterAll import java.util.TimeZone @@ -189,6 +190,14 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { sql("SELECT * FROM testData2 ORDER BY a DESC, b ASC"), Seq((3,1), (3,2), (2,1), (2,2), (1,1), (1,2))) + checkAnswer( + sql("SELECT b FROM binaryData ORDER BY a ASC"), + (1 to 5).map(Row(_)).toSeq) + + checkAnswer( + sql("SELECT b FROM binaryData ORDER BY a DESC"), + (1 to 5).map(Row(_)).toSeq.reverse) + checkAnswer( sql("SELECT * FROM arrayData ORDER BY data[0] ASC"), arrayData.collect().sortBy(_.data(0)).toSeq) @@ -380,7 +389,6 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { } test("SPARK-3349 partitioning after limit") { - /* sql("SELECT DISTINCT n FROM lowerCaseData ORDER BY n DESC") .limit(2) .registerTempTable("subset1") @@ -395,7 +403,6 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { sql("SELECT * FROM lowerCaseData INNER JOIN subset2 ON subset2.n = lowerCaseData.n"), (1, "a", 1) :: (2, "b", 2) :: Nil) - */ } test("mixed-case keywords") { @@ -649,24 +656,24 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { (3, null) :: (4, 2147483644) :: Nil) } - + test("SPARK-3423 BETWEEN") { checkAnswer( sql("SELECT key, value FROM testData WHERE key BETWEEN 5 and 7"), Seq((5, "5"), (6, "6"), (7, "7")) ) - + checkAnswer( sql("SELECT key, value FROM testData WHERE key BETWEEN 7 and 7"), Seq((7, "7")) ) - + checkAnswer( sql("SELECT key, value FROM testData WHERE key BETWEEN 9 and 7"), Seq() ) } - + test("cast boolean to string") { // TODO Ensure true/false string letter casing is consistent with Hive in all cases. checkAnswer( @@ -694,4 +701,10 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { validateMetadata(sql("SELECT * FROM personWithMeta JOIN salary ON id = personId")) validateMetadata(sql("SELECT name, salary FROM personWithMeta JOIN salary ON id = personId")) } + + test("SPARK-3371 Renaming a function expression with group by gives error") { + registerFunction("len", (s: String) => s.length) + checkAnswer( + sql("SELECT len(value) as temp FROM testData WHERE key = 1 group by len(value)"), 1) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala index 9600ebbd0da59..c61b0aee37863 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 @@ -54,6 +54,16 @@ object TestData { TestData2(3, 2) :: Nil) testData2.registerTempTable("testData2") + case class BinaryData(a: Array[Byte], b: Int) + val binaryData: SchemaRDD = + TestSQLContext.sparkContext.parallelize( + BinaryData("12".getBytes(), 1) :: + BinaryData("22".getBytes(), 5) :: + BinaryData("122".getBytes(), 3) :: + BinaryData("121".getBytes(), 2) :: + BinaryData("123".getBytes(), 4) :: Nil) + binaryData.registerTempTable("binaryData") + // TODO: There is no way to express null primitives as case classes currently... val testData3 = logical.LocalRelation('a.int, 'b.int).loadData( 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 c1278248ef655..9775dd26b7773 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 @@ -20,6 +20,7 @@ package org.apache.spark.sql.columnar import org.apache.spark.sql.catalyst.expressions.Row import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.{QueryTest, TestData} +import org.apache.spark.storage.StorageLevel.MEMORY_ONLY class InMemoryColumnarQuerySuite extends QueryTest { import org.apache.spark.sql.TestData._ @@ -27,7 +28,7 @@ class InMemoryColumnarQuerySuite extends QueryTest { test("simple columnar query") { val plan = TestSQLContext.executePlan(testData.logicalPlan).executedPlan - val scan = InMemoryRelation(useCompression = true, 5, plan) + val scan = InMemoryRelation(useCompression = true, 5, MEMORY_ONLY, plan) checkAnswer(scan, testData.collect().toSeq) } @@ -42,7 +43,7 @@ class InMemoryColumnarQuerySuite extends QueryTest { test("projection") { val plan = TestSQLContext.executePlan(testData.select('value, 'key).logicalPlan).executedPlan - val scan = InMemoryRelation(useCompression = true, 5, plan) + val scan = InMemoryRelation(useCompression = true, 5, MEMORY_ONLY, plan) checkAnswer(scan, testData.collect().map { case Row(key: Int, value: String) => value -> key @@ -51,7 +52,7 @@ class InMemoryColumnarQuerySuite extends QueryTest { test("SPARK-1436 regression: in-memory columns must be able to be accessed multiple times") { val plan = TestSQLContext.executePlan(testData.logicalPlan).executedPlan - val scan = InMemoryRelation(useCompression = true, 5, plan) + val scan = InMemoryRelation(useCompression = true, 5, MEMORY_ONLY, plan) checkAnswer(scan, testData.collect().toSeq) checkAnswer(scan, testData.collect().toSeq) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala index 37d64f0de7bab..bfbf431a11913 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala @@ -22,7 +22,7 @@ import org.scalatest.FunSuite import org.apache.spark.sql.TestData._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans._ -import org.apache.spark.sql.execution +import org.apache.spark.sql.{SQLConf, execution} import org.apache.spark.sql.test.TestSQLContext._ import org.apache.spark.sql.test.TestSQLContext.planner._ @@ -57,4 +57,22 @@ class PlannerSuite extends FunSuite { val planned = HashAggregation(query) assert(planned.nonEmpty) } + + test("sizeInBytes estimation of limit operator for broadcast hash join optimization") { + val origThreshold = autoBroadcastJoinThreshold + setConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD, 81920.toString) + + // Using a threshold that is definitely larger than the small testing table (b) below + val a = testData.as('a) + val b = testData.limit(3).as('b) + val planned = a.join(b, Inner, Some("a.key".attr === "b.key".attr)).queryExecution.executedPlan + + val broadcastHashJoins = planned.collect { case join: BroadcastHashJoin => join } + val shuffledHashJoins = planned.collect { case join: ShuffledHashJoin => join } + + assert(broadcastHashJoins.size === 1, "Should use broadcast hash join") + assert(shuffledHashJoins.isEmpty, "Should not use shuffled hash join") + + setConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD, origThreshold.toString) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index 08f7358446b29..07adf731405af 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -789,4 +789,13 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA assert(result3(0)(1) === "the answer") Utils.deleteRecursively(tmpdir) } + + test("Querying on empty parquet throws exception (SPARK-3536)") { + val tmpdir = Utils.createTempDir() + Utils.deleteRecursively(tmpdir) + createParquetFile[TestRDDEntry](tmpdir.toString()).registerTempTable("tmpemptytable") + val result1 = sql("SELECT * FROM tmpemptytable").collect() + assert(result1.size === 0) + Utils.deleteRecursively(tmpdir) + } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala index cadf7aaf42157..3d468d804622c 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala @@ -26,6 +26,7 @@ import org.apache.hive.service.cli.thrift.ThriftBinaryCLIService import org.apache.hive.service.server.{HiveServer2, ServerOptionsProcessor} import org.apache.spark.Logging +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.sql.hive.HiveContext import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ @@ -33,9 +34,21 @@ import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ * The main entry point for the Spark SQL port of HiveServer2. Starts up a `SparkSQLContext` and a * `HiveThriftServer2` thrift server. */ -private[hive] object HiveThriftServer2 extends Logging { +object HiveThriftServer2 extends Logging { var LOG = LogFactory.getLog(classOf[HiveServer2]) + /** + * :: DeveloperApi :: + * Starts a new thrift server with the given context. + */ + @DeveloperApi + def startWithContext(sqlContext: HiveContext): Unit = { + val server = new HiveThriftServer2(sqlContext) + server.init(sqlContext.hiveconf) + server.start() + } + + def main(args: Array[String]) { val optionsProcessor = new ServerOptionsProcessor("HiveThriftServer2") diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala index b092f42372171..7ba4564602ecd 100755 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala @@ -73,18 +73,6 @@ private[hive] object SparkSQLCLIDriver { System.exit(1) } - // NOTE: It is critical to do this here so that log4j is reinitialized - // before any of the other core hive classes are loaded - var logInitFailed = false - var logInitDetailMessage: String = null - try { - logInitDetailMessage = LogUtils.initHiveLog4j() - } catch { - case e: LogInitializationException => - logInitFailed = true - logInitDetailMessage = e.getMessage - } - val sessionState = new CliSessionState(new HiveConf(classOf[SessionState])) sessionState.in = System.in @@ -100,11 +88,6 @@ private[hive] object SparkSQLCLIDriver { System.exit(2) } - if (!sessionState.getIsSilent) { - if (logInitFailed) System.err.println(logInitDetailMessage) - else SessionState.getConsole.printInfo(logInitDetailMessage) - } - // Set all properties specified via command line. val conf: HiveConf = sessionState.getConf sessionState.cmdProperties.entrySet().foreach { item: java.util.Map.Entry[Object, Object] => diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala index bd3f68d92d8c7..910174a153768 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala @@ -113,7 +113,7 @@ private[thriftserver] class SparkSQLOperationManager(hiveContext: HiveContext) case ByteType => to.addColumnValue(ColumnValue.byteValue(from.getByte(ordinal))) case ShortType => - to.addColumnValue(ColumnValue.intValue(from.getShort(ordinal))) + to.addColumnValue(ColumnValue.shortValue(from.getShort(ordinal))) case TimestampType => to.addColumnValue( ColumnValue.timestampValue(from.get(ordinal).asInstanceOf[Timestamp])) @@ -145,7 +145,7 @@ private[thriftserver] class SparkSQLOperationManager(hiveContext: HiveContext) case ByteType => to.addColumnValue(ColumnValue.byteValue(null)) case ShortType => - to.addColumnValue(ColumnValue.intValue(null)) + to.addColumnValue(ColumnValue.shortValue(null)) case TimestampType => to.addColumnValue(ColumnValue.timestampValue(null)) case BinaryType | _: ArrayType | _: StructType | _: MapType => diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index ab487d673e813..35e9c9939d4b7 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -220,6 +220,23 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { */ override def whiteList = Seq( "add_part_exist", + "dynamic_partition_skip_default", + "infer_bucket_sort_dyn_part", + "load_dyn_part1", + "load_dyn_part2", + "load_dyn_part3", + "load_dyn_part4", + "load_dyn_part5", + "load_dyn_part6", + "load_dyn_part7", + "load_dyn_part8", + "load_dyn_part9", + "load_dyn_part10", + "load_dyn_part11", + "load_dyn_part12", + "load_dyn_part13", + "load_dyn_part14", + "load_dyn_part14_win", "add_part_multiple", "add_partition_no_whitelist", "add_partition_with_whitelist", @@ -801,6 +818,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "udf_or", "udf_parse_url", "udf_PI", + "udf_pmod", "udf_positive", "udf_pow", "udf_power", diff --git a/sql/hive/src/main/scala/org/apache/spark/SparkHadoopWriter.scala b/sql/hive/src/main/scala/org/apache/spark/SparkHadoopWriter.scala deleted file mode 100644 index ab7862f4f9e06..0000000000000 --- a/sql/hive/src/main/scala/org/apache/spark/SparkHadoopWriter.scala +++ /dev/null @@ -1,195 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.hive - -import java.io.IOException -import java.text.NumberFormat -import java.util.Date - -import org.apache.hadoop.fs.Path -import org.apache.hadoop.hive.ql.exec.{FileSinkOperator, Utilities} -import org.apache.hadoop.hive.ql.io.{HiveFileFormatUtils, HiveOutputFormat} -import org.apache.hadoop.hive.ql.plan.FileSinkDesc -import org.apache.hadoop.mapred._ -import org.apache.hadoop.io.Writable - -import org.apache.spark.{Logging, SerializableWritable, SparkHadoopWriter} - -/** - * Internal helper class that saves an RDD using a Hive OutputFormat. - * It is based on [[SparkHadoopWriter]]. - */ -private[hive] class SparkHiveHadoopWriter( - @transient jobConf: JobConf, - fileSinkConf: FileSinkDesc) - extends Logging - with SparkHadoopMapRedUtil - with Serializable { - - private val now = new Date() - private val conf = new SerializableWritable(jobConf) - - private var jobID = 0 - private var splitID = 0 - private var attemptID = 0 - private var jID: SerializableWritable[JobID] = null - private var taID: SerializableWritable[TaskAttemptID] = null - - @transient private var writer: FileSinkOperator.RecordWriter = null - @transient private var format: HiveOutputFormat[AnyRef, Writable] = null - @transient private var committer: OutputCommitter = null - @transient private var jobContext: JobContext = null - @transient private var taskContext: TaskAttemptContext = null - - def preSetup() { - setIDs(0, 0, 0) - setConfParams() - - val jCtxt = getJobContext() - getOutputCommitter().setupJob(jCtxt) - } - - - def setup(jobid: Int, splitid: Int, attemptid: Int) { - setIDs(jobid, splitid, attemptid) - setConfParams() - } - - def open() { - val numfmt = NumberFormat.getInstance() - numfmt.setMinimumIntegerDigits(5) - numfmt.setGroupingUsed(false) - - val extension = Utilities.getFileExtension( - conf.value, - fileSinkConf.getCompressed, - getOutputFormat()) - - val outputName = "part-" + numfmt.format(splitID) + extension - val path = FileOutputFormat.getTaskOutputPath(conf.value, outputName) - - getOutputCommitter().setupTask(getTaskContext()) - writer = HiveFileFormatUtils.getHiveRecordWriter( - conf.value, - fileSinkConf.getTableInfo, - conf.value.getOutputValueClass.asInstanceOf[Class[Writable]], - fileSinkConf, - path, - null) - } - - def write(value: Writable) { - if (writer != null) { - writer.write(value) - } else { - throw new IOException("Writer is null, open() has not been called") - } - } - - def close() { - // Seems the boolean value passed into close does not matter. - writer.close(false) - } - - def commit() { - val taCtxt = getTaskContext() - val cmtr = getOutputCommitter() - if (cmtr.needsTaskCommit(taCtxt)) { - try { - cmtr.commitTask(taCtxt) - logInfo (taID + ": Committed") - } catch { - case e: IOException => - logError("Error committing the output of task: " + taID.value, e) - cmtr.abortTask(taCtxt) - throw e - } - } else { - logWarning ("No need to commit output of task: " + taID.value) - } - } - - def commitJob() { - // always ? Or if cmtr.needsTaskCommit ? - val cmtr = getOutputCommitter() - cmtr.commitJob(getJobContext()) - } - - // ********* Private Functions ********* - - private def getOutputFormat(): HiveOutputFormat[AnyRef,Writable] = { - if (format == null) { - format = conf.value.getOutputFormat() - .asInstanceOf[HiveOutputFormat[AnyRef,Writable]] - } - format - } - - private def getOutputCommitter(): OutputCommitter = { - if (committer == null) { - committer = conf.value.getOutputCommitter - } - committer - } - - private def getJobContext(): JobContext = { - if (jobContext == null) { - jobContext = newJobContext(conf.value, jID.value) - } - jobContext - } - - private def getTaskContext(): TaskAttemptContext = { - if (taskContext == null) { - taskContext = newTaskAttemptContext(conf.value, taID.value) - } - taskContext - } - - private def setIDs(jobId: Int, splitId: Int, attemptId: Int) { - jobID = jobId - splitID = splitId - attemptID = attemptId - - jID = new SerializableWritable[JobID](SparkHadoopWriter.createJobID(now, jobId)) - taID = new SerializableWritable[TaskAttemptID]( - new TaskAttemptID(new TaskID(jID.value, true, splitID), attemptID)) - } - - private def setConfParams() { - conf.value.set("mapred.job.id", jID.value.toString) - conf.value.set("mapred.tip.id", taID.value.getTaskID.toString) - conf.value.set("mapred.task.id", taID.value.toString) - conf.value.setBoolean("mapred.task.is.map", true) - conf.value.setInt("mapred.task.partition", splitID) - } -} - -private[hive] object SparkHiveHadoopWriter { - def createPathFromString(path: String, conf: JobConf): Path = { - if (path == null) { - throw new IllegalArgumentException("Output path is null") - } - val outputPath = new Path(path) - val fs = outputPath.getFileSystem(conf) - if (outputPath == null || fs == null) { - throw new IllegalArgumentException("Incorrectly formatted output path") - } - outputPath.makeQualified(fs) - } -} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/ExtendedHiveQlParser.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/ExtendedHiveQlParser.scala new file mode 100644 index 0000000000000..c5844e92eaaa9 --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/ExtendedHiveQlParser.scala @@ -0,0 +1,135 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive + +import scala.language.implicitConversions +import scala.util.parsing.combinator.syntactical.StandardTokenParsers +import scala.util.parsing.combinator.PackratParsers +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.SqlLexical + +/** + * A parser that recognizes all HiveQL constructs together with several Spark SQL specific + * extensions like CACHE TABLE and UNCACHE TABLE. + */ +private[hive] class ExtendedHiveQlParser extends StandardTokenParsers with PackratParsers { + + def apply(input: String): LogicalPlan = { + // Special-case out set commands since the value fields can be + // complex to handle without RegexParsers. Also this approach + // is clearer for the several possible cases of set commands. + if (input.trim.toLowerCase.startsWith("set")) { + input.trim.drop(3).split("=", 2).map(_.trim) match { + case Array("") => // "set" + SetCommand(None, None) + case Array(key) => // "set key" + SetCommand(Some(key), None) + case Array(key, value) => // "set key=value" + SetCommand(Some(key), Some(value)) + } + } else if (input.trim.startsWith("!")) { + ShellCommand(input.drop(1)) + } else { + phrase(query)(new lexical.Scanner(input)) match { + case Success(r, x) => r + case x => sys.error(x.toString) + } + } + } + + protected case class Keyword(str: String) + + protected val ADD = Keyword("ADD") + protected val AS = Keyword("AS") + protected val CACHE = Keyword("CACHE") + protected val DFS = Keyword("DFS") + protected val FILE = Keyword("FILE") + protected val JAR = Keyword("JAR") + protected val LAZY = Keyword("LAZY") + protected val SET = Keyword("SET") + protected val SOURCE = Keyword("SOURCE") + protected val TABLE = Keyword("TABLE") + protected val UNCACHE = Keyword("UNCACHE") + + protected implicit def asParser(k: Keyword): Parser[String] = + lexical.allCaseVersions(k.str).map(x => x : Parser[String]).reduce(_ | _) + + protected def allCaseConverse(k: String): Parser[String] = + lexical.allCaseVersions(k).map(x => x : Parser[String]).reduce(_ | _) + + protected val reservedWords = + this.getClass + .getMethods + .filter(_.getReturnType == classOf[Keyword]) + .map(_.invoke(this).asInstanceOf[Keyword].str) + + override val lexical = new SqlLexical(reservedWords) + + protected lazy val query: Parser[LogicalPlan] = + cache | uncache | addJar | addFile | dfs | source | hiveQl + + protected lazy val hiveQl: Parser[LogicalPlan] = + restInput ^^ { + case statement => HiveQl.createPlan(statement.trim()) + } + + // Returns the whole input string + protected lazy val wholeInput: Parser[String] = new Parser[String] { + def apply(in: Input) = + Success(in.source.toString, in.drop(in.source.length())) + } + + // Returns the rest of the input string that are not parsed yet + protected lazy val restInput: Parser[String] = new Parser[String] { + def apply(in: Input) = + Success( + in.source.subSequence(in.offset, in.source.length).toString, + in.drop(in.source.length())) + } + + protected lazy val cache: Parser[LogicalPlan] = + CACHE ~> opt(LAZY) ~ (TABLE ~> ident) ~ opt(AS ~> hiveQl) ^^ { + case isLazy ~ tableName ~ plan => + CacheTableCommand(tableName, plan, isLazy.isDefined) + } + + protected lazy val uncache: Parser[LogicalPlan] = + UNCACHE ~ TABLE ~> ident ^^ { + case tableName => UncacheTableCommand(tableName) + } + + protected lazy val addJar: Parser[LogicalPlan] = + ADD ~ JAR ~> restInput ^^ { + case jar => AddJar(jar.trim()) + } + + protected lazy val addFile: Parser[LogicalPlan] = + ADD ~ FILE ~> restInput ^^ { + case file => AddFile(file.trim()) + } + + protected lazy val dfs: Parser[LogicalPlan] = + DFS ~> wholeInput ^^ { + case command => NativeCommand(command.trim()) + } + + protected lazy val source: Parser[LogicalPlan] = + SOURCE ~> restInput ^^ { + case file => SourceCommand(file.trim()) + } +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index e0be09e6793ea..fad3b39f81413 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -231,12 +231,13 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { @transient protected[hive] lazy val sessionState = { val ss = new SessionState(hiveconf) setConf(hiveconf.getAllProperties) // Have SQLConf pick up the initial set of HiveConf. + SessionState.start(ss) + ss.err = new PrintStream(outputBuffer, true, "UTF-8") + ss.out = new PrintStream(outputBuffer, true, "UTF-8") + ss } - sessionState.err = new PrintStream(outputBuffer, true, "UTF-8") - sessionState.out = new PrintStream(outputBuffer, true, "UTF-8") - override def setConf(key: String, value: String): Unit = { super.setConf(key, value) runSqlHive(s"SET $key=$value") @@ -244,15 +245,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { /* A catalyst metadata catalog that points to the Hive Metastore. */ @transient - override protected[sql] lazy val catalog = new HiveMetastoreCatalog(this) with OverrideCatalog { - override def lookupRelation( - databaseName: Option[String], - tableName: String, - alias: Option[String] = None): LogicalPlan = { - - LowerCaseSchema(super.lookupRelation(databaseName, tableName, alias)) - } - } + override protected[sql] lazy val catalog = new HiveMetastoreCatalog(this) with OverrideCatalog // Note that HiveUDFs will be overridden by functions registered in this context. @transient @@ -275,13 +268,12 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { */ protected[sql] def runSqlHive(sql: String): Seq[String] = { val maxResults = 100000 - val results = runHive(sql, 100000) + val results = runHive(sql, maxResults) // It is very confusing when you only get back some of the results... if (results.size == maxResults) sys.error("RESULTS POSSIBLY TRUNCATED") results } - SessionState.start(sessionState) /** * Execute the command using Hive and return the results as a sequence. Each element @@ -289,13 +281,14 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { */ protected def runHive(cmd: String, maxRows: Int = 1000): Seq[String] = { try { + // Session state must be initilized before the CommandProcessor is created . + SessionState.start(sessionState) + val cmd_trimmed: String = cmd.trim() val tokens: Array[String] = cmd_trimmed.split("\\s+") val cmd_1: String = cmd_trimmed.substring(tokens(0).length()).trim() val proc: CommandProcessor = CommandProcessorFactory.get(tokens(0), hiveconf) - SessionState.start(sessionState) - proc match { case driver: Driver => driver.init() @@ -412,7 +405,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { // be similar with Hive. describeHiveTableCommand.hiveString case command: PhysicalCommand => - command.sideEffectResult.map(_.head.toString) + command.executeCollect().map(_.head.toString) case other => val result: Seq[Seq[Any]] = toRdd.collect().toSeq diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala index 943bbaa8ce25e..d633c42c6bd67 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala @@ -137,7 +137,7 @@ private[hive] trait HiveInspectors { /** Converts native catalyst types to the types expected by Hive */ def wrap(a: Any): AnyRef = a match { - case s: String => new hadoopIo.Text(s) // TODO why should be Text? + case s: String => s: java.lang.String case i: Int => i: java.lang.Integer case b: Boolean => b: java.lang.Boolean case f: Float => f: java.lang.Float @@ -145,7 +145,7 @@ private[hive] trait HiveInspectors { case l: Long => l: java.lang.Long case l: Short => l: java.lang.Short case l: Byte => l: java.lang.Byte - case b: BigDecimal => b.bigDecimal + case b: BigDecimal => new HiveDecimal(b.underlying()) case b: Array[Byte] => b case t: java.sql.Timestamp => t case s: Seq[_] => seqAsJavaList(s.map(wrap)) @@ -213,6 +213,8 @@ private[hive] trait HiveInspectors { case _: JavaHiveDecimalObjectInspector => DecimalType case _: WritableTimestampObjectInspector => TimestampType case _: JavaTimestampObjectInspector => TimestampType + case _: WritableVoidObjectInspector => NullType + case _: JavaVoidObjectInspector => NullType } implicit class typeInfoConversions(dt: DataType) { 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 2c0db9be57e54..cc0605b0adb35 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 @@ -96,10 +96,12 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with serDeInfo.setParameters(Map[String, String]()) sd.setSerdeInfo(serDeInfo) - try client.createTable(table) catch { - case e: org.apache.hadoop.hive.ql.metadata.HiveException - if e.getCause.isInstanceOf[org.apache.hadoop.hive.metastore.api.AlreadyExistsException] && - allowExisting => // Do nothing. + synchronized { + try client.createTable(table) catch { + case e: org.apache.hadoop.hive.ql.metadata.HiveException + if e.getCause.isInstanceOf[org.apache.hadoop.hive.metastore.api.AlreadyExistsException] && + allowExisting => // Do nothing. + } } } @@ -129,14 +131,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with // Wait until children are resolved. case p: LogicalPlan if !p.childrenResolved => p - case p @ InsertIntoTable( - LowerCaseSchema(table: MetastoreRelation), _, child, _) => - castChildOutput(p, table, child) - - case p @ logical.InsertIntoTable( - LowerCaseSchema( - InMemoryRelation(_, _, _, - HiveTableScan(_, table, _))), _, child, _) => + case p @ InsertIntoTable(table: MetastoreRelation, _, child, _) => castChildOutput(p, table, child) } @@ -144,7 +139,8 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with val childOutputDataTypes = child.output.map(_.dataType) // Only check attributes, not partitionKeys since they are always strings. // TODO: Fully support inserting into partitioned tables. - val tableOutputDataTypes = table.attributes.map(_.dataType) + val tableOutputDataTypes = + table.attributes.map(_.dataType) ++ table.partitionKeys.map(_.dataType) if (childOutputDataTypes == tableOutputDataTypes) { p @@ -246,6 +242,7 @@ object HiveMetastoreTypes extends RegexParsers { case BooleanType => "boolean" case DecimalType => "decimal" case TimestampType => "timestamp" + case NullType => "void" } } @@ -304,7 +301,7 @@ private[hive] case class MetastoreRelation HiveMetastoreTypes.toDataType(f.getType), // Since data can be dumped in randomly with no validation, everything is nullable. nullable = true - )(qualifiers = tableName +: alias.toSeq) + )(qualifiers = Seq(alias.getOrElse(tableName))) } // Must be a stable value since new attributes are born here. 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 21ecf17028dbc..32c9175f181bb 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 @@ -126,6 +126,9 @@ private[hive] object HiveQl { "TOK_CREATETABLE", "TOK_DESCTABLE" ) ++ nativeCommands + + // It parses hive sql query along with with several Spark SQL specific extensions + protected val hiveSqlParser = new ExtendedHiveQlParser /** * A set of implicit transformations that allow Hive ASTNodes to be rewritten by transformations @@ -215,42 +218,18 @@ private[hive] object HiveQl { def getAst(sql: String): ASTNode = ParseUtils.findRootNonNullToken((new ParseDriver).parse(sql)) /** Returns a LogicalPlan for a given HiveQL string. */ - def parseSql(sql: String): LogicalPlan = { + def parseSql(sql: String): LogicalPlan = hiveSqlParser(sql) + + /** Creates LogicalPlan for a given HiveQL string. */ + def createPlan(sql: String) = { try { - if (sql.trim.toLowerCase.startsWith("set")) { - // Split in two parts since we treat the part before the first "=" - // as key, and the part after as value, which may contain other "=" signs. - sql.trim.drop(3).split("=", 2).map(_.trim) match { - case Array("") => // "set" - SetCommand(None, None) - case Array(key) => // "set key" - SetCommand(Some(key), None) - case Array(key, value) => // "set key=value" - SetCommand(Some(key), Some(value)) - } - } else if (sql.trim.toLowerCase.startsWith("cache table")) { - CacheCommand(sql.trim.drop(12).trim, true) - } else if (sql.trim.toLowerCase.startsWith("uncache table")) { - CacheCommand(sql.trim.drop(14).trim, false) - } else if (sql.trim.toLowerCase.startsWith("add jar")) { - AddJar(sql.trim.drop(8).trim) - } else if (sql.trim.toLowerCase.startsWith("add file")) { - AddFile(sql.trim.drop(9)) - } else if (sql.trim.toLowerCase.startsWith("dfs")) { + val tree = getAst(sql) + if (nativeCommands contains tree.getText) { NativeCommand(sql) - } else if (sql.trim.startsWith("source")) { - SourceCommand(sql.split(" ").toSeq match { case Seq("source", filePath) => filePath }) - } else if (sql.trim.startsWith("!")) { - ShellCommand(sql.drop(1)) } else { - val tree = getAst(sql) - if (nativeCommands contains tree.getText) { - NativeCommand(sql) - } else { - nodeToPlan(tree) match { - case NativePlaceholder => NativeCommand(sql) - case other => other - } + nodeToPlan(tree) match { + case NativePlaceholder => NativeCommand(sql) + case other => other } } } catch { @@ -827,11 +806,6 @@ private[hive] object HiveQl { cleanIdentifier(key.toLowerCase) -> None }.toMap).getOrElse(Map.empty) - if (partitionKeys.values.exists(p => p.isEmpty)) { - throw new NotImplementedError(s"Do not support INSERT INTO/OVERWRITE with" + - s"dynamic partitioning.") - } - InsertIntoTable(UnresolvedRelation(db, tableName, None), partitionKeys, query, overwrite) case a: ASTNode => @@ -845,7 +819,7 @@ private[hive] object HiveQl { case Token("TOK_SELEXPR", e :: Token(alias, Nil) :: Nil) => - Some(Alias(nodeToExpr(e), alias)()) + Some(Alias(nodeToExpr(e), cleanIdentifier(alias))()) /* Hints are ignored */ case Token("TOK_HINTLIST", _) => None 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 43dd3d234f73a..508d8239c7628 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 @@ -23,9 +23,8 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate import org.apache.spark.sql.catalyst.planning._ import org.apache.spark.sql.catalyst.plans._ -import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, LowerCaseSchema} +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.types.StringType -import org.apache.spark.sql.columnar.InMemoryRelation import org.apache.spark.sql.execution.{DescribeCommand, OutputFaker, SparkPlan} import org.apache.spark.sql.hive import org.apache.spark.sql.hive.execution._ @@ -55,7 +54,7 @@ private[hive] trait HiveStrategies { object ParquetConversion extends Strategy { implicit class LogicalPlanHacks(s: SchemaRDD) { def lowerCase = - new SchemaRDD(s.sqlContext, LowerCaseSchema(s.logicalPlan)) + new SchemaRDD(s.sqlContext, s.logicalPlan) def addPartitioningAttributes(attrs: Seq[Attribute]) = new SchemaRDD( @@ -161,10 +160,7 @@ private[hive] trait HiveStrategies { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case logical.InsertIntoTable(table: MetastoreRelation, partition, child, overwrite) => InsertIntoHiveTable(table, partition, planLater(child), overwrite)(hiveContext) :: Nil - case logical.InsertIntoTable( - InMemoryRelation(_, _, _, - HiveTableScan(_, table, _)), partition, child, overwrite) => - InsertIntoHiveTable(table, partition, planLater(child), overwrite)(hiveContext) :: Nil + case logical.CreateTableAsSelect(database, tableName, child) => val query = planLater(child) CreateTableAsSelect( diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala index 70fb15259e7d7..a4354c1379c63 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala @@ -32,7 +32,7 @@ import org.apache.hadoop.hive.serde2.avro.AvroSerDe import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.sql.catalyst.analysis._ -import org.apache.spark.sql.catalyst.plans.logical.{CacheCommand, LogicalPlan, NativeCommand} +import org.apache.spark.sql.catalyst.plans.logical.{CacheTableCommand, LogicalPlan, NativeCommand} import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.hive._ import org.apache.spark.sql.SQLConf @@ -40,8 +40,10 @@ import org.apache.spark.sql.SQLConf /* Implicit conversions */ import scala.collection.JavaConversions._ +// SPARK-3729: Test key required to check for initialization errors with config. object TestHive - extends TestHiveContext(new SparkContext("local[2]", "TestSQLContext", new SparkConf())) + extends TestHiveContext( + new SparkContext("local[2]", "TestSQLContext", new SparkConf().set("spark.sql.test", ""))) /** * A locally running test instance of Spark's Hive execution engine. @@ -65,7 +67,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { lazy val metastorePath = getTempFilePath("sparkHiveMetastore").getCanonicalPath /** Sets up the system initially or after a RESET command */ - protected def configure() { + protected def configure(): Unit = { setConf("javax.jdo.option.ConnectionURL", s"jdbc:derby:;databaseName=$metastorePath;create=true") setConf("hive.metastore.warehouse.dir", warehousePath) @@ -152,7 +154,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { override lazy val analyzed = { val describedTables = logical match { case NativeCommand(describedTable(tbl)) => tbl :: Nil - case CacheCommand(tbl, _) => tbl :: Nil + case CacheTableCommand(tbl, _, _) => tbl :: Nil case _ => Nil } @@ -351,7 +353,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { var cacheTables: Boolean = false def loadTestTable(name: String) { if (!(loadedTables contains name)) { - // Marks the table as loaded first to prevent infite mutually recursive table loading. + // Marks the table as loaded first to prevent infinite mutually recursive table loading. loadedTables += name logInfo(s"Loading test table $name") val createCmds = @@ -381,6 +383,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { log.asInstanceOf[org.apache.log4j.Logger].setLevel(org.apache.log4j.Level.WARN) } + clearCache() loadedTables.clear() catalog.client.getAllTables("default").foreach { t => logDebug(s"Deleting table $t") @@ -426,7 +429,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { loadTestTable("srcpart") } catch { case e: Exception => - logError(s"FATAL ERROR: Failed to reset TestDB state. $e") + logError("FATAL ERROR: Failed to reset TestDB state.", e) // At this point there is really no reason to continue, but the test framework traps exits. // So instead we just pause forever so that at least the developer can see where things // started to go wrong. diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala index 71ea774d77795..3625708d03175 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala @@ -21,7 +21,6 @@ import org.apache.spark.annotation.Experimental import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.expressions.Row import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.catalyst.plans.logical.LowerCaseSchema import org.apache.spark.sql.execution.{SparkPlan, Command, LeafNode} import org.apache.spark.sql.hive.HiveContext import org.apache.spark.sql.hive.MetastoreRelation @@ -31,33 +30,32 @@ import org.apache.spark.sql.hive.MetastoreRelation * Create table and insert the query result into it. * @param database the database name of the new relation * @param tableName the table name of the new relation - * @param insertIntoRelation function of creating the `InsertIntoHiveTable` + * @param insertIntoRelation function of creating the `InsertIntoHiveTable` * by specifying the `MetaStoreRelation`, the data will be inserted into that table. * TODO Add more table creating properties, e.g. SerDe, StorageHandler, in-memory cache etc. */ @Experimental case class CreateTableAsSelect( - database: String, - tableName: String, - query: SparkPlan, - insertIntoRelation: MetastoreRelation => InsertIntoHiveTable) - extends LeafNode with Command { + database: String, + tableName: String, + query: SparkPlan, + insertIntoRelation: MetastoreRelation => InsertIntoHiveTable) + extends LeafNode with Command { def output = Seq.empty // A lazy computing of the metastoreRelation private[this] lazy val metastoreRelation: MetastoreRelation = { - // Create the table + // Create the table val sc = sqlContext.asInstanceOf[HiveContext] sc.catalog.createTable(database, tableName, query.output, false) // Get the Metastore Relation sc.catalog.lookupRelation(Some(database), tableName, None) match { - case LowerCaseSchema(r: MetastoreRelation) => r - case o: MetastoreRelation => o + case r: MetastoreRelation => r } } - override protected[sql] lazy val sideEffectResult: Seq[Row] = { + override protected lazy val sideEffectResult: Seq[Row] = { insertIntoRelation(metastoreRelation).execute Seq.empty[Row] } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala index 317801001c7a4..106cede9788ec 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala @@ -48,7 +48,7 @@ case class DescribeHiveTableCommand( .mkString("\t") } - override protected[sql] lazy val sideEffectResult: Seq[Row] = { + override protected lazy val sideEffectResult: Seq[Row] = { // Trying to mimic the format of Hive's output. But not exactly the same. var results: Seq[(String, String, String)] = Nil diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index a284a91a91e31..f8b4e898ec41d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -19,27 +19,25 @@ package org.apache.spark.sql.hive.execution import scala.collection.JavaConversions._ -import java.util.{HashMap => JHashMap} - import org.apache.hadoop.hive.common.`type`.{HiveDecimal, HiveVarchar} +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.apache.hadoop.hive.metastore.MetaStoreUtils -import org.apache.hadoop.hive.ql.Context import org.apache.hadoop.hive.ql.metadata.Hive import org.apache.hadoop.hive.ql.plan.{FileSinkDesc, TableDesc} +import org.apache.hadoop.hive.ql.{Context, ErrorMsg} import org.apache.hadoop.hive.serde2.Serializer -import org.apache.hadoop.hive.serde2.objectinspector._ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption -import org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaHiveDecimalObjectInspector -import org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaHiveVarcharObjectInspector -import org.apache.hadoop.io.Writable +import org.apache.hadoop.hive.serde2.objectinspector._ +import org.apache.hadoop.hive.serde2.objectinspector.primitive.{JavaHiveDecimalObjectInspector, JavaHiveVarcharObjectInspector} import org.apache.hadoop.mapred.{FileOutputCommitter, FileOutputFormat, JobConf} -import org.apache.spark.{SparkException, TaskContext} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.expressions.Row -import org.apache.spark.sql.execution.{SparkPlan, UnaryNode} -import org.apache.spark.sql.hive.{HiveContext, MetastoreRelation, SparkHiveHadoopWriter} +import org.apache.spark.sql.execution.{Command, SparkPlan, UnaryNode} +import org.apache.spark.sql.hive._ +import org.apache.spark.{SerializableWritable, SparkException, TaskContext} /** * :: DeveloperApi :: @@ -51,7 +49,7 @@ case class InsertIntoHiveTable( child: SparkPlan, overwrite: Boolean) (@transient sc: HiveContext) - extends UnaryNode { + extends UnaryNode with Command { @transient lazy val outputClass = newSerializer(table.tableDesc).getSerializedClass @transient private lazy val hiveContext = new Context(sc.hiveconf) @@ -101,66 +99,61 @@ case class InsertIntoHiveTable( } def saveAsHiveFile( - rdd: RDD[Writable], + rdd: RDD[Row], valueClass: Class[_], fileSinkConf: FileSinkDesc, - conf: JobConf, - isCompressed: Boolean) { - if (valueClass == null) { - throw new SparkException("Output value class not set") - } - conf.setOutputValueClass(valueClass) - if (fileSinkConf.getTableInfo.getOutputFileFormatClassName == null) { - throw new SparkException("Output format class not set") - } - // Doesn't work in Scala 2.9 due to what may be a generics bug - // TODO: Should we uncomment this for Scala 2.10? - // conf.setOutputFormat(outputFormatClass) - conf.set("mapred.output.format.class", fileSinkConf.getTableInfo.getOutputFileFormatClassName) - if (isCompressed) { - // Please note that isCompressed, "mapred.output.compress", "mapred.output.compression.codec", - // and "mapred.output.compression.type" have no impact on ORC because it uses table properties - // to store compression information. - conf.set("mapred.output.compress", "true") - fileSinkConf.setCompressed(true) - fileSinkConf.setCompressCodec(conf.get("mapred.output.compression.codec")) - fileSinkConf.setCompressType(conf.get("mapred.output.compression.type")) - } - conf.setOutputCommitter(classOf[FileOutputCommitter]) - FileOutputFormat.setOutputPath( - conf, - SparkHiveHadoopWriter.createPathFromString(fileSinkConf.getDirName, conf)) + conf: SerializableWritable[JobConf], + writerContainer: SparkHiveWriterContainer) { + assert(valueClass != null, "Output value class not set") + conf.value.setOutputValueClass(valueClass) + + val outputFileFormatClassName = fileSinkConf.getTableInfo.getOutputFileFormatClassName + assert(outputFileFormatClassName != null, "Output format class not set") + conf.value.set("mapred.output.format.class", outputFileFormatClassName) + conf.value.setOutputCommitter(classOf[FileOutputCommitter]) + FileOutputFormat.setOutputPath( + conf.value, + SparkHiveWriterContainer.createPathFromString(fileSinkConf.getDirName, conf.value)) log.debug("Saving as hadoop file of type " + valueClass.getSimpleName) - val writer = new SparkHiveHadoopWriter(conf, fileSinkConf) - writer.preSetup() + writerContainer.driverSideSetup() + sc.sparkContext.runJob(rdd, writeToFile _) + writerContainer.commitJob() + + // Note that this function is executed on executor side + def writeToFile(context: TaskContext, iterator: Iterator[Row]) { + val serializer = newSerializer(fileSinkConf.getTableInfo) + val standardOI = ObjectInspectorUtils + .getStandardObjectInspector( + fileSinkConf.getTableInfo.getDeserializer.getObjectInspector, + ObjectInspectorCopyOption.JAVA) + .asInstanceOf[StructObjectInspector] + + val fieldOIs = standardOI.getAllStructFieldRefs.map(_.getFieldObjectInspector).toArray + val outputData = new Array[Any](fieldOIs.length) - def writeToFile(context: TaskContext, iter: Iterator[Writable]) { // Hadoop wants a 32-bit task attempt ID, so if ours is bigger than Int.MaxValue, roll it // around by taking a mod. We expect that no task will be attempted 2 billion times. val attemptNumber = (context.attemptId % Int.MaxValue).toInt + writerContainer.executorSideSetup(context.stageId, context.partitionId, attemptNumber) - writer.setup(context.stageId, context.partitionId, attemptNumber) - writer.open() + iterator.foreach { row => + var i = 0 + while (i < fieldOIs.length) { + // TODO (lian) avoid per row dynamic dispatching and pattern matching cost in `wrap` + outputData(i) = wrap(row(i), fieldOIs(i)) + i += 1 + } - var count = 0 - while(iter.hasNext) { - val record = iter.next() - count += 1 - writer.write(record) + val writer = writerContainer.getLocalFileWriter(row) + writer.write(serializer.serialize(outputData, standardOI)) } - writer.close() - writer.commit() + writerContainer.close() } - - sc.sparkContext.runJob(rdd, writeToFile _) - writer.commitJob() } - override def execute() = result - /** * Inserts all the rows in the table into Hive. Row objects are properly serialized with the * `org.apache.hadoop.hive.serde2.SerDe` and the @@ -168,50 +161,69 @@ case class InsertIntoHiveTable( * * Note: this is run once and then kept to avoid double insertions. */ - private lazy val result: RDD[Row] = { - val childRdd = child.execute() - assert(childRdd != null) - + override protected[sql] lazy val sideEffectResult: Seq[Row] = { // Have to pass the TableDesc object to RDD.mapPartitions and then instantiate new serializer // instances within the closure, since Serializer is not serializable while TableDesc is. val tableDesc = table.tableDesc val tableLocation = table.hiveQlTable.getDataLocation val tmpLocation = hiveContext.getExternalTmpFileURI(tableLocation) val fileSinkConf = new FileSinkDesc(tmpLocation.toString, tableDesc, false) - val rdd = childRdd.mapPartitions { iter => - val serializer = newSerializer(fileSinkConf.getTableInfo) - val standardOI = ObjectInspectorUtils - .getStandardObjectInspector( - fileSinkConf.getTableInfo.getDeserializer.getObjectInspector, - ObjectInspectorCopyOption.JAVA) - .asInstanceOf[StructObjectInspector] + val isCompressed = sc.hiveconf.getBoolean( + ConfVars.COMPRESSRESULT.varname, ConfVars.COMPRESSRESULT.defaultBoolVal) + if (isCompressed) { + // Please note that isCompressed, "mapred.output.compress", "mapred.output.compression.codec", + // and "mapred.output.compression.type" have no impact on ORC because it uses table properties + // to store compression information. + sc.hiveconf.set("mapred.output.compress", "true") + fileSinkConf.setCompressed(true) + fileSinkConf.setCompressCodec(sc.hiveconf.get("mapred.output.compression.codec")) + fileSinkConf.setCompressType(sc.hiveconf.get("mapred.output.compression.type")) + } - val fieldOIs = standardOI.getAllStructFieldRefs.map(_.getFieldObjectInspector).toArray - val outputData = new Array[Any](fieldOIs.length) - iter.map { row => - var i = 0 - while (i < row.length) { - // Casts Strings to HiveVarchars when necessary. - outputData(i) = wrap(row(i), fieldOIs(i)) - i += 1 - } + val numDynamicPartitions = partition.values.count(_.isEmpty) + val numStaticPartitions = partition.values.count(_.nonEmpty) + val partitionSpec = partition.map { + case (key, Some(value)) => key -> value + case (key, None) => key -> "" + } + + // All partition column names in the format of "//..." + val partitionColumns = fileSinkConf.getTableInfo.getProperties.getProperty("partition_columns") + val partitionColumnNames = Option(partitionColumns).map(_.split("/")).orNull - serializer.serialize(outputData, standardOI) + // Validate partition spec if there exist any dynamic partitions + if (numDynamicPartitions > 0) { + // Report error if dynamic partitioning is not enabled + if (!sc.hiveconf.getBoolVar(HiveConf.ConfVars.DYNAMICPARTITIONING)) { + throw new SparkException(ErrorMsg.DYNAMIC_PARTITION_DISABLED.getMsg) + } + + // Report error if dynamic partition strict mode is on but no static partition is found + if (numStaticPartitions == 0 && + sc.hiveconf.getVar(HiveConf.ConfVars.DYNAMICPARTITIONINGMODE).equalsIgnoreCase("strict")) { + throw new SparkException(ErrorMsg.DYNAMIC_PARTITION_STRICT_MODE.getMsg) + } + + // Report error if any static partition appears after a dynamic partition + val isDynamic = partitionColumnNames.map(partitionSpec(_).isEmpty) + isDynamic.init.zip(isDynamic.tail).find(_ == (true, false)).foreach { _ => + throw new SparkException(ErrorMsg.PARTITION_DYN_STA_ORDER.getMsg) } } - // ORC stores compression information in table properties. While, there are other formats - // (e.g. RCFile) that rely on hadoop configurations to store compression information. val jobConf = new JobConf(sc.hiveconf) - saveAsHiveFile( - rdd, - outputClass, - fileSinkConf, - jobConf, - sc.hiveconf.getBoolean("hive.exec.compress.output", false)) - - // TODO: Handle dynamic partitioning. + val jobConfSer = new SerializableWritable(jobConf) + + val writerContainer = if (numDynamicPartitions > 0) { + val dynamicPartColNames = partitionColumnNames.takeRight(numDynamicPartitions) + new SparkHiveDynamicPartitionWriterContainer(jobConf, fileSinkConf, dynamicPartColNames) + } else { + new SparkHiveWriterContainer(jobConf, fileSinkConf) + } + + saveAsHiveFile(child.execute(), outputClass, fileSinkConf, jobConfSer, writerContainer) + val outputPath = FileOutputFormat.getOutputPath(jobConf) // Have to construct the format of dbname.tablename. val qualifiedTableName = s"${table.databaseName}.${table.tableName}" @@ -220,10 +232,6 @@ case class InsertIntoHiveTable( // holdDDLTime will be true when TOK_HOLD_DDLTIME presents in the query as a hint. val holdDDLTime = false if (partition.nonEmpty) { - val partitionSpec = partition.map { - case (key, Some(value)) => key -> value - case (key, None) => key -> "" // Should not reach here right now. - } val partVals = MetaStoreUtils.getPvals(table.hiveQlTable.getPartCols, partitionSpec) db.validatePartitionNameCharacters(partVals) // inheritTableSpecs is set to true. It should be set to false for a IMPORT query @@ -231,14 +239,26 @@ case class InsertIntoHiveTable( val inheritTableSpecs = true // TODO: Correctly set isSkewedStoreAsSubdir. val isSkewedStoreAsSubdir = false - db.loadPartition( - outputPath, - qualifiedTableName, - partitionSpec, - overwrite, - holdDDLTime, - inheritTableSpecs, - isSkewedStoreAsSubdir) + if (numDynamicPartitions > 0) { + db.loadDynamicPartitions( + outputPath, + qualifiedTableName, + partitionSpec, + overwrite, + numDynamicPartitions, + holdDDLTime, + isSkewedStoreAsSubdir + ) + } else { + db.loadPartition( + outputPath, + qualifiedTableName, + partitionSpec, + overwrite, + holdDDLTime, + inheritTableSpecs, + isSkewedStoreAsSubdir) + } } else { db.loadTable( outputPath, @@ -247,10 +267,13 @@ case class InsertIntoHiveTable( holdDDLTime) } + // Invalidate the cache. + sqlContext.invalidateCache(table) + // It would be nice to just return the childRdd unchanged so insert operations could be chained, // however for now we return an empty list to simplify compatibility checks with hive, which // does not return anything for insert operations. // TODO: implement hive compatibility as rules. - sc.sparkContext.makeRDD(Nil, 1) + Seq.empty[Row] } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/NativeCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/NativeCommand.scala index 8f10e1ba7f426..6930c2babd117 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/NativeCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/NativeCommand.scala @@ -32,7 +32,7 @@ case class NativeCommand( @transient context: HiveContext) extends LeafNode with Command { - override protected[sql] lazy val sideEffectResult: Seq[Row] = context.runSqlHive(sql).map(Row(_)) + override protected lazy val sideEffectResult: Seq[Row] = context.runSqlHive(sql).map(Row(_)) override def otherCopyArgs = context :: Nil } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala index d61c5e274a596..0fc674af31885 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala @@ -37,7 +37,7 @@ case class AnalyzeTable(tableName: String) extends LeafNode with Command { def output = Seq.empty - override protected[sql] lazy val sideEffectResult: Seq[Row] = { + override protected lazy val sideEffectResult: Seq[Row] = { hiveContext.analyze(tableName) Seq.empty[Row] } @@ -53,7 +53,7 @@ case class DropTable(tableName: String, ifExists: Boolean) extends LeafNode with def output = Seq.empty - override protected[sql] lazy val sideEffectResult: Seq[Row] = { + override protected lazy val sideEffectResult: Seq[Row] = { val ifExistsClause = if (ifExists) "IF EXISTS " else "" hiveContext.runSqlHive(s"DROP TABLE $ifExistsClause$tableName") hiveContext.catalog.unregisterTable(None, tableName) @@ -70,7 +70,7 @@ case class AddJar(path: String) extends LeafNode with Command { override def output = Seq.empty - override protected[sql] lazy val sideEffectResult: Seq[Row] = { + override protected lazy val sideEffectResult: Seq[Row] = { hiveContext.runSqlHive(s"ADD JAR $path") hiveContext.sparkContext.addJar(path) Seq.empty[Row] diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala index 7d1ad53d8bdb3..68f93f247d9bb 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 @@ -17,10 +17,12 @@ package org.apache.spark.sql.hive +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFUtils.ConversionHelper + 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.{UDF, UDAF} 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._ @@ -42,30 +44,21 @@ private[hive] abstract class HiveFunctionRegistry def lookupFunction(name: String, children: Seq[Expression]): Expression = { // We only look it up to see if it exists, but do not include it in the HiveUDF since it is // not always serializable. - val functionInfo: FunctionInfo = Option(FunctionRegistry.getFunctionInfo(name)).getOrElse( - sys.error(s"Couldn't find function $name")) + val functionInfo: FunctionInfo = + Option(FunctionRegistry.getFunctionInfo(name.toLowerCase)).getOrElse( + sys.error(s"Couldn't find function $name")) - val functionClassName = functionInfo.getFunctionClass.getName() + val functionClassName = functionInfo.getFunctionClass.getName if (classOf[UDF].isAssignableFrom(functionInfo.getFunctionClass)) { - val function = functionInfo.getFunctionClass.newInstance().asInstanceOf[UDF] - val method = function.getResolver.getEvalMethod(children.map(_.dataType.toTypeInfo)) - - lazy val expectedDataTypes = method.getParameterTypes.map(javaClassToDataType) - - HiveSimpleUdf( - functionClassName, - children.zip(expectedDataTypes).map { - case (e, NullType) => e - case (e, t) => Cast(e, t) - } - ) + HiveSimpleUdf(functionClassName, children) } else if (classOf[GenericUDF].isAssignableFrom(functionInfo.getFunctionClass)) { HiveGenericUdf(functionClassName, children) } else if ( classOf[AbstractGenericUDAFResolver].isAssignableFrom(functionInfo.getFunctionClass)) { HiveGenericUdaf(functionClassName, children) - + } else if (classOf[UDAF].isAssignableFrom(functionInfo.getFunctionClass)) { + HiveUdaf(functionClassName, children) } else if (classOf[GenericUDTF].isAssignableFrom(functionInfo.getFunctionClass)) { HiveGenericUdtf(functionClassName, Nil, children) } else { @@ -104,52 +97,21 @@ private[hive] case class HiveSimpleUdf(functionClassName: String, children: Seq[ function.getResolver.getEvalMethod(children.map(_.dataType.toTypeInfo)) @transient - lazy val dataType = javaClassToDataType(method.getReturnType) + protected lazy val arguments = children.map(c => toInspector(c.dataType)).toArray - protected lazy val wrappers: Array[(Any) => AnyRef] = method.getParameterTypes.map { argClass => - val primitiveClasses = Seq( - Integer.TYPE, classOf[java.lang.Integer], classOf[java.lang.String], java.lang.Double.TYPE, - classOf[java.lang.Double], java.lang.Long.TYPE, classOf[java.lang.Long], - classOf[HiveDecimal], java.lang.Byte.TYPE, classOf[java.lang.Byte], - classOf[java.sql.Timestamp] - ) - val matchingConstructor = argClass.getConstructors.find { c => - c.getParameterTypes.size == 1 && primitiveClasses.contains(c.getParameterTypes.head) - } + // Create parameter converters + @transient + protected lazy val conversionHelper = new ConversionHelper(method, arguments) - matchingConstructor match { - case Some(constructor) => - (a: Any) => { - logDebug( - s"Wrapping $a of type ${if (a == null) "null" else a.getClass.getName} $constructor.") - // We must make sure that primitives get boxed java style. - if (a == null) { - null - } else { - constructor.newInstance(a match { - case i: Int => i: java.lang.Integer - case bd: BigDecimal => new HiveDecimal(bd.underlying()) - case other: AnyRef => other - }).asInstanceOf[AnyRef] - } - } - case None => - (a: Any) => a match { - case wrapper => wrap(wrapper) - } - } - } + @transient + lazy val dataType = javaClassToDataType(method.getReturnType) // TODO: Finish input output types. override def eval(input: Row): Any = { - val evaluatedChildren = children.map(_.eval(input)) - // Wrap the function arguments in the expected types. - val args = evaluatedChildren.zip(wrappers).map { - case (arg, wrapper) => wrapper(arg) - } + val evaluatedChildren = children.map(c => wrap(c.eval(input))) - // Invoke the udf and unwrap the result. - unwrap(method.invoke(function, args: _*)) + unwrap(FunctionRegistry.invoke(method, function, conversionHelper + .convertIfNecessary(evaluatedChildren: _*): _*)) } } @@ -190,7 +152,7 @@ private[hive] case class HiveGenericUdf(functionClassName: String, children: Seq override def get(): AnyRef = wrap(func()) } - val dataType: DataType = inspectorToDataType(returnInspector) + lazy val dataType: DataType = inspectorToDataType(returnInspector) override def eval(input: Row): Any = { returnInspector // Make sure initialized. @@ -233,6 +195,37 @@ private[hive] case class HiveGenericUdaf( def newInstance() = new HiveUdafFunction(functionClassName, children, this) } +/** It is used as a wrapper for the hive functions which uses UDAF interface */ +private[hive] case class HiveUdaf( + functionClassName: String, + children: Seq[Expression]) extends AggregateExpression + with HiveInspectors + with HiveFunctionFactory { + + type UDFType = UDAF + + @transient + protected lazy val resolver: AbstractGenericUDAFResolver = new GenericUDAFBridge(createFunction()) + + @transient + protected lazy val objectInspector = { + resolver.getEvaluator(children.map(_.dataType.toTypeInfo).toArray) + .init(GenericUDAFEvaluator.Mode.COMPLETE, inspectors.toArray) + } + + @transient + protected lazy val inspectors = children.map(_.dataType).map(toInspector) + + def dataType: DataType = inspectorToDataType(objectInspector) + + def nullable: Boolean = true + + override def toString = s"$nodeName#$functionClassName(${children.mkString(",")})" + + def newInstance() = + new HiveUdafFunction(functionClassName, children, this, true) +} + /** * Converts a Hive Generic User Defined Table Generating Function (UDTF) to a * [[catalyst.expressions.Generator Generator]]. Note that the semantics of Generators do not allow @@ -314,14 +307,20 @@ private[hive] case class HiveGenericUdtf( private[hive] case class HiveUdafFunction( functionClassName: String, exprs: Seq[Expression], - base: AggregateExpression) + base: AggregateExpression, + isUDAFBridgeRequired: Boolean = false) extends AggregateFunction with HiveInspectors with HiveFunctionFactory { def this() = this(null, null, null) - private val resolver = createFunction[AbstractGenericUDAFResolver]() + private val resolver = + if (isUDAFBridgeRequired) { + new GenericUDAFBridge(createFunction[UDAF]()) + } else { + createFunction[AbstractGenericUDAFResolver]() + } private val inspectors = exprs.map(_.dataType).map(toInspector).toArray diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala new file mode 100644 index 0000000000000..6ccbc22a4acfb --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala @@ -0,0 +1,235 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive + +import java.io.IOException +import java.text.NumberFormat +import java.util.Date + +import scala.collection.mutable + +import org.apache.hadoop.fs.Path +import org.apache.hadoop.hive.conf.HiveConf.ConfVars +import org.apache.hadoop.hive.ql.exec.{FileSinkOperator, Utilities} +import org.apache.hadoop.hive.ql.io.{HiveFileFormatUtils, HiveOutputFormat} +import org.apache.hadoop.hive.ql.plan.FileSinkDesc +import org.apache.hadoop.io.Writable +import org.apache.hadoop.mapred._ + +import org.apache.spark.sql.Row +import org.apache.spark.{Logging, SerializableWritable, SparkHadoopWriter} + +/** + * Internal helper class that saves an RDD using a Hive OutputFormat. + * It is based on [[SparkHadoopWriter]]. + */ +private[hive] class SparkHiveWriterContainer( + @transient jobConf: JobConf, + fileSinkConf: FileSinkDesc) + extends Logging + with SparkHadoopMapRedUtil + with Serializable { + + private val now = new Date() + protected val conf = new SerializableWritable(jobConf) + + private var jobID = 0 + private var splitID = 0 + private var attemptID = 0 + private var jID: SerializableWritable[JobID] = null + private var taID: SerializableWritable[TaskAttemptID] = null + + @transient private var writer: FileSinkOperator.RecordWriter = null + @transient protected lazy val committer = conf.value.getOutputCommitter + @transient protected lazy val jobContext = newJobContext(conf.value, jID.value) + @transient private lazy val taskContext = newTaskAttemptContext(conf.value, taID.value) + @transient private lazy val outputFormat = + conf.value.getOutputFormat.asInstanceOf[HiveOutputFormat[AnyRef,Writable]] + + def driverSideSetup() { + setIDs(0, 0, 0) + setConfParams() + committer.setupJob(jobContext) + } + + def executorSideSetup(jobId: Int, splitId: Int, attemptId: Int) { + setIDs(jobId, splitId, attemptId) + setConfParams() + committer.setupTask(taskContext) + initWriters() + } + + protected def getOutputName: String = { + val numberFormat = NumberFormat.getInstance() + numberFormat.setMinimumIntegerDigits(5) + numberFormat.setGroupingUsed(false) + val extension = Utilities.getFileExtension(conf.value, fileSinkConf.getCompressed, outputFormat) + "part-" + numberFormat.format(splitID) + extension + } + + def getLocalFileWriter(row: Row): FileSinkOperator.RecordWriter = writer + + def close() { + // Seems the boolean value passed into close does not matter. + writer.close(false) + commit() + } + + def commitJob() { + committer.commitJob(jobContext) + } + + protected def initWriters() { + // NOTE this method is executed at the executor side. + // For Hive tables without partitions or with only static partitions, only 1 writer is needed. + writer = HiveFileFormatUtils.getHiveRecordWriter( + conf.value, + fileSinkConf.getTableInfo, + conf.value.getOutputValueClass.asInstanceOf[Class[Writable]], + fileSinkConf, + FileOutputFormat.getTaskOutputPath(conf.value, getOutputName), + Reporter.NULL) + } + + protected def commit() { + if (committer.needsTaskCommit(taskContext)) { + try { + committer.commitTask(taskContext) + logInfo (taID + ": Committed") + } catch { + case e: IOException => + logError("Error committing the output of task: " + taID.value, e) + committer.abortTask(taskContext) + throw e + } + } else { + logInfo("No need to commit output of task: " + taID.value) + } + } + + private def setIDs(jobId: Int, splitId: Int, attemptId: Int) { + jobID = jobId + splitID = splitId + attemptID = attemptId + + jID = new SerializableWritable[JobID](SparkHadoopWriter.createJobID(now, jobId)) + taID = new SerializableWritable[TaskAttemptID]( + new TaskAttemptID(new TaskID(jID.value, true, splitID), attemptID)) + } + + private def setConfParams() { + conf.value.set("mapred.job.id", jID.value.toString) + conf.value.set("mapred.tip.id", taID.value.getTaskID.toString) + conf.value.set("mapred.task.id", taID.value.toString) + conf.value.setBoolean("mapred.task.is.map", true) + conf.value.setInt("mapred.task.partition", splitID) + } +} + +private[hive] object SparkHiveWriterContainer { + def createPathFromString(path: String, conf: JobConf): Path = { + if (path == null) { + throw new IllegalArgumentException("Output path is null") + } + val outputPath = new Path(path) + val fs = outputPath.getFileSystem(conf) + if (outputPath == null || fs == null) { + throw new IllegalArgumentException("Incorrectly formatted output path") + } + outputPath.makeQualified(fs.getUri, fs.getWorkingDirectory) + } +} + +private[spark] object SparkHiveDynamicPartitionWriterContainer { + val SUCCESSFUL_JOB_OUTPUT_DIR_MARKER = "mapreduce.fileoutputcommitter.marksuccessfuljobs" +} + +private[spark] class SparkHiveDynamicPartitionWriterContainer( + @transient jobConf: JobConf, + fileSinkConf: FileSinkDesc, + dynamicPartColNames: Array[String]) + extends SparkHiveWriterContainer(jobConf, fileSinkConf) { + + import SparkHiveDynamicPartitionWriterContainer._ + + private val defaultPartName = jobConf.get( + ConfVars.DEFAULTPARTITIONNAME.varname, ConfVars.DEFAULTPARTITIONNAME.defaultVal) + + @transient private var writers: mutable.HashMap[String, FileSinkOperator.RecordWriter] = _ + + override protected def initWriters(): Unit = { + // NOTE: This method is executed at the executor side. + // Actual writers are created for each dynamic partition on the fly. + writers = mutable.HashMap.empty[String, FileSinkOperator.RecordWriter] + } + + override def close(): Unit = { + writers.values.foreach(_.close(false)) + commit() + } + + override def commitJob(): Unit = { + // This is a hack to avoid writing _SUCCESS mark file. In lower versions of Hadoop (e.g. 1.0.4), + // semantics of FileSystem.globStatus() is different from higher versions (e.g. 2.4.1) and will + // include _SUCCESS file when glob'ing for dynamic partition data files. + // + // Better solution is to add a step similar to what Hive FileSinkOperator.jobCloseOp does: + // calling something like Utilities.mvFileToFinalPath to cleanup the output directory and then + // load it with loadDynamicPartitions/loadPartition/loadTable. + val oldMarker = jobConf.getBoolean(SUCCESSFUL_JOB_OUTPUT_DIR_MARKER, true) + jobConf.setBoolean(SUCCESSFUL_JOB_OUTPUT_DIR_MARKER, false) + super.commitJob() + jobConf.setBoolean(SUCCESSFUL_JOB_OUTPUT_DIR_MARKER, oldMarker) + } + + override def getLocalFileWriter(row: Row): FileSinkOperator.RecordWriter = { + val dynamicPartPath = dynamicPartColNames + .zip(row.takeRight(dynamicPartColNames.length)) + .map { case (col, rawVal) => + val string = if (rawVal == null) null else String.valueOf(rawVal) + s"/$col=${if (string == null || string.isEmpty) defaultPartName else string}" + } + .mkString + + def newWriter = { + val newFileSinkDesc = new FileSinkDesc( + fileSinkConf.getDirName + dynamicPartPath, + fileSinkConf.getTableInfo, + fileSinkConf.getCompressed) + newFileSinkDesc.setCompressCodec(fileSinkConf.getCompressCodec) + newFileSinkDesc.setCompressType(fileSinkConf.getCompressType) + + val path = { + val outputPath = FileOutputFormat.getOutputPath(conf.value) + assert(outputPath != null, "Undefined job output-path") + val workPath = new Path(outputPath, dynamicPartPath.stripPrefix("/")) + new Path(workPath, getOutputName) + } + + HiveFileFormatUtils.getHiveRecordWriter( + conf.value, + fileSinkConf.getTableInfo, + conf.value.getOutputValueClass.asInstanceOf[Class[Writable]], + newFileSinkDesc, + path, + Reporter.NULL) + } + + writers.getOrElseUpdate(dynamicPartPath, newWriter) + } +} diff --git a/sql/hive/src/test/resources/golden/Cast Timestamp to Timestamp in UDF-0-db6d4503454e4dbb9edcbab9a8718d7f b/sql/hive/src/test/resources/golden/Cast Timestamp to Timestamp in UDF-0-db6d4503454e4dbb9edcbab9a8718d7f new file mode 100644 index 0000000000000..7951defec192a --- /dev/null +++ b/sql/hive/src/test/resources/golden/Cast Timestamp to Timestamp in UDF-0-db6d4503454e4dbb9edcbab9a8718d7f @@ -0,0 +1 @@ +NULL diff --git a/sql/hive/src/test/resources/golden/database.table table.attr case insensitive-0-98b2e34c9134208e9fe7c62d33010005 b/sql/hive/src/test/resources/golden/database.table table.attr case insensitive-0-98b2e34c9134208e9fe7c62d33010005 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/database.table table.attr case insensitive-0-98b2e34c9134208e9fe7c62d33010005 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/division-0-63b19f8a22471c8ba0415c1d3bc276f7 b/sql/hive/src/test/resources/golden/division-0-63b19f8a22471c8ba0415c1d3bc276f7 deleted file mode 100644 index 7b7a9175114ce..0000000000000 --- a/sql/hive/src/test/resources/golden/division-0-63b19f8a22471c8ba0415c1d3bc276f7 +++ /dev/null @@ -1 +0,0 @@ -2.0 0.5 0.3333333333333333 0.002 diff --git a/sql/hive/src/test/resources/golden/dynamic_partition-0-be33aaa7253c8f248ff3921cd7dae340 b/sql/hive/src/test/resources/golden/dynamic_partition-0-be33aaa7253c8f248ff3921cd7dae340 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/dynamic_partition-1-640552dd462707563fd255a713f83b41 b/sql/hive/src/test/resources/golden/dynamic_partition-1-640552dd462707563fd255a713f83b41 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/dynamic_partition-2-36456c9d0d2e3ef72ab5ba9ba48e5493 b/sql/hive/src/test/resources/golden/dynamic_partition-2-36456c9d0d2e3ef72ab5ba9ba48e5493 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/dynamic_partition-2-36456c9d0d2e3ef72ab5ba9ba48e5493 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/dynamic_partition-3-b7f7fa7ebf666f4fee27e149d8c6961f b/sql/hive/src/test/resources/golden/dynamic_partition-3-b7f7fa7ebf666f4fee27e149d8c6961f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/dynamic_partition-4-8bdb71ad8cb3cc3026043def2525de3a b/sql/hive/src/test/resources/golden/dynamic_partition-4-8bdb71ad8cb3cc3026043def2525de3a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/dynamic_partition-5-c630dce438f3792e7fb0f523fbbb3e1e b/sql/hive/src/test/resources/golden/dynamic_partition-5-c630dce438f3792e7fb0f523fbbb3e1e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/dynamic_partition-6-7abc9ec8a36cdc5e89e955265a7fd7cf b/sql/hive/src/test/resources/golden/dynamic_partition-6-7abc9ec8a36cdc5e89e955265a7fd7cf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/dynamic_partition-7-be33aaa7253c8f248ff3921cd7dae340 b/sql/hive/src/test/resources/golden/dynamic_partition-7-be33aaa7253c8f248ff3921cd7dae340 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/modulus-0-6afd4a359a478cfa3ebd9ad00ae3868e b/sql/hive/src/test/resources/golden/modulus-0-6afd4a359a478cfa3ebd9ad00ae3868e new file mode 100644 index 0000000000000..52eab0653c505 --- /dev/null +++ b/sql/hive/src/test/resources/golden/modulus-0-6afd4a359a478cfa3ebd9ad00ae3868e @@ -0,0 +1 @@ +1 true 0.5 diff --git a/sql/hive/src/test/resources/golden/select null from table-0-5bb53cca754cc8afe9cd22feb8c586d1 b/sql/hive/src/test/resources/golden/select null from table-0-5bb53cca754cc8afe9cd22feb8c586d1 new file mode 100644 index 0000000000000..7951defec192a --- /dev/null +++ b/sql/hive/src/test/resources/golden/select null from table-0-5bb53cca754cc8afe9cd22feb8c586d1 @@ -0,0 +1 @@ +NULL diff --git a/sql/hive/src/test/resources/golden/timestamp cast #2-0-732ed232ac592c5e7f7c913a88874fd2 b/sql/hive/src/test/resources/golden/timestamp cast #2-0-732ed232ac592c5e7f7c913a88874fd2 new file mode 100644 index 0000000000000..5625e59da8873 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp cast #2-0-732ed232ac592c5e7f7c913a88874fd2 @@ -0,0 +1 @@ +1.2 diff --git a/sql/hive/src/test/resources/golden/timestamp cast #3-0-76ee270337f664b36cacfc6528ac109 b/sql/hive/src/test/resources/golden/timestamp cast #3-0-76ee270337f664b36cacfc6528ac109 new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp cast #3-0-76ee270337f664b36cacfc6528ac109 @@ -0,0 +1 @@ +1 diff --git a/sql/hive/src/test/resources/golden/timestamp cast #4-0-732ed232ac592c5e7f7c913a88874fd2 b/sql/hive/src/test/resources/golden/timestamp cast #4-0-732ed232ac592c5e7f7c913a88874fd2 new file mode 100644 index 0000000000000..5625e59da8873 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp cast #4-0-732ed232ac592c5e7f7c913a88874fd2 @@ -0,0 +1 @@ +1.2 diff --git a/sql/hive/src/test/resources/golden/timestamp cast #5-0-dbd7bcd167d322d6617b884c02c7f247 b/sql/hive/src/test/resources/golden/timestamp cast #5-0-dbd7bcd167d322d6617b884c02c7f247 new file mode 100644 index 0000000000000..27de46fdf22ac --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp cast #5-0-dbd7bcd167d322d6617b884c02c7f247 @@ -0,0 +1 @@ +-0.0010000000000000009 diff --git a/sql/hive/src/test/resources/golden/timestamp cast #6-0-6d2da5cfada03605834e38bc4075bc79 b/sql/hive/src/test/resources/golden/timestamp cast #6-0-6d2da5cfada03605834e38bc4075bc79 new file mode 100644 index 0000000000000..1d94c8a014fb4 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp cast #6-0-6d2da5cfada03605834e38bc4075bc79 @@ -0,0 +1 @@ +-1.2 diff --git a/sql/hive/src/test/resources/golden/timestamp cast #7-0-1d70654217035f8ce5f64344f4c5a80f b/sql/hive/src/test/resources/golden/timestamp cast #7-0-1d70654217035f8ce5f64344f4c5a80f new file mode 100644 index 0000000000000..3fbedf693b51d --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp cast #7-0-1d70654217035f8ce5f64344f4c5a80f @@ -0,0 +1 @@ +-2 diff --git a/sql/hive/src/test/resources/golden/timestamp cast #8-0-6d2da5cfada03605834e38bc4075bc79 b/sql/hive/src/test/resources/golden/timestamp cast #8-0-6d2da5cfada03605834e38bc4075bc79 new file mode 100644 index 0000000000000..1d94c8a014fb4 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp cast #8-0-6d2da5cfada03605834e38bc4075bc79 @@ -0,0 +1 @@ +-1.2 diff --git a/sql/hive/src/test/resources/log4j.properties b/sql/hive/src/test/resources/log4j.properties index c07d8fedf1993..9fdb526d945e0 100644 --- a/sql/hive/src/test/resources/log4j.properties +++ b/sql/hive/src/test/resources/log4j.properties @@ -30,7 +30,7 @@ log4j.appender.FA=org.apache.log4j.FileAppender log4j.appender.FA.append=false log4j.appender.FA.file=target/unit-tests.log log4j.appender.FA.layout=org.apache.log4j.PatternLayout -log4j.appender.FA.layout.ConversionPattern=%d{HH:mm:ss.SSS} %p %c{1}: %m%n +log4j.appender.FA.layout.ConversionPattern=%d{HH:mm:ss.SSS} %t %p %c{1}: %m%n # Set the logger level of File Appender to WARN log4j.appender.FA.Threshold = INFO diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala index 188579edd7bdd..2060e1f1a7a4b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala @@ -17,22 +17,73 @@ package org.apache.spark.sql.hive -import org.apache.spark.sql.execution.SparkLogicalPlan -import org.apache.spark.sql.columnar.{InMemoryRelation, InMemoryColumnarTableScan} -import org.apache.spark.sql.hive.execution.HiveComparisonTest +import org.apache.spark.sql.columnar.{InMemoryColumnarTableScan, InMemoryRelation} import org.apache.spark.sql.hive.test.TestHive +import org.apache.spark.sql.hive.test.TestHive._ +import org.apache.spark.sql.{QueryTest, SchemaRDD} +import org.apache.spark.storage.RDDBlockId -class CachedTableSuite extends HiveComparisonTest { - import TestHive._ +class CachedTableSuite extends QueryTest { + /** + * Throws a test failed exception when the number of cached tables differs from the expected + * number. + */ + def assertCached(query: SchemaRDD, numCachedTables: Int = 1): Unit = { + val planWithCaching = query.queryExecution.withCachedData + val cachedData = planWithCaching collect { + case cached: InMemoryRelation => cached + } - TestHive.loadTestTable("src") + assert( + cachedData.size == numCachedTables, + s"Expected query to contain $numCachedTables, but it actually had ${cachedData.size}\n" + + planWithCaching) + } + + def rddIdOf(tableName: String): Int = { + val executedPlan = table(tableName).queryExecution.executedPlan + executedPlan.collect { + case InMemoryColumnarTableScan(_, _, relation) => + relation.cachedColumnBuffers.id + case _ => + fail(s"Table $tableName is not cached\n" + executedPlan) + }.head + } + + def isMaterialized(rddId: Int): Boolean = { + sparkContext.env.blockManager.get(RDDBlockId(rddId, 0)).nonEmpty + } test("cache table") { - TestHive.cacheTable("src") + val preCacheResults = sql("SELECT * FROM src").collect().toSeq + + cacheTable("src") + assertCached(sql("SELECT * FROM src")) + + checkAnswer( + sql("SELECT * FROM src"), + preCacheResults) + + uncacheTable("src") + assertCached(sql("SELECT * FROM src"), 0) } - createQueryTest("read from cached table", - "SELECT * FROM src LIMIT 1", reset = false) + test("cache invalidation") { + sql("CREATE TABLE cachedTable(key INT, value STRING)") + + sql("INSERT INTO TABLE cachedTable SELECT * FROM src") + checkAnswer(sql("SELECT * FROM cachedTable"), table("src").collect().toSeq) + + cacheTable("cachedTable") + checkAnswer(sql("SELECT * FROM cachedTable"), table("src").collect().toSeq) + + sql("INSERT INTO TABLE cachedTable SELECT * FROM src") + checkAnswer( + sql("SELECT * FROM cachedTable"), + table("src").collect().toSeq ++ table("src").collect().toSeq) + + sql("DROP TABLE cachedTable") + } test("Drop cached table") { sql("CREATE TABLE test(a INT)") @@ -48,25 +99,6 @@ class CachedTableSuite extends HiveComparisonTest { sql("DROP TABLE IF EXISTS nonexistantTable") } - test("check that table is cached and uncache") { - TestHive.table("src").queryExecution.analyzed match { - case _ : InMemoryRelation => // Found evidence of caching - case noCache => fail(s"No cache node found in plan $noCache") - } - TestHive.uncacheTable("src") - } - - createQueryTest("read from uncached table", - "SELECT * FROM src LIMIT 1", reset = false) - - test("make sure table is uncached") { - TestHive.table("src").queryExecution.analyzed match { - case cachePlan: InMemoryRelation => - fail(s"Table still cached after uncache: $cachePlan") - case noCache => // Table uncached successfully - } - } - test("correct error on uncache of non-cached table") { intercept[IllegalArgumentException] { TestHive.uncacheTable("src") @@ -75,17 +107,55 @@ class CachedTableSuite extends HiveComparisonTest { test("'CACHE TABLE' and 'UNCACHE TABLE' HiveQL statement") { TestHive.sql("CACHE TABLE src") - TestHive.table("src").queryExecution.executedPlan match { - case _: InMemoryColumnarTableScan => // Found evidence of caching - case _ => fail(s"Table 'src' should be cached") - } + assertCached(table("src")) assert(TestHive.isCached("src"), "Table 'src' should be cached") TestHive.sql("UNCACHE TABLE src") - TestHive.table("src").queryExecution.executedPlan match { - case _: InMemoryColumnarTableScan => fail(s"Table 'src' should not be cached") - case _ => // Found evidence of uncaching - } + assertCached(table("src"), 0) assert(!TestHive.isCached("src"), "Table 'src' should not be cached") } + + test("CACHE TABLE tableName AS SELECT * FROM anotherTable") { + sql("CACHE TABLE testCacheTable AS SELECT * FROM src") + assertCached(table("testCacheTable")) + + val rddId = rddIdOf("testCacheTable") + assert( + isMaterialized(rddId), + "Eagerly cached in-memory table should have already been materialized") + + uncacheTable("testCacheTable") + assert(!isMaterialized(rddId), "Uncached in-memory table should have been unpersisted") + } + + test("CACHE TABLE tableName AS SELECT ...") { + sql("CACHE TABLE testCacheTable AS SELECT key FROM src LIMIT 10") + assertCached(table("testCacheTable")) + + val rddId = rddIdOf("testCacheTable") + assert( + isMaterialized(rddId), + "Eagerly cached in-memory table should have already been materialized") + + uncacheTable("testCacheTable") + assert(!isMaterialized(rddId), "Uncached in-memory table should have been unpersisted") + } + + test("CACHE LAZY TABLE tableName") { + sql("CACHE LAZY TABLE src") + assertCached(table("src")) + + val rddId = rddIdOf("src") + assert( + !isMaterialized(rddId), + "Lazily cached in-memory table shouldn't be materialized eagerly") + + sql("SELECT COUNT(*) FROM src").collect() + assert( + isMaterialized(rddId), + "Lazily cached in-memory table should have been materialized") + + uncacheTable("src") + assert(!isMaterialized(rddId), "Uncached in-memory table should have been unpersisted") + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/api/java/JavaHiveQLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/api/java/JavaHiveQLSuite.scala index 9644b707eb1a0..46b11b582b26d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/api/java/JavaHiveQLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/api/java/JavaHiveQLSuite.scala @@ -25,34 +25,30 @@ import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.sql.api.java.JavaSchemaRDD import org.apache.spark.sql.execution.ExplainCommand import org.apache.spark.sql.hive.test.TestHive -import org.apache.spark.sql.test.TestSQLContext // Implicits import scala.collection.JavaConversions._ class JavaHiveQLSuite extends FunSuite { - lazy val javaCtx = new JavaSparkContext(TestSQLContext.sparkContext) + lazy val javaCtx = new JavaSparkContext(TestHive.sparkContext) // There is a little trickery here to avoid instantiating two HiveContexts in the same JVM lazy val javaHiveCtx = new JavaHiveContext(javaCtx) { override val sqlContext = TestHive } - ignore("SELECT * FROM src") { + test("SELECT * FROM src") { assert( javaHiveCtx.sql("SELECT * FROM src").collect().map(_.getInt(0)) === TestHive.sql("SELECT * FROM src").collect().map(_.getInt(0)).toSeq) } - private val explainCommandClassName = - classOf[ExplainCommand].getSimpleName.stripSuffix("$") - def isExplanation(result: JavaSchemaRDD) = { val explanation = result.collect().map(_.getString(0)) - explanation.size > 1 && explanation.head.startsWith(explainCommandClassName) + explanation.size > 1 && explanation.head.startsWith("== Physical Plan ==") } - ignore("Query Hive native command execution result") { + test("Query Hive native command execution result") { val tableName = "test_native_commands" assertResult(0) { @@ -63,23 +59,18 @@ class JavaHiveQLSuite extends FunSuite { javaHiveCtx.sql(s"CREATE TABLE $tableName(key INT, value STRING)").count() } - javaHiveCtx.sql("SHOW TABLES").registerTempTable("show_tables") - assert( javaHiveCtx - .sql("SELECT result FROM show_tables") + .sql("SHOW TABLES") .collect() .map(_.getString(0)) .contains(tableName)) - assertResult(Array(Array("key", "int", "None"), Array("value", "string", "None"))) { - javaHiveCtx.sql(s"DESCRIBE $tableName").registerTempTable("describe_table") - - + assertResult(Array(Array("key", "int"), Array("value", "string"))) { javaHiveCtx - .sql("SELECT result FROM describe_table") + .sql(s"describe $tableName") .collect() - .map(_.getString(0).split("\t").map(_.trim)) + .map(row => Array(row.get(0).asInstanceOf[String], row.get(1).asInstanceOf[String])) .toArray } @@ -89,7 +80,7 @@ class JavaHiveQLSuite extends FunSuite { TestHive.reset() } - ignore("Exactly once semantics for DDL and command statements") { + test("Exactly once semantics for DDL and command statements") { val tableName = "test_exactly_once" val q0 = javaHiveCtx.sql(s"CREATE TABLE $tableName(key INT, value STRING)") 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 8c8a8b124ac69..2e282a9ade40c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -19,6 +19,9 @@ package org.apache.spark.sql.hive.execution import scala.util.Try +import org.apache.hadoop.hive.conf.HiveConf.ConfVars + +import org.apache.spark.SparkException import org.apache.spark.sql.hive._ import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.hive.test.TestHive._ @@ -135,23 +138,39 @@ class HiveQuerySuite extends HiveComparisonTest { createQueryTest("div", "SELECT 1 DIV 2, 1 div 2, 1 dIv 2, 100 DIV 51, 100 DIV 49 FROM src LIMIT 1") - createQueryTest("division", - "SELECT 2 / 1, 1 / 2, 1 / 3, 1 / COUNT(*) FROM src LIMIT 1") + // Jdk version leads to different query output for double, so not use createQueryTest here + test("division") { + val res = sql("SELECT 2 / 1, 1 / 2, 1 / 3, 1 / COUNT(*) FROM src LIMIT 1").collect().head + Seq(2.0, 0.5, 0.3333333333333333, 0.002).zip(res).foreach( x => + assert(x._1 == x._2.asInstanceOf[Double])) + } + + createQueryTest("modulus", + "SELECT 11 % 10, IF((101.1 % 100.0) BETWEEN 1.01 AND 1.11, \"true\", \"false\"), (101 / 2) % 10 FROM src LIMIT 1") test("Query expressed in SQL") { setConf("spark.sql.dialect", "sql") assert(sql("SELECT 1").collect() === Array(Seq(1))) setConf("spark.sql.dialect", "hiveql") - } test("Query expressed in HiveQL") { sql("FROM src SELECT key").collect() } + test("Query with constant folding the CAST") { + sql("SELECT CAST(CAST('123' AS binary) AS binary) FROM src LIMIT 1").collect() + } + createQueryTest("Constant Folding Optimization for AVG_SUM_COUNT", "SELECT AVG(0), SUM(0), COUNT(null), COUNT(value) FROM src GROUP BY key") + createQueryTest("Cast Timestamp to Timestamp in UDF", + """ + | SELECT DATEDIFF(CAST(value AS timestamp), CAST('2002-03-21 00:00:00' AS timestamp)) + | FROM src LIMIT 1 + """.stripMargin) + createQueryTest("Simple Average", "SELECT AVG(key) FROM src") @@ -294,6 +313,36 @@ class HiveQuerySuite extends HiveComparisonTest { createQueryTest("case statements WITHOUT key #4", "SELECT (CASE WHEN key > 2 THEN 3 WHEN 2 > key THEN 2 ELSE 0 END) FROM src WHERE key < 15") + // Jdk version leads to different query output for double, so not use createQueryTest here + test("timestamp cast #1") { + val res = sql("SELECT CAST(CAST(1 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1").collect().head + assert(0.001 == res.getDouble(0)) + } + + createQueryTest("timestamp cast #2", + "SELECT CAST(CAST(1.2 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1") + + createQueryTest("timestamp cast #3", + "SELECT CAST(CAST(1200 AS TIMESTAMP) AS INT) FROM src LIMIT 1") + + createQueryTest("timestamp cast #4", + "SELECT CAST(CAST(1.2 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1") + + createQueryTest("timestamp cast #5", + "SELECT CAST(CAST(-1 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1") + + createQueryTest("timestamp cast #6", + "SELECT CAST(CAST(-1.2 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1") + + createQueryTest("timestamp cast #7", + "SELECT CAST(CAST(-1200 AS TIMESTAMP) AS INT) FROM src LIMIT 1") + + createQueryTest("timestamp cast #8", + "SELECT CAST(CAST(-1.2 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1") + + createQueryTest("select null from table", + "SELECT null FROM src LIMIT 1") + test("implement identity function using case statement") { val actual = sql("SELECT (CASE key WHEN key THEN key END) FROM src") .map { case Row(i: Int) => i } @@ -334,7 +383,7 @@ class HiveQuerySuite extends HiveComparisonTest { def isExplanation(result: SchemaRDD) = { val explanation = result.select('plan).collect().map { case Row(plan: String) => plan } - explanation.exists(_ == "== Physical Plan ==") + explanation.contains("== Physical Plan ==") } test("SPARK-1704: Explain commands as a SchemaRDD") { @@ -522,6 +571,91 @@ class HiveQuerySuite extends HiveComparisonTest { case class LogEntry(filename: String, message: String) case class LogFile(name: String) + createQueryTest("dynamic_partition", + """ + |DROP TABLE IF EXISTS dynamic_part_table; + |CREATE TABLE dynamic_part_table(intcol INT) PARTITIONED BY (partcol1 INT, partcol2 INT); + | + |SET hive.exec.dynamic.partition.mode=nonstrict; + | + |INSERT INTO TABLE dynamic_part_table PARTITION(partcol1, partcol2) + |SELECT 1, 1, 1 FROM src WHERE key=150; + | + |INSERT INTO TABLE dynamic_part_table PARTITION(partcol1, partcol2) + |SELECT 1, NULL, 1 FROM src WHERE key=150; + | + |INSERT INTO TABLE dynamic_part_table PARTITION(partcol1, partcol2) + |SELECT 1, 1, NULL FROM src WHERE key=150; + | + |INSERT INTO TABLe dynamic_part_table PARTITION(partcol1, partcol2) + |SELECT 1, NULL, NULL FROM src WHERE key=150; + | + |DROP TABLE IF EXISTS dynamic_part_table; + """.stripMargin) + + test("Dynamic partition folder layout") { + sql("DROP TABLE IF EXISTS dynamic_part_table") + sql("CREATE TABLE dynamic_part_table(intcol INT) PARTITIONED BY (partcol1 INT, partcol2 INT)") + sql("SET hive.exec.dynamic.partition.mode=nonstrict") + + val data = Map( + Seq("1", "1") -> 1, + Seq("1", "NULL") -> 2, + Seq("NULL", "1") -> 3, + Seq("NULL", "NULL") -> 4) + + data.foreach { case (parts, value) => + sql( + s"""INSERT INTO TABLE dynamic_part_table PARTITION(partcol1, partcol2) + |SELECT $value, ${parts.mkString(", ")} FROM src WHERE key=150 + """.stripMargin) + + val partFolder = Seq("partcol1", "partcol2") + .zip(parts) + .map { case (k, v) => + if (v == "NULL") { + s"$k=${ConfVars.DEFAULTPARTITIONNAME.defaultVal}" + } else { + s"$k=$v" + } + } + .mkString("/") + + // Loads partition data to a temporary table to verify contents + val path = s"$warehousePath/dynamic_part_table/$partFolder/part-00000" + + sql("DROP TABLE IF EXISTS dp_verify") + sql("CREATE TABLE dp_verify(intcol INT)") + sql(s"LOAD DATA LOCAL INPATH '$path' INTO TABLE dp_verify") + + assert(sql("SELECT * FROM dp_verify").collect() === Array(Row(value))) + } + } + + test("Partition spec validation") { + sql("DROP TABLE IF EXISTS dp_test") + sql("CREATE TABLE dp_test(key INT, value STRING) PARTITIONED BY (dp INT, sp INT)") + sql("SET hive.exec.dynamic.partition.mode=strict") + + // Should throw when using strict dynamic partition mode without any static partition + intercept[SparkException] { + sql( + """INSERT INTO TABLE dp_test PARTITION(dp) + |SELECT key, value, key % 5 FROM src + """.stripMargin) + } + + sql("SET hive.exec.dynamic.partition.mode=nonstrict") + + // Should throw when a static partition appears after a dynamic partition + intercept[SparkException] { + sql( + """INSERT INTO TABLE dp_test PARTITION(dp, sp = 1) + |SELECT key, value, key % 5 FROM src + """.stripMargin) + } + } + test("SPARK-3414 regression: should store analyzed logical plan when registering a temp table") { sparkContext.makeRDD(Seq.empty[LogEntry]).registerTempTable("rawLogs") sparkContext.makeRDD(Seq.empty[LogFile]).registerTempTable("logFiles") @@ -579,27 +713,27 @@ class HiveQuerySuite extends HiveComparisonTest { assert(sql("SET").collect().size == 0) assertResult(Set(testKey -> testVal)) { - collectResults(hql(s"SET $testKey=$testVal")) + collectResults(sql(s"SET $testKey=$testVal")) } assert(hiveconf.get(testKey, "") == testVal) assertResult(Set(testKey -> testVal)) { - collectResults(hql("SET")) + collectResults(sql("SET")) } sql(s"SET ${testKey + testKey}=${testVal + testVal}") assert(hiveconf.get(testKey + testKey, "") == testVal + testVal) assertResult(Set(testKey -> testVal, (testKey + testKey) -> (testVal + testVal))) { - collectResults(hql("SET")) + collectResults(sql("SET")) } // "set key" assertResult(Set(testKey -> testVal)) { - collectResults(hql(s"SET $testKey")) + collectResults(sql(s"SET $testKey")) } assertResult(Set(nonexistentKey -> "")) { - collectResults(hql(s"SET $nonexistentKey")) + collectResults(sql(s"SET $nonexistentKey")) } // Assert that sql() should have the same effects as sql() by repeating the above using sql(). diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala index b6be6bc1bfefe..ee9d08ff75450 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala @@ -36,6 +36,9 @@ class HiveResolutionSuite extends HiveComparisonTest { createQueryTest("database.table table.attr", "SELECT src.key FROM default.src ORDER BY key LIMIT 1") + createQueryTest("database.table table.attr case insensitive", + "SELECT SRC.Key FROM Default.Src ORDER BY key LIMIT 1") + createQueryTest("alias.attr", "SELECT a.key FROM src a ORDER BY key LIMIT 1") @@ -56,14 +59,18 @@ class HiveResolutionSuite extends HiveComparisonTest { TestHive.sparkContext.parallelize(Data(1, 2, Nested(1,2), Seq(Nested(1,2))) :: Nil) .registerTempTable("caseSensitivityTest") - sql("SELECT a, b, A, B, n.a, n.b, n.A, n.B FROM caseSensitivityTest") - - println(sql("SELECT * FROM casesensitivitytest one JOIN casesensitivitytest two ON one.a = two.a").queryExecution) - - sql("SELECT * FROM casesensitivitytest one JOIN casesensitivitytest two ON one.a = two.a").collect() + val query = sql("SELECT a, b, A, B, n.a, n.b, n.A, n.B FROM caseSensitivityTest") + assert(query.schema.fields.map(_.name) === Seq("a", "b", "A", "B", "a", "b", "A", "B"), + "The output schema did not preserve the case of the query.") + query.collect() + } - // TODO: sql("SELECT * FROM casesensitivitytest a JOIN casesensitivitytest b ON a.a = b.a") + ignore("case insensitivity with scala reflection joins") { + // Test resolution with Scala Reflection + TestHive.sparkContext.parallelize(Data(1, 2, Nested(1,2), Seq(Nested(1,2))) :: Nil) + .registerTempTable("caseSensitivityTest") + sql("SELECT * FROM casesensitivitytest a JOIN casesensitivitytest b ON a.a = b.a").collect() } test("nested repeated resolution") { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala index cc125d539c3c2..e4324e9528f9b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala @@ -79,6 +79,10 @@ class HiveUdfSuite extends HiveComparisonTest { sql("SELECT testUdf(pair) FROM hiveUdfTestTable") sql("DROP TEMPORARY FUNCTION IF EXISTS testUdf") } + + test("SPARK-2693 udaf aggregates test") { + assert(sql("SELECT percentile(key,1) FROM src").first === sql("SELECT max(key) FROM src").first) + } } class TestPair(x: Int, y: Int) extends Writable with Serializable { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala index 8275e2d3bcce3..8474d850c9c6c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala @@ -28,8 +28,6 @@ import scala.collection.JavaConversions._ * A set of test cases that validate partition and column pruning. */ class PruningSuite extends HiveComparisonTest with BeforeAndAfter { - // MINOR HACK: You must run a query before calling reset the first time. - TestHive.sql("SHOW TABLES") TestHive.cacheTables = false // Column/partition pruning is not implemented for `InMemoryColumnarTableScan` yet, need to reset diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 679efe082f2a0..3647bb1c4ce7d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -63,4 +63,10 @@ class SQLQuerySuite extends QueryTest { sql("SELECT key, value FROM test_ctas_123 ORDER BY key"), sql("SELECT key, value FROM src ORDER BY key").collect().toSeq) } + + test("SPARK-3708 Backticks aren't handled correctly is aliases") { + checkAnswer( + sql("SELECT k FROM (SELECT `key` AS `k` FROM src) a"), + sql("SELECT `key` FROM src").collect().toSeq) + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/ParquetMetastoreSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/ParquetMetastoreSuite.scala index e380280f301c1..86adbbf3ad2d8 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/ParquetMetastoreSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/ParquetMetastoreSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.parquet import java.io.File +import org.apache.spark.sql.catalyst.expressions.Row import org.scalatest.BeforeAndAfterAll import org.apache.spark.sql.QueryTest @@ -142,15 +143,21 @@ class ParquetMetastoreSuite extends QueryTest with BeforeAndAfterAll { test("sum") { checkAnswer( sql("SELECT SUM(intField) FROM partitioned_parquet WHERE intField IN (1,2,3) AND p = 1"), - 1 + 2 + 3 - ) + 1 + 2 + 3) + } + + test("hive udfs") { + checkAnswer( + sql("SELECT concat(stringField, stringField) FROM partitioned_parquet"), + sql("SELECT stringField FROM partitioned_parquet").map { + case Row(s: String) => Row(s + s) + }.collect().toSeq) } test("non-part select(*)") { checkAnswer( sql("SELECT COUNT(*) FROM normal_parquet"), - 10 - ) + 10) } test("conversion is working") { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Duration.scala b/streaming/src/main/scala/org/apache/spark/streaming/Duration.scala index 6bf275f5afcb2..a0d8fb5ab93ec 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Duration.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Duration.scala @@ -37,6 +37,25 @@ case class Duration (private val millis: Long) { def / (that: Duration): Double = millis.toDouble / that.millis.toDouble + // Java-friendlier versions of the above. + + def less(that: Duration): Boolean = this < that + + def lessEq(that: Duration): Boolean = this <= that + + def greater(that: Duration): Boolean = this > that + + def greaterEq(that: Duration): Boolean = this >= that + + def plus(that: Duration): Duration = this + that + + def minus(that: Duration): Duration = this - that + + def times(times: Int): Duration = this * times + + def div(that: Duration): Double = this / that + + def isMultipleOf(that: Duration): Boolean = (this.millis % that.millis == 0) @@ -80,4 +99,24 @@ object Minutes { def apply(minutes: Long) = new Duration(minutes * 60000) } +// Java-friendlier versions of the objects above. +// Named "Durations" instead of "Duration" to avoid changing the case class's implied API. + +object Durations { + + /** + * @return [[org.apache.spark.streaming.Duration]] representing given number of milliseconds. + */ + def milliseconds(milliseconds: Long) = Milliseconds(milliseconds) + /** + * @return [[org.apache.spark.streaming.Duration]] representing given number of seconds. + */ + def seconds(seconds: Long) = Seconds(seconds) + + /** + * @return [[org.apache.spark.streaming.Duration]] representing given number of minutes. + */ + def minutes(minutes: Long) = Minutes(minutes) + +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index f63560dcb5b89..5a8eef1372e23 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -35,10 +35,9 @@ import org.apache.spark._ import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.dstream._ -import org.apache.spark.streaming.receiver.{ActorSupervisorStrategy, ActorReceiver, Receiver} +import org.apache.spark.streaming.receiver.{ActorReceiver, ActorSupervisorStrategy, Receiver} import org.apache.spark.streaming.scheduler._ import org.apache.spark.streaming.ui.{StreamingJobProgressListener, StreamingTab} -import org.apache.spark.util.MetadataCleaner /** * Main entry point for Spark Streaming functionality. It provides methods used to create @@ -448,6 +447,7 @@ class StreamingContext private[streaming] ( throw new SparkException("StreamingContext has already been stopped") } validate() + sparkContext.setCallSite(DStream.getCreationSite()) scheduler.start() state = Started } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Time.scala b/streaming/src/main/scala/org/apache/spark/streaming/Time.scala index 37b3b28fa01cb..42c49678d24f0 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Time.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Time.scala @@ -41,10 +41,26 @@ case class Time(private val millis: Long) { def - (that: Duration): Time = new Time(millis - that.milliseconds) + // Java-friendlier versions of the above. + + def less(that: Time): Boolean = this < that + + def lessEq(that: Time): Boolean = this <= that + + def greater(that: Time): Boolean = this > that + + def greaterEq(that: Time): Boolean = this >= that + + def plus(that: Duration): Time = this + that + + def minus(that: Time): Duration = this - that + + def minus(that: Duration): Time = this - that + + def floor(that: Duration): Time = { val t = that.milliseconds - val m = math.floor(this.millis / t).toLong - new Time(m * t) + new Time((this.millis / t) * t) } def isMultipleOf(that: Duration): Boolean = 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 c00e11d11910f..59d4423086ef0 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 @@ -606,8 +606,9 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( } /** - * Return a new DStream by applying 'join' between RDDs of `this` DStream and `other` DStream. - * The supplied org.apache.spark.Partitioner is used to control the partitioning of each RDD. + * Return a new DStream by applying 'left outer join' between RDDs of `this` DStream and + * `other` DStream. The supplied org.apache.spark.Partitioner is used to control + * the partitioning of each RDD. */ def leftOuterJoin[W]( other: JavaPairDStream[K, W], @@ -624,8 +625,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * number of partitions. */ def rightOuterJoin[W](other: JavaPairDStream[K, W]): JavaPairDStream[K, (Optional[V], W)] = { - implicit val cm: ClassTag[W] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[W]] + implicit val cm: ClassTag[W] = fakeClassTag val joinResult = dstream.rightOuterJoin(other.dstream) joinResult.mapValues{case (v, w) => (JavaUtils.optionToOptional(v), w)} } @@ -658,6 +658,52 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( joinResult.mapValues{case (v, w) => (JavaUtils.optionToOptional(v), w)} } + /** + * Return a new DStream by applying 'full outer join' between RDDs of `this` DStream and + * `other` DStream. Hash partitioning is used to generate the RDDs with Spark's default + * number of partitions. + */ + def fullOuterJoin[W](other: JavaPairDStream[K, W]) + : JavaPairDStream[K, (Optional[V], Optional[W])] = { + implicit val cm: ClassTag[W] = fakeClassTag + val joinResult = dstream.fullOuterJoin(other.dstream) + joinResult.mapValues{ case (v, w) => + (JavaUtils.optionToOptional(v), JavaUtils.optionToOptional(w)) + } + } + + /** + * Return a new DStream by applying 'full outer join' between RDDs of `this` DStream and + * `other` DStream. Hash partitioning is used to generate the RDDs with `numPartitions` + * partitions. + */ + def fullOuterJoin[W]( + other: JavaPairDStream[K, W], + numPartitions: Int + ): JavaPairDStream[K, (Optional[V], Optional[W])] = { + implicit val cm: ClassTag[W] = fakeClassTag + val joinResult = dstream.fullOuterJoin(other.dstream, numPartitions) + joinResult.mapValues{ case (v, w) => + (JavaUtils.optionToOptional(v), JavaUtils.optionToOptional(w)) + } + } + + /** + * Return a new DStream by applying 'full outer join' between RDDs of `this` DStream and + * `other` DStream. The supplied org.apache.spark.Partitioner is used to control + * the partitioning of each RDD. + */ + def fullOuterJoin[W]( + other: JavaPairDStream[K, W], + partitioner: Partitioner + ): JavaPairDStream[K, (Optional[V], Optional[W])] = { + implicit val cm: ClassTag[W] = fakeClassTag + val joinResult = dstream.fullOuterJoin(other.dstream, partitioner) + joinResult.mapValues{ case (v, w) => + (JavaUtils.optionToOptional(v), JavaUtils.optionToOptional(w)) + } + } + /** * Save each RDD in `this` DStream as a Hadoop file. The file name at each batch interval is * generated based on `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix". diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala index e05db236addca..65f7ccd318684 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala @@ -23,6 +23,7 @@ import java.io.{IOException, ObjectInputStream, ObjectOutputStream} import scala.deprecated import scala.collection.mutable.HashMap import scala.reflect.ClassTag +import scala.util.matching.Regex import org.apache.spark.{Logging, SparkException} import org.apache.spark.rdd.{BlockRDD, RDD} @@ -30,7 +31,7 @@ import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming._ import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.streaming.scheduler.Job -import org.apache.spark.util.MetadataCleaner +import org.apache.spark.util.{CallSite, MetadataCleaner} /** * A Discretized Stream (DStream), the basic abstraction in Spark Streaming, is a continuous @@ -106,6 +107,9 @@ abstract class DStream[T: ClassTag] ( /** Return the StreamingContext associated with this DStream */ def context = ssc + /* Set the creation call site */ + private[streaming] val creationSite = DStream.getCreationSite() + /** Persist the RDDs of this DStream with the given storage level */ def persist(level: StorageLevel): DStream[T] = { if (this.isInitialized) { @@ -272,43 +276,41 @@ abstract class DStream[T: ClassTag] ( } /** - * Retrieve a precomputed RDD of this DStream, or computes the RDD. This is an internal - * method that should not be called directly. + * Get the RDD corresponding to the given time; either retrieve it from cache + * or compute-and-cache it. */ private[streaming] def getOrCompute(time: Time): Option[RDD[T]] = { - // If this DStream was not initialized (i.e., zeroTime not set), then do it - // If RDD was already generated, then retrieve it from HashMap - generatedRDDs.get(time) match { - - // If an RDD was already generated and is being reused, then - // probably all RDDs in this DStream will be reused and hence should be cached - case Some(oldRDD) => Some(oldRDD) - - // if RDD was not generated, and if the time is valid - // (based on sliding time of this DStream), then generate the RDD - case None => { - if (isTimeValid(time)) { - compute(time) match { - case Some(newRDD) => - if (storageLevel != StorageLevel.NONE) { - newRDD.persist(storageLevel) - logInfo("Persisting RDD " + newRDD.id + " for time " + - time + " to " + storageLevel + " at time " + time) - } - if (checkpointDuration != null && - (time - zeroTime).isMultipleOf(checkpointDuration)) { - newRDD.checkpoint() - logInfo("Marking RDD " + newRDD.id + " for time " + time + - " for checkpointing at time " + time) - } - generatedRDDs.put(time, newRDD) - Some(newRDD) - case None => - None + // If RDD was already generated, then retrieve it from HashMap, + // or else compute the RDD + generatedRDDs.get(time).orElse { + // Compute the RDD if time is valid (e.g. correct time in a sliding window) + // of RDD generation, else generate nothing. + if (isTimeValid(time)) { + // Set the thread-local property for call sites to this DStream's creation site + // such that RDDs generated by compute gets that as their creation site. + // Note that this `getOrCompute` may get called from another DStream which may have + // set its own call site. So we store its call site in a temporary variable, + // set this DStream's creation site, generate RDDs and then restore the previous call site. + val prevCallSite = ssc.sparkContext.getCallSite() + ssc.sparkContext.setCallSite(creationSite) + val rddOption = compute(time) + ssc.sparkContext.setCallSite(prevCallSite) + + rddOption.foreach { case newRDD => + // Register the generated RDD for caching and checkpointing + if (storageLevel != StorageLevel.NONE) { + newRDD.persist(storageLevel) + logDebug(s"Persisting RDD ${newRDD.id} for time $time to $storageLevel") } - } else { - None + if (checkpointDuration != null && (time - zeroTime).isMultipleOf(checkpointDuration)) { + newRDD.checkpoint() + logInfo(s"Marking RDD ${newRDD.id} for time $time for checkpointing") + } + generatedRDDs.put(time, newRDD) } + rddOption + } else { + None } } } @@ -799,3 +801,29 @@ abstract class DStream[T: ClassTag] ( this } } + +private[streaming] object DStream { + + /** Get the creation site of a DStream from the stack trace of when the DStream is created. */ + def getCreationSite(): CallSite = { + val SPARK_CLASS_REGEX = """^org\.apache\.spark""".r + val SPARK_STREAMING_TESTCLASS_REGEX = """^org\.apache\.spark\.streaming\.test""".r + val SPARK_EXAMPLES_CLASS_REGEX = """^org\.apache\.spark\.examples""".r + val SCALA_CLASS_REGEX = """^scala""".r + + /** Filtering function that excludes non-user classes for a streaming application */ + def streamingExclustionFunction(className: String): Boolean = { + def doesMatch(r: Regex) = r.findFirstIn(className).isDefined + val isSparkClass = doesMatch(SPARK_CLASS_REGEX) + val isSparkExampleClass = doesMatch(SPARK_EXAMPLES_CLASS_REGEX) + val isSparkStreamingTestClass = doesMatch(SPARK_STREAMING_TESTCLASS_REGEX) + val isScalaClass = doesMatch(SCALA_CLASS_REGEX) + + // If the class is a spark example class or a streaming test class then it is considered + // as a streaming application class and don't exclude. Otherwise, exclude any + // non-Spark and non-Scala class, as the rest would streaming application classes. + (isSparkClass || isScalaClass) && !isSparkExampleClass && !isSparkStreamingTestClass + } + org.apache.spark.util.Utils.getCallSite(streamingExclustionFunction) + } +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala index 826bf39e860e1..9467595d307a2 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala @@ -568,6 +568,42 @@ class PairDStreamFunctions[K, V](self: DStream[(K,V)]) ) } + /** + * Return a new DStream by applying 'full outer join' between RDDs of `this` DStream and + * `other` DStream. Hash partitioning is used to generate the RDDs with Spark's default + * number of partitions. + */ + def fullOuterJoin[W: ClassTag](other: DStream[(K, W)]): DStream[(K, (Option[V], Option[W]))] = { + fullOuterJoin[W](other, defaultPartitioner()) + } + + /** + * Return a new DStream by applying 'full outer join' between RDDs of `this` DStream and + * `other` DStream. Hash partitioning is used to generate the RDDs with `numPartitions` + * partitions. + */ + def fullOuterJoin[W: ClassTag]( + other: DStream[(K, W)], + numPartitions: Int + ): DStream[(K, (Option[V], Option[W]))] = { + fullOuterJoin[W](other, defaultPartitioner(numPartitions)) + } + + /** + * Return a new DStream by applying 'full outer join' between RDDs of `this` DStream and + * `other` DStream. The supplied org.apache.spark.Partitioner is used to control + * the partitioning of each RDD. + */ + def fullOuterJoin[W: ClassTag]( + other: DStream[(K, W)], + partitioner: Partitioner + ): DStream[(K, (Option[V], Option[W]))] = { + self.transformWith( + other, + (rdd1: RDD[(K, V)], rdd2: RDD[(K, W)]) => rdd1.fullOuterJoin(rdd2, partitioner) + ) + } + /** * Save each RDD in `this` DStream as a Hadoop file. The file name at each batch interval * is generated based on `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix" diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala index 374848358e700..7d73ada12d107 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala @@ -217,7 +217,6 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { /** Generate jobs and perform checkpoint for the given `time`. */ private def generateJobs(time: Time) { - SparkEnv.set(ssc.env) Try(graph.generateJobs(time)) match { case Success(jobs) => val receivedBlockInfo = graph.getReceiverInputStreams.map { stream => diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala index 1b034b9fb187c..cfa3cd8925c80 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala @@ -138,7 +138,6 @@ class JobScheduler(val ssc: StreamingContext) extends Logging { } jobSet.handleJobStart(job) logInfo("Starting job " + job.id + " from job set of time " + jobSet.time) - SparkEnv.set(ssc.env) } private def handleJobCompletion(job: Job) { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala index 5307fe189d717..7149dbc12a365 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala @@ -202,7 +202,6 @@ class ReceiverTracker(ssc: StreamingContext) extends Logging { @transient val thread = new Thread() { override def run() { try { - SparkEnv.set(env) startReceivers() } catch { case ie: InterruptedException => logInfo("ReceiverLauncher interrupted") diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaDurationSuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaDurationSuite.java new file mode 100644 index 0000000000000..76425fe2aa2d3 --- /dev/null +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaDurationSuite.java @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +package org.apache.spark.streaming; + +import org.junit.Assert; +import org.junit.Test; + +public class JavaDurationSuite { + + // Just testing the methods that are specially exposed for Java. + // This does not repeat all tests found in the Scala suite. + + @Test + public void testLess() { + Assert.assertTrue(new Duration(999).less(new Duration(1000))); + } + + @Test + public void testLessEq() { + Assert.assertTrue(new Duration(1000).lessEq(new Duration(1000))); + } + + @Test + public void testGreater() { + Assert.assertTrue(new Duration(1000).greater(new Duration(999))); + } + + @Test + public void testGreaterEq() { + Assert.assertTrue(new Duration(1000).greaterEq(new Duration(1000))); + } + + @Test + public void testPlus() { + Assert.assertEquals(new Duration(1100), new Duration(1000).plus(new Duration(100))); + } + + @Test + public void testMinus() { + Assert.assertEquals(new Duration(900), new Duration(1000).minus(new Duration(100))); + } + + @Test + public void testTimes() { + Assert.assertEquals(new Duration(200), new Duration(100).times(2)); + } + + @Test + public void testDiv() { + Assert.assertEquals(200.0, new Duration(1000).div(new Duration(5)), 1.0e-12); + } + + @Test + public void testMilliseconds() { + Assert.assertEquals(new Duration(100), Durations.milliseconds(100)); + } + + @Test + public void testSeconds() { + Assert.assertEquals(new Duration(30 * 1000), Durations.seconds(30)); + } + + @Test + public void testMinutes() { + Assert.assertEquals(new Duration(2 * 60 * 1000), Durations.minutes(2)); + } + +} diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaTimeSuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaTimeSuite.java new file mode 100644 index 0000000000000..ad6b1853e3d12 --- /dev/null +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaTimeSuite.java @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming; + +import org.junit.Assert; +import org.junit.Test; + +public class JavaTimeSuite { + + // Just testing the methods that are specially exposed for Java. + // This does not repeat all tests found in the Scala suite. + + @Test + public void testLess() { + Assert.assertTrue(new Time(999).less(new Time(1000))); + } + + @Test + public void testLessEq() { + Assert.assertTrue(new Time(1000).lessEq(new Time(1000))); + } + + @Test + public void testGreater() { + Assert.assertTrue(new Time(1000).greater(new Time(999))); + } + + @Test + public void testGreaterEq() { + Assert.assertTrue(new Time(1000).greaterEq(new Time(1000))); + } + + @Test + public void testPlus() { + Assert.assertEquals(new Time(1100), new Time(1000).plus(new Duration(100))); + } + + @Test + public void testMinusTime() { + Assert.assertEquals(new Duration(900), new Time(1000).minus(new Time(100))); + } + + @Test + public void testMinusDuration() { + Assert.assertEquals(new Time(900), new Time(1000).minus(new Duration(100))); + } + +} diff --git a/streaming/src/test/resources/log4j.properties b/streaming/src/test/resources/log4j.properties index 45d2ec676df66..4411d6e20c52a 100644 --- a/streaming/src/test/resources/log4j.properties +++ b/streaming/src/test/resources/log4j.properties @@ -22,7 +22,7 @@ log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false log4j.appender.file.file=target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout -log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose log4j.logger.org.eclipse.jetty=WARN diff --git a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala index 059ac6c2dbee2..6c8bb50145367 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala @@ -303,6 +303,21 @@ class BasicOperationsSuite extends TestSuiteBase { testOperation(inputData1, inputData2, operation, outputData, true) } + test("fullOuterJoin") { + val inputData1 = Seq( Seq("a", "b"), Seq("a", ""), Seq(""), Seq() ) + val inputData2 = Seq( Seq("a", "b"), Seq("b", ""), Seq(), Seq("") ) + val outputData = Seq( + Seq( ("a", (Some(1), Some("x"))), ("b", (Some(1), Some("x"))) ), + Seq( ("", (Some(1), Some("x"))), ("a", (Some(1), None)), ("b", (None, Some("x"))) ), + Seq( ("", (Some(1), None)) ), + Seq( ("", (None, Some("x"))) ) + ) + val operation = (s1: DStream[String], s2: DStream[String]) => { + s1.map(x => (x, 1)).fullOuterJoin(s2.map(x => (x, "x"))) + } + testOperation(inputData1, inputData2, operation, outputData, true) + } + test("updateStateByKey") { val inputData = Seq( diff --git a/streaming/src/test/scala/org/apache/spark/streaming/DurationSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/DurationSuite.scala new file mode 100644 index 0000000000000..6202250e897f2 --- /dev/null +++ b/streaming/src/test/scala/org/apache/spark/streaming/DurationSuite.scala @@ -0,0 +1,110 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming + +class DurationSuite extends TestSuiteBase { + + test("less") { + assert(new Duration(999) < new Duration(1000)) + assert(new Duration(0) < new Duration(1)) + assert(!(new Duration(1000) < new Duration(999))) + assert(!(new Duration(1000) < new Duration(1000))) + } + + test("lessEq") { + assert(new Duration(999) <= new Duration(1000)) + assert(new Duration(0) <= new Duration(1)) + assert(!(new Duration(1000) <= new Duration(999))) + assert(new Duration(1000) <= new Duration(1000)) + } + + test("greater") { + assert(!(new Duration(999) > new Duration(1000))) + assert(!(new Duration(0) > new Duration(1))) + assert(new Duration(1000) > new Duration(999)) + assert(!(new Duration(1000) > new Duration(1000))) + } + + test("greaterEq") { + assert(!(new Duration(999) >= new Duration(1000))) + assert(!(new Duration(0) >= new Duration(1))) + assert(new Duration(1000) >= new Duration(999)) + assert(new Duration(1000) >= new Duration(1000)) + } + + test("plus") { + assert((new Duration(1000) + new Duration(100)) == new Duration(1100)) + assert((new Duration(1000) + new Duration(0)) == new Duration(1000)) + } + + test("minus") { + assert((new Duration(1000) - new Duration(100)) == new Duration(900)) + assert((new Duration(1000) - new Duration(0)) == new Duration(1000)) + assert((new Duration(1000) - new Duration(1000)) == new Duration(0)) + } + + test("times") { + assert((new Duration(100) * 2) == new Duration(200)) + assert((new Duration(100) * 1) == new Duration(100)) + assert((new Duration(100) * 0) == new Duration(0)) + } + + test("div") { + assert((new Duration(1000) / new Duration(5)) == 200.0) + assert((new Duration(1000) / new Duration(1)) == 1000.0) + assert((new Duration(1000) / new Duration(1000)) == 1.0) + assert((new Duration(1000) / new Duration(2000)) == 0.5) + } + + test("isMultipleOf") { + assert(new Duration(1000).isMultipleOf(new Duration(5))) + assert(new Duration(1000).isMultipleOf(new Duration(1000))) + assert(new Duration(1000).isMultipleOf(new Duration(1))) + assert(!new Duration(1000).isMultipleOf(new Duration(6))) + } + + test("min") { + assert(new Duration(999).min(new Duration(1000)) == new Duration(999)) + assert(new Duration(1000).min(new Duration(999)) == new Duration(999)) + assert(new Duration(1000).min(new Duration(1000)) == new Duration(1000)) + } + + test("max") { + assert(new Duration(999).max(new Duration(1000)) == new Duration(1000)) + assert(new Duration(1000).max(new Duration(999)) == new Duration(1000)) + assert(new Duration(1000).max(new Duration(1000)) == new Duration(1000)) + } + + test("isZero") { + assert(new Duration(0).isZero) + assert(!(new Duration(1).isZero)) + } + + test("Milliseconds") { + assert(new Duration(100) == Milliseconds(100)) + } + + test("Seconds") { + assert(new Duration(30 * 1000) == Seconds(30)) + } + + test("Minutes") { + assert(new Duration(2 * 60 * 1000) == Minutes(2)) + } + +} diff --git a/streaming/src/test/scala/org/apache/spark/streaming/FailureSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/FailureSuite.scala index 92e1b76d28301..40434b1f9b709 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/FailureSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/FailureSuite.scala @@ -18,7 +18,6 @@ package org.apache.spark.streaming import org.apache.spark.Logging -import org.apache.spark.streaming.util.MasterFailureTest import org.apache.spark.util.Utils import java.io.File diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala b/streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala similarity index 91% rename from streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala rename to streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala index 98e17ff92e205..c53c01706083a 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala @@ -15,20 +15,18 @@ * limitations under the License. */ -package org.apache.spark.streaming.util +package org.apache.spark.streaming import org.apache.spark.Logging -import org.apache.spark.rdd.RDD -import org.apache.spark.streaming._ -import org.apache.spark.streaming.dstream.{DStream, ForEachDStream} +import org.apache.spark.streaming.dstream.DStream import org.apache.spark.util.Utils -import StreamingContext._ +import org.apache.spark.streaming.StreamingContext._ import scala.util.Random -import scala.collection.mutable.{SynchronizedBuffer, ArrayBuffer} +import scala.collection.mutable.ArrayBuffer import scala.reflect.ClassTag -import java.io.{File, ObjectInputStream, IOException} +import java.io.{File, IOException} import java.nio.charset.Charset import java.util.UUID @@ -91,7 +89,7 @@ object MasterFailureTest extends Logging { // Input: time=1 ==> [ a ] , time=2 ==> [ a, a ] , time=3 ==> [ a, a, a ] , ... val input = (1 to numBatches).map(i => (1 to i).map(_ => "a").mkString(" ")).toSeq // Expected output: time=1 ==> [ (a, 1) ] , time=2 ==> [ (a, 3) ] , time=3 ==> [ (a,6) ] , ... - val expectedOutput = (1L to numBatches).map(i => (1L to i).reduce(_ + _)).map(j => ("a", j)) + val expectedOutput = (1L to numBatches).map(i => (1L to i).sum).map(j => ("a", j)) val operation = (st: DStream[String]) => { val updateFunc = (values: Seq[Long], state: Option[Long]) => { @@ -218,7 +216,7 @@ object MasterFailureTest extends Logging { while(!isLastOutputGenerated && !isTimedOut) { // Get the output buffer - val outputBuffer = ssc.graph.getOutputStreams.head.asInstanceOf[TestOutputStream[T]].output + val outputBuffer = ssc.graph.getOutputStreams().head.asInstanceOf[TestOutputStream[T]].output def output = outputBuffer.flatMap(x => x) // Start the thread to kill the streaming after some time @@ -239,7 +237,7 @@ object MasterFailureTest extends Logging { while (!killed && !isLastOutputGenerated && !isTimedOut) { Thread.sleep(100) timeRan = System.currentTimeMillis() - startTime - isLastOutputGenerated = (!output.isEmpty && output.last == lastExpectedOutput) + isLastOutputGenerated = (output.nonEmpty && output.last == lastExpectedOutput) isTimedOut = (timeRan + totalTimeRan > maxTimeToRun) } } catch { @@ -313,31 +311,6 @@ object MasterFailureTest extends Logging { } } -/** - * This is a output stream just for testing. All the output is collected into a - * ArrayBuffer. This buffer is wiped clean on being restored from checkpoint. - */ -private[streaming] -class TestOutputStream[T: ClassTag]( - parent: DStream[T], - val output: ArrayBuffer[Seq[T]] = new ArrayBuffer[Seq[T]] with SynchronizedBuffer[Seq[T]] - ) extends ForEachDStream[T]( - parent, - (rdd: RDD[T], t: Time) => { - val collected = rdd.collect() - output += collected - } - ) { - - // This is to clear the output buffer every it is read from a checkpoint - @throws(classOf[IOException]) - private def readObject(ois: ObjectInputStream) { - ois.defaultReadObject() - output.clear() - } -} - - /** * Thread to kill streaming context after a random period of time. */ diff --git a/streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala index 99c8d13231aac..eb6e88cf5520d 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.streaming import java.nio.ByteBuffer +import java.util.concurrent.Semaphore import scala.collection.mutable.ArrayBuffer @@ -36,6 +37,7 @@ class NetworkReceiverSuite extends FunSuite with Timeouts { val receiver = new FakeReceiver val executor = new FakeReceiverSupervisor(receiver) + val executorStarted = new Semaphore(0) assert(executor.isAllEmpty) @@ -43,6 +45,7 @@ class NetworkReceiverSuite extends FunSuite with Timeouts { val executingThread = new Thread() { override def run() { executor.start() + executorStarted.release(1) executor.awaitTermination() } } @@ -57,6 +60,9 @@ class NetworkReceiverSuite extends FunSuite with Timeouts { } } + // Ensure executor is started + executorStarted.acquire() + // Verify that receiver was started assert(receiver.onStartCalled) assert(executor.isReceiverStarted) @@ -186,10 +192,10 @@ class NetworkReceiverSuite extends FunSuite with Timeouts { * An implementation of NetworkReceiver that is used for testing a receiver's life cycle. */ class FakeReceiver extends Receiver[Int](StorageLevel.MEMORY_ONLY) { - var otherThread: Thread = null - var receiving = false - var onStartCalled = false - var onStopCalled = false + @volatile var otherThread: Thread = null + @volatile var receiving = false + @volatile var onStartCalled = false + @volatile var onStopCalled = false def onStart() { otherThread = new Thread() { 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 a3cabd6be02fe..655cec1573f58 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -19,15 +19,18 @@ package org.apache.spark.streaming import java.util.concurrent.atomic.AtomicInteger +import org.scalatest.{Assertions, BeforeAndAfter, FunSuite} +import org.scalatest.concurrent.Timeouts +import org.scalatest.concurrent.Eventually._ +import org.scalatest.exceptions.TestFailedDueToTimeoutException +import org.scalatest.time.SpanSugar._ + import org.apache.spark.{Logging, SparkConf, SparkContext, SparkException} import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.dstream.DStream import org.apache.spark.streaming.receiver.Receiver -import org.apache.spark.util.{MetadataCleaner, Utils} -import org.scalatest.{BeforeAndAfter, FunSuite} -import org.scalatest.concurrent.Timeouts -import org.scalatest.exceptions.TestFailedDueToTimeoutException -import org.scalatest.time.SpanSugar._ +import org.apache.spark.util.Utils + class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts with Logging { @@ -65,7 +68,7 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w test("from no conf + spark home + env") { ssc = new StreamingContext(master, appName, batchDuration, sparkHome, Nil, Map(envPair)) - assert(ssc.conf.getExecutorEnv.exists(_ == envPair)) + assert(ssc.conf.getExecutorEnv.contains(envPair)) } test("from conf with settings") { @@ -91,7 +94,7 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w val myConf = SparkContext.updatedConf(new SparkConf(false), master, appName) myConf.set("spark.cleaner.ttl", "10") val ssc1 = new StreamingContext(myConf, batchDuration) - addInputStream(ssc1).register + addInputStream(ssc1).register() ssc1.start() val cp = new Checkpoint(ssc1, Time(1000)) assert(cp.sparkConfPairs.toMap.getOrElse("spark.cleaner.ttl", "-1") === "10") @@ -104,7 +107,7 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w test("start and stop state check") { ssc = new StreamingContext(master, appName, batchDuration) - addInputStream(ssc).register + addInputStream(ssc).register() assert(ssc.state === ssc.StreamingContextState.Initialized) ssc.start() @@ -115,7 +118,7 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w test("start multiple times") { ssc = new StreamingContext(master, appName, batchDuration) - addInputStream(ssc).register + addInputStream(ssc).register() ssc.start() intercept[SparkException] { ssc.start() @@ -124,7 +127,7 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w test("stop multiple times") { ssc = new StreamingContext(master, appName, batchDuration) - addInputStream(ssc).register + addInputStream(ssc).register() ssc.start() ssc.stop() ssc.stop() @@ -132,7 +135,7 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w test("stop before start and start after stop") { ssc = new StreamingContext(master, appName, batchDuration) - addInputStream(ssc).register + addInputStream(ssc).register() ssc.stop() // stop before start should not throw exception ssc.start() ssc.stop() @@ -144,12 +147,12 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w test("stop only streaming context") { ssc = new StreamingContext(master, appName, batchDuration) sc = ssc.sparkContext - addInputStream(ssc).register + addInputStream(ssc).register() ssc.start() - ssc.stop(false) + ssc.stop(stopSparkContext = false) assert(sc.makeRDD(1 to 100).collect().size === 100) ssc = new StreamingContext(sc, batchDuration) - addInputStream(ssc).register + addInputStream(ssc).register() ssc.start() ssc.stop() } @@ -164,11 +167,11 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w var runningCount = 0 TestReceiver.counter.set(1) val input = ssc.receiverStream(new TestReceiver) - input.count.foreachRDD(rdd => { + input.count().foreachRDD { rdd => val count = rdd.first() runningCount += count.toInt logInfo("Count = " + count + ", Running count = " + runningCount) - }) + } ssc.start() ssc.awaitTermination(500) ssc.stop(stopSparkContext = false, stopGracefully = true) @@ -188,7 +191,7 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w test("awaitTermination") { ssc = new StreamingContext(master, appName, batchDuration) val inputStream = addInputStream(ssc) - inputStream.map(x => x).register + inputStream.map(x => x).register() // test whether start() blocks indefinitely or not failAfter(2000 millis) { @@ -212,7 +215,7 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w // test whether wait exits if context is stopped failAfter(10000 millis) { // 10 seconds because spark takes a long time to shutdown new Thread() { - override def run { + override def run() { Thread.sleep(500) ssc.stop() } @@ -236,8 +239,9 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w test("awaitTermination with error in task") { ssc = new StreamingContext(master, appName, batchDuration) val inputStream = addInputStream(ssc) - inputStream.map(x => { throw new TestException("error in map task"); x}) - .foreachRDD(_.count) + inputStream + .map { x => throw new TestException("error in map task"); x } + .foreachRDD(_.count()) val exception = intercept[Exception] { ssc.start() @@ -249,7 +253,7 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w test("awaitTermination with error in job generation") { ssc = new StreamingContext(master, appName, batchDuration) val inputStream = addInputStream(ssc) - inputStream.transform(rdd => { throw new TestException("error in transform"); rdd }).register + inputStream.transform { rdd => throw new TestException("error in transform"); rdd }.register() val exception = intercept[TestException] { ssc.start() ssc.awaitTermination(5000) @@ -257,8 +261,12 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w assert(exception.getMessage.contains("transform"), "Expected exception not thrown") } + test("DStream and generated RDD creation sites") { + testPackage.test() + } + def addInputStream(s: StreamingContext): DStream[Int] = { - val input = (1 to 100).map(i => (1 to i)) + val input = (1 to 100).map(i => 1 to i) val inputStream = new TestInputStream(s, input, 1) inputStream } @@ -293,3 +301,37 @@ class TestReceiver extends Receiver[Int](StorageLevel.MEMORY_ONLY) with Logging object TestReceiver { val counter = new AtomicInteger(1) } + +/** Streaming application for testing DStream and RDD creation sites */ +package object testPackage extends Assertions { + def test() { + val conf = new SparkConf().setMaster("local").setAppName("CreationSite test") + val ssc = new StreamingContext(conf , Milliseconds(100)) + try { + val inputStream = ssc.receiverStream(new TestReceiver) + + // Verify creation site of DStream + val creationSite = inputStream.creationSite + assert(creationSite.shortForm.contains("receiverStream") && + creationSite.shortForm.contains("StreamingContextSuite") + ) + assert(creationSite.longForm.contains("testPackage")) + + // Verify creation site of generated RDDs + var rddGenerated = false + var rddCreationSiteCorrect = true + + inputStream.foreachRDD { rdd => + rddCreationSiteCorrect = rdd.creationSite == creationSite + rddGenerated = true + } + ssc.start() + + eventually(timeout(10000 millis), interval(10 millis)) { + assert(rddGenerated && rddCreationSiteCorrect, "RDD creation site was not correct") + } + } finally { + ssc.stop() + } + } +} diff --git a/streaming/src/test/scala/org/apache/spark/streaming/TimeSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/TimeSuite.scala new file mode 100644 index 0000000000000..5579ac364346c --- /dev/null +++ b/streaming/src/test/scala/org/apache/spark/streaming/TimeSuite.scala @@ -0,0 +1,111 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming + +class TimeSuite extends TestSuiteBase { + + test("less") { + assert(new Time(999) < new Time(1000)) + assert(new Time(0) < new Time(1)) + assert(!(new Time(1000) < new Time(999))) + assert(!(new Time(1000) < new Time(1000))) + } + + test("lessEq") { + assert(new Time(999) <= new Time(1000)) + assert(new Time(0) <= new Time(1)) + assert(!(new Time(1000) <= new Time(999))) + assert(new Time(1000) <= new Time(1000)) + } + + test("greater") { + assert(!(new Time(999) > new Time(1000))) + assert(!(new Time(0) > new Time(1))) + assert(new Time(1000) > new Time(999)) + assert(!(new Time(1000) > new Time(1000))) + } + + test("greaterEq") { + assert(!(new Time(999) >= new Time(1000))) + assert(!(new Time(0) >= new Time(1))) + assert(new Time(1000) >= new Time(999)) + assert(new Time(1000) >= new Time(1000)) + } + + test("plus") { + assert((new Time(1000) + new Duration(100)) == new Time(1100)) + assert((new Time(1000) + new Duration(0)) == new Time(1000)) + } + + test("minus Time") { + assert((new Time(1000) - new Time(100)) == new Duration(900)) + assert((new Time(1000) - new Time(0)) == new Duration(1000)) + assert((new Time(1000) - new Time(1000)) == new Duration(0)) + } + + test("minus Duration") { + assert((new Time(1000) - new Duration(100)) == new Time(900)) + assert((new Time(1000) - new Duration(0)) == new Time(1000)) + assert((new Time(1000) - new Duration(1000)) == new Time(0)) + } + + test("floor") { + assert(new Time(1350).floor(new Duration(200)) == new Time(1200)) + assert(new Time(1200).floor(new Duration(200)) == new Time(1200)) + assert(new Time(199).floor(new Duration(200)) == new Time(0)) + assert(new Time(1).floor(new Duration(1)) == new Time(1)) + } + + test("isMultipleOf") { + assert(new Time(1000).isMultipleOf(new Duration(5))) + assert(new Time(1000).isMultipleOf(new Duration(1000))) + assert(new Time(1000).isMultipleOf(new Duration(1))) + assert(!new Time(1000).isMultipleOf(new Duration(6))) + } + + test("min") { + assert(new Time(999).min(new Time(1000)) == new Time(999)) + assert(new Time(1000).min(new Time(999)) == new Time(999)) + assert(new Time(1000).min(new Time(1000)) == new Time(1000)) + } + + test("max") { + assert(new Time(999).max(new Time(1000)) == new Time(1000)) + assert(new Time(1000).max(new Time(999)) == new Time(1000)) + assert(new Time(1000).max(new Time(1000)) == new Time(1000)) + } + + test("until") { + assert(new Time(1000).until(new Time(1100), new Duration(100)) == + Seq(Time(1000))) + assert(new Time(1000).until(new Time(1000), new Duration(100)) == + Seq()) + assert(new Time(1000).until(new Time(1100), new Duration(30)) == + Seq(Time(1000), Time(1030), Time(1060), Time(1090))) + } + + test("to") { + assert(new Time(1000).to(new Time(1100), new Duration(100)) == + Seq(Time(1000), Time(1100))) + assert(new Time(1000).to(new Time(1000), new Duration(100)) == + Seq(Time(1000))) + assert(new Time(1000).to(new Time(1100), new Duration(30)) == + Seq(Time(1000), Time(1030), Time(1060), Time(1090))) + } + +} diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index aff9ab71f0937..5a20532315e59 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -23,13 +23,11 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.io.DataOutputBuffer import org.apache.hadoop.security.UserGroupInformation import org.apache.hadoop.yarn.api._ -import org.apache.hadoop.yarn.api.ApplicationConstants.Environment import org.apache.hadoop.yarn.api.protocolrecords._ import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.client.YarnClientImpl import org.apache.hadoop.yarn.conf.YarnConfiguration -import org.apache.hadoop.yarn.ipc.YarnRPC -import org.apache.hadoop.yarn.util.{Apps, Records} +import org.apache.hadoop.yarn.util.Records import org.apache.spark.{Logging, SparkConf} import org.apache.spark.deploy.SparkHadoopUtil @@ -37,7 +35,10 @@ import org.apache.spark.deploy.SparkHadoopUtil /** * Version of [[org.apache.spark.deploy.yarn.ClientBase]] tailored to YARN's alpha API. */ -class Client(clientArgs: ClientArguments, hadoopConf: Configuration, spConf: SparkConf) +private[spark] class Client( + val args: ClientArguments, + val hadoopConf: Configuration, + val sparkConf: SparkConf) extends YarnClientImpl with ClientBase with Logging { def this(clientArgs: ClientArguments, spConf: SparkConf) = @@ -45,112 +46,86 @@ class Client(clientArgs: ClientArguments, hadoopConf: Configuration, spConf: Spa def this(clientArgs: ClientArguments) = this(clientArgs, new SparkConf()) - val args = clientArgs - val conf = hadoopConf - val sparkConf = spConf - var rpc: YarnRPC = YarnRPC.create(conf) - val yarnConf: YarnConfiguration = new YarnConfiguration(conf) + val yarnConf: YarnConfiguration = new YarnConfiguration(hadoopConf) + /* ------------------------------------------------------------------------------------- * + | The following methods have much in common in the stable and alpha versions of Client, | + | but cannot be implemented in the parent trait due to subtle API differences across | + | hadoop versions. | + * ------------------------------------------------------------------------------------- */ - // for client user who want to monitor app status by itself. - def runApp() = { - validateArgs() - + /** Submit an application running our ApplicationMaster to the ResourceManager. */ + override def submitApplication(): ApplicationId = { init(yarnConf) start() - logClusterResourceDetails() - val newApp = super.getNewApplication() - val appId = newApp.getApplicationId() + logInfo("Requesting a new application from cluster with %d NodeManagers" + .format(getYarnClusterMetrics.getNumNodeManagers)) - verifyClusterResources(newApp) - val appContext = createApplicationSubmissionContext(appId) - val appStagingDir = getAppStagingDir(appId) - val localResources = prepareLocalResources(appStagingDir) - val env = setupLaunchEnv(localResources, appStagingDir) - val amContainer = createContainerLaunchContext(newApp, localResources, env) + // Get a new application from our RM + val newAppResponse = getNewApplication() + val appId = newAppResponse.getApplicationId() - val capability = Records.newRecord(classOf[Resource]).asInstanceOf[Resource] - // Memory for the ApplicationMaster. - capability.setMemory(args.amMemory + memoryOverhead) - amContainer.setResource(capability) + // Verify whether the cluster has enough resources for our AM + verifyClusterResources(newAppResponse) - appContext.setQueue(args.amQueue) - appContext.setAMContainerSpec(amContainer) - appContext.setUser(UserGroupInformation.getCurrentUser().getShortUserName()) + // Set up the appropriate contexts to launch our AM + val containerContext = createContainerLaunchContext(newAppResponse) + val appContext = createApplicationSubmissionContext(appId, containerContext) - submitApp(appContext) + // Finally, submit and monitor the application + logInfo(s"Submitting application ${appId.getId} to ResourceManager") + submitApplication(appContext) appId } - def run() { - val appId = runApp() - monitorApplication(appId) - } - - def logClusterResourceDetails() { - val clusterMetrics: YarnClusterMetrics = super.getYarnClusterMetrics - logInfo("Got cluster metric info from ASM, numNodeManagers = " + - clusterMetrics.getNumNodeManagers) + /** + * Set up a context for launching our ApplicationMaster container. + * In the Yarn alpha API, the memory requirements of this container must be set in + * the ContainerLaunchContext instead of the ApplicationSubmissionContext. + */ + override def createContainerLaunchContext(newAppResponse: GetNewApplicationResponse) + : ContainerLaunchContext = { + val containerContext = super.createContainerLaunchContext(newAppResponse) + val capability = Records.newRecord(classOf[Resource]) + capability.setMemory(args.amMemory + amMemoryOverhead) + containerContext.setResource(capability) + containerContext } - - def createApplicationSubmissionContext(appId: ApplicationId): ApplicationSubmissionContext = { - logInfo("Setting up application submission context for ASM") + /** Set up the context for submitting our ApplicationMaster. */ + def createApplicationSubmissionContext( + appId: ApplicationId, + containerContext: ContainerLaunchContext): ApplicationSubmissionContext = { val appContext = Records.newRecord(classOf[ApplicationSubmissionContext]) appContext.setApplicationId(appId) appContext.setApplicationName(args.appName) + appContext.setQueue(args.amQueue) + appContext.setAMContainerSpec(containerContext) + appContext.setUser(UserGroupInformation.getCurrentUser.getShortUserName) appContext } - def setupSecurityToken(amContainer: ContainerLaunchContext) = { - // Setup security tokens. + /** + * Set up security tokens for launching our ApplicationMaster container. + * ContainerLaunchContext#setContainerTokens is renamed `setTokens` in the stable API. + */ + override def setupSecurityToken(amContainer: ContainerLaunchContext): Unit = { val dob = new DataOutputBuffer() credentials.writeTokenStorageToStream(dob) amContainer.setContainerTokens(ByteBuffer.wrap(dob.getData())) } - def submitApp(appContext: ApplicationSubmissionContext) = { - // Submit the application to the applications manager. - logInfo("Submitting application to ASM") - super.submitApplication(appContext) - } - - def monitorApplication(appId: ApplicationId): Boolean = { - val interval = sparkConf.getLong("spark.yarn.report.interval", 1000) - - while (true) { - Thread.sleep(interval) - val report = super.getApplicationReport(appId) - - logInfo("Application report from ASM: \n" + - "\t application identifier: " + appId.toString() + "\n" + - "\t appId: " + appId.getId() + "\n" + - "\t clientToken: " + report.getClientToken() + "\n" + - "\t appDiagnostics: " + report.getDiagnostics() + "\n" + - "\t appMasterHost: " + report.getHost() + "\n" + - "\t appQueue: " + report.getQueue() + "\n" + - "\t appMasterRpcPort: " + report.getRpcPort() + "\n" + - "\t appStartTime: " + report.getStartTime() + "\n" + - "\t yarnAppState: " + report.getYarnApplicationState() + "\n" + - "\t distributedFinalState: " + report.getFinalApplicationStatus() + "\n" + - "\t appTrackingUrl: " + report.getTrackingUrl() + "\n" + - "\t appUser: " + report.getUser() - ) - - val state = report.getYarnApplicationState() - if (state == YarnApplicationState.FINISHED || - state == YarnApplicationState.FAILED || - state == YarnApplicationState.KILLED) { - return true - } - } - true - } + /** + * Return the security token used by this client to communicate with the ApplicationMaster. + * If no security is enabled, the token returned by the report is null. + * ApplicationReport#getClientToken is renamed `getClientToAMToken` in the stable API. + */ + override def getClientToken(report: ApplicationReport): String = + Option(report.getClientToken).getOrElse("") } object Client { - def main(argStrings: Array[String]) { if (!sys.props.contains("SPARK_SUBMIT")) { println("WARNING: This client is deprecated and will be removed in a " + @@ -158,19 +133,17 @@ object Client { } // Set an env variable indicating we are running in YARN mode. - // Note that anything with SPARK prefix gets propagated to all (remote) processes + // Note that any env variable with the SPARK_ prefix gets propagated to all (remote) processes System.setProperty("SPARK_YARN_MODE", "true") - val sparkConf = new SparkConf try { val args = new ClientArguments(argStrings, sparkConf) new Client(args, sparkConf).run() } catch { - case e: Exception => { + case e: Exception => Console.err.println(e.getMessage) System.exit(1) - } } System.exit(0) diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala index 10cbeb8b94325..229b7a09f456b 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala @@ -47,6 +47,7 @@ class ExecutorRunnable( hostname: String, executorMemory: Int, executorCores: Int, + appAttemptId: String, securityMgr: SecurityManager) extends Runnable with ExecutorRunnableUtil with Logging { @@ -83,7 +84,7 @@ class ExecutorRunnable( ctx.setContainerTokens(ByteBuffer.wrap(dob.getData())) val commands = prepareCommand(masterAddress, slaveId, hostname, executorMemory, executorCores, - localResources) + appAttemptId, localResources) logInfo("Setting up executor with commands: " + commands) ctx.setCommands(commands) diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala index acf26505e4cf9..7faf55bc63372 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala @@ -40,6 +40,7 @@ private class YarnRMClientImpl(args: ApplicationMasterArguments) extends YarnRMC private var rpc: YarnRPC = null private var resourceManager: AMRMProtocol = _ private var uiHistoryAddress: String = _ + private var registered: Boolean = false override def register( conf: YarnConfiguration, @@ -51,8 +52,11 @@ private class YarnRMClientImpl(args: ApplicationMasterArguments) extends YarnRMC this.rpc = YarnRPC.create(conf) this.uiHistoryAddress = uiHistoryAddress - resourceManager = registerWithResourceManager(conf) - registerApplicationMaster(uiAddress) + synchronized { + resourceManager = registerWithResourceManager(conf) + registerApplicationMaster(uiAddress) + registered = true + } new YarnAllocationHandler(conf, sparkConf, resourceManager, getAttemptId(), args, preferredNodeLocations, securityMgr) @@ -66,18 +70,24 @@ private class YarnRMClientImpl(args: ApplicationMasterArguments) extends YarnRMC appAttemptId } - override def shutdown(status: FinalApplicationStatus, diagnostics: String = "") = { - val finishReq = Records.newRecord(classOf[FinishApplicationMasterRequest]) - .asInstanceOf[FinishApplicationMasterRequest] - finishReq.setAppAttemptId(getAttemptId()) - finishReq.setFinishApplicationStatus(status) - finishReq.setDiagnostics(diagnostics) - finishReq.setTrackingUrl(uiHistoryAddress) - resourceManager.finishApplicationMaster(finishReq) + override def unregister(status: FinalApplicationStatus, diagnostics: String = "") = synchronized { + if (registered) { + val finishReq = Records.newRecord(classOf[FinishApplicationMasterRequest]) + .asInstanceOf[FinishApplicationMasterRequest] + finishReq.setAppAttemptId(getAttemptId()) + finishReq.setFinishApplicationStatus(status) + finishReq.setDiagnostics(diagnostics) + finishReq.setTrackingUrl(uiHistoryAddress) + resourceManager.finishApplicationMaster(finishReq) + } } - override def getProxyHostAndPort(conf: YarnConfiguration) = - YarnConfiguration.getProxyHostAndPort(conf) + override def getAmIpFilterParams(conf: YarnConfiguration, proxyBase: String) = { + val proxy = YarnConfiguration.getProxyHostAndPort(conf) + val parts = proxy.split(":") + val uriBase = "http://" + proxy + proxyBase + Map("PROXY_HOST" -> parts(0), "PROXY_URI_BASE" -> uriBase) + } override def getMaxRegAttempts(conf: YarnConfiguration) = conf.getInt(YarnConfiguration.RM_AM_MAX_RETRIES, YarnConfiguration.DEFAULT_RM_AM_MAX_RETRIES) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index cde5fff637a39..a3c43b43848d2 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -17,7 +17,10 @@ package org.apache.spark.deploy.yarn +import scala.util.control.NonFatal + import java.io.IOException +import java.lang.reflect.InvocationTargetException import java.net.Socket import java.util.concurrent.atomic.AtomicReference @@ -30,6 +33,7 @@ import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext, SparkEnv} +import org.apache.spark.SparkException import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.history.HistoryServer import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend @@ -53,8 +57,12 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, private val maxNumExecutorFailures = sparkConf.getInt("spark.yarn.max.executor.failures", sparkConf.getInt("spark.yarn.max.worker.failures", math.max(args.numExecutors * 2, 3))) + @volatile private var exitCode = 0 + @volatile private var unregistered = false @volatile private var finished = false @volatile private var finalStatus = FinalApplicationStatus.UNDEFINED + @volatile private var finalMsg: String = "" + @volatile private var userClassThread: Thread = _ private var reporterThread: Thread = _ private var allocator: YarnAllocator = _ @@ -67,80 +75,107 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, private val sparkContextRef = new AtomicReference[SparkContext](null) final def run(): Int = { - val appAttemptId = client.getAttemptId() + try { + val appAttemptId = client.getAttemptId() - if (isDriver) { - // Set the web ui port to be ephemeral for yarn so we don't conflict with - // other spark processes running on the same box - System.setProperty("spark.ui.port", "0") + if (isDriver) { + // Set the web ui port to be ephemeral for yarn so we don't conflict with + // other spark processes running on the same box + System.setProperty("spark.ui.port", "0") - // Set the master property to match the requested mode. - System.setProperty("spark.master", "yarn-cluster") + // Set the master property to match the requested mode. + System.setProperty("spark.master", "yarn-cluster") - // Propagate the application ID so that YarnClusterSchedulerBackend can pick it up. - System.setProperty("spark.yarn.app.id", appAttemptId.getApplicationId().toString()) - } + // Propagate the application ID so that YarnClusterSchedulerBackend can pick it up. + System.setProperty("spark.yarn.app.id", appAttemptId.getApplicationId().toString()) + } - logInfo("ApplicationAttemptId: " + appAttemptId) + logInfo("ApplicationAttemptId: " + appAttemptId) - val cleanupHook = new Runnable { - override def run() { - // If the SparkContext is still registered, shut it down as a best case effort in case - // users do not call sc.stop or do System.exit(). - val sc = sparkContextRef.get() - if (sc != null) { - logInfo("Invoking sc stop from shutdown hook") - sc.stop() - finish(FinalApplicationStatus.SUCCEEDED) - } + val cleanupHook = new Runnable { + override def run() { + // If the SparkContext is still registered, shut it down as a best case effort in case + // users do not call sc.stop or do System.exit(). + val sc = sparkContextRef.get() + if (sc != null) { + logInfo("Invoking sc stop from shutdown hook") + sc.stop() + } + val maxAppAttempts = client.getMaxRegAttempts(yarnConf) + val isLastAttempt = client.getAttemptId().getAttemptId() >= maxAppAttempts - // Cleanup the staging dir after the app is finished, or if it's the last attempt at - // running the AM. - val maxAppAttempts = client.getMaxRegAttempts(yarnConf) - val isLastAttempt = client.getAttemptId().getAttemptId() >= maxAppAttempts - if (finished || isLastAttempt) { - cleanupStagingDir() + if (!finished) { + // this shouldn't ever happen, but if it does assume weird failure + finish(FinalApplicationStatus.FAILED, + ApplicationMaster.EXIT_UNCAUGHT_EXCEPTION, + "shutdown hook called without cleanly finishing") + } + + if (!unregistered) { + // we only want to unregister if we don't want the RM to retry + if (finalStatus == FinalApplicationStatus.SUCCEEDED || isLastAttempt) { + unregister(finalStatus, finalMsg) + cleanupStagingDir() + } + } } } - } - // Use higher priority than FileSystem. - assert(ApplicationMaster.SHUTDOWN_HOOK_PRIORITY > FileSystem.SHUTDOWN_HOOK_PRIORITY) - ShutdownHookManager - .get().addShutdownHook(cleanupHook, ApplicationMaster.SHUTDOWN_HOOK_PRIORITY) + // Use higher priority than FileSystem. + assert(ApplicationMaster.SHUTDOWN_HOOK_PRIORITY > FileSystem.SHUTDOWN_HOOK_PRIORITY) + ShutdownHookManager + .get().addShutdownHook(cleanupHook, ApplicationMaster.SHUTDOWN_HOOK_PRIORITY) - // Call this to force generation of secret so it gets populated into the - // Hadoop UGI. This has to happen before the startUserClass which does a - // doAs in order for the credentials to be passed on to the executor containers. - val securityMgr = new SecurityManager(sparkConf) + // Call this to force generation of secret so it gets populated into the + // Hadoop UGI. This has to happen before the startUserClass which does a + // doAs in order for the credentials to be passed on to the executor containers. + val securityMgr = new SecurityManager(sparkConf) - if (isDriver) { - runDriver(securityMgr) - } else { - runExecutorLauncher(securityMgr) + if (isDriver) { + runDriver(securityMgr) + } else { + runExecutorLauncher(securityMgr) + } + } catch { + case e: Exception => + // catch everything else if not specifically handled + logError("Uncaught exception: ", e) + finish(FinalApplicationStatus.FAILED, + ApplicationMaster.EXIT_UNCAUGHT_EXCEPTION, + "Uncaught exception: " + e.getMessage()) } + exitCode + } - if (finalStatus != FinalApplicationStatus.UNDEFINED) { - finish(finalStatus) - 0 - } else { - 1 + /** + * unregister is used to completely unregister the application from the ResourceManager. + * This means the ResourceManager will not retry the application attempt on your behalf if + * a failure occurred. + */ + final def unregister(status: FinalApplicationStatus, diagnostics: String = null) = synchronized { + if (!unregistered) { + logInfo(s"Unregistering ApplicationMaster with $status" + + Option(diagnostics).map(msg => s" (diag message: $msg)").getOrElse("")) + unregistered = true + client.unregister(status, Option(diagnostics).getOrElse("")) } } - final def finish(status: FinalApplicationStatus, diagnostics: String = null) = synchronized { + final def finish(status: FinalApplicationStatus, code: Int, msg: String = null) = synchronized { if (!finished) { - logInfo(s"Finishing ApplicationMaster with $status" + - Option(diagnostics).map(msg => s" (diag message: $msg)").getOrElse("")) - finished = true + logInfo(s"Final app status: ${status}, exitCode: ${code}" + + Option(msg).map(msg => s", (reason: $msg)").getOrElse("")) + exitCode = code finalStatus = status - try { - if (Thread.currentThread() != reporterThread) { - reporterThread.interrupt() - reporterThread.join() - } - } finally { - client.shutdown(status, Option(diagnostics).getOrElse("")) + finalMsg = msg + finished = true + if (Thread.currentThread() != reporterThread && reporterThread != null) { + logDebug("shutting down reporter thread") + reporterThread.interrupt() + } + if (Thread.currentThread() != userClassThread && userClassThread != null) { + logDebug("shutting down user thread") + userClassThread.interrupt() } } } @@ -178,7 +213,8 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, private def runDriver(securityMgr: SecurityManager): Unit = { addAmIpFilter() - val userThread = startUserClass() + setupSystemSecurityManager() + userClassThread = startUserClass() // This a bit hacky, but we need to wait until the spark.driver.port property has // been set by the Thread executing the user class. @@ -186,15 +222,12 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, // If there is no SparkContext at this point, just fail the app. if (sc == null) { - finish(FinalApplicationStatus.FAILED, "Timed out waiting for SparkContext.") + finish(FinalApplicationStatus.FAILED, + ApplicationMaster.EXIT_SC_NOT_INITED, + "Timed out waiting for SparkContext.") } else { registerAM(sc.ui.map(_.appUIAddress).getOrElse(""), securityMgr) - try { - userThread.join() - } finally { - // In cluster mode, ask the reporter thread to stop since the user app is finished. - reporterThread.interrupt() - } + userClassThread.join() } } @@ -207,7 +240,6 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, // In client mode the actor will stop the reporter thread. reporterThread.join() - finalStatus = FinalApplicationStatus.SUCCEEDED } private def launchReporterThread(): Thread = { @@ -221,19 +253,42 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, // must be <= expiryInterval / 2. val interval = math.max(0, math.min(expiryInterval / 2, schedulerInterval)) + // The number of failures in a row until Reporter thread give up + val reporterMaxFailures = sparkConf.getInt("spark.yarn.scheduler.reporterThread.maxFailures", 5) + val t = new Thread { override def run() { + var failureCount = 0 while (!finished) { - checkNumExecutorsFailed() - if (!finished) { - logDebug("Sending progress") - allocator.allocateResources() - try { - Thread.sleep(interval) - } catch { - case e: InterruptedException => + try { + if (allocator.getNumExecutorsFailed >= maxNumExecutorFailures) { + finish(FinalApplicationStatus.FAILED, + ApplicationMaster.EXIT_MAX_EXECUTOR_FAILURES, + "Max number of executor failures reached") + } else { + logDebug("Sending progress") + allocator.allocateResources() + } + failureCount = 0 + } catch { + case i: InterruptedException => + case e: Throwable => { + failureCount += 1 + if (!NonFatal(e) || failureCount >= reporterMaxFailures) { + finish(FinalApplicationStatus.FAILED, + ApplicationMaster.EXIT_REPORTER_FAILURE, "Exception was thrown " + + s"${failureCount} time(s) from Reporter thread.") + + } else { + logWarning(s"Reporter thread fails ${failureCount} time(s) in a row.", e) + } } } + try { + Thread.sleep(interval) + } catch { + case e: InterruptedException => + } } } } @@ -274,7 +329,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, sparkContextRef.synchronized { var count = 0 val waitTime = 10000L - val numTries = sparkConf.getInt("spark.yarn.ApplicationMaster.waitTries", 10) + val numTries = sparkConf.getInt("spark.yarn.applicationMaster.waitTries", 10) while (sparkContextRef.get() == null && count < numTries && !finished) { logInfo("Waiting for spark context initialization ... " + count) count = count + 1 @@ -294,10 +349,19 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, private def waitForSparkDriver(): ActorRef = { logInfo("Waiting for Spark driver to be reachable.") var driverUp = false + var count = 0 val hostport = args.userArgs(0) val (driverHost, driverPort) = Utils.parseHostPort(hostport) - while (!driverUp) { + + // spark driver should already be up since it launched us, but we don't want to + // wait forever, so wait 100 seconds max to match the cluster mode setting. + // Leave this config unpublished for now. SPARK-3779 to investigating changing + // this config to be time based. + val numTries = sparkConf.getInt("spark.yarn.applicationMaster.waitTries", 1000) + + while (!driverUp && !finished && count < numTries) { try { + count = count + 1 val socket = new Socket(driverHost, driverPort) socket.close() logInfo("Driver now available: %s:%s".format(driverHost, driverPort)) @@ -309,6 +373,11 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, Thread.sleep(100) } } + + if (!driverUp) { + throw new SparkException("Failed to connect to driver!") + } + sparkConf.set("spark.driver.host", driverHost) sparkConf.set("spark.driver.port", driverPort.toString) @@ -320,61 +389,94 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, actorSystem.actorOf(Props(new MonitorActor(driverUrl)), name = "YarnAM") } - private def checkNumExecutorsFailed() = { - if (allocator.getNumExecutorsFailed >= maxNumExecutorFailures) { - finish(FinalApplicationStatus.FAILED, "Max number of executor failures reached.") - - val sc = sparkContextRef.get() - if (sc != null) { - logInfo("Invoking sc stop from checkNumExecutorsFailed") - sc.stop() - } - } - } - /** Add the Yarn IP filter that is required for properly securing the UI. */ private def addAmIpFilter() = { - val amFilter = "org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter" - val proxy = client.getProxyHostAndPort(yarnConf) - val parts = proxy.split(":") val proxyBase = System.getenv(ApplicationConstants.APPLICATION_WEB_PROXY_BASE_ENV) - val uriBase = "http://" + proxy + proxyBase - val params = "PROXY_HOST=" + parts(0) + "," + "PROXY_URI_BASE=" + uriBase - + val amFilter = "org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter" + val params = client.getAmIpFilterParams(yarnConf, proxyBase) if (isDriver) { System.setProperty("spark.ui.filters", amFilter) - System.setProperty(s"spark.$amFilter.params", params) + params.foreach { case (k, v) => System.setProperty(s"spark.$amFilter.param.$k", v) } } else { - actor ! AddWebUIFilter(amFilter, params, proxyBase) + actor ! AddWebUIFilter(amFilter, params.toMap, proxyBase) } } + /** + * This system security manager applies to the entire process. + * It's main purpose is to handle the case if the user code does a System.exit. + * This allows us to catch that and properly set the YARN application status and + * cleanup if needed. + */ + private def setupSystemSecurityManager(): Unit = { + try { + var stopped = false + System.setSecurityManager(new java.lang.SecurityManager() { + override def checkExit(paramInt: Int) { + if (!stopped) { + logInfo("In securityManager checkExit, exit code: " + paramInt) + if (paramInt == 0) { + finish(FinalApplicationStatus.SUCCEEDED, ApplicationMaster.EXIT_SUCCESS) + } else { + finish(FinalApplicationStatus.FAILED, + paramInt, + "User class exited with non-zero exit code") + } + stopped = true + } + } + // required for the checkExit to work properly + override def checkPermission(perm: java.security.Permission): Unit = {} + }) + } + catch { + case e: SecurityException => + finish(FinalApplicationStatus.FAILED, + ApplicationMaster.EXIT_SECURITY, + "Error in setSecurityManager") + logError("Error in setSecurityManager:", e) + } + } + + /** + * Start the user class, which contains the spark driver, in a separate Thread. + * If the main routine exits cleanly or exits with System.exit(0) we + * assume it was successful, for all other cases we assume failure. + * + * Returns the user thread that was started. + */ private def startUserClass(): Thread = { logInfo("Starting the user JAR in a separate Thread") System.setProperty("spark.executor.instances", args.numExecutors.toString) val mainMethod = Class.forName(args.userClass, false, Thread.currentThread.getContextClassLoader).getMethod("main", classOf[Array[String]]) - val t = new Thread { + val userThread = new Thread { override def run() { - var status = FinalApplicationStatus.FAILED try { - // Copy val mainArgs = new Array[String](args.userArgs.size) args.userArgs.copyToArray(mainArgs, 0, args.userArgs.size) mainMethod.invoke(null, mainArgs) - // Some apps have "System.exit(0)" at the end. The user thread will stop here unless - // it has an uncaught exception thrown out. It needs a shutdown hook to set SUCCEEDED. - status = FinalApplicationStatus.SUCCEEDED - } finally { - logDebug("Finishing main") + finish(FinalApplicationStatus.SUCCEEDED, ApplicationMaster.EXIT_SUCCESS) + logDebug("Done running users class") + } catch { + case e: InvocationTargetException => + e.getCause match { + case _: InterruptedException => + // Reporter thread can interrupt to stop user class + case e: Exception => + finish(FinalApplicationStatus.FAILED, + ApplicationMaster.EXIT_EXCEPTION_USER_CLASS, + "User class threw exception: " + e.getMessage) + // re-throw to get it logged + throw e + } } - finalStatus = status } } - t.setName("Driver") - t.start() - t + userThread.setName("Driver") + userThread.start() + userThread } // Actor used to monitor the driver when running in client deploy mode. @@ -394,7 +496,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, override def receive = { case x: DisassociatedEvent => logInfo(s"Driver terminated or disconnected! Shutting down. $x") - finish(FinalApplicationStatus.SUCCEEDED) + finish(FinalApplicationStatus.SUCCEEDED, ApplicationMaster.EXIT_SUCCESS) case x: AddWebUIFilter => logInfo(s"Add WebUI Filter. $x") driver ! x @@ -408,6 +510,15 @@ object ApplicationMaster extends Logging { val SHUTDOWN_HOOK_PRIORITY: Int = 30 + // exit codes for different causes, no reason behind the values + private val EXIT_SUCCESS = 0 + private val EXIT_UNCAUGHT_EXCEPTION = 10 + private val EXIT_MAX_EXECUTOR_FAILURES = 11 + private val EXIT_REPORTER_FAILURE = 12 + private val EXIT_SC_NOT_INITED = 13 + private val EXIT_SECURITY = 14 + private val EXIT_EXCEPTION_USER_CLASS = 15 + private var master: ApplicationMaster = _ def main(args: Array[String]) = { diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala index 40d8d6d6e6961..a12f82d2fbe70 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala @@ -17,15 +17,14 @@ package org.apache.spark.deploy.yarn -import scala.collection.mutable.{ArrayBuffer, HashMap} +import scala.collection.mutable.ArrayBuffer import org.apache.spark.SparkConf -import org.apache.spark.scheduler.InputFormatInfo import org.apache.spark.util.{Utils, IntParam, MemoryParam} - +import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil._ // TODO: Add code and support for ensuring that yarn resource 'tasks' are location aware ! -class ClientArguments(val args: Array[String], val sparkConf: SparkConf) { +private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) { var addJars: String = null var files: String = null var archives: String = null @@ -35,28 +34,51 @@ class ClientArguments(val args: Array[String], val sparkConf: SparkConf) { var executorMemory = 1024 // MB var executorCores = 1 var numExecutors = 2 - var amQueue = sparkConf.get("QUEUE", "default") + var amQueue = sparkConf.get("spark.yarn.queue", "default") var amMemory: Int = 512 // MB var appName: String = "Spark" var priority = 0 parseArgs(args.toList) + loadEnvironmentArgs() + + // Additional memory to allocate to containers + // For now, use driver's memory overhead as our AM container's memory overhead + val amMemoryOverhead = sparkConf.getInt("spark.yarn.driver.memoryOverhead", + math.max((MEMORY_OVERHEAD_FACTOR * amMemory).toInt, MEMORY_OVERHEAD_MIN)) + + val executorMemoryOverhead = sparkConf.getInt("spark.yarn.executor.memoryOverhead", + math.max((MEMORY_OVERHEAD_FACTOR * executorMemory).toInt, MEMORY_OVERHEAD_MIN)) + + validateArgs() + + /** Load any default arguments provided through environment variables and Spark properties. */ + private def loadEnvironmentArgs(): Unit = { + // For backward compatibility, SPARK_YARN_DIST_{ARCHIVES/FILES} should be resolved to hdfs://, + // while spark.yarn.dist.{archives/files} should be resolved to file:// (SPARK-2051). + files = Option(files) + .orElse(sys.env.get("SPARK_YARN_DIST_FILES")) + .orElse(sparkConf.getOption("spark.yarn.dist.files").map(p => Utils.resolveURIs(p))) + .orNull + archives = Option(archives) + .orElse(sys.env.get("SPARK_YARN_DIST_ARCHIVES")) + .orElse(sparkConf.getOption("spark.yarn.dist.archives").map(p => Utils.resolveURIs(p))) + .orNull + } - // env variable SPARK_YARN_DIST_ARCHIVES/SPARK_YARN_DIST_FILES set in yarn-client then - // it should default to hdfs:// - files = Option(files).getOrElse(sys.env.get("SPARK_YARN_DIST_FILES").orNull) - archives = Option(archives).getOrElse(sys.env.get("SPARK_YARN_DIST_ARCHIVES").orNull) - - // spark.yarn.dist.archives/spark.yarn.dist.files defaults to use file:// if not specified, - // for both yarn-client and yarn-cluster - files = Option(files).getOrElse(sparkConf.getOption("spark.yarn.dist.files"). - map(p => Utils.resolveURIs(p)).orNull) - archives = Option(archives).getOrElse(sparkConf.getOption("spark.yarn.dist.archives"). - map(p => Utils.resolveURIs(p)).orNull) + /** + * Fail fast if any arguments provided are invalid. + * This is intended to be called only after the provided arguments have been parsed. + */ + private def validateArgs(): Unit = { + if (numExecutors <= 0) { + throw new IllegalArgumentException( + "You must specify at least 1 executor!\n" + getUsageMessage()) + } + } private def parseArgs(inputArgs: List[String]): Unit = { - val userArgsBuffer: ArrayBuffer[String] = new ArrayBuffer[String]() - + val userArgsBuffer = new ArrayBuffer[String]() var args = inputArgs while (!args.isEmpty) { @@ -138,16 +160,14 @@ class ClientArguments(val args: Array[String], val sparkConf: SparkConf) { userArgs = userArgsBuffer.readOnly } - - def getUsageMessage(unknownParam: Any = null): String = { + private def getUsageMessage(unknownParam: List[String] = null): String = { val message = if (unknownParam != null) s"Unknown/unsupported param $unknownParam\n" else "" - message + "Usage: org.apache.spark.deploy.yarn.Client [options] \n" + "Options:\n" + " --jar JAR_PATH Path to your application's JAR file (required in yarn-cluster mode)\n" + " --class CLASS_NAME Name of your application's main class (required)\n" + - " --arg ARGS Argument to be passed to your application's main class.\n" + + " --arg ARG Argument to be passed to your application's main class.\n" + " Multiple invocations are possible, each will be passed in order.\n" + " --num-executors NUM Number of executors to start (Default: 2)\n" + " --executor-cores NUM Number of cores for the executors (Default: 1).\n" + 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 c96f731923d22..6ecac6eae6e03 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 @@ -17,7 +17,6 @@ package org.apache.spark.deploy.yarn -import java.io.File import java.net.{InetAddress, UnknownHostException, URI, URISyntaxException} import scala.collection.JavaConversions._ @@ -37,154 +36,111 @@ import org.apache.hadoop.yarn.api.protocolrecords._ import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.util.Records + import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext, SparkException} /** - * The entry point (starting in Client#main() and Client#run()) for launching Spark on YARN. The - * Client submits an application to the YARN ResourceManager. + * The entry point (starting in Client#main() and Client#run()) for launching Spark on YARN. + * The Client submits an application to the YARN ResourceManager. */ -trait ClientBase extends Logging { - val args: ClientArguments - val conf: Configuration - val sparkConf: SparkConf - val yarnConf: YarnConfiguration - val credentials = UserGroupInformation.getCurrentUser().getCredentials() - private val SPARK_STAGING: String = ".sparkStaging" +private[spark] trait ClientBase extends Logging { + import ClientBase._ + + protected val args: ClientArguments + protected val hadoopConf: Configuration + protected val sparkConf: SparkConf + protected val yarnConf: YarnConfiguration + protected val credentials = UserGroupInformation.getCurrentUser.getCredentials + protected val amMemoryOverhead = args.amMemoryOverhead // MB + protected val executorMemoryOverhead = args.executorMemoryOverhead // MB private val distCacheMgr = new ClientDistributedCacheManager() - // Staging directory is private! -> rwx-------- - val STAGING_DIR_PERMISSION: FsPermission = - FsPermission.createImmutable(Integer.parseInt("700", 8).toShort) - // App files are world-wide readable and owner writable -> rw-r--r-- - val APP_FILE_PERMISSION: FsPermission = - FsPermission.createImmutable(Integer.parseInt("644", 8).toShort) - - // Additional memory overhead - in mb. - protected def memoryOverhead: Int = sparkConf.getInt("spark.yarn.driver.memoryOverhead", - YarnSparkHadoopUtil.DEFAULT_MEMORY_OVERHEAD) - - // TODO(harvey): This could just go in ClientArguments. - def validateArgs() = { - Map( - (args.numExecutors <= 0) -> "Error: You must specify at least 1 executor!", - (args.amMemory <= memoryOverhead) -> ("Error: AM memory size must be" + - "greater than: " + memoryOverhead), - (args.executorMemory <= memoryOverhead) -> ("Error: Executor memory size" + - "must be greater than: " + memoryOverhead.toString) - ).foreach { case(cond, errStr) => - if (cond) { - logError(errStr) - throw new IllegalArgumentException(args.getUsageMessage()) - } - } - } - - def getAppStagingDir(appId: ApplicationId): String = { - SPARK_STAGING + Path.SEPARATOR + appId.toString() + Path.SEPARATOR - } - - def verifyClusterResources(app: GetNewApplicationResponse) = { - val maxMem = app.getMaximumResourceCapability().getMemory() - logInfo("Max mem capabililty of a single resource in this cluster " + maxMem) - - // If we have requested more then the clusters max for a single resource then exit. - if (args.executorMemory > maxMem) { - val errorMessage = - "Required executor memory (%d MB), is above the max threshold (%d MB) of this cluster." - .format(args.executorMemory, maxMem) - - logError(errorMessage) - throw new IllegalArgumentException(errorMessage) - } - val amMem = args.amMemory + memoryOverhead + /** + * Fail fast if we have requested more resources per container than is available in the cluster. + */ + protected def verifyClusterResources(newAppResponse: GetNewApplicationResponse): Unit = { + val maxMem = newAppResponse.getMaximumResourceCapability().getMemory() + logInfo("Verifying our application has not requested more than the maximum " + + s"memory capability of the cluster ($maxMem MB per container)") + val executorMem = args.executorMemory + executorMemoryOverhead + if (executorMem > maxMem) { + throw new IllegalArgumentException(s"Required executor memory (${args.executorMemory}" + + s"+$executorMemoryOverhead MB) is above the max threshold ($maxMem MB) of this cluster!") + } + val amMem = args.amMemory + amMemoryOverhead if (amMem > maxMem) { - - val errorMessage = "Required AM memory (%d) is above the max threshold (%d) of this cluster." - .format(amMem, maxMem) - logError(errorMessage) - throw new IllegalArgumentException(errorMessage) + throw new IllegalArgumentException(s"Required AM memory (${args.amMemory}" + + s"+$amMemoryOverhead MB) is above the max threshold ($maxMem MB) of this cluster!") } + logInfo("Will allocate AM container, with %d MB memory including %d MB overhead".format( + amMem, + amMemoryOverhead)) // We could add checks to make sure the entire cluster has enough resources but that involves // getting all the node reports and computing ourselves. } - /** See if two file systems are the same or not. */ - private def compareFs(srcFs: FileSystem, destFs: FileSystem): Boolean = { - val srcUri = srcFs.getUri() - val dstUri = destFs.getUri() - if (srcUri.getScheme() == null) { - return false - } - if (!srcUri.getScheme().equals(dstUri.getScheme())) { - return false - } - var srcHost = srcUri.getHost() - var dstHost = dstUri.getHost() - if ((srcHost != null) && (dstHost != null)) { - try { - srcHost = InetAddress.getByName(srcHost).getCanonicalHostName() - dstHost = InetAddress.getByName(dstHost).getCanonicalHostName() - } catch { - case e: UnknownHostException => - return false - } - if (!srcHost.equals(dstHost)) { - return false - } - } else if (srcHost == null && dstHost != null) { - return false - } else if (srcHost != null && dstHost == null) { - return false - } - if (srcUri.getPort() != dstUri.getPort()) { - false - } else { - true - } - } - - /** Copy the file into HDFS if needed. */ - private[yarn] def copyRemoteFile( - dstDir: Path, - originalPath: Path, + /** + * Copy the given file to a remote file system (e.g. HDFS) if needed. + * The file is only copied if the source and destination file systems are different. This is used + * for preparing resources for launching the ApplicationMaster container. Exposed for testing. + */ + def copyFileToRemote( + destDir: Path, + srcPath: Path, replication: Short, setPerms: Boolean = false): Path = { - val fs = FileSystem.get(conf) - val remoteFs = originalPath.getFileSystem(conf) - var newPath = originalPath - if (!compareFs(remoteFs, fs)) { - newPath = new Path(dstDir, originalPath.getName()) - logInfo("Uploading " + originalPath + " to " + newPath) - FileUtil.copy(remoteFs, originalPath, fs, newPath, false, conf) - fs.setReplication(newPath, replication) - if (setPerms) fs.setPermission(newPath, new FsPermission(APP_FILE_PERMISSION)) + val destFs = destDir.getFileSystem(hadoopConf) + val srcFs = srcPath.getFileSystem(hadoopConf) + var destPath = srcPath + if (!compareFs(srcFs, destFs)) { + destPath = new Path(destDir, srcPath.getName()) + logInfo(s"Uploading resource $srcPath -> $destPath") + FileUtil.copy(srcFs, srcPath, destFs, destPath, false, hadoopConf) + destFs.setReplication(destPath, replication) + if (setPerms) { + destFs.setPermission(destPath, new FsPermission(APP_FILE_PERMISSION)) + } + } else { + logInfo(s"Source and destination file systems are the same. Not copying $srcPath") } // Resolve any symlinks in the URI path so using a "current" symlink to point to a specific // version shows the specific version in the distributed cache configuration - val qualPath = fs.makeQualified(newPath) - val fc = FileContext.getFileContext(qualPath.toUri(), conf) - val destPath = fc.resolvePath(qualPath) - destPath + val qualifiedDestPath = destFs.makeQualified(destPath) + val fc = FileContext.getFileContext(qualifiedDestPath.toUri(), hadoopConf) + fc.resolvePath(qualifiedDestPath) } - private def qualifyForLocal(localURI: URI): Path = { - var qualifiedURI = localURI - // If not specified, assume these are in the local filesystem to keep behavior like Hadoop - if (qualifiedURI.getScheme() == null) { - qualifiedURI = new URI(FileSystem.getLocal(conf).makeQualified(new Path(qualifiedURI)).toString) - } + /** + * Given a local URI, resolve it and return a qualified local path that corresponds to the URI. + * This is used for preparing local resources to be included in the container launch context. + */ + private def getQualifiedLocalPath(localURI: URI): Path = { + val qualifiedURI = + if (localURI.getScheme == null) { + // If not specified, assume this is in the local filesystem to keep the behavior + // consistent with that of Hadoop + new URI(FileSystem.getLocal(hadoopConf).makeQualified(new Path(localURI)).toString) + } else { + localURI + } new Path(qualifiedURI) } + /** + * Upload any resources to the distributed cache if needed. If a resource is intended to be + * consumed locally, set up the appropriate config for downstream code to handle it properly. + * This is used for setting up a container launch context for our ApplicationMaster. + * Exposed for testing. + */ def prepareLocalResources(appStagingDir: String): HashMap[String, LocalResource] = { - logInfo("Preparing Local resources") - // Upload Spark and the application JAR to the remote file system if necessary. Add them as - // local resources to the application master. - val fs = FileSystem.get(conf) + logInfo("Preparing resources for our AM container") + // Upload Spark and the application JAR to the remote file system if necessary, + // and add them as local resources to the application master. + val fs = FileSystem.get(hadoopConf) val dst = new Path(fs.getHomeDirectory(), appStagingDir) - val nns = ClientBase.getNameNodesToAccess(sparkConf) + dst - ClientBase.obtainTokensForNamenodes(nns, conf, credentials) + val nns = getNameNodesToAccess(sparkConf) + dst + obtainTokensForNamenodes(nns, hadoopConf, credentials) val replication = sparkConf.getInt("spark.yarn.submit.file.replication", 3).toShort val localResources = HashMap[String, LocalResource]() @@ -200,73 +156,84 @@ trait ClientBase extends Logging { "for alternatives.") } + /** + * Copy the given main resource to the distributed cache if the scheme is not "local". + * Otherwise, set the corresponding key in our SparkConf to handle it downstream. + * Each resource is represented by a 4-tuple of: + * (1) destination resource name, + * (2) local path to the resource, + * (3) Spark property key to set if the scheme is not local, and + * (4) whether to set permissions for this resource + */ List( - (ClientBase.SPARK_JAR, ClientBase.sparkJar(sparkConf), ClientBase.CONF_SPARK_JAR), - (ClientBase.APP_JAR, args.userJar, ClientBase.CONF_SPARK_USER_JAR), - ("log4j.properties", oldLog4jConf.getOrElse(null), null) - ).foreach { case(destName, _localPath, confKey) => + (SPARK_JAR, sparkJar(sparkConf), CONF_SPARK_JAR, false), + (APP_JAR, args.userJar, CONF_SPARK_USER_JAR, true), + ("log4j.properties", oldLog4jConf.orNull, null, false) + ).foreach { case (destName, _localPath, confKey, setPermissions) => val localPath: String = if (_localPath != null) _localPath.trim() else "" - if (! localPath.isEmpty()) { + if (!localPath.isEmpty()) { val localURI = new URI(localPath) - if (!ClientBase.LOCAL_SCHEME.equals(localURI.getScheme())) { - val setPermissions = destName.equals(ClientBase.APP_JAR) - val destPath = copyRemoteFile(dst, qualifyForLocal(localURI), replication, setPermissions) - val destFs = FileSystem.get(destPath.toUri(), conf) - distCacheMgr.addResource(destFs, conf, destPath, localResources, LocalResourceType.FILE, - destName, statCache) + if (localURI.getScheme != LOCAL_SCHEME) { + val src = getQualifiedLocalPath(localURI) + val destPath = copyFileToRemote(dst, src, replication, setPermissions) + val destFs = FileSystem.get(destPath.toUri(), hadoopConf) + distCacheMgr.addResource(destFs, hadoopConf, destPath, + localResources, LocalResourceType.FILE, destName, statCache) } else if (confKey != null) { + // If the resource is intended for local use only, handle this downstream + // by setting the appropriate property sparkConf.set(confKey, localPath) } } } + /** + * Do the same for any additional resources passed in through ClientArguments. + * Each resource category is represented by a 3-tuple of: + * (1) comma separated list of resources in this category, + * (2) resource type, and + * (3) whether to add these resources to the classpath + */ val cachedSecondaryJarLinks = ListBuffer.empty[String] - val fileLists = List( (args.addJars, LocalResourceType.FILE, true), + List( + (args.addJars, LocalResourceType.FILE, true), (args.files, LocalResourceType.FILE, false), - (args.archives, LocalResourceType.ARCHIVE, false) ) - fileLists.foreach { case (flist, resType, addToClasspath) => + (args.archives, LocalResourceType.ARCHIVE, false) + ).foreach { case (flist, resType, addToClasspath) => if (flist != null && !flist.isEmpty()) { - flist.split(',').foreach { case file: String => + flist.split(',').foreach { file => val localURI = new URI(file.trim()) - if (!ClientBase.LOCAL_SCHEME.equals(localURI.getScheme())) { + if (localURI.getScheme != LOCAL_SCHEME) { val localPath = new Path(localURI) val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName()) - val destPath = copyRemoteFile(dst, localPath, replication) - distCacheMgr.addResource(fs, conf, destPath, localResources, resType, - linkname, statCache) + val destPath = copyFileToRemote(dst, localPath, replication) + distCacheMgr.addResource( + fs, hadoopConf, destPath, localResources, resType, linkname, statCache) if (addToClasspath) { cachedSecondaryJarLinks += linkname } } else if (addToClasspath) { + // Resource is intended for local use only and should be added to the class path cachedSecondaryJarLinks += file.trim() } } } } - logInfo("Prepared Local resources " + localResources) - sparkConf.set(ClientBase.CONF_SPARK_YARN_SECONDARY_JARS, cachedSecondaryJarLinks.mkString(",")) + if (cachedSecondaryJarLinks.nonEmpty) { + sparkConf.set(CONF_SPARK_YARN_SECONDARY_JARS, cachedSecondaryJarLinks.mkString(",")) + } - UserGroupInformation.getCurrentUser().addCredentials(credentials) localResources } - /** Get all application master environment variables set on this SparkConf */ - def getAppMasterEnv: Seq[(String, String)] = { - val prefix = "spark.yarn.appMasterEnv." - sparkConf.getAll.filter{case (k, v) => k.startsWith(prefix)} - .map{case (k, v) => (k.substring(prefix.length), v)} - } - - - def setupLaunchEnv( - localResources: HashMap[String, LocalResource], - stagingDir: String): HashMap[String, String] = { - logInfo("Setting up the launch environment") - + /** + * Set up the environment for launching our ApplicationMaster container. + */ + private def setupLaunchEnv(stagingDir: String): HashMap[String, String] = { + logInfo("Setting up the launch environment for our AM container") val env = new HashMap[String, String]() - val extraCp = sparkConf.getOption("spark.driver.extraClassPath") - ClientBase.populateClasspath(args, yarnConf, sparkConf, env, extraCp) + populateClasspath(args, yarnConf, sparkConf, env, extraCp) env("SPARK_YARN_MODE") = "true" env("SPARK_YARN_STAGING_DIR") = stagingDir env("SPARK_USER") = UserGroupInformation.getCurrentUser().getShortUserName() @@ -275,42 +242,20 @@ trait ClientBase extends Logging { distCacheMgr.setDistFilesEnv(env) distCacheMgr.setDistArchivesEnv(env) - getAppMasterEnv.foreach { case (key, value) => - YarnSparkHadoopUtil.addToEnvironment(env, key, value, File.pathSeparator) - } + // Pick up any environment variables for the AM provided through spark.yarn.appMasterEnv.* + val amEnvPrefix = "spark.yarn.appMasterEnv." + sparkConf.getAll + .filter { case (k, v) => k.startsWith(amEnvPrefix) } + .map { case (k, v) => (k.substring(amEnvPrefix.length), v) } + .foreach { case (k, v) => YarnSparkHadoopUtil.addPathToEnvironment(env, k, v) } // Keep this for backwards compatibility but users should move to the config sys.env.get("SPARK_YARN_USER_ENV").foreach { userEnvs => // Allow users to specify some environment variables. - YarnSparkHadoopUtil.setEnvFromInputString(env, userEnvs, File.pathSeparator) - + YarnSparkHadoopUtil.setEnvFromInputString(env, userEnvs) // 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 - } - - def userArgsToString(clientArgs: ClientArguments): String = { - val prefix = " --arg " - val args = clientArgs.userArgs - val retval = new StringBuilder() - for (arg <- args) { - retval.append(prefix).append(" ").append(YarnSparkHadoopUtil.escapeForShell(arg)) - } - retval.toString - } - - def setupSecurityToken(amContainer: ContainerLaunchContext) - - def createContainerLaunchContext( - newApp: GetNewApplicationResponse, - localResources: HashMap[String, LocalResource], - env: HashMap[String, String]): ContainerLaunchContext = { - logInfo("Setting up container launch context") - val amContainer = Records.newRecord(classOf[ContainerLaunchContext]) - amContainer.setLocalResources(localResources) - - val isLaunchingDriver = args.userClass != null // In cluster mode, if the deprecated SPARK_JAVA_OPTS is set, we need to propagate it to // executors. But we can't just set spark.executor.extraJavaOptions, because the driver's @@ -320,6 +265,7 @@ trait ClientBase extends Logging { // Note that to warn the user about the deprecation in cluster mode, some code from // SparkConf#validateSettings() is duplicated here (to avoid triggering the condition // described above). + val isLaunchingDriver = args.userClass != null if (isLaunchingDriver) { sys.env.get("SPARK_JAVA_OPTS").foreach { value => val warning = @@ -342,14 +288,30 @@ trait ClientBase extends Logging { env("SPARK_JAVA_OPTS") = value } } - amContainer.setEnvironment(env) - val amMemory = args.amMemory + env + } + + /** + * Set up a ContainerLaunchContext to launch our ApplicationMaster container. + * This sets up the launch environment, java options, and the command for launching the AM. + */ + protected def createContainerLaunchContext(newAppResponse: GetNewApplicationResponse) + : ContainerLaunchContext = { + logInfo("Setting up container launch context for our AM") + + val appId = newAppResponse.getApplicationId + val appStagingDir = getAppStagingDir(appId) + val localResources = prepareLocalResources(appStagingDir) + val launchEnv = setupLaunchEnv(appStagingDir) + val amContainer = Records.newRecord(classOf[ContainerLaunchContext]) + amContainer.setLocalResources(localResources) + amContainer.setEnvironment(launchEnv) val javaOpts = ListBuffer[String]() // Add Xmx for AM memory - javaOpts += "-Xmx" + amMemory + "m" + javaOpts += "-Xmx" + args.amMemory + "m" val tmpDir = new Path(Environment.PWD.$(), YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR) javaOpts += "-Djava.io.tmpdir=" + tmpDir @@ -361,8 +323,7 @@ trait ClientBase extends Logging { // Instead of using this, rely on cpusets by YARN to enforce "proper" Spark behavior in // multi-tenant environments. Not sure how default Java GC behaves if it is limited to subset // of cores on a node. - val useConcurrentAndIncrementalGC = env.isDefinedAt("SPARK_USE_CONC_INCR_GC") && - java.lang.Boolean.parseBoolean(env("SPARK_USE_CONC_INCR_GC")) + val useConcurrentAndIncrementalGC = launchEnv.get("SPARK_USE_CONC_INCR_GC").exists(_.toBoolean) if (useConcurrentAndIncrementalGC) { // In our expts, using (default) throughput collector has severe perf ramifications in // multi-tenant machines @@ -380,6 +341,8 @@ trait ClientBase extends Logging { javaOpts += YarnSparkHadoopUtil.escapeForShell(s"-D$k=$v") } + // Include driver-specific java options if we are launching a driver + val isLaunchingDriver = args.userClass != null if (isLaunchingDriver) { sparkConf.getOption("spark.driver.extraJavaOptions") .orElse(sys.env.get("SPARK_JAVA_OPTS")) @@ -388,25 +351,36 @@ trait ClientBase extends Logging { .foreach(p => javaOpts += s"-Djava.library.path=$p") } + // For log4j configuration to reference + javaOpts += ("-Dspark.yarn.app.container.log.dir=" + ApplicationConstants.LOG_DIR_EXPANSION_VAR) + val userClass = if (args.userClass != null) { Seq("--class", YarnSparkHadoopUtil.escapeForShell(args.userClass)) } else { Nil } + val userJar = + if (args.userJar != null) { + Seq("--jar", args.userJar) + } else { + Nil + } val amClass = if (isLaunchingDriver) { - classOf[ApplicationMaster].getName() + Class.forName("org.apache.spark.deploy.yarn.ApplicationMaster").getName } else { - classOf[ApplicationMaster].getName().replace("ApplicationMaster", "ExecutorLauncher") + Class.forName("org.apache.spark.deploy.yarn.ExecutorLauncher").getName } + val userArgs = args.userArgs.flatMap { arg => + Seq("--arg", YarnSparkHadoopUtil.escapeForShell(arg)) + } val amArgs = - Seq(amClass) ++ userClass ++ - (if (args.userJar != null) Seq("--jar", args.userJar) else Nil) ++ - Seq("--executor-memory", args.executorMemory.toString, + Seq(amClass) ++ userClass ++ userJar ++ userArgs ++ + Seq( + "--executor-memory", args.executorMemory.toString, "--executor-cores", args.executorCores.toString, - "--num-executors ", args.numExecutors.toString, - userArgsToString(args)) + "--num-executors ", args.numExecutors.toString) // Command for the ApplicationMaster val commands = Seq(Environment.JAVA_HOME.$() + "/bin/java", "-server") ++ @@ -415,41 +389,153 @@ trait ClientBase extends Logging { "1>", ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout", "2>", ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr") - logInfo("Yarn AM launch context:") - logInfo(s" user class: ${args.userClass}") - logInfo(s" env: $env") - logInfo(s" command: ${commands.mkString(" ")}") - // TODO: it would be nicer to just make sure there are no null commands here val printableCommands = commands.map(s => if (s == null) "null" else s).toList amContainer.setCommands(printableCommands) - setupSecurityToken(amContainer) + logDebug("===============================================================================") + logDebug("Yarn AM launch context:") + logDebug(s" user class: ${Option(args.userClass).getOrElse("N/A")}") + logDebug(" env:") + launchEnv.foreach { case (k, v) => logDebug(s" $k -> $v") } + logDebug(" resources:") + localResources.foreach { case (k, v) => logDebug(s" $k -> $v")} + logDebug(" command:") + logDebug(s" ${printableCommands.mkString(" ")}") + logDebug("===============================================================================") // send the acl settings into YARN to control who has access via YARN interfaces val securityManager = new SecurityManager(sparkConf) amContainer.setApplicationACLs(YarnSparkHadoopUtil.getApplicationAclsForYarn(securityManager)) + setupSecurityToken(amContainer) + UserGroupInformation.getCurrentUser().addCredentials(credentials) amContainer } + + /** + * Report the state of an application until it has exited, either successfully or + * due to some failure, then return the application state. + * + * @param appId ID of the application to monitor. + * @param returnOnRunning Whether to also return the application state when it is RUNNING. + * @param logApplicationReport Whether to log details of the application report every iteration. + * @return state of the application, one of FINISHED, FAILED, KILLED, and RUNNING. + */ + def monitorApplication( + appId: ApplicationId, + returnOnRunning: Boolean = false, + logApplicationReport: Boolean = true): YarnApplicationState = { + val interval = sparkConf.getLong("spark.yarn.report.interval", 1000) + var lastState: YarnApplicationState = null + while (true) { + Thread.sleep(interval) + val report = getApplicationReport(appId) + val state = report.getYarnApplicationState + + if (logApplicationReport) { + logInfo(s"Application report for $appId (state: $state)") + val details = Seq[(String, String)]( + ("client token", getClientToken(report)), + ("diagnostics", report.getDiagnostics), + ("ApplicationMaster host", report.getHost), + ("ApplicationMaster RPC port", report.getRpcPort.toString), + ("queue", report.getQueue), + ("start time", report.getStartTime.toString), + ("final status", report.getFinalApplicationStatus.toString), + ("tracking URL", report.getTrackingUrl), + ("user", report.getUser) + ) + + // Use more loggable format if value is null or empty + val formattedDetails = details + .map { case (k, v) => + val newValue = Option(v).filter(_.nonEmpty).getOrElse("N/A") + s"\n\t $k: $newValue" } + .mkString("") + + // If DEBUG is enabled, log report details every iteration + // Otherwise, log them every time the application changes state + if (log.isDebugEnabled) { + logDebug(formattedDetails) + } else if (lastState != state) { + logInfo(formattedDetails) + } + } + + if (state == YarnApplicationState.FINISHED || + state == YarnApplicationState.FAILED || + state == YarnApplicationState.KILLED) { + return state + } + + if (returnOnRunning && state == YarnApplicationState.RUNNING) { + return state + } + + lastState = state + } + + // Never reached, but keeps compiler happy + throw new SparkException("While loop is depleted! This should never happen...") + } + + /** + * Submit an application to the ResourceManager and monitor its state. + * This continues until the application has exited for any reason. + */ + def run(): Unit = monitorApplication(submitApplication()) + + /* --------------------------------------------------------------------------------------- * + | Methods that cannot be implemented here due to API differences across hadoop versions | + * --------------------------------------------------------------------------------------- */ + + /** Submit an application running our ApplicationMaster to the ResourceManager. */ + def submitApplication(): ApplicationId + + /** Set up security tokens for launching our ApplicationMaster container. */ + protected def setupSecurityToken(containerContext: ContainerLaunchContext): Unit + + /** Get the application report from the ResourceManager for an application we have submitted. */ + protected def getApplicationReport(appId: ApplicationId): ApplicationReport + + /** + * Return the security token used by this client to communicate with the ApplicationMaster. + * If no security is enabled, the token returned by the report is null. + */ + protected def getClientToken(report: ApplicationReport): String } -object ClientBase extends Logging { +private[spark] object ClientBase extends Logging { + + // Alias for the Spark assembly jar and the user jar val SPARK_JAR: String = "__spark__.jar" val APP_JAR: String = "__app__.jar" + + // URI scheme that identifies local resources val LOCAL_SCHEME = "local" + + // Staging directory for any temporary jars or files + val SPARK_STAGING: String = ".sparkStaging" + + // Location of any user-defined Spark jars val CONF_SPARK_JAR = "spark.yarn.jar" - /** - * This is an internal config used to propagate the location of the user's jar file to the - * driver/executors. - */ + val ENV_SPARK_JAR = "SPARK_JAR" + + // Internal config to propagate the location of the user's jar to the driver/executors val CONF_SPARK_USER_JAR = "spark.yarn.user.jar" - /** - * This is an internal config used to propagate the list of extra jars to add to the classpath - * of executors. - */ + + // Internal config to propagate the locations of any extra jars to add to the classpath + // of the executors val CONF_SPARK_YARN_SECONDARY_JARS = "spark.yarn.secondary.jars" - val ENV_SPARK_JAR = "SPARK_JAR" + + // Staging directory is private! -> rwx-------- + val STAGING_DIR_PERMISSION: FsPermission = + FsPermission.createImmutable(Integer.parseInt("700", 8).toShort) + + // App files are world-wide readable and owner writable -> rw-r--r-- + val APP_FILE_PERMISSION: FsPermission = + FsPermission.createImmutable(Integer.parseInt("644", 8).toShort) /** * Find the user-defined Spark jar if configured, or return the jar containing this @@ -458,7 +544,7 @@ object ClientBase extends Logging { * This method first looks in the SparkConf object for the CONF_SPARK_JAR key, and in the * user environment if that is not found (for backwards compatibility). */ - def sparkJar(conf: SparkConf) = { + private def sparkJar(conf: SparkConf): String = { if (conf.contains(CONF_SPARK_JAR)) { conf.get(CONF_SPARK_JAR) } else if (System.getenv(ENV_SPARK_JAR) != null) { @@ -471,16 +557,22 @@ object ClientBase extends Logging { } } - def populateHadoopClasspath(conf: Configuration, env: HashMap[String, String]) = { + /** + * Return the path to the given application's staging directory. + */ + private def getAppStagingDir(appId: ApplicationId): String = { + SPARK_STAGING + Path.SEPARATOR + appId.toString() + Path.SEPARATOR + } + + /** + * Populate the classpath entry in the given environment map with any application + * classpath specified through the Hadoop and Yarn configurations. + */ + def populateHadoopClasspath(conf: Configuration, env: HashMap[String, String]): Unit = { val classPathElementsToAdd = getYarnAppClasspath(conf) ++ getMRAppClasspath(conf) for (c <- classPathElementsToAdd.flatten) { - YarnSparkHadoopUtil.addToEnvironment( - env, - Environment.CLASSPATH.name, - c.trim, - File.pathSeparator) + YarnSparkHadoopUtil.addPathToEnvironment(env, Environment.CLASSPATH.name, c.trim) } - classPathElementsToAdd } private def getYarnAppClasspath(conf: Configuration): Option[Seq[String]] = @@ -516,7 +608,7 @@ object ClientBase extends Logging { /** * In Hadoop 0.23, the MR application classpath comes with the YARN application - * classpath. In Hadoop 2.0, it's an array of Strings, and in 2.2+ it's a String. + * classpath. In Hadoop 2.0, it's an array of Strings, and in 2.2+ it's a String. * So we need to use reflection to retrieve it. */ def getDefaultMRApplicationClasspath: Option[Seq[String]] = { @@ -542,8 +634,16 @@ object ClientBase extends Logging { triedDefault.toOption } - def populateClasspath(args: ClientArguments, conf: Configuration, sparkConf: SparkConf, - env: HashMap[String, String], extraClassPath: Option[String] = None) { + /** + * Populate the classpath entry in the given environment map. + * This includes the user jar, Spark jar, and any extra application jars. + */ + def populateClasspath( + args: ClientArguments, + conf: Configuration, + sparkConf: SparkConf, + env: HashMap[String, String], + extraClassPath: Option[String] = None): Unit = { extraClassPath.foreach(addClasspathEntry(_, env)) addClasspathEntry(Environment.PWD.$(), env) @@ -551,36 +651,40 @@ object ClientBase extends Logging { if (sparkConf.get("spark.yarn.user.classpath.first", "false").toBoolean) { addUserClasspath(args, sparkConf, env) addFileToClasspath(sparkJar(sparkConf), SPARK_JAR, env) - ClientBase.populateHadoopClasspath(conf, env) + populateHadoopClasspath(conf, env) } else { addFileToClasspath(sparkJar(sparkConf), SPARK_JAR, env) - ClientBase.populateHadoopClasspath(conf, env) + populateHadoopClasspath(conf, env) addUserClasspath(args, sparkConf, env) } // Append all jar files under the working directory to the classpath. - addClasspathEntry(Environment.PWD.$() + Path.SEPARATOR + "*", env); + addClasspathEntry(Environment.PWD.$() + Path.SEPARATOR + "*", env) } /** * Adds the user jars which have local: URIs (or alternate names, such as APP_JAR) explicitly * to the classpath. */ - private def addUserClasspath(args: ClientArguments, conf: SparkConf, - env: HashMap[String, String]) = { - if (args != null) { - addFileToClasspath(args.userJar, APP_JAR, env) - if (args.addJars != null) { - args.addJars.split(",").foreach { case file: String => - addFileToClasspath(file, null, env) - } + private def addUserClasspath( + args: ClientArguments, + conf: SparkConf, + env: HashMap[String, String]): Unit = { + + // If `args` is not null, we are launching an AM container. + // Otherwise, we are launching executor containers. + val (mainJar, secondaryJars) = + if (args != null) { + (args.userJar, args.addJars) + } else { + (conf.get(CONF_SPARK_USER_JAR, null), conf.get(CONF_SPARK_YARN_SECONDARY_JARS, null)) } - } else { - val userJar = conf.get(CONF_SPARK_USER_JAR, null) - addFileToClasspath(userJar, APP_JAR, env) - val cachedSecondaryJarLinks = conf.get(CONF_SPARK_YARN_SECONDARY_JARS, "").split(",") - cachedSecondaryJarLinks.foreach(jar => addFileToClasspath(jar, null, env)) + addFileToClasspath(mainJar, APP_JAR, env) + if (secondaryJars != null) { + secondaryJars.split(",").filter(_.nonEmpty).foreach { jar => + addFileToClasspath(jar, null, env) + } } } @@ -596,46 +700,44 @@ object ClientBase extends Logging { * @param fileName Alternate name for the file (optional). * @param env Map holding the environment variables. */ - private def addFileToClasspath(path: String, fileName: String, - env: HashMap[String, String]) : Unit = { + private def addFileToClasspath( + path: String, + fileName: String, + env: HashMap[String, String]): Unit = { if (path != null) { scala.util.control.Exception.ignoring(classOf[URISyntaxException]) { - val localPath = getLocalPath(path) - if (localPath != null) { - addClasspathEntry(localPath, env) + val uri = new URI(path) + if (uri.getScheme == LOCAL_SCHEME) { + addClasspathEntry(uri.getPath, env) return } } } if (fileName != null) { - addClasspathEntry(Environment.PWD.$() + Path.SEPARATOR + fileName, env); + addClasspathEntry(Environment.PWD.$() + Path.SEPARATOR + fileName, env) } } /** - * Returns the local path if the URI is a "local:" URI, or null otherwise. + * Add the given path to the classpath entry of the given environment map. + * If the classpath is already set, this appends the new path to the existing classpath. */ - private def getLocalPath(resource: String): String = { - val uri = new URI(resource) - if (LOCAL_SCHEME.equals(uri.getScheme())) { - return uri.getPath() - } - null - } - - private def addClasspathEntry(path: String, env: HashMap[String, String]) = - YarnSparkHadoopUtil.addToEnvironment(env, Environment.CLASSPATH.name, path, - File.pathSeparator) + private def addClasspathEntry(path: String, env: HashMap[String, String]): Unit = + YarnSparkHadoopUtil.addPathToEnvironment(env, Environment.CLASSPATH.name, path) /** * Get the list of namenodes the user may access. */ - private[yarn] def getNameNodesToAccess(sparkConf: SparkConf): Set[Path] = { - sparkConf.get("spark.yarn.access.namenodes", "").split(",").map(_.trim()).filter(!_.isEmpty) - .map(new Path(_)).toSet + def getNameNodesToAccess(sparkConf: SparkConf): Set[Path] = { + sparkConf.get("spark.yarn.access.namenodes", "") + .split(",") + .map(_.trim()) + .filter(!_.isEmpty) + .map(new Path(_)) + .toSet } - private[yarn] def getTokenRenewer(conf: Configuration): String = { + def getTokenRenewer(conf: Configuration): String = { val delegTokenRenewer = Master.getMasterPrincipal(conf) logDebug("delegation token renewer is: " + delegTokenRenewer) if (delegTokenRenewer == null || delegTokenRenewer.length() == 0) { @@ -649,17 +751,54 @@ object ClientBase extends Logging { /** * Obtains tokens for the namenodes passed in and adds them to the credentials. */ - private[yarn] def obtainTokensForNamenodes(paths: Set[Path], conf: Configuration, - creds: Credentials) { + def obtainTokensForNamenodes( + paths: Set[Path], + conf: Configuration, + creds: Credentials): Unit = { if (UserGroupInformation.isSecurityEnabled()) { val delegTokenRenewer = getTokenRenewer(conf) + paths.foreach { dst => + val dstFs = dst.getFileSystem(conf) + logDebug("getting token for namenode: " + dst) + dstFs.addDelegationTokens(delegTokenRenewer, creds) + } + } + } - paths.foreach { - dst => - val dstFs = dst.getFileSystem(conf) - logDebug("getting token for namenode: " + dst) - dstFs.addDelegationTokens(delegTokenRenewer, creds) + /** + * Return whether the two file systems are the same. + */ + private def compareFs(srcFs: FileSystem, destFs: FileSystem): Boolean = { + val srcUri = srcFs.getUri() + val dstUri = destFs.getUri() + if (srcUri.getScheme() == null) { + return false + } + if (!srcUri.getScheme().equals(dstUri.getScheme())) { + return false + } + var srcHost = srcUri.getHost() + var dstHost = dstUri.getHost() + if ((srcHost != null) && (dstHost != null)) { + try { + srcHost = InetAddress.getByName(srcHost).getCanonicalHostName() + dstHost = InetAddress.getByName(dstHost).getCanonicalHostName() + } catch { + case e: UnknownHostException => + return false } + if (!srcHost.equals(dstHost)) { + return false + } + } else if (srcHost == null && dstHost != null) { + return false + } else if (srcHost != null && dstHost == null) { + return false + } + if (srcUri.getPort() != dstUri.getPort()) { + false + } else { + true } } diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala index 9b7f1fca96c6d..c592ecfdfce06 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala @@ -19,29 +19,24 @@ package org.apache.spark.deploy.yarn import java.net.URI +import scala.collection.mutable.{HashMap, LinkedHashMap, Map} + import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.FileStatus -import org.apache.hadoop.fs.FileSystem -import org.apache.hadoop.fs.Path +import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} import org.apache.hadoop.fs.permission.FsAction -import org.apache.hadoop.yarn.api.records.LocalResource -import org.apache.hadoop.yarn.api.records.LocalResourceVisibility -import org.apache.hadoop.yarn.api.records.LocalResourceType +import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.util.{Records, ConverterUtils} -import org.apache.spark.Logging - -import scala.collection.mutable.HashMap -import scala.collection.mutable.LinkedHashMap -import scala.collection.mutable.Map - +import org.apache.spark.Logging /** Client side methods to setup the Hadoop distributed cache */ -class ClientDistributedCacheManager() extends Logging { - private val distCacheFiles: Map[String, Tuple3[String, String, String]] = - LinkedHashMap[String, Tuple3[String, String, String]]() - private val distCacheArchives: Map[String, Tuple3[String, String, String]] = - LinkedHashMap[String, Tuple3[String, String, String]]() +private[spark] class ClientDistributedCacheManager() extends Logging { + + // Mappings from remote URI to (file status, modification time, visibility) + private val distCacheFiles: Map[String, (String, String, String)] = + LinkedHashMap[String, (String, String, String)]() + private val distCacheArchives: Map[String, (String, String, String)] = + LinkedHashMap[String, (String, String, String)]() /** @@ -68,9 +63,9 @@ class ClientDistributedCacheManager() extends Logging { resourceType: LocalResourceType, link: String, statCache: Map[URI, FileStatus], - appMasterOnly: Boolean = false) = { + appMasterOnly: Boolean = false): Unit = { val destStatus = fs.getFileStatus(destPath) - val amJarRsrc = Records.newRecord(classOf[LocalResource]).asInstanceOf[LocalResource] + val amJarRsrc = Records.newRecord(classOf[LocalResource]) amJarRsrc.setType(resourceType) val visibility = getVisibility(conf, destPath.toUri(), statCache) amJarRsrc.setVisibility(visibility) @@ -80,7 +75,7 @@ class ClientDistributedCacheManager() extends Logging { if (link == null || link.isEmpty()) throw new Exception("You must specify a valid link name") localResources(link) = amJarRsrc - if (appMasterOnly == false) { + if (!appMasterOnly) { val uri = destPath.toUri() val pathURI = new URI(uri.getScheme(), uri.getAuthority(), uri.getPath(), null, link) if (resourceType == LocalResourceType.FILE) { @@ -95,12 +90,10 @@ class ClientDistributedCacheManager() extends Logging { /** * Adds the necessary cache file env variables to the env passed in - * @param env */ - def setDistFilesEnv(env: Map[String, String]) = { + def setDistFilesEnv(env: Map[String, String]): Unit = { val (keys, tupleValues) = distCacheFiles.unzip val (sizes, timeStamps, visibilities) = tupleValues.unzip3 - if (keys.size > 0) { env("SPARK_YARN_CACHE_FILES") = keys.reduceLeft[String] { (acc,n) => acc + "," + n } env("SPARK_YARN_CACHE_FILES_TIME_STAMPS") = @@ -114,12 +107,10 @@ class ClientDistributedCacheManager() extends Logging { /** * Adds the necessary cache archive env variables to the env passed in - * @param env */ - def setDistArchivesEnv(env: Map[String, String]) = { + def setDistArchivesEnv(env: Map[String, String]): Unit = { val (keys, tupleValues) = distCacheArchives.unzip val (sizes, timeStamps, visibilities) = tupleValues.unzip3 - if (keys.size > 0) { env("SPARK_YARN_CACHE_ARCHIVES") = keys.reduceLeft[String] { (acc,n) => acc + "," + n } env("SPARK_YARN_CACHE_ARCHIVES_TIME_STAMPS") = @@ -133,25 +124,21 @@ class ClientDistributedCacheManager() extends Logging { /** * Returns the local resource visibility depending on the cache file permissions - * @param conf - * @param uri - * @param statCache * @return LocalResourceVisibility */ - def getVisibility(conf: Configuration, uri: URI, statCache: Map[URI, FileStatus]): - LocalResourceVisibility = { + def getVisibility( + conf: Configuration, + uri: URI, + statCache: Map[URI, FileStatus]): LocalResourceVisibility = { if (isPublic(conf, uri, statCache)) { - return LocalResourceVisibility.PUBLIC - } - LocalResourceVisibility.PRIVATE + LocalResourceVisibility.PUBLIC + } else { + LocalResourceVisibility.PRIVATE + } } /** - * Returns a boolean to denote whether a cache file is visible to all(public) - * or not - * @param conf - * @param uri - * @param statCache + * Returns a boolean to denote whether a cache file is visible to all (public) * @return true if the path in the uri is visible to all, false otherwise */ def isPublic(conf: Configuration, uri: URI, statCache: Map[URI, FileStatus]): Boolean = { @@ -167,13 +154,12 @@ class ClientDistributedCacheManager() extends Logging { /** * Returns true if all ancestors of the specified path have the 'execute' * permission set for all users (i.e. that other users can traverse - * the directory heirarchy to the given path) - * @param fs - * @param path - * @param statCache + * the directory hierarchy to the given path) * @return true if all ancestors have the 'execute' permission set for all users */ - def ancestorsHaveExecutePermissions(fs: FileSystem, path: Path, + def ancestorsHaveExecutePermissions( + fs: FileSystem, + path: Path, statCache: Map[URI, FileStatus]): Boolean = { var current = path while (current != null) { @@ -187,32 +173,25 @@ class ClientDistributedCacheManager() extends Logging { } /** - * Checks for a given path whether the Other permissions on it + * Checks for a given path whether the Other permissions on it * imply the permission in the passed FsAction - * @param fs - * @param path - * @param action - * @param statCache * @return true if the path in the uri is visible to all, false otherwise */ - def checkPermissionOfOther(fs: FileSystem, path: Path, - action: FsAction, statCache: Map[URI, FileStatus]): Boolean = { + def checkPermissionOfOther( + fs: FileSystem, + path: Path, + action: FsAction, + statCache: Map[URI, FileStatus]): Boolean = { val status = getFileStatus(fs, path.toUri(), statCache) val perms = status.getPermission() val otherAction = perms.getOtherAction() - if (otherAction.implies(action)) { - return true - } - false + otherAction.implies(action) } /** - * Checks to see if the given uri exists in the cache, if it does it + * Checks to see if the given uri exists in the cache, if it does it * returns the existing FileStatus, otherwise it stats the uri, stores * it in the cache, and returns the FileStatus. - * @param fs - * @param uri - * @param statCache * @return FileStatus */ def getFileStatus(fs: FileSystem, uri: URI, statCache: Map[URI, FileStatus]): FileStatus = { 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 312d82a649792..5cb4753de2e84 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 @@ -17,7 +17,6 @@ package org.apache.spark.deploy.yarn -import java.io.File import java.net.URI import scala.collection.JavaConversions._ @@ -44,6 +43,7 @@ trait ExecutorRunnableUtil extends Logging { hostname: String, executorMemory: Int, executorCores: Int, + appId: String, localResources: HashMap[String, LocalResource]): List[String] = { // Extra options for the JVM val javaOpts = ListBuffer[String]() @@ -98,6 +98,9 @@ trait ExecutorRunnableUtil extends Logging { } */ + // For log4j configuration to reference + javaOpts += ("-Dspark.yarn.app.container.log.dir=" + ApplicationConstants.LOG_DIR_EXPANSION_VAR) + val commands = Seq(Environment.JAVA_HOME.$() + "/bin/java", "-server", // Kill if OOM is raised - leverage yarn's failure handling to cause rescheduling. @@ -112,6 +115,7 @@ trait ExecutorRunnableUtil extends Logging { slaveId.toString, hostname.toString, executorCores.toString, + appId, "1>", ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout", "2>", ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr") @@ -125,9 +129,9 @@ trait ExecutorRunnableUtil extends Logging { localResources: HashMap[String, LocalResource], timestamp: String, size: String, - vis: String) = { + vis: String): Unit = { val uri = new URI(file) - val amJarRsrc = Records.newRecord(classOf[LocalResource]).asInstanceOf[LocalResource] + val amJarRsrc = Records.newRecord(classOf[LocalResource]) amJarRsrc.setType(rtype) amJarRsrc.setVisibility(LocalResourceVisibility.valueOf(vis)) amJarRsrc.setResource(ConverterUtils.getYarnUrlFromURI(uri)) @@ -172,14 +176,17 @@ trait ExecutorRunnableUtil extends Logging { ClientBase.populateClasspath(null, yarnConf, sparkConf, env, extraCp) sparkConf.getExecutorEnv.foreach { case (key, value) => - YarnSparkHadoopUtil.addToEnvironment(env, key, value, File.pathSeparator) + // This assumes each executor environment variable set here is a path + // This is kept for backward compatibility and consistency with hadoop + YarnSparkHadoopUtil.addPathToEnvironment(env, key, value) } // Keep this for backwards compatibility but users should move to the config - YarnSparkHadoopUtil.setEnvFromInputString(env, System.getenv("SPARK_YARN_USER_ENV"), - File.pathSeparator) + sys.env.get("SPARK_YARN_USER_ENV").foreach { userEnvs => + YarnSparkHadoopUtil.setEnvFromInputString(env, userEnvs) + } - System.getenv().filterKeys(_.startsWith("SPARK")).foreach { case (k,v) => env(k) = v } + System.getenv().filterKeys(_.startsWith("SPARK")).foreach { case (k, v) => env(k) = v } env } diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index 299e38a5eb9c0..e1af8d5a74cb1 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -33,6 +33,7 @@ import org.apache.spark.scheduler.{SplitInfo, TaskSchedulerImpl} import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import com.google.common.util.concurrent.ThreadFactoryBuilder +import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil._ object AllocationType extends Enumeration { type AllocationType = Value @@ -56,6 +57,7 @@ object AllocationType extends Enumeration { private[yarn] abstract class YarnAllocator( conf: Configuration, sparkConf: SparkConf, + appAttemptId: ApplicationAttemptId, args: ApplicationMasterArguments, preferredNodes: collection.Map[String, collection.Set[SplitInfo]], securityMgr: SecurityManager) @@ -78,10 +80,6 @@ private[yarn] abstract class YarnAllocator( // Containers to be released in next request to RM private val releasedContainers = new ConcurrentHashMap[ContainerId, Boolean] - // Additional memory overhead - in mb. - protected val memoryOverhead: Int = sparkConf.getInt("spark.yarn.executor.memoryOverhead", - YarnSparkHadoopUtil.DEFAULT_MEMORY_OVERHEAD) - // Number of container requests that have been sent to, but not yet allocated by the // ApplicationMaster. private val numPendingAllocate = new AtomicInteger() @@ -97,6 +95,10 @@ private[yarn] abstract class YarnAllocator( protected val (preferredHostToCount, preferredRackToCount) = generateNodeToWeight(conf, preferredNodes) + // Additional memory overhead - in mb. + protected val memoryOverhead: Int = sparkConf.getInt("spark.yarn.executor.memoryOverhead", + math.max((MEMORY_OVERHEAD_FACTOR * executorMemory).toInt, MEMORY_OVERHEAD_MIN)) + private val launcherPool = new ThreadPoolExecutor( // max pool size of Integer.MAX_VALUE is ignored because we use an unbounded queue sparkConf.getInt("spark.yarn.containerLauncherMaxThreads", 25), Integer.MAX_VALUE, @@ -114,12 +116,11 @@ private[yarn] abstract class YarnAllocator( // this is needed by alpha, do it here since we add numPending right after this val executorsPending = numPendingAllocate.get() - if (missing > 0) { + val totalExecutorMemory = executorMemory + memoryOverhead numPendingAllocate.addAndGet(missing) - logInfo("Will Allocate %d executor containers, each with %d memory".format( - missing, - (executorMemory + memoryOverhead))) + logInfo(s"Will allocate $missing executor containers, each with $totalExecutorMemory MB " + + s"memory including $memoryOverhead MB overhead") } else { logDebug("Empty allocation request ...") } @@ -295,6 +296,7 @@ private[yarn] abstract class YarnAllocator( executorHostname, executorMemory, executorCores, + appAttemptId.getApplicationId.toString, securityMgr) launcherPool.execute(executorRunnable) } diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala index ed65e56b3e413..2510b9c9cef68 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala @@ -49,18 +49,18 @@ trait YarnRMClient { securityMgr: SecurityManager): YarnAllocator /** - * Shuts down the AM. Guaranteed to only be called once. + * Unregister the AM. Guaranteed to only be called once. * * @param status The final status of the AM. * @param diagnostics Diagnostics message to include in the final status. */ - def shutdown(status: FinalApplicationStatus, diagnostics: String = ""): Unit + def unregister(status: FinalApplicationStatus, diagnostics: String = ""): Unit /** Returns the attempt ID. */ def getAttemptId(): ApplicationAttemptId - /** Returns the RM's proxy host and port. */ - def getProxyHostAndPort(conf: YarnConfiguration): String + /** Returns the configuration for the AmIpFilter to add to the Spark UI. */ + def getAmIpFilterParams(conf: YarnConfiguration, proxyBase: String): Map[String, String] /** Returns the maximum number of attempts to register the AM. */ def getMaxRegAttempts(conf: YarnConfiguration): Int diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index 4a33e34c3bfc7..e1e0144f46fe9 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -18,6 +18,7 @@ package org.apache.spark.deploy.yarn import java.lang.{Boolean => JBoolean} +import java.io.File import java.util.{Collections, Set => JSet} import java.util.regex.Matcher import java.util.regex.Pattern @@ -29,14 +30,12 @@ import org.apache.hadoop.io.Text import org.apache.hadoop.mapred.JobConf import org.apache.hadoop.security.Credentials import org.apache.hadoop.security.UserGroupInformation -import org.apache.hadoop.util.StringInterner import org.apache.hadoop.yarn.conf.YarnConfiguration -import org.apache.hadoop.yarn.api.ApplicationConstants import org.apache.hadoop.yarn.api.records.ApplicationAccessType import org.apache.hadoop.yarn.util.RackResolver import org.apache.hadoop.conf.Configuration -import org.apache.spark.{SecurityManager, SparkConf, SparkContext} +import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.util.Utils @@ -85,8 +84,12 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { } object YarnSparkHadoopUtil { - // Additional memory overhead - in mb. - val DEFAULT_MEMORY_OVERHEAD = 384 + // Additional memory overhead + // 7% was arrived at experimentally. In the interest of minimizing memory waste while covering + // the common cases. Memory overhead tends to grow with container size. + + val MEMORY_OVERHEAD_FACTOR = 0.07 + val MEMORY_OVERHEAD_MIN = 384 val ANY_HOST = "*" @@ -100,30 +103,26 @@ object YarnSparkHadoopUtil { private val hostToRack = new ConcurrentHashMap[String, String]() private val rackToHostSet = new ConcurrentHashMap[String, JSet[String]]() - def addToEnvironment( - env: HashMap[String, String], - variable: String, - value: String, - classPathSeparator: String) = { - var envVariable = "" - if (env.get(variable) == None) { - envVariable = value - } else { - envVariable = env.get(variable).get + classPathSeparator + value - } - env put (StringInterner.weakIntern(variable), StringInterner.weakIntern(envVariable)) + /** + * Add a path variable to the given environment map. + * If the map already contains this key, append the value to the existing value instead. + */ + def addPathToEnvironment(env: HashMap[String, String], key: String, value: String): Unit = { + val newValue = if (env.contains(key)) { env(key) + File.pathSeparator + value } else value + env.put(key, newValue) } - def setEnvFromInputString( - env: HashMap[String, String], - envString: String, - classPathSeparator: String) = { - if (envString != null && envString.length() > 0) { - var childEnvs = envString.split(",") - var p = Pattern.compile(getEnvironmentVariableRegex()) + /** + * Set zero or more environment variables specified by the given input string. + * The input string is expected to take the form "KEY1=VAL1,KEY2=VAL2,KEY3=VAL3". + */ + def setEnvFromInputString(env: HashMap[String, String], inputString: String): Unit = { + if (inputString != null && inputString.length() > 0) { + val childEnvs = inputString.split(",") + val p = Pattern.compile(environmentVariableRegex) for (cEnv <- childEnvs) { - var parts = cEnv.split("=") // split on '=' - var m = p.matcher(parts(1)) + val parts = cEnv.split("=") // split on '=' + val m = p.matcher(parts(1)) val sb = new StringBuffer while (m.find()) { val variable = m.group(1) @@ -131,8 +130,7 @@ object YarnSparkHadoopUtil { if (env.get(variable) != None) { replace = env.get(variable).get } else { - // if this key is not configured for the child .. get it - // from the env + // if this key is not configured for the child .. get it from the env replace = System.getenv(variable) if (replace == null) { // the env key is note present anywhere .. simply set it @@ -142,14 +140,15 @@ object YarnSparkHadoopUtil { m.appendReplacement(sb, Matcher.quoteReplacement(replace)) } m.appendTail(sb) - addToEnvironment(env, parts(0), sb.toString(), classPathSeparator) + // This treats the environment variable as path variable delimited by `File.pathSeparator` + // This is kept for backward compatibility and consistency with Hadoop's behavior + addPathToEnvironment(env, parts(0), sb.toString) } } } - private def getEnvironmentVariableRegex() : String = { - val osName = System.getProperty("os.name") - if (osName startsWith "Windows") { + private val environmentVariableRegex: String = { + if (Utils.isWindows) { "%([A-Za-z_][A-Za-z0-9_]*?)%" } else { "\\$([A-Za-z_][A-Za-z0-9_]*)" @@ -181,14 +180,14 @@ object YarnSparkHadoopUtil { } } - private[spark] def lookupRack(conf: Configuration, host: String): String = { + def lookupRack(conf: Configuration, host: String): String = { if (!hostToRack.contains(host)) { populateRackInfo(conf, host) } hostToRack.get(host) } - private[spark] def populateRackInfo(conf: Configuration, hostname: String) { + def populateRackInfo(conf: Configuration, hostname: String) { Utils.checkHost(hostname) if (!hostToRack.containsKey(hostname)) { @@ -212,8 +211,8 @@ object YarnSparkHadoopUtil { } } - private[spark] def getApplicationAclsForYarn(securityMgr: SecurityManager): - Map[ApplicationAccessType, String] = { + def getApplicationAclsForYarn(securityMgr: SecurityManager) + : Map[ApplicationAccessType, String] = { Map[ApplicationAccessType, String] ( ApplicationAccessType.VIEW_APP -> securityMgr.getViewAcls, ApplicationAccessType.MODIFY_APP -> securityMgr.getModifyAcls diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala index 6aa6475fe4a18..6bb4b82316ad4 100644 --- a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -19,7 +19,7 @@ package org.apache.spark.scheduler.cluster import org.apache.hadoop.yarn.api.records.{ApplicationId, YarnApplicationState} import org.apache.spark.{SparkException, Logging, SparkContext} -import org.apache.spark.deploy.yarn.{Client, ClientArguments, YarnSparkHadoopUtil} +import org.apache.spark.deploy.yarn.{Client, ClientArguments} import org.apache.spark.scheduler.TaskSchedulerImpl import scala.collection.mutable.ArrayBuffer @@ -34,115 +34,120 @@ private[spark] class YarnClientSchedulerBackend( minRegisteredRatio = 0.8 } - var client: Client = null - var appId: ApplicationId = null - var checkerThread: Thread = null - var stopping: Boolean = false - var totalExpectedExecutors = 0 - - private[spark] def addArg(optionName: String, envVar: String, sysProp: String, - arrayBuf: ArrayBuffer[String]) { - if (System.getenv(envVar) != null) { - arrayBuf += (optionName, System.getenv(envVar)) - } else if (sc.getConf.contains(sysProp)) { - arrayBuf += (optionName, sc.getConf.get(sysProp)) - } - } + private var client: Client = null + private var appId: ApplicationId = null + private var stopping: Boolean = false + private var totalExpectedExecutors = 0 + /** + * Create a Yarn client to submit an application to the ResourceManager. + * This waits until the application is running. + */ override def start() { super.start() - val driverHost = conf.get("spark.driver.host") val driverPort = conf.get("spark.driver.port") val hostport = driverHost + ":" + driverPort sc.ui.foreach { ui => conf.set("spark.driver.appUIAddress", ui.appUIHostPort) } val argsArrayBuf = new ArrayBuffer[String]() - argsArrayBuf += ( - "--args", hostport - ) - - // process any optional arguments, given either as environment variables - // or system properties. use the defaults already defined in ClientArguments - // if things aren't specified. system properties override environment - // variables. - List(("--driver-memory", "SPARK_MASTER_MEMORY", "spark.master.memory"), - ("--driver-memory", "SPARK_DRIVER_MEMORY", "spark.driver.memory"), - ("--num-executors", "SPARK_WORKER_INSTANCES", "spark.executor.instances"), - ("--num-executors", "SPARK_EXECUTOR_INSTANCES", "spark.executor.instances"), - ("--executor-memory", "SPARK_WORKER_MEMORY", "spark.executor.memory"), - ("--executor-memory", "SPARK_EXECUTOR_MEMORY", "spark.executor.memory"), - ("--executor-cores", "SPARK_WORKER_CORES", "spark.executor.cores"), - ("--executor-cores", "SPARK_EXECUTOR_CORES", "spark.executor.cores"), - ("--queue", "SPARK_YARN_QUEUE", "spark.yarn.queue"), - ("--name", "SPARK_YARN_APP_NAME", "spark.app.name")) - .foreach { case (optName, envVar, sysProp) => addArg(optName, envVar, sysProp, argsArrayBuf) } - - logDebug("ClientArguments called with: " + argsArrayBuf) + argsArrayBuf += ("--arg", hostport) + argsArrayBuf ++= getExtraClientArguments + + logDebug("ClientArguments called with: " + argsArrayBuf.mkString(" ")) val args = new ClientArguments(argsArrayBuf.toArray, conf) totalExpectedExecutors = args.numExecutors client = new Client(args, conf) - appId = client.runApp() - waitForApp() - checkerThread = yarnApplicationStateCheckerThread() + appId = client.submitApplication() + waitForApplication() + asyncMonitorApplication() } - def waitForApp() { - - // TODO : need a better way to find out whether the executors are ready or not - // maybe by resource usage report? - while(true) { - val report = client.getApplicationReport(appId) - - logInfo("Application report from ASM: \n" + - "\t appMasterRpcPort: " + report.getRpcPort() + "\n" + - "\t appStartTime: " + report.getStartTime() + "\n" + - "\t yarnAppState: " + report.getYarnApplicationState() + "\n" + /** + * Return any extra command line arguments to be passed to Client provided in the form of + * environment variables or Spark properties. + */ + private def getExtraClientArguments: Seq[String] = { + val extraArgs = new ArrayBuffer[String] + val optionTuples = // List of (target Client argument, environment variable, Spark property) + List( + ("--driver-memory", "SPARK_MASTER_MEMORY", "spark.master.memory"), + ("--driver-memory", "SPARK_DRIVER_MEMORY", "spark.driver.memory"), + ("--num-executors", "SPARK_WORKER_INSTANCES", "spark.executor.instances"), + ("--num-executors", "SPARK_EXECUTOR_INSTANCES", "spark.executor.instances"), + ("--executor-memory", "SPARK_WORKER_MEMORY", "spark.executor.memory"), + ("--executor-memory", "SPARK_EXECUTOR_MEMORY", "spark.executor.memory"), + ("--executor-cores", "SPARK_WORKER_CORES", "spark.executor.cores"), + ("--executor-cores", "SPARK_EXECUTOR_CORES", "spark.executor.cores"), + ("--queue", "SPARK_YARN_QUEUE", "spark.yarn.queue"), + ("--name", "SPARK_YARN_APP_NAME", "spark.app.name") ) - - // Ready to go, or already gone. - val state = report.getYarnApplicationState() - if (state == YarnApplicationState.RUNNING) { - return - } else if (state == YarnApplicationState.FINISHED || - state == YarnApplicationState.FAILED || - state == YarnApplicationState.KILLED) { - throw new SparkException("Yarn application already ended," + - "might be killed or not able to launch application master.") + optionTuples.foreach { case (optionName, envVar, sparkProp) => + if (System.getenv(envVar) != null) { + extraArgs += (optionName, System.getenv(envVar)) + } else if (sc.getConf.contains(sparkProp)) { + extraArgs += (optionName, sc.getConf.get(sparkProp)) } + } + extraArgs + } - Thread.sleep(1000) + /** + * Report the state of the application until it is running. + * If the application has finished, failed or been killed in the process, throw an exception. + * This assumes both `client` and `appId` have already been set. + */ + private def waitForApplication(): Unit = { + assert(client != null && appId != null, "Application has not been submitted yet!") + val state = client.monitorApplication(appId, returnOnRunning = true) // blocking + if (state == YarnApplicationState.FINISHED || + state == YarnApplicationState.FAILED || + state == YarnApplicationState.KILLED) { + throw new SparkException("Yarn application has already ended! " + + "It might have been killed or unable to launch application master.") + } + if (state == YarnApplicationState.RUNNING) { + logInfo(s"Application $appId has started running.") } } - private def yarnApplicationStateCheckerThread(): Thread = { + /** + * Monitor the application state in a separate thread. + * If the application has exited for any reason, stop the SparkContext. + * This assumes both `client` and `appId` have already been set. + */ + private def asyncMonitorApplication(): Unit = { + assert(client != null && appId != null, "Application has not been submitted yet!") val t = new Thread { override def run() { while (!stopping) { val report = client.getApplicationReport(appId) val state = report.getYarnApplicationState() - if (state == YarnApplicationState.FINISHED || state == YarnApplicationState.KILLED - || state == YarnApplicationState.FAILED) { - logError(s"Yarn application already ended: $state") + if (state == YarnApplicationState.FINISHED || + state == YarnApplicationState.KILLED || + state == YarnApplicationState.FAILED) { + logError(s"Yarn application has already exited with state $state!") sc.stop() stopping = true } Thread.sleep(1000L) } - checkerThread = null Thread.currentThread().interrupt() } } - t.setName("Yarn Application State Checker") + t.setName("Yarn application state monitor") t.setDaemon(true) t.start() - t } + /** + * Stop the scheduler. This assumes `start()` has already been called. + */ override def stop() { + assert(client != null, "Attempted to stop this scheduler before starting it!") stopping = true super.stop() - client.stop + client.stop() logInfo("Stopped") } @@ -150,6 +155,10 @@ private[spark] class YarnClientSchedulerBackend( totalRegisteredExecutors.get() >= totalExpectedExecutors * minRegisteredRatio } - override def applicationId(): Option[String] = Option(appId).map(_.toString()) + override def applicationId(): String = + Option(appId).map(_.toString).getOrElse { + logWarning("Application ID is not initialized yet.") + super.applicationId + } } diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala index 39436d0999663..3a186cfeb4eeb 100644 --- a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala +++ b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala @@ -48,6 +48,13 @@ private[spark] class YarnClusterSchedulerBackend( totalRegisteredExecutors.get() >= totalExpectedExecutors * minRegisteredRatio } - override def applicationId(): Option[String] = sc.getConf.getOption("spark.yarn.app.id") + override def applicationId(): String = + // In YARN Cluster mode, spark.yarn.app.id is expect to be set + // before user application is launched. + // So, if spark.yarn.app.id is not set, it is something wrong. + sc.getConf.getOption("spark.yarn.app.id").getOrElse { + logError("Application ID is not set.") + super.applicationId + } } diff --git a/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala b/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala index c3b7a2c8f02e5..9bd916100dd2c 100644 --- a/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala +++ b/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala @@ -27,7 +27,7 @@ import org.apache.hadoop.mapreduce.MRJobConfig import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.api.ApplicationConstants.Environment import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse -import org.apache.hadoop.yarn.api.records.ContainerLaunchContext +import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.mockito.Matchers._ import org.mockito.Mockito._ @@ -90,7 +90,7 @@ class ClientBaseSuite extends FunSuite with Matchers { val env = new MutableHashMap[String, String]() val args = new ClientArguments(Array("--jar", USER, "--addJars", ADDED), sparkConf) - ClientBase.populateClasspath(args, conf, sparkConf, env, None) + ClientBase.populateClasspath(args, conf, sparkConf, env) val cp = env("CLASSPATH").split(File.pathSeparator) s"$SPARK,$USER,$ADDED".split(",").foreach({ entry => @@ -114,10 +114,10 @@ class ClientBaseSuite extends FunSuite with Matchers { val args = new ClientArguments(Array("--jar", USER, "--addJars", ADDED), sparkConf) val client = spy(new DummyClient(args, conf, sparkConf, yarnConf)) - doReturn(new Path("/")).when(client).copyRemoteFile(any(classOf[Path]), + doReturn(new Path("/")).when(client).copyFileToRemote(any(classOf[Path]), any(classOf[Path]), anyShort(), anyBoolean()) - var tempDir = Files.createTempDir(); + val tempDir = Files.createTempDir() try { client.prepareLocalResources(tempDir.getAbsolutePath()) sparkConf.getOption(ClientBase.CONF_SPARK_USER_JAR) should be (Some(USER)) @@ -247,13 +247,13 @@ class ClientBaseSuite extends FunSuite with Matchers { private class DummyClient( val args: ClientArguments, - val conf: Configuration, + val hadoopConf: Configuration, val sparkConf: SparkConf, val yarnConf: YarnConfiguration) extends ClientBase { - - override def setupSecurityToken(amContainer: ContainerLaunchContext): Unit = - throw new UnsupportedOperationException() - + override def setupSecurityToken(amContainer: ContainerLaunchContext): Unit = ??? + override def submitApplication(): ApplicationId = ??? + override def getApplicationReport(appId: ApplicationId): ApplicationReport = ??? + override def getClientToken(report: ApplicationReport): String = ??? } } diff --git a/yarn/pom.xml b/yarn/pom.xml index 815a736c2e8fd..8a7035c85e9f1 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -140,7 +140,6 @@ ${basedir}/../.. - ${spark.classpath} @@ -148,7 +147,7 @@ target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes - + ../common/src/main/resources diff --git a/yarn/stable/pom.xml b/yarn/stable/pom.xml index fd934b7726181..97eb0548e77c3 100644 --- a/yarn/stable/pom.xml +++ b/yarn/stable/pom.xml @@ -32,4 +32,13 @@ jar Spark Project YARN Stable API + + + org.apache.hadoop + hadoop-yarn-server-tests + tests + test + + + diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 82e45e3e7ad54..0b43e6ee20538 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -21,11 +21,9 @@ import java.nio.ByteBuffer import org.apache.hadoop.conf.Configuration import org.apache.hadoop.io.DataOutputBuffer -import org.apache.hadoop.yarn.api.protocolrecords._ import org.apache.hadoop.yarn.api.records._ -import org.apache.hadoop.yarn.client.api.YarnClient +import org.apache.hadoop.yarn.client.api.{YarnClient, YarnClientApplication} import org.apache.hadoop.yarn.conf.YarnConfiguration -import org.apache.hadoop.yarn.ipc.YarnRPC import org.apache.hadoop.yarn.util.Records import org.apache.spark.{Logging, SparkConf} @@ -34,128 +32,98 @@ import org.apache.spark.deploy.SparkHadoopUtil /** * Version of [[org.apache.spark.deploy.yarn.ClientBase]] tailored to YARN's stable API. */ -class Client(clientArgs: ClientArguments, hadoopConf: Configuration, spConf: SparkConf) +private[spark] class Client( + val args: ClientArguments, + val hadoopConf: Configuration, + val sparkConf: SparkConf) extends ClientBase with Logging { - val yarnClient = YarnClient.createYarnClient - def this(clientArgs: ClientArguments, spConf: SparkConf) = this(clientArgs, SparkHadoopUtil.get.newConfiguration(spConf), spConf) def this(clientArgs: ClientArguments) = this(clientArgs, new SparkConf()) - val args = clientArgs - val conf = hadoopConf - val sparkConf = spConf - var rpc: YarnRPC = YarnRPC.create(conf) - val yarnConf: YarnConfiguration = new YarnConfiguration(conf) - - def runApp(): ApplicationId = { - validateArgs() - // Initialize and start the client service. + val yarnClient = YarnClient.createYarnClient + val yarnConf = new YarnConfiguration(hadoopConf) + + def stop(): Unit = yarnClient.stop() + + /* ------------------------------------------------------------------------------------- * + | The following methods have much in common in the stable and alpha versions of Client, | + | but cannot be implemented in the parent trait due to subtle API differences across | + | hadoop versions. | + * ------------------------------------------------------------------------------------- */ + + /** + * Submit an application running our ApplicationMaster to the ResourceManager. + * + * The stable Yarn API provides a convenience method (YarnClient#createApplication) for + * creating applications and setting up the application submission context. This was not + * available in the alpha API. + */ + override def submitApplication(): ApplicationId = { yarnClient.init(yarnConf) yarnClient.start() - // Log details about this YARN cluster (e.g, the number of slave machines/NodeManagers). - logClusterResourceDetails() - - // Prepare to submit a request to the ResourcManager (specifically its ApplicationsManager (ASM) - // interface). + logInfo("Requesting a new application from cluster with %d NodeManagers" + .format(yarnClient.getYarnClusterMetrics.getNumNodeManagers)) - // Get a new client application. + // Get a new application from our RM val newApp = yarnClient.createApplication() val newAppResponse = newApp.getNewApplicationResponse() val appId = newAppResponse.getApplicationId() + // Verify whether the cluster has enough resources for our AM verifyClusterResources(newAppResponse) - // Set up resource and environment variables. - val appStagingDir = getAppStagingDir(appId) - val localResources = prepareLocalResources(appStagingDir) - val launchEnv = setupLaunchEnv(localResources, appStagingDir) - val amContainer = createContainerLaunchContext(newAppResponse, localResources, launchEnv) + // Set up the appropriate contexts to launch our AM + val containerContext = createContainerLaunchContext(newAppResponse) + val appContext = createApplicationSubmissionContext(newApp, containerContext) - // Set up an application submission context. - val appContext = newApp.getApplicationSubmissionContext() - appContext.setApplicationName(args.appName) - appContext.setQueue(args.amQueue) - appContext.setAMContainerSpec(amContainer) - appContext.setApplicationType("SPARK") - - // Memory for the ApplicationMaster. - val memoryResource = Records.newRecord(classOf[Resource]).asInstanceOf[Resource] - memoryResource.setMemory(args.amMemory + memoryOverhead) - appContext.setResource(memoryResource) - - // Finally, submit and monitor the application. - submitApp(appContext) + // Finally, submit and monitor the application + logInfo(s"Submitting application ${appId.getId} to ResourceManager") + yarnClient.submitApplication(appContext) appId } - def run() { - val appId = runApp() - monitorApplication(appId) - } - - def logClusterResourceDetails() { - val clusterMetrics: YarnClusterMetrics = yarnClient.getYarnClusterMetrics - logInfo("Got cluster metric info from ResourceManager, number of NodeManagers: " + - clusterMetrics.getNumNodeManagers) + /** + * Set up the context for submitting our ApplicationMaster. + * This uses the YarnClientApplication not available in the Yarn alpha API. + */ + def createApplicationSubmissionContext( + newApp: YarnClientApplication, + containerContext: ContainerLaunchContext): ApplicationSubmissionContext = { + val appContext = newApp.getApplicationSubmissionContext + appContext.setApplicationName(args.appName) + appContext.setQueue(args.amQueue) + appContext.setAMContainerSpec(containerContext) + appContext.setApplicationType("SPARK") + val capability = Records.newRecord(classOf[Resource]) + capability.setMemory(args.amMemory + amMemoryOverhead) + appContext.setResource(capability) + appContext } - def setupSecurityToken(amContainer: ContainerLaunchContext) = { - // Setup security tokens. - val dob = new DataOutputBuffer() + /** Set up security tokens for launching our ApplicationMaster container. */ + override def setupSecurityToken(amContainer: ContainerLaunchContext): Unit = { + val dob = new DataOutputBuffer credentials.writeTokenStorageToStream(dob) - amContainer.setTokens(ByteBuffer.wrap(dob.getData())) + amContainer.setTokens(ByteBuffer.wrap(dob.getData)) } - def submitApp(appContext: ApplicationSubmissionContext) = { - // Submit the application to the applications manager. - logInfo("Submitting application to ResourceManager") - yarnClient.submitApplication(appContext) - } + /** Get the application report from the ResourceManager for an application we have submitted. */ + override def getApplicationReport(appId: ApplicationId): ApplicationReport = + yarnClient.getApplicationReport(appId) - def getApplicationReport(appId: ApplicationId) = - yarnClient.getApplicationReport(appId) - - def stop = yarnClient.stop - - def monitorApplication(appId: ApplicationId): Boolean = { - val interval = sparkConf.getLong("spark.yarn.report.interval", 1000) - - while (true) { - Thread.sleep(interval) - val report = yarnClient.getApplicationReport(appId) - - logInfo("Application report from ResourceManager: \n" + - "\t application identifier: " + appId.toString() + "\n" + - "\t appId: " + appId.getId() + "\n" + - "\t clientToAMToken: " + report.getClientToAMToken() + "\n" + - "\t appDiagnostics: " + report.getDiagnostics() + "\n" + - "\t appMasterHost: " + report.getHost() + "\n" + - "\t appQueue: " + report.getQueue() + "\n" + - "\t appMasterRpcPort: " + report.getRpcPort() + "\n" + - "\t appStartTime: " + report.getStartTime() + "\n" + - "\t yarnAppState: " + report.getYarnApplicationState() + "\n" + - "\t distributedFinalState: " + report.getFinalApplicationStatus() + "\n" + - "\t appTrackingUrl: " + report.getTrackingUrl() + "\n" + - "\t appUser: " + report.getUser() - ) - - val state = report.getYarnApplicationState() - if (state == YarnApplicationState.FINISHED || - state == YarnApplicationState.FAILED || - state == YarnApplicationState.KILLED) { - return true - } - } - true - } + /** + * Return the security token used by this client to communicate with the ApplicationMaster. + * If no security is enabled, the token returned by the report is null. + */ + override def getClientToken(report: ApplicationReport): String = + Option(report.getClientToAMToken).map(_.toString).getOrElse("") } object Client { - def main(argStrings: Array[String]) { if (!sys.props.contains("SPARK_SUBMIT")) { println("WARNING: This client is deprecated and will be removed in a " + @@ -163,22 +131,19 @@ object Client { } // Set an env variable indicating we are running in YARN mode. - // Note: anything env variable with SPARK_ prefix gets propagated to all (remote) processes - - // see Client#setupLaunchEnv(). + // Note that any env variable with the SPARK_ prefix gets propagated to all (remote) processes System.setProperty("SPARK_YARN_MODE", "true") - val sparkConf = new SparkConf() + val sparkConf = new SparkConf try { val args = new ClientArguments(argStrings, sparkConf) new Client(args, sparkConf).run() } catch { - case e: Exception => { + case e: Exception => Console.err.println(e.getMessage) System.exit(1) - } } System.exit(0) } - } diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala index 833be12982e71..0b5a92d87d722 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala @@ -47,6 +47,7 @@ class ExecutorRunnable( hostname: String, executorMemory: Int, executorCores: Int, + appId: String, securityMgr: SecurityManager) extends Runnable with ExecutorRunnableUtil with Logging { @@ -80,7 +81,7 @@ class ExecutorRunnable( ctx.setTokens(ByteBuffer.wrap(dob.getData())) val commands = prepareCommand(masterAddress, slaveId, hostname, executorMemory, executorCores, - localResources) + appId, localResources) logInfo(s"Setting up executor with environment: $env") logInfo("Setting up executor with commands: " + commands) diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index e44a8db41b97e..2bbf5d7db8668 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -41,7 +41,7 @@ private[yarn] class YarnAllocationHandler( args: ApplicationMasterArguments, preferredNodes: collection.Map[String, collection.Set[SplitInfo]], securityMgr: SecurityManager) - extends YarnAllocator(conf, sparkConf, args, preferredNodes, securityMgr) { + extends YarnAllocator(conf, sparkConf, appAttemptId, args, preferredNodes, securityMgr) { override protected def releaseContainer(container: Container) = { amClient.releaseAssignedContainer(container.getId()) diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala index 54bc6b14c44ce..8d4b96ed79933 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala @@ -17,8 +17,13 @@ package org.apache.spark.deploy.yarn +import java.util.{List => JList} + import scala.collection.{Map, Set} +import scala.collection.JavaConversions._ +import scala.util._ +import org.apache.hadoop.conf.Configuration import org.apache.hadoop.yarn.api._ import org.apache.hadoop.yarn.api.protocolrecords._ import org.apache.hadoop.yarn.api.records._ @@ -40,6 +45,7 @@ private class YarnRMClientImpl(args: ApplicationMasterArguments) extends YarnRMC private var amClient: AMRMClient[ContainerRequest] = _ private var uiHistoryAddress: String = _ + private var registered: Boolean = false override def register( conf: YarnConfiguration, @@ -54,13 +60,19 @@ private class YarnRMClientImpl(args: ApplicationMasterArguments) extends YarnRMC this.uiHistoryAddress = uiHistoryAddress logInfo("Registering the ApplicationMaster") - amClient.registerApplicationMaster(Utils.localHostName(), 0, uiAddress) + synchronized { + amClient.registerApplicationMaster(Utils.localHostName(), 0, uiAddress) + registered = true + } new YarnAllocationHandler(conf, sparkConf, amClient, getAttemptId(), args, preferredNodeLocations, securityMgr) } - override def shutdown(status: FinalApplicationStatus, diagnostics: String = "") = - amClient.unregisterApplicationMaster(status, diagnostics, uiHistoryAddress) + override def unregister(status: FinalApplicationStatus, diagnostics: String = "") = synchronized { + if (registered) { + amClient.unregisterApplicationMaster(status, diagnostics, uiHistoryAddress) + } + } override def getAttemptId() = { val containerIdString = System.getenv(ApplicationConstants.Environment.CONTAINER_ID.name()) @@ -69,7 +81,28 @@ private class YarnRMClientImpl(args: ApplicationMasterArguments) extends YarnRMC appAttemptId } - override def getProxyHostAndPort(conf: YarnConfiguration) = WebAppUtils.getProxyHostAndPort(conf) + override def getAmIpFilterParams(conf: YarnConfiguration, proxyBase: String) = { + // Figure out which scheme Yarn is using. Note the method seems to have been added after 2.2, + // so not all stable releases have it. + val prefix = Try(classOf[WebAppUtils].getMethod("getHttpSchemePrefix", classOf[Configuration]) + .invoke(null, conf).asInstanceOf[String]).getOrElse("http://") + + // If running a new enough Yarn, use the HA-aware API for retrieving the RM addresses. + try { + val method = classOf[WebAppUtils].getMethod("getProxyHostsAndPortsForAmFilter", + classOf[Configuration]) + val proxies = method.invoke(null, conf).asInstanceOf[JList[String]] + val hosts = proxies.map { proxy => proxy.split(":")(0) } + val uriBases = proxies.map { proxy => prefix + proxy + proxyBase } + Map("PROXY_HOSTS" -> hosts.mkString(","), "PROXY_URI_BASES" -> uriBases.mkString(",")) + } catch { + case e: NoSuchMethodException => + val proxy = WebAppUtils.getProxyHostAndPort(conf) + val parts = proxy.split(":") + val uriBase = prefix + proxy + proxyBase + Map("PROXY_HOST" -> parts(0), "PROXY_URI_BASE" -> uriBase) + } + } override def getMaxRegAttempts(conf: YarnConfiguration) = conf.getInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS) diff --git a/yarn/stable/src/test/resources/log4j.properties b/yarn/stable/src/test/resources/log4j.properties new file mode 100644 index 0000000000000..9dd05f17f012b --- /dev/null +++ b/yarn/stable/src/test/resources/log4j.properties @@ -0,0 +1,28 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Set everything to be logged to the file core/target/unit-tests.log +log4j.rootCategory=INFO, file +log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file.append=false +log4j.appender.file.file=target/unit-tests.log +log4j.appender.file.layout=org.apache.log4j.PatternLayout +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n + +# Ignore messages below warning level from Jetty, because it's a bit verbose +log4j.logger.org.eclipse.jetty=WARN +org.eclipse.jetty.LEVEL=WARN diff --git a/yarn/stable/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/yarn/stable/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala new file mode 100644 index 0000000000000..a826b2a78a8f5 --- /dev/null +++ b/yarn/stable/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala @@ -0,0 +1,181 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.yarn + +import java.io.File +import java.util.concurrent.TimeUnit + +import scala.collection.JavaConversions._ + +import com.google.common.base.Charsets +import com.google.common.io.Files +import org.scalatest.{BeforeAndAfterAll, FunSuite, Matchers} + +import org.apache.hadoop.yarn.conf.YarnConfiguration +import org.apache.hadoop.yarn.server.MiniYARNCluster + +import org.apache.spark.{Logging, SparkConf, SparkContext} +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.util.Utils + +class YarnClusterSuite extends FunSuite with BeforeAndAfterAll with Matchers with Logging { + + // log4j configuration for the Yarn containers, so that their output is collected + // by Yarn instead of trying to overwrite unit-tests.log. + private val LOG4J_CONF = """ + |log4j.rootCategory=DEBUG, console + |log4j.appender.console=org.apache.log4j.ConsoleAppender + |log4j.appender.console.target=System.err + |log4j.appender.console.layout=org.apache.log4j.PatternLayout + |log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n + """.stripMargin + + private var yarnCluster: MiniYARNCluster = _ + private var tempDir: File = _ + private var fakeSparkJar: File = _ + private var oldConf: Map[String, String] = _ + + override def beforeAll() { + tempDir = Utils.createTempDir() + + val logConfDir = new File(tempDir, "log4j") + logConfDir.mkdir() + + val logConfFile = new File(logConfDir, "log4j.properties") + Files.write(LOG4J_CONF, logConfFile, Charsets.UTF_8) + + val childClasspath = logConfDir.getAbsolutePath() + File.pathSeparator + + sys.props("java.class.path") + + oldConf = sys.props.filter { case (k, v) => k.startsWith("spark.") }.toMap + + yarnCluster = new MiniYARNCluster(getClass().getName(), 1, 1, 1) + yarnCluster.init(new YarnConfiguration()) + yarnCluster.start() + + // There's a race in MiniYARNCluster in which start() may return before the RM has updated + // its address in the configuration. You can see this in the logs by noticing that when + // MiniYARNCluster prints the address, it still has port "0" assigned, although later the + // test works sometimes: + // + // INFO MiniYARNCluster: MiniYARN ResourceManager address: blah:0 + // + // That log message prints the contents of the RM_ADDRESS config variable. If you check it + // later on, it looks something like this: + // + // INFO YarnClusterSuite: RM address in configuration is blah:42631 + // + // This hack loops for a bit waiting for the port to change, and fails the test if it hasn't + // done so in a timely manner (defined to be 10 seconds). + val config = yarnCluster.getConfig() + val deadline = System.currentTimeMillis() + TimeUnit.SECONDS.toMillis(10) + while (config.get(YarnConfiguration.RM_ADDRESS).split(":")(1) == "0") { + if (System.currentTimeMillis() > deadline) { + throw new IllegalStateException("Timed out waiting for RM to come up.") + } + logDebug("RM address still not set in configuration, waiting...") + TimeUnit.MILLISECONDS.sleep(100) + } + + logInfo(s"RM address in configuration is ${config.get(YarnConfiguration.RM_ADDRESS)}") + config.foreach { e => + sys.props += ("spark.hadoop." + e.getKey() -> e.getValue()) + } + + fakeSparkJar = File.createTempFile("sparkJar", null, tempDir) + sys.props += ("spark.yarn.jar" -> ("local:" + fakeSparkJar.getAbsolutePath())) + sys.props += ("spark.executor.instances" -> "1") + sys.props += ("spark.driver.extraClassPath" -> childClasspath) + sys.props += ("spark.executor.extraClassPath" -> childClasspath) + + super.beforeAll() + } + + override def afterAll() { + yarnCluster.stop() + sys.props.retain { case (k, v) => !k.startsWith("spark.") } + sys.props ++= oldConf + super.afterAll() + } + + test("run Spark in yarn-client mode") { + var result = File.createTempFile("result", null, tempDir) + YarnClusterDriver.main(Array("yarn-client", result.getAbsolutePath())) + checkResult(result) + } + + test("run Spark in yarn-cluster mode") { + val main = YarnClusterDriver.getClass.getName().stripSuffix("$") + var result = File.createTempFile("result", null, tempDir) + + // The Client object will call System.exit() after the job is done, and we don't want + // that because it messes up the scalatest monitoring. So replicate some of what main() + // does here. + val args = Array("--class", main, + "--jar", "file:" + fakeSparkJar.getAbsolutePath(), + "--arg", "yarn-cluster", + "--arg", result.getAbsolutePath(), + "--num-executors", "1") + val sparkConf = new SparkConf() + val yarnConf = SparkHadoopUtil.get.newConfiguration(sparkConf) + val clientArgs = new ClientArguments(args, sparkConf) + new Client(clientArgs, yarnConf, sparkConf).run() + checkResult(result) + } + + /** + * This is a workaround for an issue with yarn-cluster mode: the Client class will not provide + * any sort of error when the job process finishes successfully, but the job itself fails. So + * the tests enforce that something is written to a file after everything is ok to indicate + * that the job succeeded. + */ + private def checkResult(result: File) = { + var resultString = Files.toString(result, Charsets.UTF_8) + resultString should be ("success") + } + +} + +private object YarnClusterDriver extends Logging with Matchers { + + def main(args: Array[String]) = { + if (args.length != 2) { + System.err.println( + s""" + |Invalid command line: ${args.mkString(" ")} + | + |Usage: YarnClusterDriver [master] [result file] + """.stripMargin) + System.exit(1) + } + + val sc = new SparkContext(new SparkConf().setMaster(args(0)) + .setAppName("yarn \"test app\" 'with quotes' and \\back\\slashes and $dollarSigns")) + val status = new File(args(1)) + var result = "failure" + try { + val data = sc.parallelize(1 to 4, 4).collect().toSet + data should be (Set(1, 2, 3, 4)) + result = "success" + } finally { + sc.stop() + Files.write(result, status, Charsets.UTF_8) + } + } + +}