diff --git a/assembly/pom.xml b/assembly/pom.xml index 78fb908f9a9e..b2a9d0780ee2 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -354,5 +354,25 @@ + + + + hadoop-provided + + provided + + + + hive-provided + + provided + + + + parquet-provided + + provided + + diff --git a/bagel/pom.xml b/bagel/pom.xml index 0327ffa40267..510e92640eff 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -40,15 +40,6 @@ spark-core_${scala.binary.version} ${project.version} - - org.eclipse.jetty - jetty-server - - - org.scalatest - scalatest_${scala.binary.version} - test - org.scalacheck scalacheck_${scala.binary.version} @@ -58,11 +49,5 @@ target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes - - - org.scalatest - scalatest-maven-plugin - - diff --git a/bagel/src/test/resources/log4j.properties b/bagel/src/test/resources/log4j.properties index 789869f72e3b..853ef0ed2986 100644 --- a/bagel/src/test/resources/log4j.properties +++ b/bagel/src/test/resources/log4j.properties @@ -15,10 +15,10 @@ # limitations under the License. # -# Set everything to be logged to the file bagel/target/unit-tests.log +# Set everything to be logged to the file target/unit-tests.log log4j.rootCategory=INFO, file log4j.appender.file=org.apache.log4j.FileAppender -log4j.appender.file.append=false +log4j.appender.file.append=true 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 diff --git a/bin/compute-classpath.cmd b/bin/compute-classpath.cmd index a4c099fb45b1..088f993954d9 100644 --- a/bin/compute-classpath.cmd +++ b/bin/compute-classpath.cmd @@ -109,6 +109,13 @@ if "x%YARN_CONF_DIR%"=="x" goto no_yarn_conf_dir set CLASSPATH=%CLASSPATH%;%YARN_CONF_DIR% :no_yarn_conf_dir +rem To allow for distributions to append needed libraries to the classpath (e.g. when +rem using the "hadoop-provided" profile to build Spark), check SPARK_DIST_CLASSPATH and +rem append it to tbe final classpath. +if not "x%$SPARK_DIST_CLASSPATH%"=="x" ( + set CLASSPATH=%CLASSPATH%;%SPARK_DIST_CLASSPATH% +) + rem A bit of a hack to allow calling this script within run2.cmd without seeing output if "%DONT_PRINT_CLASSPATH%"=="1" goto exit diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index a31ea73d3ce1..9e8d0b785194 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -72,22 +72,25 @@ else assembly_folder="$ASSEMBLY_DIR" fi -num_jars="$(ls "$assembly_folder" | grep "spark-assembly.*hadoop.*\.jar$" | wc -l)" -if [ "$num_jars" -eq "0" ]; then - echo "Failed to find Spark assembly in $assembly_folder" - echo "You need to build Spark before running this program." - exit 1 -fi +num_jars=0 + +for f in ${assembly_folder}/spark-assembly*hadoop*.jar; do + if [[ ! -e "$f" ]]; then + echo "Failed to find Spark assembly in $assembly_folder" 1>&2 + echo "You need to build Spark before running this program." 1>&2 + exit 1 + fi + ASSEMBLY_JAR="$f" + num_jars=$((num_jars+1)) +done + if [ "$num_jars" -gt "1" ]; then - jars_list=$(ls "$assembly_folder" | grep "spark-assembly.*hadoop.*.jar$") - echo "Found multiple Spark assembly jars in $assembly_folder:" - echo "$jars_list" - echo "Please remove all but one jar." + echo "Found multiple Spark assembly jars in $assembly_folder:" 1>&2 + ls ${assembly_folder}/spark-assembly*hadoop*.jar 1>&2 + echo "Please remove all but one jar." 1>&2 exit 1 fi -ASSEMBLY_JAR="$(ls "$assembly_folder"/spark-assembly*hadoop*.jar 2>/dev/null)" - # Verify that versions of java used to build the jars and run Spark are compatible jar_error_check=$("$JAR_CMD" -tf "$ASSEMBLY_JAR" nonexistent/class/path 2>&1) if [[ "$jar_error_check" =~ "invalid CEN header" ]]; then @@ -146,4 +149,11 @@ if [ -n "$YARN_CONF_DIR" ]; then CLASSPATH="$CLASSPATH:$YARN_CONF_DIR" fi +# To allow for distributions to append needed libraries to the classpath (e.g. when +# using the "hadoop-provided" profile to build Spark), check SPARK_DIST_CLASSPATH and +# append it to tbe final classpath. +if [ -n "$SPARK_DIST_CLASSPATH" ]; then + CLASSPATH="$CLASSPATH:$SPARK_DIST_CLASSPATH" +fi + echo "$CLASSPATH" diff --git a/bin/run-example b/bin/run-example index 3d932509426f..c567acf9a6b5 100755 --- a/bin/run-example +++ b/bin/run-example @@ -35,17 +35,32 @@ else fi if [ -f "$FWDIR/RELEASE" ]; then - export SPARK_EXAMPLES_JAR="`ls "$FWDIR"/lib/spark-examples-*hadoop*.jar`" -elif [ -e "$EXAMPLES_DIR"/target/scala-$SPARK_SCALA_VERSION/spark-examples-*hadoop*.jar ]; then - export SPARK_EXAMPLES_JAR="`ls "$EXAMPLES_DIR"/target/scala-$SPARK_SCALA_VERSION/spark-examples-*hadoop*.jar`" + JAR_PATH="${FWDIR}/lib" +else + JAR_PATH="${EXAMPLES_DIR}/target/scala-${SPARK_SCALA_VERSION}" fi -if [[ -z "$SPARK_EXAMPLES_JAR" ]]; then - echo "Failed to find Spark examples assembly in $FWDIR/lib or $FWDIR/examples/target" 1>&2 - echo "You need to build Spark before running this program" 1>&2 +JAR_COUNT=0 + +for f in ${JAR_PATH}/spark-examples-*hadoop*.jar; do + if [[ ! -e "$f" ]]; then + echo "Failed to find Spark examples assembly in $FWDIR/lib or $FWDIR/examples/target" 1>&2 + echo "You need to build Spark before running this program" 1>&2 + exit 1 + fi + SPARK_EXAMPLES_JAR="$f" + JAR_COUNT=$((JAR_COUNT+1)) +done + +if [ "$JAR_COUNT" -gt "1" ]; then + echo "Found multiple Spark examples assembly jars in ${JAR_PATH}" 1>&2 + ls ${JAR_PATH}/spark-examples-*hadoop*.jar 1>&2 + echo "Please remove all but one jar." 1>&2 exit 1 fi +export SPARK_EXAMPLES_JAR + EXAMPLE_MASTER=${MASTER:-"local[*]"} if [[ ! $EXAMPLE_CLASS == org.apache.spark.examples* ]]; then diff --git a/bin/spark-class b/bin/spark-class index 0d58d95c1aee..1b945461fabc 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -71,6 +71,8 @@ case "$1" in 'org.apache.spark.executor.MesosExecutorBackend') OUR_JAVA_OPTS="$SPARK_JAVA_OPTS $SPARK_EXECUTOR_OPTS" OUR_JAVA_MEM=${SPARK_EXECUTOR_MEMORY:-$DEFAULT_MEM} + export PYTHONPATH="$FWDIR/python:$PYTHONPATH" + export PYTHONPATH="$FWDIR/python/lib/py4j-0.8.2.1-src.zip:$PYTHONPATH" ;; # Spark submit uses SPARK_JAVA_OPTS + SPARK_SUBMIT_OPTS + @@ -148,7 +150,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-$SPARK_SCALA_VERSION/" 1>&2 - echo "You need to build Spark before running $1." 1>&2 + echo "You need to run \"build/sbt tools/package\" before running $1." 1>&2 exit 1 fi CLASSPATH="$CLASSPATH:$SPARK_TOOLS_JAR" diff --git a/bin/spark-submit b/bin/spark-submit index f92d90c3a66b..3e5cbdbb2439 100755 --- a/bin/spark-submit +++ b/bin/spark-submit @@ -38,11 +38,19 @@ while (($#)); do export SPARK_SUBMIT_CLASSPATH=$2 elif [ "$1" = "--driver-java-options" ]; then export SPARK_SUBMIT_OPTS=$2 + elif [ "$1" = "--master" ]; then + export MASTER=$2 fi shift done -DEFAULT_PROPERTIES_FILE="$SPARK_HOME/conf/spark-defaults.conf" +if [ -z "$SPARK_CONF_DIR" ]; then + export SPARK_CONF_DIR="$SPARK_HOME/conf" +fi +DEFAULT_PROPERTIES_FILE="$SPARK_CONF_DIR/spark-defaults.conf" +if [ "$MASTER" == "yarn-cluster" ]; then + SPARK_SUBMIT_DEPLOY_MODE=cluster +fi export SPARK_SUBMIT_DEPLOY_MODE=${SPARK_SUBMIT_DEPLOY_MODE:-"client"} export SPARK_SUBMIT_PROPERTIES_FILE=${SPARK_SUBMIT_PROPERTIES_FILE:-"$DEFAULT_PROPERTIES_FILE"} diff --git a/bin/spark-submit2.cmd b/bin/spark-submit2.cmd index cf6046d1547a..12244a9cb04f 100644 --- a/bin/spark-submit2.cmd +++ b/bin/spark-submit2.cmd @@ -24,7 +24,11 @@ set ORIG_ARGS=%* rem Reset the values of all variables used set SPARK_SUBMIT_DEPLOY_MODE=client -set SPARK_SUBMIT_PROPERTIES_FILE=%SPARK_HOME%\conf\spark-defaults.conf + +if not defined %SPARK_CONF_DIR% ( + set SPARK_CONF_DIR=%SPARK_HOME%\conf +) +set SPARK_SUBMIT_PROPERTIES_FILE=%SPARK_CONF_DIR%\spark-defaults.conf set SPARK_SUBMIT_DRIVER_MEMORY= set SPARK_SUBMIT_LIBRARY_PATH= set SPARK_SUBMIT_CLASSPATH= @@ -45,11 +49,17 @@ if [%1] == [] goto continue set SPARK_SUBMIT_CLASSPATH=%2 ) else if [%1] == [--driver-java-options] ( set SPARK_SUBMIT_OPTS=%2 + ) else if [%1] == [--master] ( + set MASTER=%2 ) shift goto loop :continue +if [%MASTER%] == [yarn-cluster] ( + set SPARK_SUBMIT_DEPLOY_MODE=cluster +) + rem For client mode, the driver will be launched in the same JVM that launches rem SparkSubmit, so we may need to read the properties file for any extra class rem paths, library paths, java options and memory early on. Otherwise, it will diff --git a/core/pom.xml b/core/pom.xml index c5c41b2b5de4..d9a49c9e08af 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -276,11 +276,6 @@ selenium-java test - - org.scalatest - scalatest_${scala.binary.version} - test - org.mockito mockito-all @@ -326,19 +321,6 @@ target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes - - org.scalatest - scalatest-maven-plugin - - - test - - test - - - - - org.apache.maven.plugins diff --git a/core/src/main/java/org/apache/spark/JavaSparkListener.java b/core/src/main/java/org/apache/spark/JavaSparkListener.java new file mode 100644 index 000000000000..646496f31350 --- /dev/null +++ b/core/src/main/java/org/apache/spark/JavaSparkListener.java @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark; + +import org.apache.spark.scheduler.SparkListener; +import org.apache.spark.scheduler.SparkListenerApplicationEnd; +import org.apache.spark.scheduler.SparkListenerApplicationStart; +import org.apache.spark.scheduler.SparkListenerBlockManagerAdded; +import org.apache.spark.scheduler.SparkListenerBlockManagerRemoved; +import org.apache.spark.scheduler.SparkListenerEnvironmentUpdate; +import org.apache.spark.scheduler.SparkListenerExecutorAdded; +import org.apache.spark.scheduler.SparkListenerExecutorMetricsUpdate; +import org.apache.spark.scheduler.SparkListenerExecutorRemoved; +import org.apache.spark.scheduler.SparkListenerJobEnd; +import org.apache.spark.scheduler.SparkListenerJobStart; +import org.apache.spark.scheduler.SparkListenerStageCompleted; +import org.apache.spark.scheduler.SparkListenerStageSubmitted; +import org.apache.spark.scheduler.SparkListenerTaskEnd; +import org.apache.spark.scheduler.SparkListenerTaskGettingResult; +import org.apache.spark.scheduler.SparkListenerTaskStart; +import org.apache.spark.scheduler.SparkListenerUnpersistRDD; + +/** + * Java clients should extend this class instead of implementing + * SparkListener directly. This is to prevent java clients + * from breaking when new events are added to the SparkListener + * trait. + * + * This is a concrete class instead of abstract to enforce + * new events get added to both the SparkListener and this adapter + * in lockstep. + */ +public class JavaSparkListener implements SparkListener { + + @Override + public void onStageCompleted(SparkListenerStageCompleted stageCompleted) { } + + @Override + public void onStageSubmitted(SparkListenerStageSubmitted stageSubmitted) { } + + @Override + public void onTaskStart(SparkListenerTaskStart taskStart) { } + + @Override + public void onTaskGettingResult(SparkListenerTaskGettingResult taskGettingResult) { } + + @Override + public void onTaskEnd(SparkListenerTaskEnd taskEnd) { } + + @Override + public void onJobStart(SparkListenerJobStart jobStart) { } + + @Override + public void onJobEnd(SparkListenerJobEnd jobEnd) { } + + @Override + public void onEnvironmentUpdate(SparkListenerEnvironmentUpdate environmentUpdate) { } + + @Override + public void onBlockManagerAdded(SparkListenerBlockManagerAdded blockManagerAdded) { } + + @Override + public void onBlockManagerRemoved(SparkListenerBlockManagerRemoved blockManagerRemoved) { } + + @Override + public void onUnpersistRDD(SparkListenerUnpersistRDD unpersistRDD) { } + + @Override + public void onApplicationStart(SparkListenerApplicationStart applicationStart) { } + + @Override + public void onApplicationEnd(SparkListenerApplicationEnd applicationEnd) { } + + @Override + public void onExecutorMetricsUpdate(SparkListenerExecutorMetricsUpdate executorMetricsUpdate) { } + + @Override + public void onExecutorAdded(SparkListenerExecutorAdded executorAdded) { } + + @Override + public void onExecutorRemoved(SparkListenerExecutorRemoved executorRemoved) { } +} diff --git a/core/src/main/java/org/apache/spark/TaskContext.java b/core/src/main/java/org/apache/spark/TaskContext.java index 0d6973203eba..095f9fb94fdf 100644 --- a/core/src/main/java/org/apache/spark/TaskContext.java +++ b/core/src/main/java/org/apache/spark/TaskContext.java @@ -62,7 +62,7 @@ static void unset() { */ public abstract boolean isInterrupted(); - /** @deprecated: use isRunningLocally() */ + /** @deprecated use {@link #isRunningLocally()} */ @Deprecated public abstract boolean runningLocally(); @@ -87,19 +87,39 @@ static void unset() { * 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 + * @deprecated use {@link #addTaskCompletionListener(scala.Function1)} * * @param f Callback function. */ @Deprecated public abstract void addOnCompleteCallback(final Function0 f); + /** + * The ID of the stage that this task belong to. + */ public abstract int stageId(); + /** + * The ID of the RDD partition that is computed by this task. + */ public abstract int partitionId(); + /** + * How many times this task has been attempted. The first task attempt will be assigned + * attemptNumber = 0, and subsequent attempts will have increasing attempt numbers. + */ + public abstract int attemptNumber(); + + /** @deprecated use {@link #taskAttemptId()}; it was renamed to avoid ambiguity. */ + @Deprecated public abstract long attemptId(); + /** + * An ID that is unique to this task attempt (within the same SparkContext, no two task attempts + * will share the same attempt ID). This is roughly equivalent to Hadoop's TaskAttemptID. + */ + public abstract long taskAttemptId(); + /** ::DeveloperApi:: */ @DeveloperApi public abstract TaskMetrics taskMetrics(); diff --git a/core/src/main/resources/org/apache/spark/log4j-defaults.properties b/core/src/main/resources/org/apache/spark/log4j-defaults.properties index 89eec7d4b7f6..c99a61f63ea2 100644 --- a/core/src/main/resources/org/apache/spark/log4j-defaults.properties +++ b/core/src/main/resources/org/apache/spark/log4j-defaults.properties @@ -10,3 +10,4 @@ log4j.logger.org.eclipse.jetty=WARN log4j.logger.org.eclipse.jetty.util.component.AbstractLifeCycle=ERROR log4j.logger.org.apache.spark.repl.SparkIMain$exprTyper=INFO log4j.logger.org.apache.spark.repl.SparkILoop$SparkILoopInterpreter=INFO +log4j.logger.org.apache.hadoop.yarn.util.RackResolver=WARN 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 5751964b792c..a1f7133f897e 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 @@ -19,6 +19,7 @@ height: 50px; font-size: 15px; margin-bottom: 15px; + min-width: 1200px } .navbar .navbar-inner { @@ -39,12 +40,12 @@ .navbar .nav > li a { height: 30px; - line-height: 30px; + line-height: 2; } .navbar-text { height: 50px; - line-height: 50px; + line-height: 3.3; } table.sortable thead { @@ -120,6 +121,14 @@ pre { border: none; } +.description-input { + overflow: hidden; + text-overflow: ellipsis; + width: 100%; + white-space: nowrap; + display: block; +} + .stacktrace-details { max-height: 300px; overflow-y: auto; @@ -170,7 +179,7 @@ span.additional-metric-title { } .version { - line-height: 30px; + line-height: 2.5; vertical-align: bottom; font-size: 12px; padding: 0; diff --git a/core/src/main/scala/org/apache/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala index 09eb9605fb79..3b684bbeceaf 100644 --- a/core/src/main/scala/org/apache/spark/Aggregator.scala +++ b/core/src/main/scala/org/apache/spark/Aggregator.scala @@ -61,8 +61,8 @@ case class Aggregator[K, V, C] ( // Update task metrics if context is not null // TODO: Make context non optional in a future release Option(context).foreach { c => - c.taskMetrics.memoryBytesSpilled += combiners.memoryBytesSpilled - c.taskMetrics.diskBytesSpilled += combiners.diskBytesSpilled + c.taskMetrics.incMemoryBytesSpilled(combiners.memoryBytesSpilled) + c.taskMetrics.incDiskBytesSpilled(combiners.diskBytesSpilled) } combiners.iterator } @@ -95,8 +95,8 @@ case class Aggregator[K, V, C] ( // Update task metrics if context is not null // TODO: Make context non-optional in a future release Option(context).foreach { c => - c.taskMetrics.memoryBytesSpilled += combiners.memoryBytesSpilled - c.taskMetrics.diskBytesSpilled += combiners.diskBytesSpilled + c.taskMetrics.incMemoryBytesSpilled(combiners.memoryBytesSpilled) + c.taskMetrics.incDiskBytesSpilled(combiners.diskBytesSpilled) } combiners.iterator } diff --git a/core/src/main/scala/org/apache/spark/CacheManager.scala b/core/src/main/scala/org/apache/spark/CacheManager.scala index 80da62c44edc..a0c0372b7f0e 100644 --- a/core/src/main/scala/org/apache/spark/CacheManager.scala +++ b/core/src/main/scala/org/apache/spark/CacheManager.scala @@ -44,7 +44,11 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { blockManager.get(key) match { case Some(blockResult) => // Partition is already materialized, so just return its values - context.taskMetrics.inputMetrics = Some(blockResult.inputMetrics) + val inputMetrics = blockResult.inputMetrics + val existingMetrics = context.taskMetrics + .getInputMetricsForReadMethod(inputMetrics.readMethod) + existingMetrics.addBytesRead(inputMetrics.bytesRead) + new InterruptibleIterator(context, blockResult.data.asInstanceOf[Iterator[T]]) case None => diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index e9e90e3f2f65..b28da192c1c0 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -65,6 +65,9 @@ private[spark] class ExecutorAllocationManager( listenerBus: LiveListenerBus, conf: SparkConf) extends Logging { + + allocationManager => + import ExecutorAllocationManager._ // Lower and upper bounds on the number of executors. These are required. @@ -121,7 +124,7 @@ private[spark] class ExecutorAllocationManager( private var clock: Clock = new RealClock // Listener for Spark events that impact the allocation policy - private val listener = new ExecutorAllocationListener(this) + private val listener = new ExecutorAllocationListener /** * Verify that the settings specified through the config are valid. @@ -155,7 +158,7 @@ private[spark] class ExecutorAllocationManager( "shuffle service. You may enable this through spark.shuffle.service.enabled.") } if (tasksPerExecutor == 0) { - throw new SparkException("spark.executor.cores must not be less than spark.task.cpus.cores") + throw new SparkException("spark.executor.cores must not be less than spark.task.cpus.") } } @@ -209,11 +212,12 @@ private[spark] class ExecutorAllocationManager( addTime += sustainedSchedulerBacklogTimeout * 1000 } - removeTimes.foreach { case (executorId, expireTime) => - if (now >= expireTime) { + removeTimes.retain { case (executorId, expireTime) => + val expired = now >= expireTime + if (expired) { removeExecutor(executorId) - removeTimes.remove(executorId) } + !expired } } @@ -291,7 +295,7 @@ private[spark] class ExecutorAllocationManager( // Do not kill the executor if we have already reached the lower bound val numExistingExecutors = executorIds.size - executorsPendingToRemove.size if (numExistingExecutors - 1 < minNumExecutors) { - logInfo(s"Not removing idle executor $executorId because there are only " + + logDebug(s"Not removing idle executor $executorId because there are only " + s"$numExistingExecutors executor(s) left (limit $minNumExecutors)") return false } @@ -315,7 +319,11 @@ private[spark] class ExecutorAllocationManager( private def onExecutorAdded(executorId: String): Unit = synchronized { if (!executorIds.contains(executorId)) { executorIds.add(executorId) - executorIds.foreach(onExecutorIdle) + // If an executor (call this executor X) is not removed because the lower bound + // has been reached, it will no longer be marked as idle. When new executors join, + // however, we are no longer at the lower bound, and so we must mark executor X + // as idle again so as not to forget that it is a candidate for removal. (see SPARK-4951) + executorIds.filter(listener.isExecutorIdle).foreach(onExecutorIdle) logInfo(s"New executor $executorId has registered (new total is ${executorIds.size})") if (numExecutorsPending > 0) { numExecutorsPending -= 1 @@ -373,10 +381,14 @@ private[spark] class ExecutorAllocationManager( * the executor is not already marked as idle. */ private def onExecutorIdle(executorId: String): Unit = synchronized { - if (!removeTimes.contains(executorId) && !executorsPendingToRemove.contains(executorId)) { - logDebug(s"Starting idle timer for $executorId because there are no more tasks " + - s"scheduled to run on the executor (to expire in $executorIdleTimeout seconds)") - removeTimes(executorId) = clock.getTimeMillis + executorIdleTimeout * 1000 + if (executorIds.contains(executorId)) { + if (!removeTimes.contains(executorId) && !executorsPendingToRemove.contains(executorId)) { + logDebug(s"Starting idle timer for $executorId because there are no more tasks " + + s"scheduled to run on the executor (to expire in $executorIdleTimeout seconds)") + removeTimes(executorId) = clock.getTimeMillis + executorIdleTimeout * 1000 + } + } else { + logWarning(s"Attempted to mark unknown executor $executorId idle") } } @@ -396,25 +408,24 @@ private[spark] class ExecutorAllocationManager( * and consistency of events returned by the listener. For simplicity, it does not account * for speculated tasks. */ - private class ExecutorAllocationListener(allocationManager: ExecutorAllocationManager) - extends SparkListener { + private class ExecutorAllocationListener extends SparkListener { private val stageIdToNumTasks = new mutable.HashMap[Int, Int] private val stageIdToTaskIndices = new mutable.HashMap[Int, mutable.HashSet[Int]] private val executorIdToTaskIds = new mutable.HashMap[String, mutable.HashSet[Long]] override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted): Unit = { - synchronized { - val stageId = stageSubmitted.stageInfo.stageId - val numTasks = stageSubmitted.stageInfo.numTasks + val stageId = stageSubmitted.stageInfo.stageId + val numTasks = stageSubmitted.stageInfo.numTasks + allocationManager.synchronized { stageIdToNumTasks(stageId) = numTasks allocationManager.onSchedulerBacklogged() } } override def onStageCompleted(stageCompleted: SparkListenerStageCompleted): Unit = { - synchronized { - val stageId = stageCompleted.stageInfo.stageId + val stageId = stageCompleted.stageInfo.stageId + allocationManager.synchronized { stageIdToNumTasks -= stageId stageIdToTaskIndices -= stageId @@ -426,39 +437,49 @@ private[spark] class ExecutorAllocationManager( } } - override def onTaskStart(taskStart: SparkListenerTaskStart): Unit = synchronized { + override def onTaskStart(taskStart: SparkListenerTaskStart): Unit = { val stageId = taskStart.stageId val taskId = taskStart.taskInfo.taskId val taskIndex = taskStart.taskInfo.index val executorId = taskStart.taskInfo.executorId - // If this is the last pending task, mark the scheduler queue as empty - stageIdToTaskIndices.getOrElseUpdate(stageId, new mutable.HashSet[Int]) += taskIndex - val numTasksScheduled = stageIdToTaskIndices(stageId).size - val numTasksTotal = stageIdToNumTasks.getOrElse(stageId, -1) - if (numTasksScheduled == numTasksTotal) { - // No more pending tasks for this stage - stageIdToNumTasks -= stageId - if (stageIdToNumTasks.isEmpty) { - allocationManager.onSchedulerQueueEmpty() + allocationManager.synchronized { + // This guards against the race condition in which the `SparkListenerTaskStart` + // event is posted before the `SparkListenerBlockManagerAdded` event, which is + // possible because these events are posted in different threads. (see SPARK-4951) + if (!allocationManager.executorIds.contains(executorId)) { + allocationManager.onExecutorAdded(executorId) + } + + // If this is the last pending task, mark the scheduler queue as empty + stageIdToTaskIndices.getOrElseUpdate(stageId, new mutable.HashSet[Int]) += taskIndex + val numTasksScheduled = stageIdToTaskIndices(stageId).size + val numTasksTotal = stageIdToNumTasks.getOrElse(stageId, -1) + if (numTasksScheduled == numTasksTotal) { + // No more pending tasks for this stage + stageIdToNumTasks -= stageId + if (stageIdToNumTasks.isEmpty) { + allocationManager.onSchedulerQueueEmpty() + } } - } - // Mark the executor on which this task is scheduled as busy - executorIdToTaskIds.getOrElseUpdate(executorId, new mutable.HashSet[Long]) += taskId - allocationManager.onExecutorBusy(executorId) + // Mark the executor on which this task is scheduled as busy + executorIdToTaskIds.getOrElseUpdate(executorId, new mutable.HashSet[Long]) += taskId + allocationManager.onExecutorBusy(executorId) + } } - override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = synchronized { + override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = { val executorId = taskEnd.taskInfo.executorId val taskId = taskEnd.taskInfo.taskId - - // If the executor is no longer running scheduled any tasks, mark it as idle - if (executorIdToTaskIds.contains(executorId)) { - executorIdToTaskIds(executorId) -= taskId - if (executorIdToTaskIds(executorId).isEmpty) { - executorIdToTaskIds -= executorId - allocationManager.onExecutorIdle(executorId) + allocationManager.synchronized { + // If the executor is no longer running scheduled any tasks, mark it as idle + if (executorIdToTaskIds.contains(executorId)) { + executorIdToTaskIds(executorId) -= taskId + if (executorIdToTaskIds(executorId).isEmpty) { + executorIdToTaskIds -= executorId + allocationManager.onExecutorIdle(executorId) + } } } } @@ -466,7 +487,12 @@ private[spark] class ExecutorAllocationManager( override def onBlockManagerAdded(blockManagerAdded: SparkListenerBlockManagerAdded): Unit = { val executorId = blockManagerAdded.blockManagerId.executorId if (executorId != SparkContext.DRIVER_IDENTIFIER) { - allocationManager.onExecutorAdded(executorId) + // This guards against the race condition in which the `SparkListenerTaskStart` + // event is posted before the `SparkListenerBlockManagerAdded` event, which is + // possible because these events are posted in different threads. (see SPARK-4951) + if (!allocationManager.executorIds.contains(executorId)) { + allocationManager.onExecutorAdded(executorId) + } } } @@ -478,12 +504,23 @@ private[spark] class ExecutorAllocationManager( /** * An estimate of the total number of pending tasks remaining for currently running stages. Does * not account for tasks which may have failed and been resubmitted. + * + * Note: This is not thread-safe without the caller owning the `allocationManager` lock. */ def totalPendingTasks(): Int = { stageIdToNumTasks.map { case (stageId, numTasks) => numTasks - stageIdToTaskIndices.get(stageId).map(_.size).getOrElse(0) }.sum } + + /** + * Return true if an executor is not currently running a task, and false otherwise. + * + * Note: This is not thread-safe without the caller owning the `allocationManager` lock. + */ + def isExecutorIdle(executorId: String): Boolean = { + !executorIdToTaskIds.contains(executorId) + } } } diff --git a/core/src/main/scala/org/apache/spark/HttpFileServer.scala b/core/src/main/scala/org/apache/spark/HttpFileServer.scala index edc3889c9ae5..677c5e0f89d7 100644 --- a/core/src/main/scala/org/apache/spark/HttpFileServer.scala +++ b/core/src/main/scala/org/apache/spark/HttpFileServer.scala @@ -24,6 +24,7 @@ import com.google.common.io.Files import org.apache.spark.util.Utils private[spark] class HttpFileServer( + conf: SparkConf, securityManager: SecurityManager, requestedPort: Int = 0) extends Logging { @@ -41,7 +42,7 @@ private[spark] class HttpFileServer( fileDir.mkdir() jarDir.mkdir() logInfo("HTTP File server directory is " + baseDir) - httpServer = new HttpServer(baseDir, securityManager, requestedPort, "HTTP file server") + httpServer = new HttpServer(conf, baseDir, securityManager, requestedPort, "HTTP file server") httpServer.start() serverUri = httpServer.uri logDebug("HTTP file server started at: " + serverUri) diff --git a/core/src/main/scala/org/apache/spark/HttpServer.scala b/core/src/main/scala/org/apache/spark/HttpServer.scala index 912558d0cab7..fa22787ce7ea 100644 --- a/core/src/main/scala/org/apache/spark/HttpServer.scala +++ b/core/src/main/scala/org/apache/spark/HttpServer.scala @@ -42,6 +42,7 @@ private[spark] class ServerStateException(message: String) extends Exception(mes * around a Jetty server. */ private[spark] class HttpServer( + conf: SparkConf, resourceBase: File, securityManager: SecurityManager, requestedPort: Int = 0, @@ -57,7 +58,7 @@ private[spark] class HttpServer( } else { logInfo("Starting HTTP Server") val (actualServer, actualPort) = - Utils.startServiceOnPort[Server](requestedPort, doStart, serverName) + Utils.startServiceOnPort[Server](requestedPort, doStart, conf, serverName) server = actualServer port = actualPort } diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index a074ab8ece1b..6e4edc7c80d7 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -76,6 +76,8 @@ private[spark] class MapOutputTrackerMasterActor(tracker: MapOutputTrackerMaster */ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging { private val timeout = AkkaUtils.askTimeout(conf) + private val retryAttempts = AkkaUtils.numRetries(conf) + private val retryIntervalMs = AkkaUtils.retryWaitMs(conf) /** Set to the MapOutputTrackerActor living on the driver. */ var trackerActor: ActorRef = _ @@ -108,8 +110,7 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging */ protected def askTracker(message: Any): Any = { try { - val future = trackerActor.ask(message)(timeout) - Await.result(future, timeout) + AkkaUtils.askWithReply(message, trackerActor, retryAttempts, retryIntervalMs, timeout) } catch { case e: Exception => logError("Error communicating with MapOutputTracker", e) diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index c14764f77398..f9d4aa4240e9 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -18,6 +18,7 @@ package org.apache.spark import scala.collection.JavaConverters._ +import scala.collection.concurrent.TrieMap import scala.collection.mutable.{HashMap, LinkedHashSet} import org.apache.spark.serializer.KryoSerializer @@ -46,7 +47,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { /** Create a SparkConf that loads defaults from system properties and the classpath */ def this() = this(true) - private[spark] val settings = new HashMap[String, String]() + private[spark] val settings = new TrieMap[String, String]() if (loadDefaults) { // Load any spark.* system properties @@ -177,7 +178,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { } /** Get all parameters as a list of pairs */ - def getAll: Array[(String, String)] = settings.clone().toArray + def getAll: Array[(String, String)] = settings.toArray /** Get a parameter as an integer, falling back to a default if not set */ def getInt(key: String, defaultValue: Int): Int = { @@ -370,7 +371,9 @@ private[spark] object SparkConf { } /** - * Return whether the given config is a Spark port config. + * Return true if the given config matches either `spark.*.port` or `spark.port.*`. */ - def isSparkPortConf(name: String): Boolean = name.startsWith("spark.") && name.endsWith(".port") + def isSparkPortConf(name: String): Boolean = { + (name.startsWith("spark.") && name.endsWith(".port")) || name.startsWith("spark.port.") + } } diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 57bc3d4e4ae3..6a354ed4d148 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -229,7 +229,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli // An asynchronous listener bus for Spark events private[spark] val listenerBus = new LiveListenerBus - conf.set("spark.executor.id", "driver") + conf.set("spark.executor.id", SparkContext.DRIVER_IDENTIFIER) // Create the Spark execution environment (cache, map output tracker, etc) private[spark] val env = SparkEnv.createDriverEnv(conf, isLocal, listenerBus) @@ -329,8 +329,13 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli try { dagScheduler = new DAGScheduler(this) } catch { - case e: Exception => throw - new SparkException("DAGScheduler cannot be initialized due to %s".format(e.getMessage)) + case e: Exception => { + try { + stop() + } finally { + throw new SparkException("Error while constructing DAGScheduler", e) + } + } } // start TaskScheduler after taskScheduler sets DAGScheduler reference in DAGScheduler's @@ -453,7 +458,6 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli Option(localProperties.get).map(_.getProperty(key)).getOrElse(null) /** Set a human readable description of the current job. */ - @deprecated("use setJobGroup", "0.8.1") def setJobDescription(value: String) { setLocalProperty(SparkContext.SPARK_JOB_DESCRIPTION, value) } @@ -516,10 +520,9 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli /** Distribute a local Scala collection to form an RDD. * - * @note Parallelize acts lazily. If `seq` is a mutable collection and is - * altered after the call to parallelize and before the first action on the - * RDD, the resultant RDD will reflect the modified collection. Pass a copy of - * the argument to avoid this. + * @note Parallelize acts lazily. If `seq` is a mutable collection and is altered after the call + * to parallelize and before the first action on the RDD, the resultant RDD will reflect the + * modified collection. Pass a copy of the argument to avoid this. */ def parallelize[T: ClassTag](seq: Seq[T], numSlices: Int = defaultParallelism): RDD[T] = { new ParallelCollectionRDD[T](this, seq, numSlices, Map[Int, Seq[String]]()) @@ -1708,19 +1711,19 @@ object SparkContext extends Logging { // Implicit conversions to common Writable types, for saveAsSequenceFile - implicit def intToIntWritable(i: Int) = new IntWritable(i) + implicit def intToIntWritable(i: Int): IntWritable = new IntWritable(i) - implicit def longToLongWritable(l: Long) = new LongWritable(l) + implicit def longToLongWritable(l: Long): LongWritable = new LongWritable(l) - implicit def floatToFloatWritable(f: Float) = new FloatWritable(f) + implicit def floatToFloatWritable(f: Float): FloatWritable = new FloatWritable(f) - implicit def doubleToDoubleWritable(d: Double) = new DoubleWritable(d) + implicit def doubleToDoubleWritable(d: Double): DoubleWritable = new DoubleWritable(d) - implicit def boolToBoolWritable (b: Boolean) = new BooleanWritable(b) + implicit def boolToBoolWritable (b: Boolean): BooleanWritable = new BooleanWritable(b) - implicit def bytesToBytesWritable (aob: Array[Byte]) = new BytesWritable(aob) + implicit def bytesToBytesWritable (aob: Array[Byte]): BytesWritable = new BytesWritable(aob) - implicit def stringToText(s: String) = new Text(s) + implicit def stringToText(s: String): Text = new Text(s) private implicit def arrayToArrayWritable[T <% Writable: ClassTag](arr: Traversable[T]) : ArrayWritable = { diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 43436a169700..4d418037bd33 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -312,7 +312,7 @@ object SparkEnv extends Logging { val httpFileServer = if (isDriver) { val fileServerPort = conf.getInt("spark.fileserver.port", 0) - val server = new HttpFileServer(securityManager, fileServerPort) + val server = new HttpFileServer(conf, securityManager, fileServerPort) server.initialize() conf.set("spark.fileserver.uri", server.serverUri) server diff --git a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala index afd2b85d33a7..9bb0c61e441f 100644 --- a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala +++ b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala @@ -22,14 +22,19 @@ import org.apache.spark.util.{TaskCompletionListener, TaskCompletionListenerExce import scala.collection.mutable.ArrayBuffer -private[spark] class TaskContextImpl(val stageId: Int, +private[spark] class TaskContextImpl( + val stageId: Int, val partitionId: Int, - val attemptId: Long, + override val taskAttemptId: Long, + override val attemptNumber: Int, val runningLocally: Boolean = false, val taskMetrics: TaskMetrics = TaskMetrics.empty) extends TaskContext with Logging { + // For backwards-compatibility; this method is now deprecated as of 1.3.0. + override def attemptId: Long = taskAttemptId + // List of callback functions to execute when the task completes. @transient private val onCompleteCallbacks = new ArrayBuffer[TaskCompletionListener] diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/NullType.java b/core/src/main/scala/org/apache/spark/TaskNotSerializableException.scala similarity index 76% rename from sql/core/src/main/java/org/apache/spark/sql/api/java/NullType.java rename to core/src/main/scala/org/apache/spark/TaskNotSerializableException.scala index 6d5ecdf46e55..9df61062e1f8 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/NullType.java +++ b/core/src/main/scala/org/apache/spark/TaskNotSerializableException.scala @@ -15,13 +15,11 @@ * limitations under the License. */ -package org.apache.spark.sql.api.java; +package org.apache.spark + +import org.apache.spark.annotation.DeveloperApi /** - * The data type representing null and NULL values. - * - * {@code NullType} is represented by the singleton object {@link DataType#NullType}. + * Exception thrown when a task cannot be serialized. */ -public class NullType extends DataType { - protected NullType() {} -} +private[spark] class TaskNotSerializableException(error: Throwable) extends Exception(error) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index bd451634e53d..62bf18d82d9b 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -38,6 +38,10 @@ import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel import org.apache.spark.util.Utils +/** + * Defines operations common to several Java RDD implementations. + * Note that this trait is not intended to be implemented by user code. + */ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { def wrapRDD(rdd: RDD[T]): This @@ -435,6 +439,12 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { */ def first(): T = rdd.first() + /** + * @return true if and only if the RDD contains no elements at all. Note that an RDD + * may be empty even when it has at least 1 partition. + */ + def isEmpty(): Boolean = rdd.isEmpty() + /** * Save this RDD as a text file, using string representations of elements. */ diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala b/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala index 86e94931300f..71b26737b8c0 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala @@ -80,7 +80,7 @@ private[spark] object JavaUtils { prev match { case Some(k) => underlying match { - case mm: mutable.Map[a, _] => + case mm: mutable.Map[A, _] => mm remove k prev = None case _ => 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 bad40e6529f7..6f5d3dda377d 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 @@ -125,8 +125,8 @@ private[spark] class PythonRDD( init, finish)) val memoryBytesSpilled = stream.readLong() val diskBytesSpilled = stream.readLong() - context.taskMetrics.memoryBytesSpilled += memoryBytesSpilled - context.taskMetrics.diskBytesSpilled += diskBytesSpilled + context.taskMetrics.incMemoryBytesSpilled(memoryBytesSpilled) + context.taskMetrics.incDiskBytesSpilled(diskBytesSpilled) read() case SpecialLengths.PYTHON_EXCEPTION_THROWN => // Signals that an exception has been thrown in python @@ -313,6 +313,7 @@ private object SpecialLengths { val PYTHON_EXCEPTION_THROWN = -2 val TIMING_DATA = -3 val END_OF_STREAM = -4 + val NULL = -5 } private[spark] object PythonRDD extends Logging { @@ -374,49 +375,63 @@ private[spark] object PythonRDD extends Logging { // The right way to implement this would be to use TypeTags to get the full // type of T. Since I don't want to introduce breaking changes throughout the // entire Spark API, I have to use this hacky approach: + def write(bytes: Array[Byte]) { + if (bytes == null) { + dataOut.writeInt(SpecialLengths.NULL) + } else { + dataOut.writeInt(bytes.length) + dataOut.write(bytes) + } + } + + def writeS(str: String) { + if (str == null) { + dataOut.writeInt(SpecialLengths.NULL) + } else { + writeUTF(str, dataOut) + } + } + if (iter.hasNext) { val first = iter.next() val newIter = Seq(first).iterator ++ iter first match { case arr: Array[Byte] => - newIter.asInstanceOf[Iterator[Array[Byte]]].foreach { bytes => - dataOut.writeInt(bytes.length) - dataOut.write(bytes) - } + newIter.asInstanceOf[Iterator[Array[Byte]]].foreach(write) case string: String => - newIter.asInstanceOf[Iterator[String]].foreach { str => - writeUTF(str, dataOut) - } + newIter.asInstanceOf[Iterator[String]].foreach(writeS) case stream: PortableDataStream => newIter.asInstanceOf[Iterator[PortableDataStream]].foreach { stream => - val bytes = stream.toArray() - dataOut.writeInt(bytes.length) - dataOut.write(bytes) + write(stream.toArray()) } case (key: String, stream: PortableDataStream) => newIter.asInstanceOf[Iterator[(String, PortableDataStream)]].foreach { case (key, stream) => - writeUTF(key, dataOut) - val bytes = stream.toArray() - dataOut.writeInt(bytes.length) - dataOut.write(bytes) + writeS(key) + write(stream.toArray()) } case (key: String, value: String) => newIter.asInstanceOf[Iterator[(String, String)]].foreach { case (key, value) => - writeUTF(key, dataOut) - writeUTF(value, dataOut) + writeS(key) + writeS(value) } case (key: Array[Byte], value: Array[Byte]) => newIter.asInstanceOf[Iterator[(Array[Byte], Array[Byte])]].foreach { case (key, value) => - dataOut.writeInt(key.length) - dataOut.write(key) - dataOut.writeInt(value.length) - dataOut.write(value) + write(key) + write(value) } + // key is null + case (null, v:Array[Byte]) => + newIter.asInstanceOf[Iterator[(Array[Byte], Array[Byte])]].foreach { + case (key, value) => + write(key) + write(value) + } + case other => - throw new SparkException("Unexpected element type " + first.getClass) + throw new SparkException("Unexpected element type " + other.getClass) } } } 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 31f0a462f84d..31d6958c403b 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala @@ -153,7 +153,8 @@ private[broadcast] object HttpBroadcast extends Logging { private def createServer(conf: SparkConf) { broadcastDir = Utils.createTempDir(Utils.getLocalDir(conf)) val broadcastPort = conf.getInt("spark.broadcast.port", 0) - server = new HttpServer(broadcastDir, securityManager, broadcastPort, "HTTP broadcast server") + server = + new HttpServer(conf, broadcastDir, securityManager, broadcastPort, "HTTP broadcast server") server.start() serverUri = server.uri logInfo("Broadcast server started at " + serverUri) diff --git a/core/src/main/scala/org/apache/spark/deploy/Client.scala b/core/src/main/scala/org/apache/spark/deploy/Client.scala index f2687ce6b42b..7c1c831c248f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/Client.scala +++ b/core/src/main/scala/org/apache/spark/deploy/Client.scala @@ -160,6 +160,8 @@ object Client { val (actorSystem, _) = AkkaUtils.createActorSystem( "driverClient", Utils.localHostName(), 0, conf, new SecurityManager(conf)) + // Verify driverArgs.master is a valid url so that we can use it in ClientActor safely + Master.toAkkaUrl(driverArgs.master) actorSystem.actorOf(Props(classOf[ClientActor], driverArgs, conf)) actorSystem.awaitTermination() diff --git a/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala index 2e1e52906cee..e5873ce724b9 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala @@ -23,7 +23,7 @@ import scala.collection.mutable.ListBuffer import org.apache.log4j.Level -import org.apache.spark.util.MemoryParam +import org.apache.spark.util.{IntParam, MemoryParam} /** * Command-line parser for the driver client. @@ -51,8 +51,8 @@ private[spark] class ClientArguments(args: Array[String]) { parse(args.toList) def parse(args: List[String]): Unit = args match { - case ("--cores" | "-c") :: value :: tail => - cores = value.toInt + case ("--cores" | "-c") :: IntParam(value) :: tail => + cores = value parse(tail) case ("--memory" | "-m") :: MemoryParam(value) :: tail => 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 955cbd6dab96..050ba91eb2bc 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -200,6 +200,7 @@ object SparkSubmit { // Yarn cluster only OptionAssigner(args.name, YARN, CLUSTER, clOption = "--name"), OptionAssigner(args.driverMemory, YARN, CLUSTER, clOption = "--driver-memory"), + OptionAssigner(args.driverCores, YARN, CLUSTER, clOption = "--driver-cores"), OptionAssigner(args.queue, YARN, CLUSTER, clOption = "--queue"), OptionAssigner(args.numExecutors, YARN, CLUSTER, clOption = "--num-executors"), OptionAssigner(args.executorMemory, YARN, CLUSTER, clOption = "--executor-memory"), 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 f174bc1af59b..81ec08cb6d50 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -17,6 +17,7 @@ package org.apache.spark.deploy +import java.net.URI import java.util.jar.JarFile import scala.collection.mutable.{ArrayBuffer, HashMap} @@ -107,6 +108,9 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St .orElse(sparkProperties.get("spark.driver.memory")) .orElse(env.get("SPARK_DRIVER_MEMORY")) .orNull + driverCores = Option(driverCores) + .orElse(sparkProperties.get("spark.driver.cores")) + .orNull executorMemory = Option(executorMemory) .orElse(sparkProperties.get("spark.executor.memory")) .orElse(env.get("SPARK_EXECUTOR_MEMORY")) @@ -125,20 +129,34 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St // Try to set main class from JAR if no --class argument is given if (mainClass == null && !isPython && primaryResource != null) { - try { - val jar = new JarFile(primaryResource) - // Note that this might still return null if no main-class is set; we catch that later - mainClass = jar.getManifest.getMainAttributes.getValue("Main-Class") - } catch { - case e: Exception => - SparkSubmit.printErrorAndExit("Cannot load main class from JAR: " + primaryResource) - return + val uri = new URI(primaryResource) + val uriScheme = uri.getScheme() + + uriScheme match { + case "file" => + try { + val jar = new JarFile(uri.getPath) + // Note that this might still return null if no main-class is set; we catch that later + mainClass = jar.getManifest.getMainAttributes.getValue("Main-Class") + } catch { + case e: Exception => + SparkSubmit.printErrorAndExit(s"Cannot load main class from JAR $primaryResource") + } + case _ => + SparkSubmit.printErrorAndExit( + s"Cannot load main class from JAR $primaryResource with URI $uriScheme. " + + "Please specify a class through --class.") } } // Global defaults. These should be keep to minimum to avoid confusing behavior. master = Option(master).getOrElse("local[*]") + // In YARN mode, app name can be set via SPARK_YARN_APP_NAME (see SPARK-5222) + if (master.startsWith("yarn")) { + name = Option(name).orElse(env.get("SPARK_YARN_APP_NAME")).orNull + } + // Set name from main class if not given name = Option(name).orElse(Option(mainClass)).orNull if (name == null && primaryResource != null) { @@ -391,11 +409,14 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St | --total-executor-cores NUM Total cores for all executors. | | YARN-only: + | --driver-cores NUM Number of cores used by the driver, only in cluster mode + | (Default: 1). | --executor-cores NUM Number of cores per executor (Default: 1). | --queue QUEUE_NAME The YARN queue to submit to (Default: "default"). | --num-executors NUM Number of executors to launch (Default: 2). | --archives ARCHIVES Comma separated list of archives to be extracted into the - | working directory of each executor.""".stripMargin + | working directory of each executor. + """.stripMargin ) SparkSubmit.exitFn() } diff --git a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala index 4efebcaa350f..39a7b0319b6a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala @@ -26,7 +26,7 @@ import akka.actor._ import akka.pattern.ask import akka.remote.{AssociationErrorEvent, DisassociatedEvent, RemotingLifecycleEvent} -import org.apache.spark.{Logging, SparkConf, SparkException} +import org.apache.spark.{Logging, SparkConf} import org.apache.spark.deploy.{ApplicationDescription, ExecutorState} import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.Master @@ -47,6 +47,8 @@ private[spark] class AppClient( conf: SparkConf) extends Logging { + val masterAkkaUrls = masterUrls.map(Master.toAkkaUrl) + val REGISTRATION_TIMEOUT = 20.seconds val REGISTRATION_RETRIES = 3 @@ -75,9 +77,9 @@ private[spark] class AppClient( } def tryRegisterAllMasters() { - for (masterUrl <- masterUrls) { - logInfo("Connecting to master " + masterUrl + "...") - val actor = context.actorSelection(Master.toAkkaUrl(masterUrl)) + for (masterAkkaUrl <- masterAkkaUrls) { + logInfo("Connecting to master " + masterAkkaUrl + "...") + val actor = context.actorSelection(masterAkkaUrl) actor ! RegisterApplication(appDescription) } } @@ -103,20 +105,14 @@ private[spark] class AppClient( } def changeMaster(url: String) { + // activeMasterUrl is a valid Spark url since we receive it from master. activeMasterUrl = url master = context.actorSelection(Master.toAkkaUrl(activeMasterUrl)) - masterAddress = activeMasterUrl match { - case Master.sparkUrlRegex(host, port) => - Address("akka.tcp", Master.systemName, host, port.toInt) - case x => - throw new SparkException("Invalid spark URL: " + x) - } + masterAddress = Master.toAkkaAddress(activeMasterUrl) } private def isPossibleMaster(remoteUrl: Address) = { - masterUrls.map(s => Master.toAkkaUrl(s)) - .map(u => AddressFromURIString(u).hostPort) - .contains(remoteUrl.hostPort) + masterAkkaUrls.map(AddressFromURIString(_).hostPort).contains(remoteUrl.hostPort) } override def receiveWithLogging = { diff --git a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala index fbe39b27649f..553bf3cb945a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala @@ -25,7 +25,8 @@ private[spark] case class ApplicationHistoryInfo( startTime: Long, endTime: Long, lastUpdated: Long, - sparkUser: String) + sparkUser: String, + completed: Boolean = false) private[spark] abstract class ApplicationHistoryProvider { diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 792d15b99ea0..2b084a2d73b7 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -173,20 +173,9 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis val logInfos = statusList .filter { entry => try { - val isFinishedApplication = - if (isLegacyLogDirectory(entry)) { - fs.exists(new Path(entry.getPath(), APPLICATION_COMPLETE)) - } else { - !entry.getPath().getName().endsWith(EventLoggingListener.IN_PROGRESS) - } - - if (isFinishedApplication) { - val modTime = getModificationTime(entry) - newLastModifiedTime = math.max(newLastModifiedTime, modTime) - modTime >= lastModifiedTime - } else { - false - } + val modTime = getModificationTime(entry) + newLastModifiedTime = math.max(newLastModifiedTime, modTime) + modTime >= lastModifiedTime } catch { case e: AccessControlException => // Do not use "logInfo" since these messages can get pretty noisy if printed on @@ -204,7 +193,7 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis None } } - .sortBy { info => -info.endTime } + .sortBy { info => (-info.endTime, -info.startTime) } lastModifiedTime = newLastModifiedTime @@ -261,7 +250,8 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis appListener.startTime.getOrElse(-1L), appListener.endTime.getOrElse(-1L), getModificationTime(eventLog), - appListener.sparkUser.getOrElse(NOT_STARTED)) + appListener.sparkUser.getOrElse(NOT_STARTED), + isApplicationCompleted(eventLog)) } finally { logInput.close() } @@ -329,6 +319,17 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis /** Returns the system's mononotically increasing time. */ private def getMonotonicTimeMs(): Long = System.nanoTime() / (1000 * 1000) + /** + * Return true when the application has completed. + */ + private def isApplicationCompleted(entry: FileStatus): Boolean = { + if (isLegacyLogDirectory(entry)) { + fs.exists(new Path(entry.getPath(), APPLICATION_COMPLETE)) + } else { + !entry.getPath().getName().endsWith(EventLoggingListener.IN_PROGRESS) + } + } + } private object FsHistoryProvider { @@ -342,5 +343,6 @@ private class FsApplicationHistoryInfo( startTime: Long, endTime: Long, lastUpdated: Long, - sparkUser: String) - extends ApplicationHistoryInfo(id, name, startTime, endTime, lastUpdated, sparkUser) + sparkUser: String, + completed: Boolean = true) + extends ApplicationHistoryInfo(id, name, startTime, endTime, lastUpdated, sparkUser, completed) 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 0d5dcfb1ddff..e4e7bc221601 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 @@ -31,8 +31,10 @@ private[spark] class HistoryPage(parent: HistoryServer) extends WebUIPage("") { def render(request: HttpServletRequest): Seq[Node] = { val requestedPage = Option(request.getParameter("page")).getOrElse("1").toInt val requestedFirst = (requestedPage - 1) * pageSize + val requestedIncomplete = + Option(request.getParameter("showIncomplete")).getOrElse("false").toBoolean - val allApps = parent.getApplicationList() + val allApps = parent.getApplicationList().filter(_.completed != requestedIncomplete) val actualFirst = if (requestedFirst < allApps.size) requestedFirst else 0 val apps = allApps.slice(actualFirst, Math.min(actualFirst + pageSize, allApps.size)) @@ -65,25 +67,26 @@ private[spark] class HistoryPage(parent: HistoryServer) extends WebUIPage("") {

Showing {actualFirst + 1}-{last + 1} of {allApps.size} - - { - if (actualPage > 1) { - < - 1 - } + {if (requestedIncomplete) "(Incomplete applications)"} + + { + if (actualPage > 1) { + < + 1 } - {if (actualPage - plusOrMinus > secondPageFromLeft) " ... "} - {leftSideIndices} - {actualPage} - {rightSideIndices} - {if (actualPage + plusOrMinus < secondPageFromRight) " ... "} - { - if (actualPage < pageCount) { - {pageCount} - > - } + } + {if (actualPage - plusOrMinus > secondPageFromLeft) " ... "} + {leftSideIndices} + {actualPage} + {rightSideIndices} + {if (actualPage + plusOrMinus < secondPageFromRight) " ... "} + { + if (actualPage < pageCount) { + {pageCount} + > } - + } +

++ appTable } else { @@ -96,6 +99,15 @@ private[spark] class HistoryPage(parent: HistoryServer) extends WebUIPage("") {

} } + + { + if (requestedIncomplete) { + "Back to completed applications" + } else { + "Show incomplete applications" + } + } + UIUtils.basicSparkPage(content, "History Server") @@ -117,8 +129,9 @@ private[spark] class HistoryPage(parent: HistoryServer) extends WebUIPage("") { private def appRow(info: ApplicationHistoryInfo): Seq[Node] = { val uiAddress = HistoryServer.UI_PATH_PREFIX + s"/${info.id}" val startTime = UIUtils.formatDate(info.startTime) - val endTime = UIUtils.formatDate(info.endTime) - val duration = UIUtils.formatDuration(info.endTime - info.startTime) + val endTime = if (info.endTime > 0) UIUtils.formatDate(info.endTime) else "-" + val duration = + if (info.endTime > 0) UIUtils.formatDuration(info.endTime - info.startTime) else "-" val lastUpdated = UIUtils.formatDate(info.lastUpdated) {info.id} @@ -130,4 +143,11 @@ private[spark] class HistoryPage(parent: HistoryServer) extends WebUIPage("") { {lastUpdated} } + + private def makePageLink(linkPage: Int, showIncomplete: Boolean): String = { + "/?" + Array( + "page=" + linkPage, + "showIncomplete=" + showIncomplete + ).mkString("&") + } } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala index ad7d81747c37..ede0a9dbefb8 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala @@ -38,8 +38,8 @@ private[spark] class ApplicationInfo( extends Serializable { @transient var state: ApplicationState.Value = _ - @transient var executors: mutable.HashMap[Int, ExecutorInfo] = _ - @transient var removedExecutors: ArrayBuffer[ExecutorInfo] = _ + @transient var executors: mutable.HashMap[Int, ExecutorDesc] = _ + @transient var removedExecutors: ArrayBuffer[ExecutorDesc] = _ @transient var coresGranted: Int = _ @transient var endTime: Long = _ @transient var appSource: ApplicationSource = _ @@ -55,12 +55,12 @@ private[spark] class ApplicationInfo( private def init() { state = ApplicationState.WAITING - executors = new mutable.HashMap[Int, ExecutorInfo] + executors = new mutable.HashMap[Int, ExecutorDesc] coresGranted = 0 endTime = -1L appSource = new ApplicationSource(this) nextExecutorId = 0 - removedExecutors = new ArrayBuffer[ExecutorInfo] + removedExecutors = new ArrayBuffer[ExecutorDesc] } private def newExecutorId(useID: Option[Int] = None): Int = { @@ -75,14 +75,14 @@ private[spark] class ApplicationInfo( } } - def addExecutor(worker: WorkerInfo, cores: Int, useID: Option[Int] = None): ExecutorInfo = { - val exec = new ExecutorInfo(newExecutorId(useID), this, worker, cores, desc.memoryPerSlave) + def addExecutor(worker: WorkerInfo, cores: Int, useID: Option[Int] = None): ExecutorDesc = { + val exec = new ExecutorDesc(newExecutorId(useID), this, worker, cores, desc.memoryPerSlave) executors(exec.id) = exec coresGranted += cores exec } - def removeExecutor(exec: ExecutorInfo) { + def removeExecutor(exec: ExecutorDesc) { if (executors.contains(exec.id)) { removedExecutors += executors(exec.id) executors -= exec.id diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ExecutorInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/ExecutorDesc.scala similarity index 95% rename from core/src/main/scala/org/apache/spark/deploy/master/ExecutorInfo.scala rename to core/src/main/scala/org/apache/spark/deploy/master/ExecutorDesc.scala index d417070c5101..5d620dfcabad 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ExecutorInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ExecutorDesc.scala @@ -19,7 +19,7 @@ package org.apache.spark.deploy.master import org.apache.spark.deploy.{ExecutorDescription, ExecutorState} -private[spark] class ExecutorInfo( +private[spark] class ExecutorDesc( val id: Int, val application: ApplicationInfo, val worker: WorkerInfo, @@ -37,7 +37,7 @@ private[spark] class ExecutorInfo( override def equals(other: Any): Boolean = { other match { - case info: ExecutorInfo => + case info: ExecutorDesc => fullId == info.fullId && worker.id == info.worker.id && cores == info.cores && 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 e8a5cfc746fe..d92d99310a58 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 @@ -581,7 +581,7 @@ private[spark] class Master( } } - def launchExecutor(worker: WorkerInfo, exec: ExecutorInfo) { + def launchExecutor(worker: WorkerInfo, exec: ExecutorDesc) { logInfo("Launching executor " + exec.fullId + " on worker " + worker.id) worker.addExecutor(exec) worker.actor ! LaunchExecutor(masterUrl, @@ -720,26 +720,27 @@ private[spark] class Master( def rebuildSparkUI(app: ApplicationInfo): Boolean = { val appName = app.desc.name val notFoundBasePath = HistoryServer.UI_PATH_PREFIX + "/not-found" - val eventLogFile = app.desc.eventLogDir - .map { dir => EventLoggingListener.getLogPath(dir, app.id) } - .getOrElse { - // Event logging is not enabled for this application - app.desc.appUiUrl = notFoundBasePath - return false - } - val fs = Utils.getHadoopFileSystem(eventLogFile, hadoopConf) + try { + val eventLogFile = app.desc.eventLogDir + .map { dir => EventLoggingListener.getLogPath(dir, app.id) } + .getOrElse { + // Event logging is not enabled for this application + app.desc.appUiUrl = notFoundBasePath + return false + } + + val fs = Utils.getHadoopFileSystem(eventLogFile, hadoopConf) - if (fs.exists(new Path(eventLogFile + EventLoggingListener.IN_PROGRESS))) { - // Event logging is enabled for this application, but the application is still in progress - val title = s"Application history not found (${app.id})" - var msg = s"Application $appName is still in progress." - logWarning(msg) - msg = URLEncoder.encode(msg, "UTF-8") - app.desc.appUiUrl = notFoundBasePath + s"?msg=$msg&title=$title" - return false - } + if (fs.exists(new Path(eventLogFile + EventLoggingListener.IN_PROGRESS))) { + // Event logging is enabled for this application, but the application is still in progress + val title = s"Application history not found (${app.id})" + var msg = s"Application $appName is still in progress." + logWarning(msg) + msg = URLEncoder.encode(msg, "UTF-8") + app.desc.appUiUrl = notFoundBasePath + s"?msg=$msg&title=$title" + return false + } - try { val (logInput, sparkVersion) = EventLoggingListener.openEventLog(new Path(eventLogFile), fs) val replayBus = new ReplayListenerBus() val ui = SparkUI.createHistoryUI(new SparkConf, replayBus, new SecurityManager(conf), @@ -758,7 +759,7 @@ private[spark] class Master( case fnf: FileNotFoundException => // 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 $eventLogFile." + var msg = s"No event logs found for application $appName in ${app.desc.eventLogDir}." logWarning(msg) msg += " Did you specify the correct logging directory?" msg = URLEncoder.encode(msg, "UTF-8") @@ -845,7 +846,6 @@ private[spark] class Master( private[spark] object Master extends Logging { val systemName = "sparkMaster" private val actorName = "Master" - val sparkUrlRegex = "spark://([^:]+):([0-9]+)".r def main(argStrings: Array[String]) { SignalLogger.register(log) @@ -855,14 +855,24 @@ private[spark] object Master extends Logging { actorSystem.awaitTermination() } - /** Returns an `akka.tcp://...` URL for the Master actor given a sparkUrl `spark://host:ip`. */ + /** + * Returns an `akka.tcp://...` URL for the Master actor given a sparkUrl `spark://host:port`. + * + * @throws SparkException if the url is invalid + */ def toAkkaUrl(sparkUrl: String): String = { - sparkUrl match { - case sparkUrlRegex(host, port) => - "akka.tcp://%s@%s:%s/user/%s".format(systemName, host, port, actorName) - case _ => - throw new SparkException("Invalid master URL: " + sparkUrl) - } + val (host, port) = Utils.extractHostPortFromSparkUrl(sparkUrl) + "akka.tcp://%s@%s:%s/user/%s".format(systemName, host, port, actorName) + } + + /** + * Returns an akka `Address` for the Master actor given a sparkUrl `spark://host:port`. + * + * @throws SparkException if the url is invalid + */ + def toAkkaAddress(sparkUrl: String): Address = { + val (host, port) = Utils.extractHostPortFromSparkUrl(sparkUrl) + Address("akka.tcp", systemName, host, port) } def startSystemAndActor( diff --git a/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala index 473ddc23ff0f..e94aae93e449 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala @@ -38,7 +38,7 @@ private[spark] class WorkerInfo( Utils.checkHost(host, "Expected hostname") assert (port > 0) - @transient var executors: mutable.HashMap[String, ExecutorInfo] = _ // executorId => info + @transient var executors: mutable.HashMap[String, ExecutorDesc] = _ // executorId => info @transient var drivers: mutable.HashMap[String, DriverInfo] = _ // driverId => info @transient var state: WorkerState.Value = _ @transient var coresUsed: Int = _ @@ -70,13 +70,13 @@ private[spark] class WorkerInfo( host + ":" + port } - def addExecutor(exec: ExecutorInfo) { + def addExecutor(exec: ExecutorDesc) { executors(exec.fullId) = exec coresUsed += exec.cores memoryUsed += exec.memory } - def removeExecutor(exec: ExecutorInfo) { + def removeExecutor(exec: ExecutorDesc) { if (executors.contains(exec.fullId)) { executors -= exec.fullId coresUsed -= exec.cores diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala index 4588c130ef43..3aae2b95d739 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala @@ -27,7 +27,7 @@ import org.json4s.JValue import org.apache.spark.deploy.{ExecutorState, JsonProtocol} import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState} -import org.apache.spark.deploy.master.ExecutorInfo +import org.apache.spark.deploy.master.ExecutorDesc import org.apache.spark.ui.{UIUtils, WebUIPage} import org.apache.spark.util.Utils @@ -109,7 +109,7 @@ private[spark] class ApplicationPage(parent: MasterWebUI) extends WebUIPage("app UIUtils.basicSparkPage(content, "Application: " + app.desc.name) } - private def executorRow(executor: ExecutorInfo): Seq[Node] = { + private def executorRow(executor: ExecutorDesc): Seq[Node] = { {executor.id} 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 f0f3da5eec4d..13599830123d 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 @@ -40,7 +40,7 @@ import org.apache.spark.metrics.MetricsSystem import org.apache.spark.util.{ActorLogReceive, AkkaUtils, SignalLogger, Utils} /** - * @param masterUrls Each url should look like spark://host:port. + * @param masterAkkaUrls Each url should be a valid akka url. */ private[spark] class Worker( host: String, @@ -48,7 +48,7 @@ private[spark] class Worker( webUiPort: Int, cores: Int, memory: Int, - masterUrls: Array[String], + masterAkkaUrls: Array[String], actorSystemName: String, actorName: String, workDirPath: String = null, @@ -171,15 +171,11 @@ private[spark] class Worker( } def changeMaster(url: String, uiUrl: String) { + // activeMasterUrl it's a valid Spark url since we receive it from master. activeMasterUrl = url activeMasterWebUiUrl = uiUrl master = context.actorSelection(Master.toAkkaUrl(activeMasterUrl)) - masterAddress = activeMasterUrl match { - case Master.sparkUrlRegex(_host, _port) => - Address("akka.tcp", Master.systemName, _host, _port.toInt) - case x => - throw new SparkException("Invalid spark URL: " + x) - } + masterAddress = Master.toAkkaAddress(activeMasterUrl) connected = true // Cancel any outstanding re-registration attempts because we found a new master registrationRetryTimer.foreach(_.cancel()) @@ -187,9 +183,9 @@ private[spark] class Worker( } private def tryRegisterAllMasters() { - for (masterUrl <- masterUrls) { - logInfo("Connecting to master " + masterUrl + "...") - val actor = context.actorSelection(Master.toAkkaUrl(masterUrl)) + for (masterAkkaUrl <- masterAkkaUrls) { + logInfo("Connecting to master " + masterAkkaUrl + "...") + val actor = context.actorSelection(masterAkkaUrl) actor ! RegisterWorker(workerId, host, port, cores, memory, webUi.boundPort, publicAddress) } } @@ -527,8 +523,9 @@ private[spark] object Worker extends Logging { val securityMgr = new SecurityManager(conf) val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port, conf = conf, securityManager = securityMgr) + val masterAkkaUrls = masterUrls.map(Master.toAkkaUrl) actorSystem.actorOf(Props(classOf[Worker], host, boundPort, webUiPort, cores, memory, - masterUrls, systemName, actorName, workDir, conf, securityMgr), name = actorName) + masterAkkaUrls, systemName, actorName, workDir, conf, securityMgr), name = actorName) (actorSystem, boundPort) } 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 c794a7bc3599..9a4adfbbb3d7 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -71,7 +71,8 @@ private[spark] class CoarseGrainedExecutorBackend( val ser = env.closureSerializer.newInstance() val taskDesc = ser.deserialize[TaskDescription](data.value) logInfo("Got assigned task " + taskDesc.taskId) - executor.launchTask(this, taskDesc.taskId, taskDesc.name, taskDesc.serializedTask) + executor.launchTask(this, taskId = taskDesc.taskId, attemptNumber = taskDesc.attemptNumber, + taskDesc.name, taskDesc.serializedTask) } case KillTask(taskId, _, interruptThread) => 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 0f99cd9f3b08..42566d1a1409 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -108,8 +108,13 @@ private[spark] class Executor( startDriverHeartbeater() def launchTask( - context: ExecutorBackend, taskId: Long, taskName: String, serializedTask: ByteBuffer) { - val tr = new TaskRunner(context, taskId, taskName, serializedTask) + context: ExecutorBackend, + taskId: Long, + attemptNumber: Int, + taskName: String, + serializedTask: ByteBuffer) { + val tr = new TaskRunner(context, taskId = taskId, attemptNumber = attemptNumber, taskName, + serializedTask) runningTasks.put(taskId, tr) threadPool.execute(tr) } @@ -134,7 +139,11 @@ private[spark] class Executor( private def gcTime = ManagementFactory.getGarbageCollectorMXBeans.map(_.getCollectionTime).sum class TaskRunner( - execBackend: ExecutorBackend, val taskId: Long, taskName: String, serializedTask: ByteBuffer) + execBackend: ExecutorBackend, + val taskId: Long, + val attemptNumber: Int, + taskName: String, + serializedTask: ByteBuffer) extends Runnable { @volatile private var killed = false @@ -180,7 +189,7 @@ private[spark] class Executor( // Run the actual task and measure its runtime. taskStart = System.currentTimeMillis() - val value = task.run(taskId.toInt) + val value = task.run(taskAttemptId = taskId, attemptNumber = attemptNumber) val taskFinish = System.currentTimeMillis() // If the task has been killed, let's fail it. @@ -194,10 +203,10 @@ private[spark] class Executor( val afterSerialization = System.currentTimeMillis() for (m <- task.metrics) { - m.executorDeserializeTime = taskStart - deserializeStartTime - m.executorRunTime = taskFinish - taskStart - m.jvmGCTime = gcTime - startGCTime - m.resultSerializationTime = afterSerialization - beforeSerialization + m.setExecutorDeserializeTime(taskStart - deserializeStartTime) + m.setExecutorRunTime(taskFinish - taskStart) + m.setJvmGCTime(gcTime - startGCTime) + m.setResultSerializationTime(afterSerialization - beforeSerialization) } val accumUpdates = Accumulators.values @@ -248,8 +257,8 @@ private[spark] class Executor( val serviceTime = System.currentTimeMillis() - taskStart val metrics = attemptedTask.flatMap(t => t.metrics) for (m <- metrics) { - m.executorRunTime = serviceTime - m.jvmGCTime = gcTime - startGCTime + m.setExecutorRunTime(serviceTime) + m.setJvmGCTime(gcTime - startGCTime) } val reason = new ExceptionFailure(t, metrics) execBackend.statusUpdate(taskId, TaskState.FAILED, ser.serialize(reason)) @@ -367,10 +376,12 @@ private[spark] class Executor( val curGCTime = gcTime for (taskRunner <- runningTasks.values()) { - if (!taskRunner.attemptedTask.isEmpty) { + if (taskRunner.attemptedTask.nonEmpty) { Option(taskRunner.task).flatMap(_.metrics).foreach { metrics => - metrics.updateShuffleReadMetrics - metrics.jvmGCTime = curGCTime - taskRunner.startGCTime + metrics.updateShuffleReadMetrics() + metrics.updateInputMetrics() + metrics.setJvmGCTime(curGCTime - taskRunner.startGCTime) + if (isLocal) { // JobProgressListener will hold an reference of it during // onExecutorMetricsUpdate(), then JobProgressListener can not see 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 a098d07bd865..cfd672e1d8a9 100644 --- a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala @@ -22,12 +22,13 @@ import java.nio.ByteBuffer import scala.collection.JavaConversions._ import org.apache.mesos.protobuf.ByteString -import org.apache.mesos.{Executor => MesosExecutor, ExecutorDriver, MesosExecutorDriver, MesosNativeLibrary} +import org.apache.mesos.{Executor => MesosExecutor, ExecutorDriver, MesosExecutorDriver} import org.apache.mesos.Protos.{TaskStatus => MesosTaskStatus, _} import org.apache.spark.{Logging, TaskState, SparkConf, SparkEnv} import org.apache.spark.TaskState.TaskState import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.scheduler.cluster.mesos.{MesosTaskLaunchData} import org.apache.spark.util.{SignalLogger, Utils} private[spark] class MesosExecutorBackend @@ -77,10 +78,14 @@ private[spark] class MesosExecutorBackend override def launchTask(d: ExecutorDriver, taskInfo: TaskInfo) { val taskId = taskInfo.getTaskId.getValue.toLong + val taskData = MesosTaskLaunchData.fromByteString(taskInfo.getData) if (executor == null) { logError("Received launchTask but executor was null") } else { - executor.launchTask(this, taskId, taskInfo.getName, taskInfo.getData.asReadOnlyByteBuffer) + SparkHadoopUtil.get.runAsSparkUser { () => + executor.launchTask(this, taskId = taskId, attemptNumber = taskData.attemptNumber, + taskInfo.getName, taskData.serializedTask) + } } } @@ -112,11 +117,8 @@ private[spark] class MesosExecutorBackend private[spark] object MesosExecutorBackend extends Logging { def main(args: Array[String]) { SignalLogger.register(log) - SparkHadoopUtil.get.runAsSparkUser { () => - MesosNativeLibrary.load() - // Create a new Executor and start it running - val runner = new MesosExecutorBackend() - new MesosExecutorDriver(runner).run() - } + // Create a new Executor and start it running + val runner = new MesosExecutorBackend() + new MesosExecutorDriver(runner).run() } } diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 51b5328cb4c8..ddb5903bf687 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -17,6 +17,11 @@ package org.apache.spark.executor +import java.util.concurrent.atomic.AtomicLong + +import org.apache.spark.executor.DataReadMethod +import org.apache.spark.executor.DataReadMethod.DataReadMethod + import scala.collection.mutable.ArrayBuffer import org.apache.spark.annotation.DeveloperApi @@ -39,48 +44,78 @@ class TaskMetrics extends Serializable { /** * Host's name the task runs on */ - var hostname: String = _ - + private var _hostname: String = _ + def hostname = _hostname + private[spark] def setHostname(value: String) = _hostname = value + /** * Time taken on the executor to deserialize this task */ - var executorDeserializeTime: Long = _ - + private var _executorDeserializeTime: Long = _ + def executorDeserializeTime = _executorDeserializeTime + private[spark] def setExecutorDeserializeTime(value: Long) = _executorDeserializeTime = value + + /** * Time the executor spends actually running the task (including fetching shuffle data) */ - var executorRunTime: Long = _ - + private var _executorRunTime: Long = _ + def executorRunTime = _executorRunTime + private[spark] def setExecutorRunTime(value: Long) = _executorRunTime = value + /** * The number of bytes this task transmitted back to the driver as the TaskResult */ - var resultSize: Long = _ + private var _resultSize: Long = _ + def resultSize = _resultSize + private[spark] def setResultSize(value: Long) = _resultSize = value + /** * Amount of time the JVM spent in garbage collection while executing this task */ - var jvmGCTime: Long = _ + private var _jvmGCTime: Long = _ + def jvmGCTime = _jvmGCTime + private[spark] def setJvmGCTime(value: Long) = _jvmGCTime = value /** * Amount of time spent serializing the task result */ - var resultSerializationTime: Long = _ + private var _resultSerializationTime: Long = _ + def resultSerializationTime = _resultSerializationTime + private[spark] def setResultSerializationTime(value: Long) = _resultSerializationTime = value /** * The number of in-memory bytes spilled by this task */ - var memoryBytesSpilled: Long = _ + private var _memoryBytesSpilled: Long = _ + def memoryBytesSpilled = _memoryBytesSpilled + private[spark] def incMemoryBytesSpilled(value: Long) = _memoryBytesSpilled += value + private[spark] def decMemoryBytesSpilled(value: Long) = _memoryBytesSpilled -= value /** * The number of on-disk bytes spilled by this task */ - var diskBytesSpilled: Long = _ + private var _diskBytesSpilled: Long = _ + def diskBytesSpilled = _diskBytesSpilled + def incDiskBytesSpilled(value: Long) = _diskBytesSpilled += value + def decDiskBytesSpilled(value: Long) = _diskBytesSpilled -= value /** * If this task reads from a HadoopRDD or from persisted data, metrics on how much data was read * are stored here. */ - var inputMetrics: Option[InputMetrics] = None + private var _inputMetrics: Option[InputMetrics] = None + + def inputMetrics = _inputMetrics + + /** + * This should only be used when recreating TaskMetrics, not when updating input metrics in + * executors + */ + private[spark] def setInputMetrics(inputMetrics: Option[InputMetrics]) { + _inputMetrics = inputMetrics + } /** * If this task writes data externally (e.g. to a distributed filesystem), metrics on how much @@ -133,19 +168,47 @@ class TaskMetrics extends Serializable { readMetrics } + /** + * Returns the input metrics object that the task should use. Currently, if + * there exists an input metric with the same readMethod, we return that one + * so the caller can accumulate bytes read. If the readMethod is different + * than previously seen by this task, we return a new InputMetric but don't + * record it. + * + * Once https://issues.apache.org/jira/browse/SPARK-5225 is addressed, + * we can store all the different inputMetrics (one per readMethod). + */ + private[spark] def getInputMetricsForReadMethod(readMethod: DataReadMethod): + InputMetrics =synchronized { + _inputMetrics match { + case None => + val metrics = new InputMetrics(readMethod) + _inputMetrics = Some(metrics) + metrics + case Some(metrics @ InputMetrics(method)) if method == readMethod => + metrics + case Some(InputMetrics(method)) => + new InputMetrics(readMethod) + } + } + /** * Aggregates shuffle read metrics for all registered dependencies into shuffleReadMetrics. */ private[spark] def updateShuffleReadMetrics() = synchronized { val merged = new ShuffleReadMetrics() for (depMetrics <- depsShuffleReadMetrics) { - merged.fetchWaitTime += depMetrics.fetchWaitTime - merged.localBlocksFetched += depMetrics.localBlocksFetched - merged.remoteBlocksFetched += depMetrics.remoteBlocksFetched - merged.remoteBytesRead += depMetrics.remoteBytesRead + merged.incFetchWaitTime(depMetrics.fetchWaitTime) + merged.incLocalBlocksFetched(depMetrics.localBlocksFetched) + merged.incRemoteBlocksFetched(depMetrics.remoteBlocksFetched) + merged.incRemoteBytesRead(depMetrics.remoteBytesRead) } _shuffleReadMetrics = Some(merged) } + + private[spark] def updateInputMetrics() = synchronized { + inputMetrics.foreach(_.updateBytesRead()) + } } private[spark] object TaskMetrics { @@ -179,10 +242,38 @@ object DataWriteMethod extends Enumeration with Serializable { */ @DeveloperApi case class InputMetrics(readMethod: DataReadMethod.Value) { + + private val _bytesRead: AtomicLong = new AtomicLong() + /** * Total bytes read. */ - var bytesRead: Long = 0L + def bytesRead: Long = _bytesRead.get() + @volatile @transient var bytesReadCallback: Option[() => Long] = None + + /** + * Adds additional bytes read for this read method. + */ + def addBytesRead(bytes: Long) = { + _bytesRead.addAndGet(bytes) + } + + /** + * Invoke the bytesReadCallback and mutate bytesRead. + */ + def updateBytesRead() { + bytesReadCallback.foreach { c => + _bytesRead.set(c()) + } + } + + /** + * Register a function that can be called to get up-to-date information on how many bytes the task + * has read from an input source. + */ + def setBytesReadCallback(f: Option[() => Long]) { + bytesReadCallback = f + } } /** @@ -194,7 +285,9 @@ case class OutputMetrics(writeMethod: DataWriteMethod.Value) { /** * Total bytes written */ - var bytesWritten: Long = 0L + private var _bytesWritten: Long = _ + def bytesWritten = _bytesWritten + private[spark] def setBytesWritten(value : Long) = _bytesWritten = value } /** @@ -203,32 +296,45 @@ case class OutputMetrics(writeMethod: DataWriteMethod.Value) { */ @DeveloperApi class ShuffleReadMetrics extends Serializable { - /** - * Number of blocks fetched in this shuffle by this task (remote or local) - */ - def totalBlocksFetched: Int = remoteBlocksFetched + localBlocksFetched - /** * Number of remote blocks fetched in this shuffle by this task */ - var remoteBlocksFetched: Int = _ - + private var _remoteBlocksFetched: Int = _ + def remoteBlocksFetched = _remoteBlocksFetched + private[spark] def incRemoteBlocksFetched(value: Int) = _remoteBlocksFetched += value + private[spark] def defRemoteBlocksFetched(value: Int) = _remoteBlocksFetched -= value + /** * Number of local blocks fetched in this shuffle by this task */ - var localBlocksFetched: Int = _ + private var _localBlocksFetched: Int = _ + def localBlocksFetched = _localBlocksFetched + private[spark] def incLocalBlocksFetched(value: Int) = _localBlocksFetched += value + private[spark] def defLocalBlocksFetched(value: Int) = _localBlocksFetched -= value + /** * Time the task spent waiting for remote shuffle blocks. This only includes the time * blocking on shuffle input data. For instance if block B is being fetched while the task is * still not finished processing block A, it is not considered to be blocking on block B. */ - var fetchWaitTime: Long = _ - + private var _fetchWaitTime: Long = _ + def fetchWaitTime = _fetchWaitTime + private[spark] def incFetchWaitTime(value: Long) = _fetchWaitTime += value + private[spark] def decFetchWaitTime(value: Long) = _fetchWaitTime -= value + /** * Total number of remote bytes read from the shuffle by this task */ - var remoteBytesRead: Long = _ + private var _remoteBytesRead: Long = _ + def remoteBytesRead = _remoteBytesRead + private[spark] def incRemoteBytesRead(value: Long) = _remoteBytesRead += value + private[spark] def decRemoteBytesRead(value: Long) = _remoteBytesRead -= value + + /** + * Number of blocks fetched in this shuffle by this task (remote or local) + */ + def totalBlocksFetched = _remoteBlocksFetched + _localBlocksFetched } /** @@ -240,10 +346,18 @@ class ShuffleWriteMetrics extends Serializable { /** * Number of bytes written for the shuffle by this task */ - @volatile var shuffleBytesWritten: Long = _ - + @volatile private var _shuffleBytesWritten: Long = _ + def shuffleBytesWritten = _shuffleBytesWritten + private[spark] def incShuffleBytesWritten(value: Long) = _shuffleBytesWritten += value + private[spark] def decShuffleBytesWritten(value: Long) = _shuffleBytesWritten -= value + /** * Time the task spent blocking on writes to disk or buffer cache, in nanoseconds */ - @volatile var shuffleWriteTime: Long = _ + @volatile private var _shuffleWriteTime: Long = _ + def shuffleWriteTime= _shuffleWriteTime + private[spark] def incShuffleWriteTime(value: Long) = _shuffleWriteTime += value + private[spark] def decShuffleWriteTime(value: Long) = _shuffleWriteTime -= value + + } 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 243b71c98086..03c4137ca0a8 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 @@ -81,7 +81,8 @@ private[nio] class ConnectionManager( private val ackTimeoutMonitor = new HashedWheelTimer(Utils.namedThreadFactory("AckTimeoutMonitor")) - private val ackTimeout = conf.getInt("spark.core.connection.ack.wait.timeout", 60) + private val ackTimeout = + conf.getInt("spark.core.connection.ack.wait.timeout", conf.getInt("spark.network.timeout", 120)) // Get the thread counts from the Spark Configuration. // @@ -173,7 +174,7 @@ private[nio] class ConnectionManager( serverChannel.socket.bind(new InetSocketAddress(port)) (serverChannel, serverChannel.socket.getLocalPort) } - Utils.startServiceOnPort[ServerSocketChannel](port, startService, name) + Utils.startServiceOnPort[ServerSocketChannel](port, startService, conf, name) serverChannel.register(selector, SelectionKey.OP_ACCEPT) val id = new ConnectionManagerId(Utils.localHostName, serverChannel.socket.getLocalPort) diff --git a/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala index 7ba1182f0ed2..1c13e2c37284 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala @@ -95,7 +95,8 @@ private[spark] object CheckpointRDD extends Logging { val finalOutputName = splitIdToFile(ctx.partitionId) val finalOutputPath = new Path(outputDir, finalOutputName) - val tempOutputPath = new Path(outputDir, "." + finalOutputName + "-attempt-" + ctx.attemptId) + val tempOutputPath = + new Path(outputDir, "." + finalOutputName + "-attempt-" + ctx.attemptNumber) if (fs.exists(tempOutputPath)) { throw new IOException("Checkpoint failed: temporary path " + @@ -119,7 +120,7 @@ private[spark] object CheckpointRDD extends Logging { logInfo("Deleting tempOutputPath " + tempOutputPath) fs.delete(tempOutputPath, false) throw new IOException("Checkpoint failed: failed to save output of task: " - + ctx.attemptId + " and final output path does not exist") + + ctx.attemptNumber + " and final output path does not exist") } else { // Some other copy of this task must've finished before us and renamed it logInfo("Final output path " + finalOutputPath + " already exists; not overwriting it") diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala index 70edf191d928..07398a6fa62f 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -159,8 +159,8 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: for ((it, depNum) <- rddIterators) { map.insertAll(it.map(pair => (pair._1, new CoGroupValue(pair._2, depNum)))) } - context.taskMetrics.memoryBytesSpilled += map.memoryBytesSpilled - context.taskMetrics.diskBytesSpilled += map.diskBytesSpilled + context.taskMetrics.incMemoryBytesSpilled(map.memoryBytesSpilled) + context.taskMetrics.incDiskBytesSpilled(map.diskBytesSpilled) new InterruptibleIterator(context, map.iterator.asInstanceOf[Iterator[(K, Array[Iterable[_]])]]) } 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 0001c2329c83..056aef0bc210 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -213,23 +213,24 @@ class HadoopRDD[K, V]( logInfo("Input split: " + split.inputSplit) val jobConf = getJobConf() - val inputMetrics = new InputMetrics(DataReadMethod.Hadoop) + val inputMetrics = context.taskMetrics + .getInputMetricsForReadMethod(DataReadMethod.Hadoop) + // Find a function that will return the FileSystem bytes read by this thread. Do this before // creating RecordReader, because RecordReader's constructor might read some bytes - val bytesReadCallback = if (split.inputSplit.value.isInstanceOf[FileSplit]) { - SparkHadoopUtil.get.getFSBytesReadOnThreadCallback( - split.inputSplit.value.asInstanceOf[FileSplit].getPath, jobConf) - } else { - None - } - if (bytesReadCallback.isDefined) { - context.taskMetrics.inputMetrics = Some(inputMetrics) - } + val bytesReadCallback = inputMetrics.bytesReadCallback.orElse( + split.inputSplit.value match { + case split: FileSplit => + SparkHadoopUtil.get.getFSBytesReadOnThreadCallback(split.getPath, jobConf) + case _ => None + } + ) + inputMetrics.setBytesReadCallback(bytesReadCallback) var reader: RecordReader[K, V] = null val inputFormat = getInputFormat(jobConf) HadoopRDD.addLocalConfiguration(new SimpleDateFormat("yyyyMMddHHmm").format(createTime), - context.stageId, theSplit.index, context.attemptId.toInt, jobConf) + context.stageId, theSplit.index, context.attemptNumber, jobConf) reader = inputFormat.getRecordReader(split.inputSplit.value, jobConf, Reporter.NULL) // Register an on-task-completion callback to close the input stream. @@ -237,8 +238,6 @@ class HadoopRDD[K, V]( val key: K = reader.createKey() val value: V = reader.createValue() - var recordsSinceMetricsUpdate = 0 - override def getNext() = { try { finished = !reader.next(key, value) @@ -247,15 +246,6 @@ class HadoopRDD[K, V]( finished = true } - // Update bytes read metric every few records - if (recordsSinceMetricsUpdate == HadoopRDD.RECORDS_BETWEEN_BYTES_READ_METRIC_UPDATES - && bytesReadCallback.isDefined) { - recordsSinceMetricsUpdate = 0 - val bytesReadFn = bytesReadCallback.get - inputMetrics.bytesRead = bytesReadFn() - } else { - recordsSinceMetricsUpdate += 1 - } (key, value) } @@ -263,14 +253,12 @@ class HadoopRDD[K, V]( try { reader.close() if (bytesReadCallback.isDefined) { - val bytesReadFn = bytesReadCallback.get - inputMetrics.bytesRead = bytesReadFn() + inputMetrics.updateBytesRead() } else if (split.inputSplit.value.isInstanceOf[FileSplit]) { // If we can't get the bytes read from the FS stats, fall back to the split size, // which may be inaccurate. try { - inputMetrics.bytesRead = split.inputSplit.value.getLength - context.taskMetrics.inputMetrics = Some(inputMetrics) + inputMetrics.addBytesRead(split.inputSplit.value.getLength) } catch { case e: java.io.IOException => logWarning("Unable to get input size to set InputMetrics for task", e) 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 e55d03d391e0..7b0e3c87ccff 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -109,18 +109,19 @@ class NewHadoopRDD[K, V]( logInfo("Input split: " + split.serializableHadoopSplit) val conf = confBroadcast.value.value - val inputMetrics = new InputMetrics(DataReadMethod.Hadoop) + val inputMetrics = context.taskMetrics + .getInputMetricsForReadMethod(DataReadMethod.Hadoop) + // Find a function that will return the FileSystem bytes read by this thread. Do this before // creating RecordReader, because RecordReader's constructor might read some bytes - val bytesReadCallback = if (split.serializableHadoopSplit.value.isInstanceOf[FileSplit]) { - SparkHadoopUtil.get.getFSBytesReadOnThreadCallback( - split.serializableHadoopSplit.value.asInstanceOf[FileSplit].getPath, conf) - } else { - None - } - if (bytesReadCallback.isDefined) { - context.taskMetrics.inputMetrics = Some(inputMetrics) - } + val bytesReadCallback = inputMetrics.bytesReadCallback.orElse( + split.serializableHadoopSplit.value match { + case split: FileSplit => + SparkHadoopUtil.get.getFSBytesReadOnThreadCallback(split.getPath, conf) + case _ => None + } + ) + inputMetrics.setBytesReadCallback(bytesReadCallback) val attemptId = newTaskAttemptID(jobTrackerId, id, isMap = true, split.index, 0) val hadoopAttemptContext = newTaskAttemptContext(conf, attemptId) @@ -154,33 +155,19 @@ class NewHadoopRDD[K, V]( } havePair = false - // Update bytes read metric every few records - if (recordsSinceMetricsUpdate == HadoopRDD.RECORDS_BETWEEN_BYTES_READ_METRIC_UPDATES - && bytesReadCallback.isDefined) { - recordsSinceMetricsUpdate = 0 - val bytesReadFn = bytesReadCallback.get - inputMetrics.bytesRead = bytesReadFn() - } else { - recordsSinceMetricsUpdate += 1 - } - (reader.getCurrentKey, reader.getCurrentValue) } private def close() { try { reader.close() - - // Update metrics with final amount if (bytesReadCallback.isDefined) { - val bytesReadFn = bytesReadCallback.get - inputMetrics.bytesRead = bytesReadFn() + inputMetrics.updateBytesRead() } else if (split.serializableHadoopSplit.value.isInstanceOf[FileSplit]) { // If we can't get the bytes read from the FS stats, fall back to the split size, // which may be inaccurate. try { - inputMetrics.bytesRead = split.serializableHadoopSplit.value.getLength - context.taskMetrics.inputMetrics = Some(inputMetrics) + inputMetrics.addBytesRead(split.serializableHadoopSplit.value.getLength) } catch { case e: java.io.IOException => logWarning("Unable to get input size to set InputMetrics for task", e) 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 4469c89e6bb1..0f37d830ef34 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -25,6 +25,7 @@ import scala.collection.{Map, mutable} import scala.collection.JavaConversions._ import scala.collection.mutable.ArrayBuffer import scala.reflect.ClassTag +import scala.util.DynamicVariable import com.clearspring.analytics.stream.cardinality.HyperLogLogPlus import org.apache.hadoop.conf.{Configurable, Configuration} @@ -436,6 +437,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * Note: This operation may be very expensive. If you are grouping in order to perform an * aggregation (such as a sum or average) over each key, using [[PairRDDFunctions.aggregateByKey]] * or [[PairRDDFunctions.reduceByKey]] will provide much better performance. + * + * Note: As currently implemented, groupByKey must be able to hold all the key-value pairs for any + * key in memory. If a key has too many values, it can result in an [[OutOfMemoryError]]. */ def groupByKey(partitioner: Partitioner): RDD[(K, Iterable[V])] = { // groupByKey shouldn't use map side combine because map side combine does not @@ -457,6 +461,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * Note: This operation may be very expensive. If you are grouping in order to perform an * aggregation (such as a sum or average) over each key, using [[PairRDDFunctions.aggregateByKey]] * or [[PairRDDFunctions.reduceByKey]] will provide much better performance. + * + * Note: As currently implemented, groupByKey must be able to hold all the key-value pairs for any + * key in memory. If a key has too many values, it can result in an [[OutOfMemoryError]]. */ def groupByKey(numPartitions: Int): RDD[(K, Iterable[V])] = { groupByKey(new HashPartitioner(numPartitions)) @@ -964,19 +971,16 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) val outfmt = job.getOutputFormatClass val jobFormat = outfmt.newInstance - if (self.conf.getBoolean("spark.hadoop.validateOutputSpecs", true)) { + if (isOutputSpecValidationEnabled) { // FileOutputFormat ignores the filesystem parameter jobFormat.checkOutputSpecs(job) } val writeShard = (context: TaskContext, iter: Iterator[(K,V)]) => { val config = wrappedConf.value - // 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 /* "reduce task" */ val attemptId = newTaskAttemptID(jobtrackerID, stageId, isMap = false, context.partitionId, - attemptNumber) + context.attemptNumber) val hadoopContext = newTaskAttemptContext(config, attemptId) val format = outfmt.newInstance format match { @@ -1003,7 +1007,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) writer.close(hadoopContext) } committer.commitTask(hadoopContext) - bytesWrittenCallback.foreach { fn => outputMetrics.bytesWritten = fn() } + bytesWrittenCallback.foreach { fn => outputMetrics.setBytesWritten(fn()) } 1 } : Int @@ -1042,7 +1046,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) logDebug("Saving as hadoop file of type (" + keyClass.getSimpleName + ", " + valueClass.getSimpleName + ")") - if (self.conf.getBoolean("spark.hadoop.validateOutputSpecs", true)) { + if (isOutputSpecValidationEnabled) { // FileOutputFormat ignores the filesystem parameter val ignoredFs = FileSystem.get(hadoopConf) hadoopConf.getOutputFormat.checkOutputSpecs(ignoredFs, hadoopConf) @@ -1055,11 +1059,11 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) val config = wrappedConf.value // 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 taskAttemptId = (context.taskAttemptId % Int.MaxValue).toInt val (outputMetrics, bytesWrittenCallback) = initHadoopOutputMetrics(context, config) - writer.setup(context.stageId, context.partitionId, attemptNumber) + writer.setup(context.stageId, context.partitionId, taskAttemptId) writer.open() try { var recordsWritten = 0L @@ -1075,7 +1079,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) writer.close() } writer.commit() - bytesWrittenCallback.foreach { fn => outputMetrics.bytesWritten = fn() } + bytesWrittenCallback.foreach { fn => outputMetrics.setBytesWritten(fn()) } } self.context.runJob(self, writeToFile) @@ -1098,7 +1102,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) outputMetrics: OutputMetrics, recordsWritten: Long): Unit = { if (recordsWritten % PairRDDFunctions.RECORDS_BETWEEN_BYTES_WRITTEN_METRIC_UPDATES == 0 && bytesWrittenCallback.isDefined) { - bytesWrittenCallback.foreach { fn => outputMetrics.bytesWritten = fn() } + bytesWrittenCallback.foreach { fn => outputMetrics.setBytesWritten(fn()) } } } @@ -1117,8 +1121,22 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) private[spark] def valueClass: Class[_] = vt.runtimeClass private[spark] def keyOrdering: Option[Ordering[K]] = Option(ord) + + // Note: this needs to be a function instead of a 'val' so that the disableOutputSpecValidation + // setting can take effect: + private def isOutputSpecValidationEnabled: Boolean = { + val validationDisabled = PairRDDFunctions.disableOutputSpecValidation.value + val enabledInConf = self.conf.getBoolean("spark.hadoop.validateOutputSpecs", true) + enabledInConf && !validationDisabled + } } private[spark] object PairRDDFunctions { val RECORDS_BETWEEN_BYTES_WRITTEN_METRIC_UPDATES = 256 + + /** + * Allows for the `spark.hadoop.validateOutputSpecs` checks to be disabled on a case-by-case + * basis; see SPARK-4835 for more details. + */ + val disableOutputSpecValidation: DynamicVariable[Boolean] = new DynamicVariable[Boolean](false) } diff --git a/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala index 87b22de6ae69..f12d0cffaba3 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala @@ -111,7 +111,8 @@ private object ParallelCollectionRDD { /** * Slice a collection into numSlices sub-collections. One extra thing we do here is to treat Range * collections specially, encoding the slices as other Ranges to minimize memory cost. This makes - * it efficient to run Spark over RDDs representing large sets of numbers. + * it efficient to run Spark over RDDs representing large sets of numbers. And if the collection + * is an inclusive Range, we use inclusive range for the last slice. */ def slice[T: ClassTag](seq: Seq[T], numSlices: Int): Seq[Seq[T]] = { if (numSlices < 1) { @@ -127,19 +128,15 @@ private object ParallelCollectionRDD { }) } seq match { - case r: Range.Inclusive => { - val sign = if (r.step < 0) { - -1 - } else { - 1 - } - slice(new Range( - r.start, r.end + sign, r.step).asInstanceOf[Seq[T]], numSlices) - } case r: Range => { - positions(r.length, numSlices).map({ - case (start, end) => + positions(r.length, numSlices).zipWithIndex.map({ case ((start, end), index) => + // If the range is inclusive, use inclusive range for the last slice + if (r.isInclusive && index == numSlices - 1) { + new Range.Inclusive(r.start + start * r.step, r.end, r.step) + } + else { new Range(r.start + start * r.step, r.start + end * r.step, r.step) + } }).toSeq.asInstanceOf[Seq[Seq[T]]] } case nr: NumericRange[_] => { 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 f47c2d1fcdcc..97012c7033f9 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -1146,15 +1146,20 @@ abstract class RDD[T: ClassTag]( if (num == 0) { Array.empty } else { - mapPartitions { items => + val mapRDDs = mapPartitions { items => // Priority keeps the largest elements, so let's reverse the ordering. val queue = new BoundedPriorityQueue[T](num)(ord.reverse) queue ++= util.collection.Utils.takeOrdered(items, num)(ord) Iterator.single(queue) - }.reduce { (queue1, queue2) => - queue1 ++= queue2 - queue1 - }.toArray.sorted(ord) + } + if (mapRDDs.partitions.size == 0) { + Array.empty + } else { + mapRDDs.reduce { (queue1, queue2) => + queue1 ++= queue2 + queue1 + }.toArray.sorted(ord) + } } } @@ -1170,6 +1175,12 @@ abstract class RDD[T: ClassTag]( * */ def min()(implicit ord: Ordering[T]): T = this.reduce(ord.min) + /** + * @return true if and only if the RDD contains no elements at all. Note that an RDD + * may be empty even when it has at least 1 partition. + */ + def isEmpty(): Boolean = partitions.length == 0 || take(1).length == 0 + /** * Save this RDD as a text file, using string representations of elements. */ 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 cb8ccfbdbdcb..1cfe98673773 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -19,6 +19,7 @@ package org.apache.spark.scheduler import java.io.NotSerializableException import java.util.Properties +import java.util.concurrent.{TimeUnit, Executors} import java.util.concurrent.atomic.AtomicInteger import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map, Stack} @@ -28,8 +29,6 @@ import scala.language.postfixOps import scala.reflect.ClassTag import scala.util.control.NonFatal -import akka.actor._ -import akka.actor.SupervisorStrategy.Stop import akka.pattern.ask import akka.util.Timeout @@ -39,7 +38,7 @@ import org.apache.spark.executor.TaskMetrics import org.apache.spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialResult} import org.apache.spark.rdd.RDD import org.apache.spark.storage._ -import org.apache.spark.util.{CallSite, SystemClock, Clock, Utils} +import org.apache.spark.util.{CallSite, EventLoop, SystemClock, Clock, Utils} import org.apache.spark.storage.BlockManagerMessages.BlockManagerHeartbeat /** @@ -67,8 +66,6 @@ class DAGScheduler( clock: Clock = SystemClock) extends Logging { - import DAGScheduler._ - def this(sc: SparkContext, taskScheduler: TaskScheduler) = { this( sc, @@ -112,14 +109,10 @@ class DAGScheduler( // stray messages to detect. private val failedEpoch = new HashMap[String, Long] - private val dagSchedulerActorSupervisor = - env.actorSystem.actorOf(Props(new DAGSchedulerActorSupervisor(this))) - // A closure serializer that we reuse. // This is only safe because DAGScheduler runs in a single thread. private val closureSerializer = SparkEnv.get.closureSerializer.newInstance() - private[scheduler] var eventProcessActor: ActorRef = _ /** If enabled, we may run certain actions like take() and first() locally. */ private val localExecutionEnabled = sc.getConf.getBoolean("spark.localExecution.enabled", false) @@ -127,26 +120,20 @@ class DAGScheduler( /** If enabled, FetchFailed will not cause stage retry, in order to surface the problem. */ private val disallowStageRetryForTest = sc.getConf.getBoolean("spark.test.noStageRetry", false) - private def initializeEventProcessActor() { - // blocking the thread until supervisor is started, which ensures eventProcessActor is - // not null before any job is submitted - implicit val timeout = Timeout(30 seconds) - val initEventActorReply = - dagSchedulerActorSupervisor ? Props(new DAGSchedulerEventProcessActor(this)) - eventProcessActor = Await.result(initEventActorReply, timeout.duration). - asInstanceOf[ActorRef] - } + private val messageScheduler = + Executors.newScheduledThreadPool(1, Utils.namedThreadFactory("dag-scheduler-message")) - initializeEventProcessActor() + private[scheduler] val eventProcessLoop = new DAGSchedulerEventProcessLoop(this) + taskScheduler.setDAGScheduler(this) // Called by TaskScheduler to report task's starting. def taskStarted(task: Task[_], taskInfo: TaskInfo) { - eventProcessActor ! BeginEvent(task, taskInfo) + eventProcessLoop.post(BeginEvent(task, taskInfo)) } // Called to report that a task has completed and results are being fetched remotely. def taskGettingResult(taskInfo: TaskInfo) { - eventProcessActor ! GettingResultEvent(taskInfo) + eventProcessLoop.post(GettingResultEvent(taskInfo)) } // Called by TaskScheduler to report task completions or failures. @@ -157,7 +144,8 @@ class DAGScheduler( accumUpdates: Map[Long, Any], taskInfo: TaskInfo, taskMetrics: TaskMetrics) { - eventProcessActor ! CompletionEvent(task, reason, result, accumUpdates, taskInfo, taskMetrics) + eventProcessLoop.post( + CompletionEvent(task, reason, result, accumUpdates, taskInfo, taskMetrics)) } /** @@ -179,18 +167,18 @@ class DAGScheduler( // Called by TaskScheduler when an executor fails. def executorLost(execId: String) { - eventProcessActor ! ExecutorLost(execId) + eventProcessLoop.post(ExecutorLost(execId)) } // Called by TaskScheduler when a host is added def executorAdded(execId: String, host: String) { - eventProcessActor ! ExecutorAdded(execId, host) + eventProcessLoop.post(ExecutorAdded(execId, host)) } // Called by TaskScheduler to cancel an entire TaskSet due to either repeated failures or // cancellation of the job itself. def taskSetFailed(taskSet: TaskSet, reason: String) { - eventProcessActor ! TaskSetFailed(taskSet, reason) + eventProcessLoop.post(TaskSetFailed(taskSet, reason)) } private def getCacheLocs(rdd: RDD[_]): Array[Seq[TaskLocation]] = { @@ -495,8 +483,8 @@ class DAGScheduler( assert(partitions.size > 0) val func2 = func.asInstanceOf[(TaskContext, Iterator[_]) => _] val waiter = new JobWaiter(this, jobId, partitions.size, resultHandler) - eventProcessActor ! JobSubmitted( - jobId, rdd, func2, partitions.toArray, allowLocal, callSite, waiter, properties) + eventProcessLoop.post(JobSubmitted( + jobId, rdd, func2, partitions.toArray, allowLocal, callSite, waiter, properties)) waiter } @@ -536,8 +524,8 @@ class DAGScheduler( val func2 = func.asInstanceOf[(TaskContext, Iterator[_]) => _] val partitions = (0 until rdd.partitions.size).toArray val jobId = nextJobId.getAndIncrement() - eventProcessActor ! JobSubmitted( - jobId, rdd, func2, partitions, allowLocal = false, callSite, listener, properties) + eventProcessLoop.post(JobSubmitted( + jobId, rdd, func2, partitions, allowLocal = false, callSite, listener, properties)) listener.awaitResult() // Will throw an exception if the job fails } @@ -546,19 +534,19 @@ class DAGScheduler( */ def cancelJob(jobId: Int) { logInfo("Asked to cancel job " + jobId) - eventProcessActor ! JobCancelled(jobId) + eventProcessLoop.post(JobCancelled(jobId)) } def cancelJobGroup(groupId: String) { logInfo("Asked to cancel job group " + groupId) - eventProcessActor ! JobGroupCancelled(groupId) + eventProcessLoop.post(JobGroupCancelled(groupId)) } /** * Cancel all jobs that are running or waiting in the queue. */ def cancelAllJobs() { - eventProcessActor ! AllJobsCancelled + eventProcessLoop.post(AllJobsCancelled) } private[scheduler] def doCancelAllJobs() { @@ -574,7 +562,7 @@ class DAGScheduler( * Cancel all jobs associated with a running or scheduled stage. */ def cancelStage(stageId: Int) { - eventProcessActor ! StageCancelled(stageId) + eventProcessLoop.post(StageCancelled(stageId)) } /** @@ -634,8 +622,8 @@ class DAGScheduler( try { val rdd = job.finalStage.rdd val split = rdd.partitions(job.partitions(0)) - val taskContext = - new TaskContextImpl(job.finalStage.id, job.partitions(0), 0, true) + val taskContext = new TaskContextImpl(job.finalStage.id, job.partitions(0), taskAttemptId = 0, + attemptNumber = 0, runningLocally = true) TaskContextHelper.setTaskContext(taskContext) try { val result = job.func(taskContext, rdd.iterator(split, taskContext)) @@ -660,7 +648,7 @@ class DAGScheduler( // completion events or stage abort stageIdToStage -= s.id jobIdToStageIds -= job.jobId - listenerBus.post(SparkListenerJobEnd(job.jobId, jobResult)) + listenerBus.post(SparkListenerJobEnd(job.jobId, clock.getTime(), jobResult)) } } @@ -709,7 +697,7 @@ class DAGScheduler( stage.latestInfo.stageFailed(stageFailedMessage) listenerBus.post(SparkListenerStageCompleted(stage.latestInfo)) } - listenerBus.post(SparkListenerJobEnd(job.jobId, JobFailed(error))) + listenerBus.post(SparkListenerJobEnd(job.jobId, clock.getTime(), JobFailed(error))) } } @@ -748,9 +736,11 @@ class DAGScheduler( logInfo("Missing parents: " + getMissingParentStages(finalStage)) val shouldRunLocally = localExecutionEnabled && allowLocal && finalStage.parents.isEmpty && partitions.length == 1 + val jobSubmissionTime = clock.getTime() if (shouldRunLocally) { // Compute very short actions like first() or take() with no parent stages locally. - listenerBus.post(SparkListenerJobStart(job.jobId, Seq.empty, properties)) + listenerBus.post( + SparkListenerJobStart(job.jobId, jobSubmissionTime, Seq.empty, properties)) runLocally(job) } else { jobIdToActiveJob(jobId) = job @@ -758,7 +748,8 @@ class DAGScheduler( finalStage.resultOfJob = Some(job) val stageIds = jobIdToStageIds(jobId).toArray val stageInfos = stageIds.flatMap(id => stageIdToStage.get(id).map(_.latestInfo)) - listenerBus.post(SparkListenerJobStart(job.jobId, stageInfos, properties)) + listenerBus.post( + SparkListenerJobStart(job.jobId, jobSubmissionTime, stageInfos, properties)) submitStage(finalStage) } } @@ -865,26 +856,6 @@ class DAGScheduler( } if (tasks.size > 0) { - // Preemptively serialize a task to make sure it can be serialized. We are catching this - // exception here because it would be fairly hard to catch the non-serializable exception - // down the road, where we have several different implementations for local scheduler and - // cluster schedulers. - // - // We've already serialized RDDs and closures in taskBinary, but here we check for all other - // objects such as Partition. - try { - closureSerializer.serialize(tasks.head) - } catch { - case e: NotSerializableException => - abortStage(stage, "Task not serializable: " + e.toString) - runningStages -= stage - return - case NonFatal(e) => // Other exceptions, such as IllegalArgumentException from Kryo. - abortStage(stage, s"Task serialization failed: $e\n${e.getStackTraceString}") - runningStages -= stage - return - } - logInfo("Submitting " + tasks.size + " missing tasks from " + stage + " (" + stage.rdd + ")") stage.pendingTasks ++= tasks logDebug("New pending tasks: " + stage.pendingTasks) @@ -984,7 +955,8 @@ class DAGScheduler( if (job.numFinished == job.numPartitions) { markStageAsFinished(stage) cleanupStateForJobAndIndependentStages(job) - listenerBus.post(SparkListenerJobEnd(job.jobId, JobSucceeded)) + listenerBus.post( + SparkListenerJobEnd(job.jobId, clock.getTime(), JobSucceeded)) } // taskSucceeded runs some user code that might throw an exception. Make sure @@ -1078,16 +1050,15 @@ class DAGScheduler( if (disallowStageRetryForTest) { abortStage(failedStage, "Fetch failure will not retry stage due to testing config") - } else if (failedStages.isEmpty && eventProcessActor != null) { + } else if (failedStages.isEmpty) { // Don't schedule an event to resubmit failed stages if failed isn't empty, because - // in that case the event will already have been scheduled. eventProcessActor may be - // null during unit tests. + // in that case the event will already have been scheduled. // TODO: Cancel running tasks in the stage - import env.actorSystem.dispatcher logInfo(s"Resubmitting $mapStage (${mapStage.name}) and " + s"$failedStage (${failedStage.name}) due to fetch failure") - env.actorSystem.scheduler.scheduleOnce( - RESUBMIT_TIMEOUT, eventProcessActor, ResubmitFailedStages) + messageScheduler.schedule(new Runnable { + override def run(): Unit = eventProcessLoop.post(ResubmitFailedStages) + }, DAGScheduler.RESUBMIT_TIMEOUT, TimeUnit.MILLISECONDS) } failedStages += failedStage failedStages += mapStage @@ -1253,7 +1224,7 @@ class DAGScheduler( if (ableToCancelStages) { job.listener.jobFailed(error) cleanupStateForJobAndIndependentStages(job) - listenerBus.post(SparkListenerJobEnd(job.jobId, JobFailed(error))) + listenerBus.post(SparkListenerJobEnd(job.jobId, clock.getTime(), JobFailed(error))) } } @@ -1345,46 +1316,21 @@ class DAGScheduler( def stop() { logInfo("Stopping DAGScheduler") - dagSchedulerActorSupervisor ! PoisonPill + eventProcessLoop.stop() taskScheduler.stop() } -} - -private[scheduler] class DAGSchedulerActorSupervisor(dagScheduler: DAGScheduler) - extends Actor with Logging { - - override val supervisorStrategy = - OneForOneStrategy() { - case x: Exception => - logError("eventProcesserActor failed; shutting down SparkContext", x) - try { - dagScheduler.doCancelAllJobs() - } catch { - case t: Throwable => logError("DAGScheduler failed to cancel all jobs.", t) - } - dagScheduler.sc.stop() - Stop - } - def receive = { - case p: Props => sender ! context.actorOf(p) - case _ => logWarning("received unknown message in DAGSchedulerActorSupervisor") - } + // Start the event thread at the end of the constructor + eventProcessLoop.start() } -private[scheduler] class DAGSchedulerEventProcessActor(dagScheduler: DAGScheduler) - extends Actor with Logging { - - override def preStart() { - // set DAGScheduler for taskScheduler to ensure eventProcessActor is always - // valid when the messages arrive - dagScheduler.taskScheduler.setDAGScheduler(dagScheduler) - } +private[scheduler] class DAGSchedulerEventProcessLoop(dagScheduler: DAGScheduler) + extends EventLoop[DAGSchedulerEvent]("dag-scheduler-event-loop") with Logging { /** * The main event loop of the DAG scheduler. */ - def receive = { + override def onReceive(event: DAGSchedulerEvent): Unit = event match { case JobSubmitted(jobId, rdd, func, partitions, allowLocal, callSite, listener, properties) => dagScheduler.handleJobSubmitted(jobId, rdd, func, partitions, allowLocal, callSite, listener, properties) @@ -1423,7 +1369,17 @@ private[scheduler] class DAGSchedulerEventProcessActor(dagScheduler: DAGSchedule dagScheduler.resubmitFailedStages() } - override def postStop() { + override def onError(e: Throwable): Unit = { + logError("DAGSchedulerEventProcessLoop failed; shutting down SparkContext", e) + try { + dagScheduler.doCancelAllJobs() + } catch { + case t: Throwable => logError("DAGScheduler failed to cancel all jobs.", t) + } + dagScheduler.sc.stop() + } + + override def onStop() { // Cancel any active jobs in postStop hook dagScheduler.cleanUpAfterSchedulerStop() } @@ -1433,9 +1389,5 @@ private[spark] object DAGScheduler { // The time, in millis, to wait for fetch failure events to stop coming in after one is detected; // this is a simplistic way to avoid resubmitting tasks in the non-fetchable map stage one by one // as more failure events come in - val RESUBMIT_TIMEOUT = 200.milliseconds - - // The time, in millis, to wake up between polls of the completion queue in order to potentially - // resubmit failed stages - val POLL_TIMEOUT = 10L + val RESUBMIT_TIMEOUT = 200 } 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 27bf4f159907..30075c172bdb 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -168,6 +168,10 @@ private[spark] class EventLoggingListener( logEvent(event, flushLogger = true) override def onApplicationEnd(event: SparkListenerApplicationEnd) = logEvent(event, flushLogger = true) + override def onExecutorAdded(event: SparkListenerExecutorAdded) = + logEvent(event, flushLogger = true) + override def onExecutorRemoved(event: SparkListenerExecutorRemoved) = + logEvent(event, flushLogger = true) // No-op because logging every update would be overkill override def onExecutorMetricsUpdate(event: SparkListenerExecutorMetricsUpdate) { } diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index b62b0c131269..e5d1eb767e10 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -25,6 +25,7 @@ import scala.collection.mutable import org.apache.spark.{Logging, TaskEndReason} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.TaskMetrics +import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.{Distribution, Utils} @@ -58,6 +59,7 @@ case class SparkListenerTaskEnd( @DeveloperApi case class SparkListenerJobStart( jobId: Int, + time: Long, stageInfos: Seq[StageInfo], properties: Properties = null) extends SparkListenerEvent { @@ -67,7 +69,11 @@ case class SparkListenerJobStart( } @DeveloperApi -case class SparkListenerJobEnd(jobId: Int, jobResult: JobResult) extends SparkListenerEvent +case class SparkListenerJobEnd( + jobId: Int, + time: Long, + jobResult: JobResult) + extends SparkListenerEvent @DeveloperApi case class SparkListenerEnvironmentUpdate(environmentDetails: Map[String, Seq[(String, String)]]) @@ -84,6 +90,14 @@ case class SparkListenerBlockManagerRemoved(time: Long, blockManagerId: BlockMan @DeveloperApi case class SparkListenerUnpersistRDD(rddId: Int) extends SparkListenerEvent +@DeveloperApi +case class SparkListenerExecutorAdded(executorId: String, executorInfo: ExecutorInfo) + extends SparkListenerEvent + +@DeveloperApi +case class SparkListenerExecutorRemoved(executorId: String) + extends SparkListenerEvent + /** * Periodic updates from executors. * @param execId executor id @@ -109,7 +123,8 @@ private[spark] case object SparkListenerShutdown extends SparkListenerEvent /** * :: DeveloperApi :: * Interface for listening to events from the Spark scheduler. Note that this is an internal - * interface which might change in different Spark releases. + * interface which might change in different Spark releases. Java clients should extend + * {@link JavaSparkListener} */ @DeveloperApi trait SparkListener { @@ -183,6 +198,16 @@ trait SparkListener { * Called when the driver receives task metrics from an executor in a heartbeat. */ def onExecutorMetricsUpdate(executorMetricsUpdate: SparkListenerExecutorMetricsUpdate) { } + + /** + * Called when the driver registers a new executor. + */ + def onExecutorAdded(executorAdded: SparkListenerExecutorAdded) { } + + /** + * Called when the driver removes an executor. + */ + def onExecutorRemoved(executorRemoved: SparkListenerExecutorRemoved) { } } /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala index e79ffd7a3587..e700c6af542f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala @@ -70,6 +70,10 @@ private[spark] trait SparkListenerBus extends Logging { foreachListener(_.onApplicationEnd(applicationEnd)) case metricsUpdate: SparkListenerExecutorMetricsUpdate => foreachListener(_.onExecutorMetricsUpdate(metricsUpdate)) + case executorAdded: SparkListenerExecutorAdded => + foreachListener(_.onExecutorAdded(executorAdded)) + case executorRemoved: SparkListenerExecutorRemoved => + foreachListener(_.onExecutorRemoved(executorRemoved)) case SparkListenerShutdown => } } 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 d7dde4fe3843..847a4912eec1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -44,10 +44,18 @@ 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 TaskContextImpl(stageId, partitionId, attemptId, runningLocally = false) + /** + * Called by Executor to run this task. + * + * @param taskAttemptId an identifier for this task attempt that is unique within a SparkContext. + * @param attemptNumber how many times this task has been attempted (0 for the first attempt) + * @return the result of the task + */ + final def run(taskAttemptId: Long, attemptNumber: Int): T = { + context = new TaskContextImpl(stageId = stageId, partitionId = partitionId, + taskAttemptId = taskAttemptId, attemptNumber = attemptNumber, runningLocally = false) TaskContextHelper.setTaskContext(context) - context.taskMetrics.hostname = Utils.localHostName() + context.taskMetrics.setHostname(Utils.localHostName()) taskThread = Thread.currentThread() if (_killed) { kill(interruptThread = false) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala index 4c96b9e5fef6..1c7c81c488c3 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala @@ -27,6 +27,7 @@ import org.apache.spark.util.SerializableBuffer */ private[spark] class TaskDescription( val taskId: Long, + val attemptNumber: Int, val executorId: String, val name: String, val index: Int, // Index within this task's TaskSet 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 4896ec845bbc..774f3d8cdb27 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala @@ -77,7 +77,7 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul (deserializedResult, size) } - result.metrics.resultSize = size + result.metrics.setResultSize(size) scheduler.handleSuccessfulTask(taskSetManager, tid, result) } catch { case cnf: ClassNotFoundException => 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 cd3c015321e8..33a7aae5d3fc 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -31,6 +31,7 @@ import scala.util.Random import org.apache.spark._ import org.apache.spark.TaskState.TaskState import org.apache.spark.scheduler.SchedulingMode.SchedulingMode +import org.apache.spark.scheduler.TaskLocality.TaskLocality import org.apache.spark.util.Utils import org.apache.spark.executor.TaskMetrics import org.apache.spark.storage.BlockManagerId @@ -167,7 +168,7 @@ private[spark] class TaskSchedulerImpl( if (!hasLaunchedTask) { logWarning("Initial job has not accepted any resources; " + "check your cluster UI to ensure that workers are registered " + - "and have sufficient memory") + "and have sufficient resources") } else { this.cancel() } @@ -209,6 +210,40 @@ private[spark] class TaskSchedulerImpl( .format(manager.taskSet.id, manager.parent.name)) } + private def resourceOfferSingleTaskSet( + taskSet: TaskSetManager, + maxLocality: TaskLocality, + shuffledOffers: Seq[WorkerOffer], + availableCpus: Array[Int], + tasks: Seq[ArrayBuffer[TaskDescription]]) : Boolean = { + var launchedTask = false + for (i <- 0 until shuffledOffers.size) { + val execId = shuffledOffers(i).executorId + val host = shuffledOffers(i).host + if (availableCpus(i) >= CPUS_PER_TASK) { + try { + for (task <- taskSet.resourceOffer(execId, host, maxLocality)) { + tasks(i) += task + val tid = task.taskId + taskIdToTaskSetId(tid) = taskSet.taskSet.id + taskIdToExecutorId(tid) = execId + executorsByHost(host) += execId + availableCpus(i) -= CPUS_PER_TASK + assert(availableCpus(i) >= 0) + launchedTask = true + } + } catch { + case e: TaskNotSerializableException => + logError(s"Resource offer failed, task set ${taskSet.name} was not serializable") + // Do not offer resources for this task, but don't throw an error to allow other + // task sets to be submitted. + return launchedTask + } + } + } + return launchedTask + } + /** * Called by cluster manager to offer resources on slaves. We respond by asking our active task * sets for tasks in order of priority. We fill each node with tasks in a round-robin manner so @@ -251,23 +286,8 @@ private[spark] class TaskSchedulerImpl( var launchedTask = false for (taskSet <- sortedTaskSets; maxLocality <- taskSet.myLocalityLevels) { do { - launchedTask = false - for (i <- 0 until shuffledOffers.size) { - val execId = shuffledOffers(i).executorId - val host = shuffledOffers(i).host - if (availableCpus(i) >= CPUS_PER_TASK) { - for (task <- taskSet.resourceOffer(execId, host, maxLocality)) { - tasks(i) += task - val tid = task.taskId - taskIdToTaskSetId(tid) = taskSet.taskSet.id - taskIdToExecutorId(tid) = execId - executorsByHost(host) += execId - availableCpus(i) -= CPUS_PER_TASK - assert(availableCpus(i) >= 0) - launchedTask = true - } - } - } + launchedTask = resourceOfferSingleTaskSet( + taskSet, maxLocality, shuffledOffers, availableCpus, tasks) } while (launchedTask) } @@ -394,9 +414,6 @@ private[spark] class TaskSchedulerImpl( taskResultGetter.stop() } starvationTimer.cancel() - - // sleeping for an arbitrary 1 seconds to ensure that messages are sent out. - Thread.sleep(1000L) } override def defaultParallelism() = backend.defaultParallelism() 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 28e6147509f7..5c94c6bbcb37 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -18,12 +18,14 @@ package org.apache.spark.scheduler import java.io.NotSerializableException +import java.nio.ByteBuffer import java.util.Arrays import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap import scala.collection.mutable.HashSet import scala.math.{min, max} +import scala.util.control.NonFatal import org.apache.spark._ import org.apache.spark.executor.TaskMetrics @@ -417,6 +419,7 @@ private[spark] class TaskSetManager( * @param host the host Id of the offered resource * @param maxLocality the maximum locality we want to schedule the tasks at */ + @throws[TaskNotSerializableException] def resourceOffer( execId: String, host: String, @@ -456,10 +459,17 @@ private[spark] class TaskSetManager( } // Serialize and return the task val startTime = clock.getTime() - // We rely on the DAGScheduler to catch non-serializable closures and RDDs, so in here - // we assume the task can be serialized without exceptions. - val serializedTask = Task.serializeWithDependencies( - task, sched.sc.addedFiles, sched.sc.addedJars, ser) + val serializedTask: ByteBuffer = try { + Task.serializeWithDependencies(task, sched.sc.addedFiles, sched.sc.addedJars, ser) + } catch { + // If the task cannot be serialized, then there's no point to re-attempt the task, + // as it will always fail. So just abort the whole task-set. + case NonFatal(e) => + val msg = s"Failed to serialize task $taskId, not attempting to retry it." + logError(msg, e) + abort(s"$msg Exception during serialization: $e") + throw new TaskNotSerializableException(e) + } if (serializedTask.limit > TaskSetManager.TASK_SIZE_TO_WARN_KB * 1024 && !emittedTaskSizeWarning) { emittedTaskSizeWarning = true @@ -477,7 +487,8 @@ private[spark] class TaskSetManager( taskName, taskId, host, taskLocality, serializedTask.limit)) sched.dagScheduler.taskStarted(task, info) - return Some(new TaskDescription(taskId, execId, taskName, index, serializedTask)) + return Some(new TaskDescription(taskId = taskId, attemptNumber = attemptNum, execId, + taskName, index, serializedTask)) } case _ => } 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 fe9914b50bc5..5786d367464f 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 @@ -28,7 +28,7 @@ import akka.pattern.ask import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} import org.apache.spark.{ExecutorAllocationClient, Logging, SparkEnv, SparkException, TaskState} -import org.apache.spark.scheduler.{SchedulerBackend, SlaveLost, TaskDescription, TaskSchedulerImpl, WorkerOffer} +import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ import org.apache.spark.util.{ActorLogReceive, SerializableBuffer, AkkaUtils, Utils} @@ -66,6 +66,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste // Number of executors requested from the cluster manager that have not registered yet private var numPendingExecutors = 0 + private val listenerBus = scheduler.sc.listenerBus + // Executors we have requested the cluster manager to kill that have not died yet private val executorsPendingToRemove = new HashSet[String] @@ -106,6 +108,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste logDebug(s"Decremented number of pending executors ($numPendingExecutors left)") } } + listenerBus.post(SparkListenerExecutorAdded(executorId, data)) makeOffers() } @@ -213,6 +216,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste totalCoreCount.addAndGet(-executorInfo.totalCores) totalRegisteredExecutors.addAndGet(-1) scheduler.executorLost(executorId, SlaveLost(reason)) + listenerBus.post(SparkListenerExecutorRemoved(executorId)) case None => logError(s"Asked to remove non-existent executor $executorId") } } 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 index b71bd5783d6d..eb52ddfb1eab 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala @@ -31,7 +31,7 @@ import akka.actor.{Address, ActorRef} private[cluster] class ExecutorData( val executorActor: ActorRef, val executorAddress: Address, - val executorHost: String , + override val executorHost: String, var freeCores: Int, - val totalCores: Int -) + override val totalCores: Int +) extends ExecutorInfo(executorHost, totalCores) diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/BooleanType.java b/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorInfo.scala similarity index 52% rename from sql/core/src/main/java/org/apache/spark/sql/api/java/BooleanType.java rename to core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorInfo.scala index 5a1f52725631..b4738e64c939 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/BooleanType.java +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorInfo.scala @@ -14,14 +14,32 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +package org.apache.spark.scheduler.cluster -package org.apache.spark.sql.api.java; +import org.apache.spark.annotation.DeveloperApi /** - * The data type representing boolean and Boolean values. - * - * {@code BooleanType} is represented by the singleton object {@link DataType#BooleanType}. + * :: DeveloperApi :: + * Stores information about an executor to pass from the scheduler to SparkListeners. */ -public class BooleanType extends DataType { - protected BooleanType() {} +@DeveloperApi +class ExecutorInfo( + val executorHost: String, + val totalCores: Int +) { + + def canEqual(other: Any): Boolean = other.isInstanceOf[ExecutorInfo] + + override def equals(other: Any): Boolean = other match { + case that: ExecutorInfo => + (that canEqual this) && + executorHost == that.executorHost && + totalCores == that.totalCores + case _ => false + } + + override def hashCode(): Int = { + val state = Seq(executorHost, totalCores) + state.map(_.hashCode()).foldLeft(0)((a, b) => 31 * a + b) + } } 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 8c7de75600b5..7eb87a564d6f 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 @@ -55,19 +55,26 @@ private[spark] class SparkDeploySchedulerBackend( "{{WORKER_URL}}") val extraJavaOpts = sc.conf.getOption("spark.executor.extraJavaOptions") .map(Utils.splitCommandString).getOrElse(Seq.empty) - val classPathEntries = sc.conf.getOption("spark.executor.extraClassPath").toSeq.flatMap { cp => - cp.split(java.io.File.pathSeparator) - } - val libraryPathEntries = - sc.conf.getOption("spark.executor.extraLibraryPath").toSeq.flatMap { cp => - cp.split(java.io.File.pathSeparator) + val classPathEntries = sc.conf.getOption("spark.executor.extraClassPath") + .map(_.split(java.io.File.pathSeparator).toSeq).getOrElse(Nil) + val libraryPathEntries = sc.conf.getOption("spark.executor.extraLibraryPath") + .map(_.split(java.io.File.pathSeparator).toSeq).getOrElse(Nil) + + // When testing, expose the parent class path to the child. This is processed by + // compute-classpath.{cmd,sh} and makes all needed jars available to child processes + // when the assembly is built with the "*-provided" profiles enabled. + val testingClassPath = + if (sys.props.contains("spark.testing")) { + sys.props("java.class.path").split(java.io.File.pathSeparator).toSeq + } else { + Nil } // Start executors with a few necessary configs for registering with the scheduler val sparkJavaOpts = Utils.sparkJavaOpts(conf, SparkConf.isExecutorStartupConf) val javaOpts = sparkJavaOpts ++ extraJavaOpts val command = Command("org.apache.spark.executor.CoarseGrainedExecutorBackend", - args, sc.executorEnvs, classPathEntries, libraryPathEntries, javaOpts) + args, sc.executorEnvs, classPathEntries ++ testingClassPath, libraryPathEntries, javaOpts) val appUIAddress = sc.ui.map(_.appUIAddress).getOrElse("") val appDesc = new ApplicationDescription(sc.appName, maxCores, sc.executorMemory, command, appUIAddress, sc.eventLogDir) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala index 50721b9d6cd6..f14aaeea0a25 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala @@ -17,6 +17,8 @@ package org.apache.spark.scheduler.cluster +import scala.concurrent.{Future, ExecutionContext} + import akka.actor.{Actor, ActorRef, Props} import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} @@ -24,7 +26,9 @@ import org.apache.spark.SparkContext import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ import org.apache.spark.scheduler.TaskSchedulerImpl import org.apache.spark.ui.JettyUtils -import org.apache.spark.util.AkkaUtils +import org.apache.spark.util.{AkkaUtils, Utils} + +import scala.util.control.NonFatal /** * Abstract Yarn scheduler backend that contains common logic @@ -97,6 +101,9 @@ private[spark] abstract class YarnSchedulerBackend( private class YarnSchedulerActor extends Actor { private var amActor: Option[ActorRef] = None + implicit val askAmActorExecutor = ExecutionContext.fromExecutor( + Utils.newDaemonCachedThreadPool("yarn-scheduler-ask-am-executor")) + override def preStart(): Unit = { // Listen for disassociation events context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) @@ -110,7 +117,12 @@ private[spark] abstract class YarnSchedulerBackend( case r: RequestExecutors => amActor match { case Some(actor) => - sender ! AkkaUtils.askWithReply[Boolean](r, actor, askTimeout) + val driverActor = sender + Future { + driverActor ! AkkaUtils.askWithReply[Boolean](r, actor, askTimeout) + } onFailure { + case NonFatal(e) => logError(s"Sending $r to AM was unsuccessful", e) + } case None => logWarning("Attempted to request executors before the AM has registered!") sender ! false @@ -119,7 +131,12 @@ private[spark] abstract class YarnSchedulerBackend( case k: KillExecutors => amActor match { case Some(actor) => - sender ! AkkaUtils.askWithReply[Boolean](k, actor, askTimeout) + val driverActor = sender + Future { + driverActor ! AkkaUtils.askWithReply[Boolean](k, actor, askTimeout) + } onFailure { + case NonFatal(e) => logError(s"Sending $k to AM was unsuccessful", e) + } case None => logWarning("Attempted to kill executors before the AM has registered!") sender ! false 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 10e6886c16a4..79c9051e8869 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 @@ -22,14 +22,17 @@ import java.util.{ArrayList => JArrayList, List => JList} import java.util.Collections import scala.collection.JavaConversions._ -import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} +import scala.collection.mutable.{HashMap, HashSet} import org.apache.mesos.protobuf.ByteString import org.apache.mesos.{Scheduler => MScheduler} import org.apache.mesos._ -import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, TaskState => MesosTaskState, _} +import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, TaskState => MesosTaskState, + ExecutorInfo => MesosExecutorInfo, _} +import org.apache.spark.executor.MesosExecutorBackend import org.apache.spark.{Logging, SparkContext, SparkException, TaskState} +import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.scheduler._ import org.apache.spark.util.Utils @@ -62,6 +65,9 @@ private[spark] class MesosSchedulerBackend( var classLoader: ClassLoader = null + // The listener bus to publish executor added/removed events. + val listenerBus = sc.listenerBus + @volatile var appId: String = _ override def start() { @@ -87,7 +93,7 @@ private[spark] class MesosSchedulerBackend( } } - def createExecutorInfo(execId: String): ExecutorInfo = { + def createExecutorInfo(execId: String): MesosExecutorInfo = { val executorSparkHome = sc.conf.getOption("spark.mesos.executor.home") .orElse(sc.getSparkHome()) // Fall back to driver Spark home for backward compatibility .getOrElse { @@ -118,14 +124,15 @@ private[spark] class MesosSchedulerBackend( val command = CommandInfo.newBuilder() .setEnvironment(environment) val uri = sc.conf.get("spark.executor.uri", null) + val executorBackendName = classOf[MesosExecutorBackend].getName if (uri == null) { - val executorPath = new File(executorSparkHome, "/sbin/spark-executor").getCanonicalPath - command.setValue("%s %s".format(prefixEnv, executorPath)) + val executorPath = new File(executorSparkHome, "/bin/spark-class").getCanonicalPath + command.setValue(s"$prefixEnv $executorPath $executorBackendName") } else { // Grab everything to the first '.'. We'll use that and '*' to // glob the directory "correctly". val basename = uri.split('/').last.split('.').head - command.setValue("cd %s*; %s ./sbin/spark-executor".format(basename, prefixEnv)) + command.setValue(s"cd ${basename}*; $prefixEnv ./bin/spark-class $executorBackendName") command.addUris(CommandInfo.URI.newBuilder().setValue(uri)) } val cpus = Resource.newBuilder() @@ -141,7 +148,7 @@ private[spark] class MesosSchedulerBackend( Value.Scalar.newBuilder() .setValue(MemoryUtils.calculateTotalMemory(sc)).build()) .build() - ExecutorInfo.newBuilder() + MesosExecutorInfo.newBuilder() .setExecutorId(ExecutorID.newBuilder().setValue(execId).build()) .setCommand(command) .setData(ByteString.copyFrom(createExecArg())) @@ -237,6 +244,7 @@ private[spark] class MesosSchedulerBackend( } val slaveIdToOffer = usableOffers.map(o => o.getSlaveId.getValue -> o).toMap + val slaveIdToWorkerOffer = workerOffers.map(o => o.executorId -> o).toMap val mesosTasks = new HashMap[String, JArrayList[MesosTaskInfo]] @@ -260,6 +268,10 @@ private[spark] class MesosSchedulerBackend( val filters = Filters.newBuilder().setRefuseSeconds(1).build() // TODO: lower timeout? mesosTasks.foreach { case (slaveId, tasks) => + slaveIdToWorkerOffer.get(slaveId).foreach(o => + listenerBus.post(SparkListenerExecutorAdded(slaveId, + new ExecutorInfo(o.host, o.cores))) + ) d.launchTasks(Collections.singleton(slaveIdToOffer(slaveId).getId), tasks, filters) } @@ -296,7 +308,7 @@ private[spark] class MesosSchedulerBackend( .setExecutor(createExecutorInfo(slaveId)) .setName(task.name) .addResources(cpuResource) - .setData(ByteString.copyFrom(task.serializedTask)) + .setData(MesosTaskLaunchData(task.serializedTask, task.attemptNumber).toByteString) .build() } @@ -315,7 +327,7 @@ private[spark] class MesosSchedulerBackend( synchronized { if (status.getState == MesosTaskState.TASK_LOST && taskIdToSlaveId.contains(tid)) { // We lost the executor on this slave, so remember that it's gone - slaveIdsWithExecutors -= taskIdToSlaveId(tid) + removeExecutor(taskIdToSlaveId(tid)) } if (isFinished(status.getState)) { taskIdToSlaveId.remove(tid) @@ -344,12 +356,20 @@ private[spark] class MesosSchedulerBackend( override def frameworkMessage(d: SchedulerDriver, e: ExecutorID, s: SlaveID, b: Array[Byte]) {} + /** + * Remove executor associated with slaveId in a thread safe manner. + */ + private def removeExecutor(slaveId: String) = { + synchronized { + listenerBus.post(SparkListenerExecutorRemoved(slaveId)) + slaveIdsWithExecutors -= slaveId + } + } + private def recordSlaveLost(d: SchedulerDriver, slaveId: SlaveID, reason: ExecutorLossReason) { inClassLoader() { logInfo("Mesos slave lost: " + slaveId.getValue) - synchronized { - slaveIdsWithExecutors -= slaveId.getValue - } + removeExecutor(slaveId.getValue) scheduler.executorLost(slaveId.getValue, reason) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosTaskLaunchData.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosTaskLaunchData.scala new file mode 100644 index 000000000000..5e7e6567a3e0 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosTaskLaunchData.scala @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.scheduler.cluster.mesos + +import java.nio.ByteBuffer + +import org.apache.mesos.protobuf.ByteString + +import org.apache.spark.Logging + +/** + * Wrapper for serializing the data sent when launching Mesos tasks. + */ +private[spark] case class MesosTaskLaunchData( + serializedTask: ByteBuffer, + attemptNumber: Int) extends Logging { + + def toByteString: ByteString = { + val dataBuffer = ByteBuffer.allocate(4 + serializedTask.limit) + dataBuffer.putInt(attemptNumber) + dataBuffer.put(serializedTask) + dataBuffer.rewind + logDebug(s"ByteBuffer size: [${dataBuffer.remaining}]") + ByteString.copyFrom(dataBuffer) + } +} + +private[spark] object MesosTaskLaunchData extends Logging { + def fromByteString(byteString: ByteString): MesosTaskLaunchData = { + val byteBuffer = byteString.asReadOnlyByteBuffer() + logDebug(s"ByteBuffer size: [${byteBuffer.remaining}]") + val attemptNumber = byteBuffer.getInt // updates the position by 4 bytes + val serializedTask = byteBuffer.slice() // subsequence starting at the current position + MesosTaskLaunchData(serializedTask, attemptNumber) + } +} 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 b3bd3110ac80..05b6fa54564b 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 @@ -76,7 +76,8 @@ private[spark] class LocalActor( val offers = Seq(new WorkerOffer(localExecutorId, localExecutorHostname, freeCores)) for (task <- scheduler.resourceOffers(offers).flatten) { freeCores -= scheduler.CPUS_PER_TASK - executor.launchTask(executorBackend, task.taskId, task.name, task.serializedTask) + executor.launchTask(executorBackend, taskId = task.taskId, attemptNumber = task.attemptNumber, + task.name, task.serializedTask) } } } diff --git a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala index 662a7b91248a..fa8a337ad63a 100644 --- a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala @@ -92,7 +92,7 @@ private[spark] class JavaSerializerInstance(counterReset: Int, defaultClassLoade } override def deserializeStream(s: InputStream): DeserializationStream = { - new JavaDeserializationStream(s, Utils.getContextOrSparkClassLoader) + new JavaDeserializationStream(s, defaultClassLoader) } def deserializeStream(s: InputStream, loader: ClassLoader): DeserializationStream = { diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index 621a951c27d0..d56e23ce4478 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -26,9 +26,10 @@ import com.esotericsoftware.kryo.serializers.{JavaSerializer => KryoJavaSerializ import com.twitter.chill.{AllScalaRegistrar, EmptyScalaKryoInstantiator} import org.apache.spark._ +import org.apache.spark.api.python.PythonBroadcast import org.apache.spark.broadcast.HttpBroadcast import org.apache.spark.network.nio.{PutBlock, GotBlock, GetBlock} -import org.apache.spark.scheduler.MapStatus +import org.apache.spark.scheduler.{CompressedMapStatus, HighlyCompressedMapStatus} import org.apache.spark.storage._ import org.apache.spark.util.BoundedPriorityQueue import org.apache.spark.util.collection.CompactBuffer @@ -90,6 +91,7 @@ class KryoSerializer(conf: SparkConf) // Allow sending SerializableWritable kryo.register(classOf[SerializableWritable[_]], new KryoJavaSerializer()) kryo.register(classOf[HttpBroadcast[_]], new KryoJavaSerializer()) + kryo.register(classOf[PythonBroadcast], new KryoJavaSerializer()) try { // Use the default classloader when calling the user registrator. @@ -205,7 +207,8 @@ private[serializer] object KryoSerializer { classOf[PutBlock], classOf[GotBlock], classOf[GetBlock], - classOf[MapStatus], + classOf[CompressedMapStatus], + classOf[HighlyCompressedMapStatus], classOf[CompactBuffer[_]], classOf[BlockManagerId], classOf[Array[Byte]], diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala index de72148ccc7a..41bafabde05b 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala @@ -59,8 +59,8 @@ private[spark] class HashShuffleReader[K, C]( // the ExternalSorter won't spill to disk. val sorter = new ExternalSorter[K, C, C](ordering = Some(keyOrd), serializer = Some(ser)) sorter.insertAll(aggregatedIter) - context.taskMetrics.memoryBytesSpilled += sorter.memoryBytesSpilled - context.taskMetrics.diskBytesSpilled += sorter.diskBytesSpilled + context.taskMetrics.incMemoryBytesSpilled(sorter.memoryBytesSpilled) + context.taskMetrics.incDiskBytesSpilled(sorter.diskBytesSpilled) sorter.iterator case None => aggregatedIter 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 d7b184f8a10e..8bc5a1cd18b6 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -34,10 +34,9 @@ import org.apache.spark.executor._ import org.apache.spark.io.CompressionCodec import org.apache.spark.network._ import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer} -import org.apache.spark.network.netty.{SparkTransportConf, NettyBlockTransferService} +import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.network.shuffle.ExternalShuffleClient import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo -import org.apache.spark.network.util.{ConfigProvider, TransportConf} import org.apache.spark.serializer.Serializer import org.apache.spark.shuffle.ShuffleManager import org.apache.spark.shuffle.hash.HashShuffleManager @@ -54,7 +53,7 @@ private[spark] class BlockResult( readMethod: DataReadMethod.Value, bytes: Long) { val inputMetrics = new InputMetrics(readMethod) - inputMetrics.bytesRead = bytes + inputMetrics.addBytesRead(bytes) } /** @@ -120,7 +119,7 @@ private[spark] class BlockManager( private[spark] var shuffleServerId: BlockManagerId = _ // Client to read other executors' shuffle files. This is either an external service, or just the - // standard BlockTranserService to directly connect to other Executors. + // standard BlockTransferService to directly connect to other Executors. private[spark] val shuffleClient = if (externalShuffleServiceEnabled) { val transConf = SparkTransportConf.fromSparkConf(conf, numUsableCores) new ExternalShuffleClient(transConf, securityManager, securityManager.isAuthenticationEnabled()) 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 9cbda41223a8..64133464d8da 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -52,8 +52,7 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus private val akkaTimeout = AkkaUtils.askTimeout(conf) - val slaveTimeout = conf.getLong("spark.storage.blockManagerSlaveTimeoutMs", - math.max(conf.getInt("spark.executor.heartbeatInterval", 10000) * 3, 45000)) + val slaveTimeout = conf.getLong("spark.storage.blockManagerSlaveTimeoutMs", 120 * 1000) val checkTimeoutInterval = conf.getLong("spark.storage.blockManagerTimeoutIntervalMs", 60000) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala index 9c469370ffe1..3198d766fca3 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala @@ -160,14 +160,14 @@ private[spark] class DiskBlockObjectWriter( } finalPosition = file.length() // In certain compression codecs, more bytes are written after close() is called - writeMetrics.shuffleBytesWritten += (finalPosition - reportedPosition) + writeMetrics.incShuffleBytesWritten(finalPosition - reportedPosition) } // Discard current writes. We do this by flushing the outstanding writes and then // truncating the file to its initial position. override def revertPartialWritesAndClose() { try { - writeMetrics.shuffleBytesWritten -= (reportedPosition - initialPosition) + writeMetrics.decShuffleBytesWritten(reportedPosition - initialPosition) if (initialized) { objOut.flush() @@ -212,14 +212,14 @@ private[spark] class DiskBlockObjectWriter( */ private def updateBytesWritten() { val pos = channel.position() - writeMetrics.shuffleBytesWritten += (pos - reportedPosition) + writeMetrics.incShuffleBytesWritten(pos - reportedPosition) reportedPosition = pos } private def callWithTiming(f: => Unit) = { val start = System.nanoTime() f - writeMetrics.shuffleWriteTime += (System.nanoTime() - start) + writeMetrics.incShuffleWriteTime(System.nanoTime() - start) } // For testing diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala index bb2ae9f3f458..af05eb3ca69c 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -166,7 +166,7 @@ private[spark] class DiskBlockManager(blockManager: BlockManager, conf: SparkCon /** Cleanup local dirs and stop shuffle sender. */ private[spark] def stop() { // Only perform cleanup if an external service is not serving our shuffle files. - if (!blockManager.externalShuffleServiceEnabled) { + if (!blockManager.externalShuffleServiceEnabled || blockManager.blockManagerId.isDriver) { localDirs.foreach { localDir => if (localDir.isDirectory() && localDir.exists()) { try { 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 8dadf6794039..61ef5ff16879 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala @@ -31,7 +31,8 @@ import org.apache.spark.util.Utils private[spark] class DiskStore(blockManager: BlockManager, diskManager: DiskBlockManager) extends BlockStore(blockManager) with Logging { - val minMemoryMapBytes = blockManager.conf.getLong("spark.storage.memoryMapThreshold", 2 * 4096L) + val minMemoryMapBytes = blockManager.conf.getLong( + "spark.storage.memoryMapThreshold", 2 * 1024L * 1024L) override def getSize(blockId: BlockId): Long = { diskManager.getFile(blockId.name).length 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 2499c11a65b0..ab9ee4f0096b 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala @@ -156,8 +156,8 @@ final class ShuffleBlockFetcherIterator( // This needs to be released after use. buf.retain() results.put(new SuccessFetchResult(BlockId(blockId), sizeMap(blockId), buf)) - shuffleMetrics.remoteBytesRead += buf.size - shuffleMetrics.remoteBlocksFetched += 1 + shuffleMetrics.incRemoteBytesRead(buf.size) + shuffleMetrics.incRemoteBlocksFetched(1) } logTrace("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime)) } @@ -233,7 +233,7 @@ final class ShuffleBlockFetcherIterator( val blockId = iter.next() try { val buf = blockManager.getBlockData(blockId) - shuffleMetrics.localBlocksFetched += 1 + shuffleMetrics.incLocalBlocksFetched(1) buf.retain() results.put(new SuccessFetchResult(blockId, 0, buf)) } catch { @@ -277,7 +277,7 @@ final class ShuffleBlockFetcherIterator( currentResult = results.take() val result = currentResult val stopFetchWait = System.currentTimeMillis() - shuffleMetrics.fetchWaitTime += (stopFetchWait - startFetchWait) + shuffleMetrics.incFetchWaitTime(stopFetchWait - startFetchWait) result match { case SuccessFetchResult(_, size, _) => bytesInFlight -= size 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 2a27d49d2de0..88fed833f922 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -201,7 +201,7 @@ private[spark] object JettyUtils extends Logging { } } - val (server, boundPort) = Utils.startServiceOnPort[Server](port, connect, serverName) + val (server, boundPort) = Utils.startServiceOnPort[Server](port, connect, conf, serverName) ServerInfo(server, boundPort, collection) } diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index 7486cb6b1bbc..b5022fe853c4 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -234,8 +234,9 @@ private[spark] object UIUtils extends Logging {

- + + {org.apache.spark.SPARK_VERSION} {title}

diff --git a/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala index b4677447c887..fc1844600f1c 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala @@ -22,20 +22,23 @@ import scala.util.Random import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.scheduler.SchedulingMode +// scalastyle:off /** * Continuously generates jobs that expose various features of the WebUI (internal testing tool). * - * Usage: ./bin/spark-class org.apache.spark.ui.UIWorkloadGenerator [master] [FIFO|FAIR] + * Usage: ./bin/spark-class org.apache.spark.ui.UIWorkloadGenerator [master] [FIFO|FAIR] [#job set (4 jobs per set)] */ +// scalastyle:on private[spark] object UIWorkloadGenerator { val NUM_PARTITIONS = 100 val INTER_JOB_WAIT_MS = 5000 def main(args: Array[String]) { - if (args.length < 2) { + if (args.length < 3) { println( - "usage: ./bin/spark-class org.apache.spark.ui.UIWorkloadGenerator [master] [FIFO|FAIR]") + "usage: ./bin/spark-class org.apache.spark.ui.UIWorkloadGenerator " + + "[master] [FIFO|FAIR] [#job set (4 jobs per set)]") System.exit(1) } @@ -45,6 +48,7 @@ private[spark] object UIWorkloadGenerator { if (schedulingMode == SchedulingMode.FAIR) { conf.set("spark.scheduler.mode", "FAIR") } + val nJobSet = args(2).toInt val sc = new SparkContext(conf) def setProperties(s: String) = { @@ -84,7 +88,7 @@ private[spark] object UIWorkloadGenerator { ("Job with delays", baseData.map(x => Thread.sleep(100)).count) ) - while (true) { + (1 to nJobSet).foreach { _ => for ((desc, job) <- jobs) { new Thread { override def run() { @@ -101,5 +105,6 @@ private[spark] object UIWorkloadGenerator { Thread.sleep(INTER_JOB_WAIT_MS) } } + sc.stop() } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala index ea2d187a0e8e..045c69da06fe 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala @@ -21,7 +21,6 @@ import scala.xml.{Node, NodeSeq} import javax.servlet.http.HttpServletRequest -import org.apache.spark.JobExecutionStatus import org.apache.spark.ui.{WebUIPage, UIUtils} import org.apache.spark.ui.jobs.UIData.JobUIData @@ -47,17 +46,17 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { val lastStageData = lastStageInfo.flatMap { s => listener.stageIdToData.get((s.stageId, s.attemptId)) } - val isComplete = job.status == JobExecutionStatus.SUCCEEDED + val lastStageName = lastStageInfo.map(_.name).getOrElse("(Unknown Stage Name)") val lastStageDescription = lastStageData.flatMap(_.description).getOrElse("") val duration: Option[Long] = { - job.startTime.map { start => - val end = job.endTime.getOrElse(System.currentTimeMillis()) + job.submissionTime.map { start => + val end = job.completionTime.getOrElse(System.currentTimeMillis()) end - start } } val formattedDuration = duration.map(d => UIUtils.formatDuration(d)).getOrElse("Unknown") - val formattedSubmissionTime = job.startTime.map(UIUtils.formatDate).getOrElse("Unknown") + val formattedSubmissionTime = job.submissionTime.map(UIUtils.formatDate).getOrElse("Unknown") val detailUrl = "%s/jobs/job?id=%s".format(UIUtils.prependBaseUri(parent.basePath), job.jobId) @@ -65,10 +64,10 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { {job.jobId} {job.jobGroup.map(id => s"($id)").getOrElse("")} -
{lastStageDescription}
+ {lastStageDescription} {lastStageName} - + {formattedSubmissionTime} {formattedDuration} @@ -101,11 +100,15 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { val now = System.currentTimeMillis val activeJobsTable = - jobsTable(activeJobs.sortBy(_.startTime.getOrElse(-1L)).reverse) + jobsTable(activeJobs.sortBy(_.submissionTime.getOrElse(-1L)).reverse) val completedJobsTable = - jobsTable(completedJobs.sortBy(_.endTime.getOrElse(-1L)).reverse) + jobsTable(completedJobs.sortBy(_.completionTime.getOrElse(-1L)).reverse) val failedJobsTable = - jobsTable(failedJobs.sortBy(_.endTime.getOrElse(-1L)).reverse) + jobsTable(failedJobs.sortBy(_.completionTime.getOrElse(-1L)).reverse) + + val shouldShowActiveJobs = activeJobs.nonEmpty + val shouldShowCompletedJobs = completedJobs.nonEmpty + val shouldShowFailedJobs = failedJobs.nonEmpty val summary: NodeSeq =
@@ -121,27 +124,47 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { Scheduling Mode: {listener.schedulingMode.map(_.toString).getOrElse("Unknown")} -
  • - Active Jobs: - {activeJobs.size} -
  • -
  • - Completed Jobs: - {completedJobs.size} -
  • -
  • - Failed Jobs: - {failedJobs.size} -
  • + { + if (shouldShowActiveJobs) { +
  • + Active Jobs: + {activeJobs.size} +
  • + } + } + { + if (shouldShowCompletedJobs) { +
  • + Completed Jobs: + {completedJobs.size} +
  • + } + } + { + if (shouldShowFailedJobs) { +
  • + Failed Jobs: + {failedJobs.size} +
  • + } + }
    - val content = summary ++ -

    Active Jobs ({activeJobs.size})

    ++ activeJobsTable ++ -

    Completed Jobs ({completedJobs.size})

    ++ completedJobsTable ++ -

    Failed Jobs ({failedJobs.size})

    ++ failedJobsTable - - val helpText = """A job is triggered by a action, like "count()" or "saveAsTextFile()".""" + + var content = summary + if (shouldShowActiveJobs) { + content ++=

    Active Jobs ({activeJobs.size})

    ++ + activeJobsTable + } + if (shouldShowCompletedJobs) { + content ++=

    Completed Jobs ({completedJobs.size})

    ++ + completedJobsTable + } + if (shouldShowFailedJobs) { + content ++=

    Failed Jobs ({failedJobs.size})

    ++ + failedJobsTable + } + val helpText = """A job is triggered by an action, like "count()" or "saveAsTextFile()".""" + " Click on a job's title to see information about the stages of tasks associated with" + " the job." diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala index b0f8ca2ab0d3..479f967fb154 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala @@ -33,6 +33,7 @@ private[ui] class AllStagesPage(parent: StagesTab) extends WebUIPage("") { def render(request: HttpServletRequest): Seq[Node] = { listener.synchronized { val activeStages = listener.activeStages.values.toSeq + val pendingStages = listener.pendingStages.values.toSeq val completedStages = listener.completedStages.reverse.toSeq val numCompletedStages = listener.numCompletedStages val failedStages = listener.failedStages.reverse.toSeq @@ -43,6 +44,10 @@ private[ui] class AllStagesPage(parent: StagesTab) extends WebUIPage("") { new StageTableBase(activeStages.sortBy(_.submissionTime).reverse, parent.basePath, parent.listener, isFairScheduler = parent.isFairScheduler, killEnabled = parent.killEnabled) + val pendingStagesTable = + new StageTableBase(pendingStages.sortBy(_.submissionTime).reverse, + parent.basePath, parent.listener, isFairScheduler = parent.isFairScheduler, + killEnabled = false) val completedStagesTable = new StageTableBase(completedStages.sortBy(_.submissionTime).reverse, parent.basePath, parent.listener, isFairScheduler = parent.isFairScheduler, killEnabled = false) @@ -54,48 +59,86 @@ private[ui] class AllStagesPage(parent: StagesTab) extends WebUIPage("") { val pools = sc.map(_.getAllPools).getOrElse(Seq.empty[Schedulable]) val poolTable = new PoolTable(pools, parent) + val shouldShowActiveStages = activeStages.nonEmpty + val shouldShowPendingStages = pendingStages.nonEmpty + val shouldShowCompletedStages = completedStages.nonEmpty + val shouldShowFailedStages = failedStages.nonEmpty + val summary: NodeSeq =
      - {if (sc.isDefined) { - // Total duration is not meaningful unless the UI is live -
    • - Total Duration: - {UIUtils.formatDuration(now - sc.get.startTime)} -
    • - }} + { + if (sc.isDefined) { + // Total duration is not meaningful unless the UI is live +
    • + Total Duration: + {UIUtils.formatDuration(now - sc.get.startTime)} +
    • + } + }
    • Scheduling Mode: {listener.schedulingMode.map(_.toString).getOrElse("Unknown")}
    • -
    • - Active Stages: - {activeStages.size} -
    • -
    • - Completed Stages: - {numCompletedStages} -
    • -
    • - Failed Stages: - {numFailedStages} -
    • + { + if (shouldShowActiveStages) { +
    • + Active Stages: + {activeStages.size} +
    • + } + } + { + if (shouldShowPendingStages) { +
    • + Pending Stages: + {pendingStages.size} +
    • + } + } + { + if (shouldShowCompletedStages) { +
    • + Completed Stages: + {numCompletedStages} +
    • + } + } + { + if (shouldShowFailedStages) { +
    • + Failed Stages: + {numFailedStages} +
    • + } + }
    - val content = summary ++ - {if (sc.isDefined && isFairScheduler) { -

    {pools.size} Fair Scheduler Pools

    ++ poolTable.toNodeSeq - } else { - Seq[Node]() - }} ++ -

    Active Stages ({activeStages.size})

    ++ - activeStagesTable.toNodeSeq ++ -

    Completed Stages ({numCompletedStages})

    ++ - completedStagesTable.toNodeSeq ++ -

    Failed Stages ({numFailedStages})

    ++ + var content = summary ++ + { + if (sc.isDefined && isFairScheduler) { +

    {pools.size} Fair Scheduler Pools

    ++ poolTable.toNodeSeq + } else { + Seq[Node]() + } + } + if (shouldShowActiveStages) { + content ++=

    Active Stages ({activeStages.size})

    ++ + activeStagesTable.toNodeSeq + } + if (shouldShowPendingStages) { + content ++=

    Pending Stages ({pendingStages.size}

    ++ + pendingStagesTable.toNodeSeq + } + if (shouldShowCompletedStages) { + content ++=

    Completed Stages ({numCompletedStages})

    ++ + completedStagesTable.toNodeSeq + } + if (shouldShowFailedStages) { + content ++=

    Failed Stages ({numFailedStages})

    ++ failedStagesTable.toNodeSeq - + } UIUtils.headerSparkPage("Spark Stages (for all jobs)", content, parent) } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index 72935beb3a34..4d200eeda86b 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -56,6 +56,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { val jobIdToData = new HashMap[JobId, JobUIData] // Stages: + val pendingStages = new HashMap[StageId, StageInfo] val activeStages = new HashMap[StageId, StageInfo] val completedStages = ListBuffer[StageInfo]() val skippedStages = ListBuffer[StageInfo]() @@ -153,14 +154,14 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { val jobData: JobUIData = new JobUIData( jobId = jobStart.jobId, - startTime = Some(System.currentTimeMillis), - endTime = None, + submissionTime = Option(jobStart.time).filter(_ >= 0), stageIds = jobStart.stageIds, jobGroup = jobGroup, status = JobExecutionStatus.RUNNING) + jobStart.stageInfos.foreach(x => pendingStages(x.stageId) = x) // Compute (a potential underestimate of) the number of tasks that will be run by this job. // This may be an underestimate because the job start event references all of the result - // stages's transitive stage dependencies, but some of these stages might be skipped if their + // stages' transitive stage dependencies, but some of these stages might be skipped if their // output is available from earlier runs. // See https://github.com/apache/spark/pull/3009 for a more extensive discussion. jobData.numTasks = { @@ -186,7 +187,9 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { logWarning(s"Job completed for unknown job ${jobEnd.jobId}") new JobUIData(jobId = jobEnd.jobId) } - jobData.endTime = Some(System.currentTimeMillis()) + jobData.completionTime = Option(jobEnd.time).filter(_ >= 0) + + jobData.stageIds.foreach(pendingStages.remove) jobEnd.jobResult match { case JobSucceeded => completedJobs += jobData @@ -257,7 +260,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = synchronized { val stage = stageSubmitted.stageInfo activeStages(stage.stageId) = stage - + pendingStages.remove(stage.stageId) val poolName = Option(stageSubmitted.properties).map { p => p.getProperty("spark.scheduler.pool", DEFAULT_POOL_NAME) }.getOrElse(DEFAULT_POOL_NAME) @@ -309,7 +312,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = synchronized { val info = taskEnd.taskInfo // If stage attempt id is -1, it means the DAGScheduler had no idea which attempt this task - // compeletion event is for. Let's just drop it here. This means we might have some speculation + // completion event is for. Let's just drop it here. This means we might have some speculation // tasks on the web ui that's never marked as complete. if (info != null && taskEnd.stageAttemptId != -1) { val stageData = stageIdToData.getOrElseUpdate((taskEnd.stageId, taskEnd.stageAttemptId), { diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index e7d6244dcd67..703d43f9c640 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -112,9 +112,8 @@ private[ui] class StageTableBase( stageData <- listener.stageIdToData.get((s.stageId, s.attemptId)) desc <- stageData.description } yield { -
    {desc}
    + {desc} } -
    {stageDesc.getOrElse("")} {killLink} {nameLink} {details}
    } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala index 48fd7caa1a1e..01f7e23212c3 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala @@ -40,15 +40,15 @@ private[jobs] object UIData { class JobUIData( var jobId: Int = -1, - var startTime: Option[Long] = None, - var endTime: Option[Long] = None, + var submissionTime: Option[Long] = None, + var completionTime: Option[Long] = None, var stageIds: Seq[Int] = Seq.empty, var jobGroup: Option[String] = None, var status: JobExecutionStatus = JobExecutionStatus.UNKNOWN, /* Tasks */ // `numTasks` is a potential underestimate of the true number of tasks that this job will run. // This may be an underestimate because the job start event references all of the result - // stages's transitive stage dependencies, but some of these stages might be skipped if their + // stages' transitive stage dependencies, but some of these stages might be skipped if their // output is available from earlier runs. // See https://github.com/apache/spark/pull/3009 for a more extensive discussion. var numTasks: Int = 0, diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index 8c2457f56bff..4c9b1e3c46f0 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -53,7 +53,7 @@ private[spark] object AkkaUtils extends Logging { val startService: Int => (ActorSystem, Int) = { actualPort => doCreateActorSystem(name, host, actualPort, conf, securityManager) } - Utils.startServiceOnPort(port, startService, name) + Utils.startServiceOnPort(port, startService, conf, name) } private def doCreateActorSystem( @@ -65,7 +65,7 @@ private[spark] object AkkaUtils extends Logging { val akkaThreads = conf.getInt("spark.akka.threads", 4) val akkaBatchSize = conf.getInt("spark.akka.batchSize", 15) - val akkaTimeout = conf.getInt("spark.akka.timeout", 100) + val akkaTimeout = conf.getInt("spark.akka.timeout", conf.getInt("spark.network.timeout", 120)) val akkaFrameSize = maxFrameSizeBytes(conf) val akkaLogLifecycleEvents = conf.getBoolean("spark.akka.logLifecycleEvents", false) val lifecycleEvents = if (akkaLogLifecycleEvents) "on" else "off" @@ -89,7 +89,7 @@ private[spark] object AkkaUtils extends Logging { } val requireCookie = if (isAuthOn) "on" else "off" val secureCookie = if (isAuthOn) secretKey else "" - logDebug("In createActorSystem, requireCookie is: " + requireCookie) + logDebug(s"In createActorSystem, requireCookie is: $requireCookie") val akkaConf = ConfigFactory.parseMap(conf.getAkkaConf.toMap[String, String]).withFallback( ConfigFactory.parseString( @@ -140,8 +140,8 @@ private[spark] object AkkaUtils extends Logging { def maxFrameSizeBytes(conf: SparkConf): Int = { val frameSizeInMB = conf.getInt("spark.akka.frameSize", 10) if (frameSizeInMB > AKKA_MAX_FRAME_SIZE_IN_MB) { - throw new IllegalArgumentException("spark.akka.frameSize should not be greater than " - + AKKA_MAX_FRAME_SIZE_IN_MB + "MB") + throw new IllegalArgumentException( + s"spark.akka.frameSize should not be greater than $AKKA_MAX_FRAME_SIZE_IN_MB MB") } frameSizeInMB * 1024 * 1024 } @@ -182,8 +182,8 @@ private[spark] object AkkaUtils extends Logging { timeout: FiniteDuration): T = { // TODO: Consider removing multiple attempts if (actor == null) { - throw new SparkException("Error sending message as actor is null " + - "[message = " + message + "]") + throw new SparkException(s"Error sending message [message = $message]" + + " as actor is null ") } var attempts = 0 var lastException: Exception = null @@ -200,13 +200,13 @@ private[spark] object AkkaUtils extends Logging { case ie: InterruptedException => throw ie case e: Exception => lastException = e - logWarning("Error sending message in " + attempts + " attempts", e) + logWarning(s"Error sending message [message = $message] in $attempts attempts", e) } Thread.sleep(retryInterval) } throw new SparkException( - "Error sending message [message = " + message + "]", lastException) + s"Error sending message [message = $message]", lastException) } def makeDriverRef(name: String, conf: SparkConf, actorSystem: ActorSystem): ActorRef = { diff --git a/core/src/main/scala/org/apache/spark/util/EventLoop.scala b/core/src/main/scala/org/apache/spark/util/EventLoop.scala new file mode 100644 index 000000000000..b0ed908b8442 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/EventLoop.scala @@ -0,0 +1,124 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util + +import java.util.concurrent.atomic.AtomicBoolean +import java.util.concurrent.{BlockingQueue, LinkedBlockingDeque} + +import scala.util.control.NonFatal + +import org.apache.spark.Logging + +/** + * An event loop to receive events from the caller and process all events in the event thread. It + * will start an exclusive event thread to process all events. + * + * Note: The event queue will grow indefinitely. So subclasses should make sure `onReceive` can + * handle events in time to avoid the potential OOM. + */ +private[spark] abstract class EventLoop[E](name: String) extends Logging { + + private val eventQueue: BlockingQueue[E] = new LinkedBlockingDeque[E]() + + private val stopped = new AtomicBoolean(false) + + private val eventThread = new Thread(name) { + setDaemon(true) + + override def run(): Unit = { + try { + while (!stopped.get) { + val event = eventQueue.take() + try { + onReceive(event) + } catch { + case NonFatal(e) => { + try { + onError(e) + } catch { + case NonFatal(e) => logError("Unexpected error in " + name, e) + } + } + } + } + } catch { + case ie: InterruptedException => // exit even if eventQueue is not empty + case NonFatal(e) => logError("Unexpected error in " + name, e) + } + } + + } + + def start(): Unit = { + if (stopped.get) { + throw new IllegalStateException(name + " has already been stopped") + } + // Call onStart before starting the event thread to make sure it happens before onReceive + onStart() + eventThread.start() + } + + def stop(): Unit = { + if (stopped.compareAndSet(false, true)) { + eventThread.interrupt() + eventThread.join() + // Call onStop after the event thread exits to make sure onReceive happens before onStop + onStop() + } else { + // Keep quiet to allow calling `stop` multiple times. + } + } + + /** + * Put the event into the event queue. The event thread will process it later. + */ + def post(event: E): Unit = { + eventQueue.put(event) + } + + /** + * Return if the event thread has already been started but not yet stopped. + */ + def isActive: Boolean = eventThread.isAlive + + /** + * Invoked when `start()` is called but before the event thread starts. + */ + protected def onStart(): Unit = {} + + /** + * Invoked when `stop()` is called and the event thread exits. + */ + protected def onStop(): Unit = {} + + /** + * Invoked in the event thread when polling events from the event queue. + * + * Note: Should avoid calling blocking actions in `onReceive`, or the event thread will be blocked + * and cannot process events in time. If you want to call some blocking actions, run them in + * another thread. + */ + protected def onReceive(event: E): Unit + + /** + * Invoked if `onReceive` throws any non fatal error. Any non fatal error thrown from `onError` + * will be ignored. + */ + protected def onError(e: Throwable): Unit + +} 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 e7b80e8774b9..f896b5072e4f 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -19,6 +19,8 @@ package org.apache.spark.util import java.util.{Properties, UUID} +import org.apache.spark.scheduler.cluster.ExecutorInfo + import scala.collection.JavaConverters._ import scala.collection.Map @@ -30,6 +32,7 @@ import org.apache.spark.executor._ import org.apache.spark.scheduler._ import org.apache.spark.storage._ import org.apache.spark._ +import org.apache.hadoop.hdfs.web.JsonUtil /** * Serializes SparkListener events to/from JSON. This protocol provides strong backwards- @@ -83,7 +86,10 @@ private[spark] object JsonProtocol { applicationStartToJson(applicationStart) case applicationEnd: SparkListenerApplicationEnd => applicationEndToJson(applicationEnd) - + case executorAdded: SparkListenerExecutorAdded => + executorAddedToJson(executorAdded) + case executorRemoved: SparkListenerExecutorRemoved => + executorRemovedToJson(executorRemoved) // These aren't used, but keeps compiler happy case SparkListenerShutdown => JNothing case SparkListenerExecutorMetricsUpdate(_, _) => JNothing @@ -136,6 +142,7 @@ private[spark] object JsonProtocol { val properties = propertiesToJson(jobStart.properties) ("Event" -> Utils.getFormattedClassName(jobStart)) ~ ("Job ID" -> jobStart.jobId) ~ + ("Submission Time" -> jobStart.time) ~ ("Stage Infos" -> jobStart.stageInfos.map(stageInfoToJson)) ~ // Added in Spark 1.2.0 ("Stage IDs" -> jobStart.stageIds) ~ ("Properties" -> properties) @@ -145,6 +152,7 @@ private[spark] object JsonProtocol { val jobResult = jobResultToJson(jobEnd.jobResult) ("Event" -> Utils.getFormattedClassName(jobEnd)) ~ ("Job ID" -> jobEnd.jobId) ~ + ("Completion Time" -> jobEnd.time) ~ ("Job Result" -> jobResult) } @@ -194,6 +202,16 @@ private[spark] object JsonProtocol { ("Timestamp" -> applicationEnd.time) } + def executorAddedToJson(executorAdded: SparkListenerExecutorAdded): JValue = { + ("Event" -> Utils.getFormattedClassName(executorAdded)) ~ + ("Executor ID" -> executorAdded.executorId) ~ + ("Executor Info" -> executorInfoToJson(executorAdded.executorInfo)) + } + + def executorRemovedToJson(executorRemoved: SparkListenerExecutorRemoved): JValue = { + ("Event" -> Utils.getFormattedClassName(executorRemoved)) ~ + ("Executor ID" -> executorRemoved.executorId) + } /** ------------------------------------------------------------------- * * JSON serialization methods for classes SparkListenerEvents depend on | @@ -362,6 +380,10 @@ private[spark] object JsonProtocol { ("Disk Size" -> blockStatus.diskSize) } + def executorInfoToJson(executorInfo: ExecutorInfo): JValue = { + ("Host" -> executorInfo.executorHost) ~ + ("Total Cores" -> executorInfo.totalCores) + } /** ------------------------------ * * Util JSON serialization methods | @@ -416,6 +438,8 @@ private[spark] object JsonProtocol { val unpersistRDD = Utils.getFormattedClassName(SparkListenerUnpersistRDD) val applicationStart = Utils.getFormattedClassName(SparkListenerApplicationStart) val applicationEnd = Utils.getFormattedClassName(SparkListenerApplicationEnd) + val executorAdded = Utils.getFormattedClassName(SparkListenerExecutorAdded) + val executorRemoved = Utils.getFormattedClassName(SparkListenerExecutorRemoved) (json \ "Event").extract[String] match { case `stageSubmitted` => stageSubmittedFromJson(json) @@ -431,6 +455,8 @@ private[spark] object JsonProtocol { case `unpersistRDD` => unpersistRDDFromJson(json) case `applicationStart` => applicationStartFromJson(json) case `applicationEnd` => applicationEndFromJson(json) + case `executorAdded` => executorAddedFromJson(json) + case `executorRemoved` => executorRemovedFromJson(json) } } @@ -469,6 +495,8 @@ private[spark] object JsonProtocol { def jobStartFromJson(json: JValue): SparkListenerJobStart = { val jobId = (json \ "Job ID").extract[Int] + val submissionTime = + Utils.jsonOption(json \ "Submission Time").map(_.extract[Long]).getOrElse(-1L) val stageIds = (json \ "Stage IDs").extract[List[JValue]].map(_.extract[Int]) val properties = propertiesFromJson(json \ "Properties") // The "Stage Infos" field was added in Spark 1.2.0 @@ -476,13 +504,15 @@ private[spark] object JsonProtocol { .map(_.extract[Seq[JValue]].map(stageInfoFromJson)).getOrElse { stageIds.map(id => new StageInfo(id, 0, "unknown", 0, Seq.empty, "unknown")) } - SparkListenerJobStart(jobId, stageInfos, properties) + SparkListenerJobStart(jobId, submissionTime, stageInfos, properties) } def jobEndFromJson(json: JValue): SparkListenerJobEnd = { val jobId = (json \ "Job ID").extract[Int] + val completionTime = + Utils.jsonOption(json \ "Completion Time").map(_.extract[Long]).getOrElse(-1L) val jobResult = jobResultFromJson(json \ "Job Result") - SparkListenerJobEnd(jobId, jobResult) + SparkListenerJobEnd(jobId, completionTime, jobResult) } def environmentUpdateFromJson(json: JValue): SparkListenerEnvironmentUpdate = { @@ -523,6 +553,16 @@ private[spark] object JsonProtocol { SparkListenerApplicationEnd((json \ "Timestamp").extract[Long]) } + def executorAddedFromJson(json: JValue): SparkListenerExecutorAdded = { + val executorId = (json \ "Executor ID").extract[String] + val executorInfo = executorInfoFromJson(json \ "Executor Info") + SparkListenerExecutorAdded(executorId, executorInfo) + } + + def executorRemovedFromJson(json: JValue): SparkListenerExecutorRemoved = { + val executorId = (json \ "Executor ID").extract[String] + SparkListenerExecutorRemoved(executorId) + } /** --------------------------------------------------------------------- * * JSON deserialization methods for classes SparkListenerEvents depend on | @@ -530,7 +570,7 @@ private[spark] object JsonProtocol { def stageInfoFromJson(json: JValue): StageInfo = { val stageId = (json \ "Stage ID").extract[Int] - val attemptId = (json \ "Attempt ID").extractOpt[Int].getOrElse(0) + val attemptId = (json \ "Stage Attempt ID").extractOpt[Int].getOrElse(0) val stageName = (json \ "Stage Name").extract[String] val numTasks = (json \ "Number of Tasks").extract[Int] val rddInfos = (json \ "RDD Info").extract[List[JValue]].map(rddInfoFromJson(_)) @@ -592,20 +632,20 @@ private[spark] object JsonProtocol { return TaskMetrics.empty } val metrics = new TaskMetrics - metrics.hostname = (json \ "Host Name").extract[String] - metrics.executorDeserializeTime = (json \ "Executor Deserialize Time").extract[Long] - metrics.executorRunTime = (json \ "Executor Run Time").extract[Long] - metrics.resultSize = (json \ "Result Size").extract[Long] - metrics.jvmGCTime = (json \ "JVM GC Time").extract[Long] - metrics.resultSerializationTime = (json \ "Result Serialization Time").extract[Long] - metrics.memoryBytesSpilled = (json \ "Memory Bytes Spilled").extract[Long] - metrics.diskBytesSpilled = (json \ "Disk Bytes Spilled").extract[Long] + metrics.setHostname((json \ "Host Name").extract[String]) + metrics.setExecutorDeserializeTime((json \ "Executor Deserialize Time").extract[Long]) + metrics.setExecutorRunTime((json \ "Executor Run Time").extract[Long]) + metrics.setResultSize((json \ "Result Size").extract[Long]) + metrics.setJvmGCTime((json \ "JVM GC Time").extract[Long]) + metrics.setResultSerializationTime((json \ "Result Serialization Time").extract[Long]) + metrics.incMemoryBytesSpilled((json \ "Memory Bytes Spilled").extract[Long]) + metrics.incDiskBytesSpilled((json \ "Disk Bytes Spilled").extract[Long]) metrics.setShuffleReadMetrics( Utils.jsonOption(json \ "Shuffle Read Metrics").map(shuffleReadMetricsFromJson)) metrics.shuffleWriteMetrics = Utils.jsonOption(json \ "Shuffle Write Metrics").map(shuffleWriteMetricsFromJson) - metrics.inputMetrics = - Utils.jsonOption(json \ "Input Metrics").map(inputMetricsFromJson) + metrics.setInputMetrics( + Utils.jsonOption(json \ "Input Metrics").map(inputMetricsFromJson)) metrics.outputMetrics = Utils.jsonOption(json \ "Output Metrics").map(outputMetricsFromJson) metrics.updatedBlocks = @@ -621,31 +661,31 @@ private[spark] object JsonProtocol { def shuffleReadMetricsFromJson(json: JValue): ShuffleReadMetrics = { val metrics = new ShuffleReadMetrics - metrics.remoteBlocksFetched = (json \ "Remote Blocks Fetched").extract[Int] - metrics.localBlocksFetched = (json \ "Local Blocks Fetched").extract[Int] - metrics.fetchWaitTime = (json \ "Fetch Wait Time").extract[Long] - metrics.remoteBytesRead = (json \ "Remote Bytes Read").extract[Long] + metrics.incRemoteBlocksFetched((json \ "Remote Blocks Fetched").extract[Int]) + metrics.incLocalBlocksFetched((json \ "Local Blocks Fetched").extract[Int]) + metrics.incFetchWaitTime((json \ "Fetch Wait Time").extract[Long]) + metrics.incRemoteBytesRead((json \ "Remote Bytes Read").extract[Long]) metrics } def shuffleWriteMetricsFromJson(json: JValue): ShuffleWriteMetrics = { val metrics = new ShuffleWriteMetrics - metrics.shuffleBytesWritten = (json \ "Shuffle Bytes Written").extract[Long] - metrics.shuffleWriteTime = (json \ "Shuffle Write Time").extract[Long] + metrics.incShuffleBytesWritten((json \ "Shuffle Bytes Written").extract[Long]) + metrics.incShuffleWriteTime((json \ "Shuffle Write Time").extract[Long]) metrics } def inputMetricsFromJson(json: JValue): InputMetrics = { val metrics = new InputMetrics( DataReadMethod.withName((json \ "Data Read Method").extract[String])) - metrics.bytesRead = (json \ "Bytes Read").extract[Long] + metrics.addBytesRead((json \ "Bytes Read").extract[Long]) metrics } def outputMetricsFromJson(json: JValue): OutputMetrics = { val metrics = new OutputMetrics( DataWriteMethod.withName((json \ "Data Write Method").extract[String])) - metrics.bytesWritten = (json \ "Bytes Written").extract[Long] + metrics.setBytesWritten((json \ "Bytes Written").extract[Long]) metrics } @@ -745,6 +785,11 @@ private[spark] object JsonProtocol { BlockStatus(storageLevel, memorySize, diskSize, tachyonSize) } + def executorInfoFromJson(json: JValue): ExecutorInfo = { + val executorHost = (json \ "Host").extract[String] + val totalCores = (json \ "Total Cores").extract[Int] + new ExecutorInfo(executorHost, totalCores) + } /** -------------------------------- * * Util JSON deserialization methods | 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 0d771baaa6ab..2c04e4ddfbcb 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -701,7 +701,7 @@ private[spark] object Utils extends Logging { } } - private var customHostname: Option[String] = None + private var customHostname: Option[String] = sys.env.get("SPARK_LOCAL_HOSTNAME") /** * Allow setting a custom host name because when we run on Mesos we need to use the same @@ -990,11 +990,12 @@ private[spark] object Utils extends Logging { for ((key, value) <- extraEnvironment) { environment.put(key, value) } + val process = builder.start() new Thread("read stderr for " + command(0)) { override def run() { for (line <- Source.fromInputStream(process.getErrorStream).getLines()) { - System.err.println(line) + logInfo(line) } } }.start() @@ -1089,7 +1090,7 @@ private[spark] object Utils extends Logging { var firstUserLine = 0 var insideSpark = true var callStack = new ArrayBuffer[String]() :+ "" - + Thread.currentThread.getStackTrace().foreach { 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 @@ -1689,17 +1690,15 @@ private[spark] object Utils extends Logging { } /** - * Default maximum number of retries when binding to a port before giving up. + * Maximum number of retries when binding to a port before giving up. */ - val portMaxRetries: Int = { - if (sys.props.contains("spark.testing")) { + def portMaxRetries(conf: SparkConf): Int = { + val maxRetries = conf.getOption("spark.port.maxRetries").map(_.toInt) + if (conf.contains("spark.testing")) { // Set a higher number of retries for tests... - sys.props.get("spark.port.maxRetries").map(_.toInt).getOrElse(100) + maxRetries.getOrElse(100) } else { - Option(SparkEnv.get) - .flatMap(_.conf.getOption("spark.port.maxRetries")) - .map(_.toInt) - .getOrElse(16) + maxRetries.getOrElse(16) } } @@ -1708,17 +1707,18 @@ private[spark] object Utils extends Logging { * Each subsequent attempt uses 1 + the port used in the previous attempt (unless the port is 0). * * @param startPort The initial port to start the service on. - * @param maxRetries Maximum number of retries to attempt. - * A value of 3 means attempting ports n, n+1, n+2, and n+3, for example. * @param startService Function to start service on a given port. * This is expected to throw java.net.BindException on port collision. + * @param conf A SparkConf used to get the maximum number of retries when binding to a port. + * @param serviceName Name of the service. */ def startServiceOnPort[T]( startPort: Int, startService: Int => (T, Int), - serviceName: String = "", - maxRetries: Int = portMaxRetries): (T, Int) = { + conf: SparkConf, + serviceName: String = ""): (T, Int) = { val serviceString = if (serviceName.isEmpty) "" else s" '$serviceName'" + val maxRetries = portMaxRetries(conf) 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) { @@ -1842,6 +1842,35 @@ private[spark] object Utils extends Logging { sparkValue } } + + /** + * Return a pair of host and port extracted from the `sparkUrl`. + * + * A spark url (`spark://host:port`) is a special URI that its scheme is `spark` and only contains + * host and port. + * + * @throws SparkException if `sparkUrl` is invalid. + */ + def extractHostPortFromSparkUrl(sparkUrl: String): (String, Int) = { + try { + val uri = new java.net.URI(sparkUrl) + val host = uri.getHost + val port = uri.getPort + if (uri.getScheme != "spark" || + host == null || + port < 0 || + (uri.getPath != null && !uri.getPath.isEmpty) || // uri.getPath returns "" instead of null + uri.getFragment != null || + uri.getQuery != null || + uri.getUserInfo != null) { + throw new SparkException("Invalid master URL: " + sparkUrl) + } + (host, port) + } catch { + case e: java.net.URISyntaxException => + throw new SparkException("Invalid master URL: " + sparkUrl, e) + } + } } /** diff --git a/core/src/main/scala/org/apache/spark/util/Vector.scala b/core/src/main/scala/org/apache/spark/util/Vector.scala index c6cab82c3e54..2ed827eab46d 100644 --- a/core/src/main/scala/org/apache/spark/util/Vector.scala +++ b/core/src/main/scala/org/apache/spark/util/Vector.scala @@ -24,9 +24,9 @@ import org.apache.spark.util.random.XORShiftRandom @deprecated("Use Vectors.dense from Spark's mllib.linalg package instead.", "1.0.0") class Vector(val elements: Array[Double]) extends Serializable { - def length = elements.length + def length: Int = elements.length - def apply(index: Int) = elements(index) + def apply(index: Int): Double = elements(index) def + (other: Vector): Vector = { if (length != other.length) { @@ -35,7 +35,7 @@ class Vector(val elements: Array[Double]) extends Serializable { Vector(length, i => this(i) + other(i)) } - def add(other: Vector) = this + other + def add(other: Vector): Vector = this + other def - (other: Vector): Vector = { if (length != other.length) { @@ -44,7 +44,7 @@ class Vector(val elements: Array[Double]) extends Serializable { Vector(length, i => this(i) - other(i)) } - def subtract(other: Vector) = this - other + def subtract(other: Vector): Vector = this - other def dot(other: Vector): Double = { if (length != other.length) { @@ -93,19 +93,19 @@ class Vector(val elements: Array[Double]) extends Serializable { this } - def addInPlace(other: Vector) = this +=other + def addInPlace(other: Vector): Vector = this +=other def * (scale: Double): Vector = Vector(length, i => this(i) * scale) - def multiply (d: Double) = this * d + def multiply (d: Double): Vector = this * d def / (d: Double): Vector = this * (1 / d) - def divide (d: Double) = this / d + def divide (d: Double): Vector = this / d - def unary_- = this * -1 + def unary_- : Vector = this * -1 - def sum = elements.reduceLeft(_ + _) + def sum: Double = elements.reduceLeft(_ + _) def squaredDist(other: Vector): Double = { var ans = 0.0 @@ -119,40 +119,40 @@ class Vector(val elements: Array[Double]) extends Serializable { def dist(other: Vector): Double = math.sqrt(squaredDist(other)) - override def toString = elements.mkString("(", ", ", ")") + override def toString: String = elements.mkString("(", ", ", ")") } object Vector { - def apply(elements: Array[Double]) = new Vector(elements) + def apply(elements: Array[Double]): Vector = new Vector(elements) - def apply(elements: Double*) = new Vector(elements.toArray) + def apply(elements: Double*): Vector = new Vector(elements.toArray) def apply(length: Int, initializer: Int => Double): Vector = { val elements: Array[Double] = Array.tabulate(length)(initializer) new Vector(elements) } - def zeros(length: Int) = new Vector(new Array[Double](length)) + def zeros(length: Int): Vector = new Vector(new Array[Double](length)) - def ones(length: Int) = Vector(length, _ => 1) + def ones(length: Int): Vector = Vector(length, _ => 1) /** * Creates this [[org.apache.spark.util.Vector]] of given length containing random numbers * between 0.0 and 1.0. Optional scala.util.Random number generator can be provided. */ - def random(length: Int, random: Random = new XORShiftRandom()) = + def random(length: Int, random: Random = new XORShiftRandom()): Vector = Vector(length, _ => random.nextDouble()) class Multiplier(num: Double) { - def * (vec: Vector) = vec * num + def * (vec: Vector): Vector = vec * num } - implicit def doubleToMultiplier(num: Double) = new Multiplier(num) + implicit def doubleToMultiplier(num: Double): Multiplier = new Multiplier(num) implicit object VectorAccumParam extends org.apache.spark.AccumulatorParam[Vector] { - def addInPlace(t1: Vector, t2: Vector) = t1 + t2 + def addInPlace(t1: Vector, t2: Vector): Vector = t1 + t2 - def zero(initialValue: Vector) = Vector.zeros(initialValue.length) + def zero(initialValue: Vector): Vector = Vector.zeros(initialValue.length) } } 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 15bda1c9cc29..6ba03841f746 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 @@ -757,12 +757,12 @@ private[spark] class ExternalSorter[K, V, C]( } } - context.taskMetrics.memoryBytesSpilled += memoryBytesSpilled - context.taskMetrics.diskBytesSpilled += diskBytesSpilled + context.taskMetrics.incMemoryBytesSpilled(memoryBytesSpilled) + context.taskMetrics.incDiskBytesSpilled(diskBytesSpilled) context.taskMetrics.shuffleWriteMetrics.filter(_ => bypassMergeSort).foreach { m => if (curWriteMetrics != null) { - m.shuffleBytesWritten += curWriteMetrics.shuffleBytesWritten - m.shuffleWriteTime += curWriteMetrics.shuffleWriteTime + m.incShuffleBytesWritten(curWriteMetrics.shuffleBytesWritten) + m.incShuffleWriteTime(curWriteMetrics.shuffleWriteTime) } } diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index 5ce299d05824..004de05c10ee 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -606,6 +606,27 @@ public void take() { rdd.takeSample(false, 2, 42); } + @Test + public void isEmpty() { + Assert.assertTrue(sc.emptyRDD().isEmpty()); + Assert.assertTrue(sc.parallelize(new ArrayList()).isEmpty()); + Assert.assertFalse(sc.parallelize(Arrays.asList(1)).isEmpty()); + Assert.assertTrue(sc.parallelize(Arrays.asList(1, 2, 3), 3).filter( + new Function() { + @Override + public Boolean call(Integer i) { + return i < 0; + } + }).isEmpty()); + Assert.assertFalse(sc.parallelize(Arrays.asList(1, 2, 3)).filter( + new Function() { + @Override + public Boolean call(Integer i) { + return i > 1; + } + }).isEmpty()); + } + @Test public void cartesian() { JavaDoubleRDD doubleRDD = sc.parallelizeDoubles(Arrays.asList(1.0, 1.0, 2.0, 3.0, 5.0, 8.0)); @@ -820,7 +841,7 @@ public void persist() { @Test public void iterator() { JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5), 2); - TaskContext context = new TaskContextImpl(0, 0, 0L, false, new TaskMetrics()); + TaskContext context = new TaskContextImpl(0, 0, 0L, 0, false, new TaskMetrics()); Assert.assertEquals(1, rdd.iterator(rdd.partitions().get(0), context).next().intValue()); } diff --git a/core/src/test/resources/log4j.properties b/core/src/test/resources/log4j.properties index 9dd05f17f012..287c8e356350 100644 --- a/core/src/test/resources/log4j.properties +++ b/core/src/test/resources/log4j.properties @@ -15,10 +15,10 @@ # limitations under the License. # -# Set everything to be logged to the file core/target/unit-tests.log +# Set everything to be logged to the file target/unit-tests.log log4j.rootCategory=INFO, file log4j.appender.file=org.apache.log4j.FileAppender -log4j.appender.file.append=false +log4j.appender.file.append=true 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 diff --git a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala index c0735f448d19..d7d9dc7b50f3 100644 --- a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala @@ -66,7 +66,7 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar // in blockManager.put is a losing battle. You have been warned. blockManager = sc.env.blockManager cacheManager = sc.env.cacheManager - val context = new TaskContextImpl(0, 0, 0) + val context = new TaskContextImpl(0, 0, 0, 0) val computeValue = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY) val getValue = blockManager.get(RDDBlockId(rdd.id, split.index)) assert(computeValue.toList === List(1, 2, 3, 4)) @@ -81,7 +81,7 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar } whenExecuting(blockManager) { - val context = new TaskContextImpl(0, 0, 0) + val context = new TaskContextImpl(0, 0, 0, 0) val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY) assert(value.toList === List(5, 6, 7)) } @@ -94,7 +94,7 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar } whenExecuting(blockManager) { - val context = new TaskContextImpl(0, 0, 0, true) + val context = new TaskContextImpl(0, 0, 0, 0, true) val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY) assert(value.toList === List(1, 2, 3, 4)) } @@ -102,7 +102,7 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar test("verify task metrics updated correctly") { cacheManager = sc.env.cacheManager - val context = new TaskContextImpl(0, 0, 0) + val context = new TaskContextImpl(0, 0, 0, 0) cacheManager.getOrCompute(rdd3, split, context, StorageLevel.MEMORY_ONLY) assert(context.taskMetrics.updatedBlocks.getOrElse(Seq()).size === 2) } diff --git a/core/src/test/scala/org/apache/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala index 998f3008ec0e..97ea3578aa8b 100644 --- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala @@ -17,7 +17,6 @@ package org.apache.spark -import org.scalatest.BeforeAndAfter import org.scalatest.FunSuite import org.scalatest.concurrent.Timeouts._ import org.scalatest.Matchers @@ -29,16 +28,10 @@ class NotSerializableClass class NotSerializableExn(val notSer: NotSerializableClass) extends Throwable() {} -class DistributedSuite extends FunSuite with Matchers with BeforeAndAfter - with LocalSparkContext { +class DistributedSuite extends FunSuite with Matchers with LocalSparkContext { val clusterUrl = "local-cluster[2,1,512]" - after { - System.clearProperty("spark.reducer.maxMbInFlight") - System.clearProperty("spark.storage.memoryFraction") - } - test("task throws not serializable exception") { // Ensures that executors do not crash when an exn is not serializable. If executors crash, // this test will hang. Correct behavior is that executors don't crash but fail tasks @@ -84,15 +77,14 @@ class DistributedSuite extends FunSuite with Matchers with BeforeAndAfter } test("groupByKey where map output sizes exceed maxMbInFlight") { - System.setProperty("spark.reducer.maxMbInFlight", "1") - sc = new SparkContext(clusterUrl, "test") + val conf = new SparkConf().set("spark.reducer.maxMbInFlight", "1") + sc = new SparkContext(clusterUrl, "test", conf) // This data should be around 20 MB, so even with 4 mappers and 2 reducers, each map output // file should be about 2.5 MB val pairs = sc.parallelize(1 to 2000, 4).map(x => (x % 16, new Array[Byte](10000))) val groups = pairs.groupByKey(2).map(x => (x._1, x._2.size)).collect() assert(groups.length === 16) assert(groups.map(_._2).sum === 2000) - // Note that spark.reducer.maxMbInFlight will be cleared in the test suite's after{} block } test("accumulators") { @@ -210,7 +202,6 @@ class DistributedSuite extends FunSuite with Matchers with BeforeAndAfter } test("compute without caching when no partitions fit in memory") { - System.setProperty("spark.storage.memoryFraction", "0.0001") sc = new SparkContext(clusterUrl, "test") // data will be 4 million * 4 bytes = 16 MB in size, but our memoryFraction set the cache // to only 50 KB (0.0001 of 512 MB), so no partitions should fit in memory @@ -218,12 +209,11 @@ class DistributedSuite extends FunSuite with Matchers with BeforeAndAfter assert(data.count() === 4000000) assert(data.count() === 4000000) assert(data.count() === 4000000) - System.clearProperty("spark.storage.memoryFraction") } test("compute when only some partitions fit in memory") { - System.setProperty("spark.storage.memoryFraction", "0.01") - sc = new SparkContext(clusterUrl, "test") + val conf = new SparkConf().set("spark.storage.memoryFraction", "0.01") + sc = new SparkContext(clusterUrl, "test", conf) // data will be 4 million * 4 bytes = 16 MB in size, but our memoryFraction set the cache // to only 5 MB (0.01 of 512 MB), so not all of it will fit in memory; we use 20 partitions // to make sure that *some* of them do fit though @@ -231,7 +221,6 @@ class DistributedSuite extends FunSuite with Matchers with BeforeAndAfter assert(data.count() === 4000000) assert(data.count() === 4000000) assert(data.count() === 4000000) - System.clearProperty("spark.storage.memoryFraction") } test("passing environment variables to cluster") { diff --git a/core/src/test/scala/org/apache/spark/DriverSuite.scala b/core/src/test/scala/org/apache/spark/DriverSuite.scala index 5265ba904032..8a54360e8179 100644 --- a/core/src/test/scala/org/apache/spark/DriverSuite.scala +++ b/core/src/test/scala/org/apache/spark/DriverSuite.scala @@ -35,7 +35,7 @@ class DriverSuite extends FunSuite with Timeouts { forAll(masters) { (master: String) => failAfter(60 seconds) { Utils.executeAndGetOutput( - Seq("./bin/spark-class", "org.apache.spark.DriverWithoutCleanup", master), + Seq(s"$sparkHome/bin/spark-class", "org.apache.spark.DriverWithoutCleanup", master), new File(sparkHome), Map("SPARK_TESTING" -> "1", "SPARK_HOME" -> sparkHome)) } @@ -50,7 +50,10 @@ class DriverSuite extends FunSuite with Timeouts { object DriverWithoutCleanup { def main(args: Array[String]) { Utils.configTestLog4j("INFO") - val sc = new SparkContext(args(0), "DriverWithoutCleanup") + // Bind the web UI to an ephemeral port in order to avoid conflicts with other tests running on + // the same machine (we shouldn't just disable the UI here, since that might mask bugs): + val conf = new SparkConf().set("spark.ui.port", "0") + val sc = new SparkContext(args(0), "DriverWithoutCleanup", conf) sc.parallelize(1 to 100, 4).count() } } diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala index c817f6dcede7..0e4df17c1bf8 100644 --- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark +import scala.collection.mutable + import org.scalatest.{FunSuite, PrivateMethodTester} import org.apache.spark.executor.TaskMetrics import org.apache.spark.scheduler._ @@ -143,11 +145,17 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { // Verify that running a task reduces the cap sc.listenerBus.postToAll(SparkListenerStageSubmitted(createStageInfo(1, 3))) + sc.listenerBus.postToAll(SparkListenerBlockManagerAdded( + 0L, BlockManagerId("executor-1", "host1", 1), 100L)) sc.listenerBus.postToAll(SparkListenerTaskStart(1, 0, createTaskInfo(0, 0, "executor-1"))) + assert(numExecutorsPending(manager) === 4) assert(addExecutors(manager) === 1) - assert(numExecutorsPending(manager) === 6) + assert(numExecutorsPending(manager) === 5) assert(numExecutorsToAdd(manager) === 2) - assert(addExecutors(manager) === 1) + assert(addExecutors(manager) === 2) + assert(numExecutorsPending(manager) === 7) + assert(numExecutorsToAdd(manager) === 4) + assert(addExecutors(manager) === 0) assert(numExecutorsPending(manager) === 7) assert(numExecutorsToAdd(manager) === 1) @@ -325,6 +333,8 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { val manager = sc.executorAllocationManager.get manager.setClock(clock) + executorIds(manager).asInstanceOf[mutable.Set[String]] ++= List("1", "2", "3") + // Starting remove timer is idempotent for each executor assert(removeTimes(manager).isEmpty) onExecutorIdle(manager, "1") @@ -597,6 +607,41 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { assert(removeTimes(manager).size === 1) } + test("SPARK-4951: call onTaskStart before onBlockManagerAdded") { + sc = createSparkContext(2, 10) + val manager = sc.executorAllocationManager.get + assert(executorIds(manager).isEmpty) + assert(removeTimes(manager).isEmpty) + + sc.listenerBus.postToAll(SparkListenerTaskStart(0, 0, createTaskInfo(0, 0, "executor-1"))) + sc.listenerBus.postToAll(SparkListenerBlockManagerAdded( + 0L, BlockManagerId("executor-1", "host1", 1), 100L)) + assert(executorIds(manager).size === 1) + assert(executorIds(manager).contains("executor-1")) + assert(removeTimes(manager).size === 0) + } + + test("SPARK-4951: onExecutorAdded should not add a busy executor to removeTimes") { + sc = createSparkContext(2, 10) + val manager = sc.executorAllocationManager.get + assert(executorIds(manager).isEmpty) + assert(removeTimes(manager).isEmpty) + sc.listenerBus.postToAll(SparkListenerBlockManagerAdded( + 0L, BlockManagerId("executor-1", "host1", 1), 100L)) + sc.listenerBus.postToAll(SparkListenerTaskStart(0, 0, createTaskInfo(0, 0, "executor-1"))) + + assert(executorIds(manager).size === 1) + assert(executorIds(manager).contains("executor-1")) + assert(removeTimes(manager).size === 0) + + sc.listenerBus.postToAll(SparkListenerBlockManagerAdded( + 0L, BlockManagerId("executor-2", "host1", 1), 100L)) + assert(executorIds(manager).size === 2) + assert(executorIds(manager).contains("executor-2")) + assert(removeTimes(manager).size === 1) + assert(removeTimes(manager).contains("executor-2")) + assert(!removeTimes(manager).contains("executor-1")) + } } /** diff --git a/core/src/test/scala/org/apache/spark/FileServerSuite.scala b/core/src/test/scala/org/apache/spark/FileServerSuite.scala index 49426545c767..0f49ce4754fb 100644 --- a/core/src/test/scala/org/apache/spark/FileServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileServerSuite.scala @@ -31,10 +31,11 @@ class FileServerSuite extends FunSuite with LocalSparkContext { @transient var tmpFile: File = _ @transient var tmpJarUrl: String = _ + def newConf: SparkConf = new SparkConf(loadDefaults = false).set("spark.authenticate", "false") + override def beforeEach() { super.beforeEach() resetSparkContext() - System.setProperty("spark.authenticate", "false") } override def beforeAll() { @@ -52,7 +53,6 @@ class FileServerSuite extends FunSuite with LocalSparkContext { val jarFile = new File(testTempDir, "test.jar") val jarStream = new FileOutputStream(jarFile) val jar = new JarOutputStream(jarStream, new java.util.jar.Manifest()) - System.setProperty("spark.authenticate", "false") val jarEntry = new JarEntry(textFile.getName) jar.putNextEntry(jarEntry) @@ -74,7 +74,7 @@ class FileServerSuite extends FunSuite with LocalSparkContext { } test("Distributing files locally") { - sc = new SparkContext("local[4]", "test") + sc = new SparkContext("local[4]", "test", newConf) sc.addFile(tmpFile.toString) val testData = Array((1,1), (1,1), (2,1), (3,5), (2,2), (3,0)) val result = sc.parallelize(testData).reduceByKey { @@ -108,7 +108,7 @@ class FileServerSuite extends FunSuite with LocalSparkContext { test("Distributing files locally using URL as input") { // addFile("file:///....") - sc = new SparkContext("local[4]", "test") + sc = new SparkContext("local[4]", "test", newConf) sc.addFile(new File(tmpFile.toString).toURI.toString) val testData = Array((1,1), (1,1), (2,1), (3,5), (2,2), (3,0)) val result = sc.parallelize(testData).reduceByKey { @@ -122,7 +122,7 @@ class FileServerSuite extends FunSuite with LocalSparkContext { } test ("Dynamically adding JARS locally") { - sc = new SparkContext("local[4]", "test") + sc = new SparkContext("local[4]", "test", newConf) sc.addJar(tmpJarUrl) val testData = Array((1, 1)) sc.parallelize(testData).foreach { x => @@ -133,7 +133,7 @@ class FileServerSuite extends FunSuite with LocalSparkContext { } test("Distributing files on a standalone cluster") { - sc = new SparkContext("local-cluster[1,1,512]", "test") + sc = new SparkContext("local-cluster[1,1,512]", "test", newConf) sc.addFile(tmpFile.toString) val testData = Array((1,1), (1,1), (2,1), (3,5), (2,2), (3,0)) val result = sc.parallelize(testData).reduceByKey { @@ -147,7 +147,7 @@ class FileServerSuite extends FunSuite with LocalSparkContext { } test ("Dynamically adding JARS on a standalone cluster") { - sc = new SparkContext("local-cluster[1,1,512]", "test") + sc = new SparkContext("local-cluster[1,1,512]", "test", newConf) sc.addJar(tmpJarUrl) val testData = Array((1,1)) sc.parallelize(testData).foreach { x => @@ -158,7 +158,7 @@ class FileServerSuite extends FunSuite with LocalSparkContext { } test ("Dynamically adding JARS on a standalone cluster using local: URL") { - sc = new SparkContext("local-cluster[1,1,512]", "test") + sc = new SparkContext("local-cluster[1,1,512]", "test", newConf) sc.addJar(tmpJarUrl.replace("file", "local")) val testData = Array((1,1)) sc.parallelize(testData).foreach { x => diff --git a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala index 41ed2bce55ce..7584ae79fc92 100644 --- a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala +++ b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala @@ -40,12 +40,11 @@ class JobCancellationSuite extends FunSuite with Matchers with BeforeAndAfter override def afterEach() { super.afterEach() resetSparkContext() - System.clearProperty("spark.scheduler.mode") } test("local mode, FIFO scheduler") { - System.setProperty("spark.scheduler.mode", "FIFO") - sc = new SparkContext("local[2]", "test") + val conf = new SparkConf().set("spark.scheduler.mode", "FIFO") + sc = new SparkContext("local[2]", "test", conf) testCount() testTake() // Make sure we can still launch tasks. @@ -53,10 +52,10 @@ class JobCancellationSuite extends FunSuite with Matchers with BeforeAndAfter } test("local mode, fair scheduler") { - System.setProperty("spark.scheduler.mode", "FAIR") + val conf = new SparkConf().set("spark.scheduler.mode", "FAIR") val xmlPath = getClass.getClassLoader.getResource("fairscheduler.xml").getFile() - System.setProperty("spark.scheduler.allocation.file", xmlPath) - sc = new SparkContext("local[2]", "test") + conf.set("spark.scheduler.allocation.file", xmlPath) + sc = new SparkContext("local[2]", "test", conf) testCount() testTake() // Make sure we can still launch tasks. @@ -64,8 +63,8 @@ class JobCancellationSuite extends FunSuite with Matchers with BeforeAndAfter } test("cluster mode, FIFO scheduler") { - System.setProperty("spark.scheduler.mode", "FIFO") - sc = new SparkContext("local-cluster[2,1,512]", "test") + val conf = new SparkConf().set("spark.scheduler.mode", "FIFO") + sc = new SparkContext("local-cluster[2,1,512]", "test", conf) testCount() testTake() // Make sure we can still launch tasks. @@ -73,10 +72,10 @@ class JobCancellationSuite extends FunSuite with Matchers with BeforeAndAfter } test("cluster mode, fair scheduler") { - System.setProperty("spark.scheduler.mode", "FAIR") + val conf = new SparkConf().set("spark.scheduler.mode", "FAIR") val xmlPath = getClass.getClassLoader.getResource("fairscheduler.xml").getFile() - System.setProperty("spark.scheduler.allocation.file", xmlPath) - sc = new SparkContext("local-cluster[2,1,512]", "test") + conf.set("spark.scheduler.allocation.file", xmlPath) + sc = new SparkContext("local-cluster[2,1,512]", "test", conf) testCount() testTake() // Make sure we can still launch tasks. diff --git a/core/src/test/scala/org/apache/spark/SharedSparkContext.scala b/core/src/test/scala/org/apache/spark/SharedSparkContext.scala index 0b6511a80df1..3d2700b7e6be 100644 --- a/core/src/test/scala/org/apache/spark/SharedSparkContext.scala +++ b/core/src/test/scala/org/apache/spark/SharedSparkContext.scala @@ -30,7 +30,7 @@ trait SharedSparkContext extends BeforeAndAfterAll { self: Suite => var conf = new SparkConf(false) override def beforeAll() { - _sc = new SparkContext("local", "test", conf) + _sc = new SparkContext("local[4]", "test", conf) super.beforeAll() } diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index 58a96245a9b5..f57921b76831 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -35,19 +35,15 @@ abstract class ShuffleSuite extends FunSuite with Matchers with LocalSparkContex conf.set("spark.test.noStageRetry", "true") test("groupByKey without compression") { - try { - System.setProperty("spark.shuffle.compress", "false") - sc = new SparkContext("local", "test", conf) - val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1)), 4) - val groups = pairs.groupByKey(4).collect() - assert(groups.size === 2) - val valuesFor1 = groups.find(_._1 == 1).get._2 - assert(valuesFor1.toList.sorted === List(1, 2, 3)) - val valuesFor2 = groups.find(_._1 == 2).get._2 - assert(valuesFor2.toList.sorted === List(1)) - } finally { - System.setProperty("spark.shuffle.compress", "true") - } + val myConf = conf.clone().set("spark.shuffle.compress", "false") + sc = new SparkContext("local", "test", myConf) + val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1)), 4) + val groups = pairs.groupByKey(4).collect() + assert(groups.size === 2) + val valuesFor1 = groups.find(_._1 == 1).get._2 + assert(valuesFor1.toList.sorted === List(1, 2, 3)) + val valuesFor2 = groups.find(_._1 == 2).get._2 + assert(valuesFor2.toList.sorted === List(1)) } test("shuffle non-zero block size") { diff --git a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala index 5d018ea9868a..790976a5ac30 100644 --- a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala @@ -19,27 +19,20 @@ package org.apache.spark import org.scalatest.FunSuite import org.apache.spark.serializer.{KryoRegistrator, KryoSerializer} +import org.apache.spark.util.ResetSystemProperties import com.esotericsoftware.kryo.Kryo -class SparkConfSuite extends FunSuite with LocalSparkContext { +class SparkConfSuite extends FunSuite with LocalSparkContext with ResetSystemProperties { test("loading from system properties") { - try { - System.setProperty("spark.test.testProperty", "2") - val conf = new SparkConf() - assert(conf.get("spark.test.testProperty") === "2") - } finally { - System.clearProperty("spark.test.testProperty") - } + System.setProperty("spark.test.testProperty", "2") + val conf = new SparkConf() + assert(conf.get("spark.test.testProperty") === "2") } test("initializing without loading defaults") { - try { - System.setProperty("spark.test.testProperty", "2") - val conf = new SparkConf(false) - assert(!conf.contains("spark.test.testProperty")) - } finally { - System.clearProperty("spark.test.testProperty") - } + System.setProperty("spark.test.testProperty", "2") + val conf = new SparkConf(false) + assert(!conf.contains("spark.test.testProperty")) } test("named set methods") { @@ -117,23 +110,17 @@ class SparkConfSuite extends FunSuite with LocalSparkContext { test("nested property names") { // This wasn't supported by some external conf parsing libraries - try { - System.setProperty("spark.test.a", "a") - System.setProperty("spark.test.a.b", "a.b") - System.setProperty("spark.test.a.b.c", "a.b.c") - val conf = new SparkConf() - assert(conf.get("spark.test.a") === "a") - assert(conf.get("spark.test.a.b") === "a.b") - assert(conf.get("spark.test.a.b.c") === "a.b.c") - conf.set("spark.test.a.b", "A.B") - assert(conf.get("spark.test.a") === "a") - assert(conf.get("spark.test.a.b") === "A.B") - assert(conf.get("spark.test.a.b.c") === "a.b.c") - } finally { - System.clearProperty("spark.test.a") - System.clearProperty("spark.test.a.b") - System.clearProperty("spark.test.a.b.c") - } + System.setProperty("spark.test.a", "a") + System.setProperty("spark.test.a.b", "a.b") + System.setProperty("spark.test.a.b.c", "a.b.c") + val conf = new SparkConf() + assert(conf.get("spark.test.a") === "a") + assert(conf.get("spark.test.a.b") === "a.b") + assert(conf.get("spark.test.a.b.c") === "a.b.c") + conf.set("spark.test.a.b", "A.B") + assert(conf.get("spark.test.a") === "a") + assert(conf.get("spark.test.a.b") === "A.B") + assert(conf.get("spark.test.a.b.c") === "a.b.c") } test("register kryo classes through registerKryoClasses") { diff --git a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala index 0390a2e4f1db..8ae4f243ec1a 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala @@ -27,10 +27,13 @@ import org.apache.spark.scheduler.local.LocalBackend class SparkContextSchedulerCreationSuite extends FunSuite with LocalSparkContext with PrivateMethodTester with Logging { - def createTaskScheduler(master: String): TaskSchedulerImpl = { + def createTaskScheduler(master: String): TaskSchedulerImpl = + createTaskScheduler(master, new SparkConf()) + + def createTaskScheduler(master: String, conf: SparkConf): TaskSchedulerImpl = { // Create local SparkContext to setup a SparkEnv. We don't actually want to start() the // real schedulers, so we don't want to create a full SparkContext with the desired scheduler. - sc = new SparkContext("local", "test") + sc = new SparkContext("local", "test", conf) val createTaskSchedulerMethod = PrivateMethod[Tuple2[SchedulerBackend, TaskScheduler]]('createTaskScheduler) val (_, sched) = SparkContext invokePrivate createTaskSchedulerMethod(sc, master) @@ -102,19 +105,13 @@ class SparkContextSchedulerCreationSuite } test("local-default-parallelism") { - val defaultParallelism = System.getProperty("spark.default.parallelism") - System.setProperty("spark.default.parallelism", "16") - val sched = createTaskScheduler("local") + val conf = new SparkConf().set("spark.default.parallelism", "16") + val sched = createTaskScheduler("local", conf) sched.backend match { case s: LocalBackend => assert(s.defaultParallelism() === 16) case _ => fail() } - - Option(defaultParallelism) match { - case Some(v) => System.setProperty("spark.default.parallelism", v) - case _ => System.clearProperty("spark.default.parallelism") - } } test("simr") { @@ -155,9 +152,10 @@ class SparkContextSchedulerCreationSuite testYarn("yarn-client", "org.apache.spark.scheduler.cluster.YarnClientClusterScheduler") } - def testMesos(master: String, expectedClass: Class[_]) { + def testMesos(master: String, expectedClass: Class[_], coarse: Boolean) { + val conf = new SparkConf().set("spark.mesos.coarse", coarse.toString) try { - val sched = createTaskScheduler(master) + val sched = createTaskScheduler(master, conf) assert(sched.backend.getClass === expectedClass) } catch { case e: UnsatisfiedLinkError => @@ -168,17 +166,14 @@ class SparkContextSchedulerCreationSuite } test("mesos fine-grained") { - System.setProperty("spark.mesos.coarse", "false") - testMesos("mesos://localhost:1234", classOf[MesosSchedulerBackend]) + testMesos("mesos://localhost:1234", classOf[MesosSchedulerBackend], coarse = false) } test("mesos coarse-grained") { - System.setProperty("spark.mesos.coarse", "true") - testMesos("mesos://localhost:1234", classOf[CoarseMesosSchedulerBackend]) + testMesos("mesos://localhost:1234", classOf[CoarseMesosSchedulerBackend], coarse = true) } test("mesos with zookeeper") { - System.setProperty("spark.mesos.coarse", "false") - testMesos("zk://localhost:1234,localhost:2345", classOf[MesosSchedulerBackend]) + testMesos("zk://localhost:1234,localhost:2345", classOf[MesosSchedulerBackend], coarse = false) } } diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index 136202210419..8b3c6871a7b3 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -23,55 +23,37 @@ import org.apache.hadoop.io.BytesWritable class SparkContextSuite extends FunSuite with LocalSparkContext { - /** Allows system properties to be changed in tests */ - private def withSystemProperty[T](property: String, value: String)(block: => T): T = { - val originalValue = System.getProperty(property) - try { - System.setProperty(property, value) - block - } finally { - if (originalValue == null) { - System.clearProperty(property) - } else { - System.setProperty(property, originalValue) - } - } - } - test("Only one SparkContext may be active at a time") { // Regression test for SPARK-4180 - withSystemProperty("spark.driver.allowMultipleContexts", "false") { - val conf = new SparkConf().setAppName("test").setMaster("local") - sc = new SparkContext(conf) - // A SparkContext is already running, so we shouldn't be able to create a second one - intercept[SparkException] { new SparkContext(conf) } - // After stopping the running context, we should be able to create a new one - resetSparkContext() - sc = new SparkContext(conf) - } + val conf = new SparkConf().setAppName("test").setMaster("local") + .set("spark.driver.allowMultipleContexts", "false") + sc = new SparkContext(conf) + // A SparkContext is already running, so we shouldn't be able to create a second one + intercept[SparkException] { new SparkContext(conf) } + // After stopping the running context, we should be able to create a new one + resetSparkContext() + sc = new SparkContext(conf) } test("Can still construct a new SparkContext after failing to construct a previous one") { - withSystemProperty("spark.driver.allowMultipleContexts", "false") { - // This is an invalid configuration (no app name or master URL) - intercept[SparkException] { - new SparkContext(new SparkConf()) - } - // Even though those earlier calls failed, we should still be able to create a new context - sc = new SparkContext(new SparkConf().setMaster("local").setAppName("test")) + val conf = new SparkConf().set("spark.driver.allowMultipleContexts", "false") + // This is an invalid configuration (no app name or master URL) + intercept[SparkException] { + new SparkContext(conf) } + // Even though those earlier calls failed, we should still be able to create a new context + sc = new SparkContext(conf.setMaster("local").setAppName("test")) } test("Check for multiple SparkContexts can be disabled via undocumented debug option") { - withSystemProperty("spark.driver.allowMultipleContexts", "true") { - var secondSparkContext: SparkContext = null - try { - val conf = new SparkConf().setAppName("test").setMaster("local") - sc = new SparkContext(conf) - secondSparkContext = new SparkContext(conf) - } finally { - Option(secondSparkContext).foreach(_.stop()) - } + var secondSparkContext: SparkContext = null + try { + val conf = new SparkConf().setAppName("test").setMaster("local") + .set("spark.driver.allowMultipleContexts", "true") + sc = new SparkContext(conf) + secondSparkContext = new SparkContext(conf) + } finally { + Option(secondSparkContext).foreach(_.stop()) } } diff --git a/core/src/test/scala/org/apache/spark/api/python/PythonBroadcastSuite.scala b/core/src/test/scala/org/apache/spark/api/python/PythonBroadcastSuite.scala new file mode 100644 index 000000000000..8959a843dbd7 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/api/python/PythonBroadcastSuite.scala @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.api.python + +import scala.io.Source + +import java.io.{PrintWriter, File} + +import org.scalatest.{Matchers, FunSuite} + +import org.apache.spark.{SharedSparkContext, SparkConf} +import org.apache.spark.serializer.KryoSerializer +import org.apache.spark.util.Utils + +// This test suite uses SharedSparkContext because we need a SparkEnv in order to deserialize +// a PythonBroadcast: +class PythonBroadcastSuite extends FunSuite with Matchers with SharedSparkContext { + test("PythonBroadcast can be serialized with Kryo (SPARK-4882)") { + val tempDir = Utils.createTempDir() + val broadcastedString = "Hello, world!" + def assertBroadcastIsValid(broadcast: PythonBroadcast): Unit = { + val source = Source.fromFile(broadcast.path) + val contents = source.mkString + source.close() + contents should be (broadcastedString) + } + try { + val broadcastDataFile: File = { + val file = new File(tempDir, "broadcastData") + val printWriter = new PrintWriter(file) + printWriter.write(broadcastedString) + printWriter.close() + file + } + val broadcast = new PythonBroadcast(broadcastDataFile.getAbsolutePath) + assertBroadcastIsValid(broadcast) + val conf = new SparkConf().set("spark.kryo.registrationRequired", "true") + val deserializedBroadcast = + Utils.clone[PythonBroadcast](broadcast, new KryoSerializer(conf).newInstance()) + assertBroadcastIsValid(deserializedBroadcast) + } finally { + Utils.deleteRecursively(tempDir) + } + } +} 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 eb7bd7ab3986..065b7534cece 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -23,11 +23,13 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark._ import org.apache.spark.deploy.SparkSubmit._ -import org.apache.spark.util.Utils +import org.apache.spark.util.{ResetSystemProperties, Utils} import org.scalatest.FunSuite import org.scalatest.Matchers -class SparkSubmitSuite extends FunSuite with Matchers { +// Note: this suite mixes in ResetSystemProperties because SparkSubmit.main() sets a bunch +// of properties that neeed to be cleared after tests. +class SparkSubmitSuite extends FunSuite with Matchers with ResetSystemProperties { def beforeAll() { System.setProperty("spark.testing", "true") } @@ -288,6 +290,7 @@ class SparkSubmitSuite extends FunSuite with Matchers { "--class", SimpleApplicationTest.getClass.getName.stripSuffix("$"), "--name", "testApp", "--master", "local", + "--conf", "spark.ui.enabled=false", unusedJar.toString) runSparkSubmit(args) } @@ -302,6 +305,7 @@ class SparkSubmitSuite extends FunSuite with Matchers { "--name", "testApp", "--master", "local-cluster[2,1,512]", "--jars", jarsString, + "--conf", "spark.ui.enabled=false", unusedJar.toString) runSparkSubmit(args) } diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala index d719e9301f4f..8379883e065e 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -64,7 +64,8 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers ) // Write an unfinished app, new-style. - writeFile(new File(testDir, "new2" + EventLoggingListener.IN_PROGRESS), true, None, + val logFile2 = new File(testDir, "new2" + EventLoggingListener.IN_PROGRESS) + writeFile(logFile2, true, None, SparkListenerApplicationStart("app2-2", None, 1L, "test") ) @@ -92,12 +93,17 @@ class FsHistoryProviderSuite extends FunSuite with BeforeAndAfter with Matchers val list = provider.getListing().toSeq list should not be (null) - list.size should be (2) + list.size should be (4) + list.count(e => e.completed) should be (2) list(0) should be (ApplicationHistoryInfo(oldLog.getName(), "app3", 2L, 3L, - oldLog.lastModified(), "test")) + oldLog.lastModified(), "test", true)) list(1) should be (ApplicationHistoryInfo(logFile1.getName(), "app1-1", 1L, 2L, - logFile1.lastModified(), "test")) + logFile1.lastModified(), "test", true)) + list(2) should be (ApplicationHistoryInfo(oldLog2.getName(), "app4", 2L, -1L, + oldLog2.lastModified(), "test", false)) + list(3) should be (ApplicationHistoryInfo(logFile2.getName(), "app2-2", 1L, -1L, + logFile2.lastModified(), "test", false)) // Make sure the UI can be rendered. list.foreach { case info => diff --git a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala new file mode 100644 index 000000000000..3d2335f9b363 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy.master + +import akka.actor.Address +import org.scalatest.FunSuite + +import org.apache.spark.SparkException + +class MasterSuite extends FunSuite { + + test("toAkkaUrl") { + val akkaUrl = Master.toAkkaUrl("spark://1.2.3.4:1234") + assert("akka.tcp://sparkMaster@1.2.3.4:1234/user/Master" === akkaUrl) + } + + test("toAkkaUrl: a typo url") { + val e = intercept[SparkException] { + Master.toAkkaUrl("spark://1.2. 3.4:1234") + } + assert("Invalid master URL: spark://1.2. 3.4:1234" === e.getMessage) + } + + test("toAkkaAddress") { + val address = Master.toAkkaAddress("spark://1.2.3.4:1234") + assert(Address("akka.tcp", "sparkMaster", "1.2.3.4", 1234) === address) + } + + test("toAkkaAddress: a typo url") { + val e = intercept[SparkException] { + Master.toAkkaAddress("spark://1.2. 3.4:1234") + } + assert("Invalid master URL: spark://1.2. 3.4:1234" === e.getMessage) + } +} diff --git a/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala b/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala index f8bcde12a371..10a39990f80c 100644 --- a/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala +++ b/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala @@ -17,66 +17,185 @@ package org.apache.spark.metrics -import java.io.{FileWriter, PrintWriter, File} +import java.io.{File, FileWriter, PrintWriter} -import org.apache.spark.SharedSparkContext -import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.scheduler.{SparkListenerTaskEnd, SparkListener} +import scala.collection.mutable.ArrayBuffer import org.scalatest.FunSuite -import org.scalatest.Matchers import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{Path, FileSystem} +import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.io.{LongWritable, Text} +import org.apache.hadoop.mapreduce.lib.input.{TextInputFormat => NewTextInputFormat} -import scala.collection.mutable.ArrayBuffer +import org.apache.spark.SharedSparkContext +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskEnd} +import org.apache.spark.util.Utils + +class InputOutputMetricsSuite extends FunSuite with SharedSparkContext { -class InputOutputMetricsSuite extends FunSuite with SharedSparkContext with Matchers { - test("input metrics when reading text file with single split") { - val file = new File(getClass.getSimpleName + ".txt") - val pw = new PrintWriter(new FileWriter(file)) - pw.println("some stuff") - pw.println("some other stuff") - pw.println("yet more stuff") - pw.println("too much stuff") + @transient var tmpDir: File = _ + @transient var tmpFile: File = _ + @transient var tmpFilePath: String = _ + + override def beforeAll() { + super.beforeAll() + + tmpDir = Utils.createTempDir() + val testTempDir = new File(tmpDir, "test") + testTempDir.mkdir() + + tmpFile = new File(testTempDir, getClass.getSimpleName + ".txt") + val pw = new PrintWriter(new FileWriter(tmpFile)) + for (x <- 1 to 1000000) { + pw.println("s") + } pw.close() - file.deleteOnExit() - val taskBytesRead = new ArrayBuffer[Long]() - sc.addSparkListener(new SparkListener() { - override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { - taskBytesRead += taskEnd.taskMetrics.inputMetrics.get.bytesRead - } - }) - sc.textFile("file://" + file.getAbsolutePath, 2).count() + // Path to tmpFile + tmpFilePath = "file://" + tmpFile.getAbsolutePath + } - // Wait for task end events to come in - sc.listenerBus.waitUntilEmpty(500) - assert(taskBytesRead.length == 2) - assert(taskBytesRead.sum >= file.length()) + override def afterAll() { + super.afterAll() + Utils.deleteRecursively(tmpDir) } - test("input metrics when reading text file with multiple splits") { - val file = new File(getClass.getSimpleName + ".txt") - val pw = new PrintWriter(new FileWriter(file)) - for (i <- 0 until 10000) { - pw.println("some stuff") + test("input metrics for old hadoop with coalesce") { + val bytesRead = runAndReturnBytesRead { + sc.textFile(tmpFilePath, 4).count() + } + val bytesRead2 = runAndReturnBytesRead { + sc.textFile(tmpFilePath, 4).coalesce(2).count() + } + assert(bytesRead != 0) + assert(bytesRead == bytesRead2) + assert(bytesRead2 >= tmpFile.length()) + } + + test("input metrics with cache and coalesce") { + // prime the cache manager + val rdd = sc.textFile(tmpFilePath, 4).cache() + rdd.collect() + + val bytesRead = runAndReturnBytesRead { + rdd.count() + } + val bytesRead2 = runAndReturnBytesRead { + rdd.coalesce(4).count() } - pw.close() - file.deleteOnExit() + // for count and coelesce, the same bytes should be read. + assert(bytesRead != 0) + assert(bytesRead2 == bytesRead) + } + + /** + * This checks the situation where we have interleaved reads from + * different sources. Currently, we only accumulate fron the first + * read method we find in the task. This test uses cartesian to create + * the interleaved reads. + * + * Once https://issues.apache.org/jira/browse/SPARK-5225 is fixed + * this test should break. + */ + test("input metrics with mixed read method") { + // prime the cache manager + val numPartitions = 2 + val rdd = sc.parallelize(1 to 100, numPartitions).cache() + rdd.collect() + + val rdd2 = sc.textFile(tmpFilePath, numPartitions) + + val bytesRead = runAndReturnBytesRead { + rdd.count() + } + val bytesRead2 = runAndReturnBytesRead { + rdd2.count() + } + + val cartRead = runAndReturnBytesRead { + rdd.cartesian(rdd2).count() + } + + assert(cartRead != 0) + assert(bytesRead != 0) + // We read from the first rdd of the cartesian once per partition. + assert(cartRead == bytesRead * numPartitions) + } + + test("input metrics for new Hadoop API with coalesce") { + val bytesRead = runAndReturnBytesRead { + sc.newAPIHadoopFile(tmpFilePath, classOf[NewTextInputFormat], classOf[LongWritable], + classOf[Text]).count() + } + val bytesRead2 = runAndReturnBytesRead { + sc.newAPIHadoopFile(tmpFilePath, classOf[NewTextInputFormat], classOf[LongWritable], + classOf[Text]).coalesce(5).count() + } + assert(bytesRead != 0) + assert(bytesRead2 == bytesRead) + assert(bytesRead >= tmpFile.length()) + } + + test("input metrics when reading text file") { + val bytesRead = runAndReturnBytesRead { + sc.textFile(tmpFilePath, 2).count() + } + assert(bytesRead >= tmpFile.length()) + } + + test("input metrics with interleaved reads") { + val numPartitions = 2 + val cartVector = 0 to 9 + val cartFile = new File(tmpDir, getClass.getSimpleName + "_cart.txt") + val cartFilePath = "file://" + cartFile.getAbsolutePath + + // write files to disk so we can read them later. + sc.parallelize(cartVector).saveAsTextFile(cartFilePath) + val aRdd = sc.textFile(cartFilePath, numPartitions) + + val tmpRdd = sc.textFile(tmpFilePath, numPartitions) + + val firstSize= runAndReturnBytesRead { + aRdd.count() + } + val secondSize = runAndReturnBytesRead { + tmpRdd.count() + } + + val cartesianBytes = runAndReturnBytesRead { + aRdd.cartesian(tmpRdd).count() + } + + // Computing the amount of bytes read for a cartesian operation is a little involved. + // Cartesian interleaves reads between two partitions eg. p1 and p2. + // Here are the steps: + // 1) First it creates an iterator for p1 + // 2) Creates an iterator for p2 + // 3) Reads the first element of p1 and then all the elements of p2 + // 4) proceeds to the next element of p1 + // 5) Creates a new iterator for p2 + // 6) rinse and repeat. + // As a result we read from the second partition n times where n is the number of keys in + // p1. Thus the math below for the test. + assert(cartesianBytes != 0) + assert(cartesianBytes == firstSize * numPartitions + (cartVector.length * secondSize)) + } + + private def runAndReturnBytesRead(job : => Unit): Long = { val taskBytesRead = new ArrayBuffer[Long]() sc.addSparkListener(new SparkListener() { override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { taskBytesRead += taskEnd.taskMetrics.inputMetrics.get.bytesRead } }) - sc.textFile("file://" + file.getAbsolutePath, 2).count() - // Wait for task end events to come in + job + sc.listenerBus.waitUntilEmpty(500) - assert(taskBytesRead.length == 2) - assert(taskBytesRead.sum >= file.length()) + taskBytesRead.sum } test("output metrics when writing text file") { diff --git a/core/src/test/scala/org/apache/spark/rdd/ParallelCollectionSplitSuite.scala b/core/src/test/scala/org/apache/spark/rdd/ParallelCollectionSplitSuite.scala index 1b112f1a41ca..cd193ae4f523 100644 --- a/core/src/test/scala/org/apache/spark/rdd/ParallelCollectionSplitSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/ParallelCollectionSplitSuite.scala @@ -76,6 +76,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { assert(slices(0).mkString(",") === (0 to 32).mkString(",")) assert(slices(1).mkString(",") === (33 to 66).mkString(",")) assert(slices(2).mkString(",") === (67 to 100).mkString(",")) + assert(slices(2).isInstanceOf[Range.Inclusive]) } test("empty data") { @@ -227,4 +228,28 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { assert(slices.map(_.size).reduceLeft(_+_) === 100) assert(slices.forall(_.isInstanceOf[NumericRange[_]])) } + + test("inclusive ranges with Int.MaxValue and Int.MinValue") { + val data1 = 1 to Int.MaxValue + val slices1 = ParallelCollectionRDD.slice(data1, 3) + assert(slices1.size === 3) + assert(slices1.map(_.size).sum === Int.MaxValue) + assert(slices1(2).isInstanceOf[Range.Inclusive]) + val data2 = -2 to Int.MinValue by -1 + val slices2 = ParallelCollectionRDD.slice(data2, 3) + assert(slices2.size == 3) + assert(slices2.map(_.size).sum === Int.MaxValue) + assert(slices2(2).isInstanceOf[Range.Inclusive]) + } + + test("empty ranges with Int.MaxValue and Int.MinValue") { + val data1 = Int.MaxValue until Int.MaxValue + val slices1 = ParallelCollectionRDD.slice(data1, 5) + assert(slices1.size === 5) + for (i <- 0 until 5) assert(slices1(i).size === 0) + val data2 = Int.MaxValue until Int.MaxValue + val slices2 = ParallelCollectionRDD.slice(data2, 5) + assert(slices2.size === 5) + for (i <- 0 until 5) assert(slices2(i).size === 0) + } } diff --git a/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala index 271a90c6646b..1a9a0e857e54 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala @@ -174,7 +174,7 @@ class PipedRDDSuite extends FunSuite with SharedSparkContext { } val hadoopPart1 = generateFakeHadoopPartition() val pipedRdd = new PipedRDD(nums, "printenv " + varName) - val tContext = new TaskContextImpl(0, 0, 0) + val tContext = new TaskContextImpl(0, 0, 0, 0) val rddIter = pipedRdd.compute(hadoopPart1, tContext) val arr = rddIter.toArray assert(arr(0) == "/some/path") 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 6836e9ab0fd6..381ee2d45630 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -17,6 +17,10 @@ package org.apache.spark.rdd +import java.io.{ObjectInputStream, ObjectOutputStream, IOException} + +import com.esotericsoftware.kryo.KryoException + import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.collection.JavaConverters._ import scala.reflect.ClassTag @@ -48,6 +52,7 @@ class RDDSuite extends FunSuite with SharedSparkContext { assert(nums.glom().map(_.toList).collect().toList === List(List(1, 2), List(3, 4))) assert(nums.collect({ case i if i >= 3 => i.toString }).collect().toList === List("3", "4")) assert(nums.keyBy(_.toString).collect().toList === List(("1", 1), ("2", 2), ("3", 3), ("4", 4))) + assert(!nums.isEmpty()) assert(nums.max() === 4) assert(nums.min() === 1) val partitionSums = nums.mapPartitions(iter => Iterator(iter.reduceLeft(_ + _))) @@ -541,6 +546,14 @@ class RDDSuite extends FunSuite with SharedSparkContext { assert(sortedTopK === nums.sorted(ord).take(5)) } + test("isEmpty") { + assert(sc.emptyRDD.isEmpty()) + assert(sc.parallelize(Seq[Int]()).isEmpty()) + assert(!sc.parallelize(Seq(1)).isEmpty()) + assert(sc.parallelize(Seq(1,2,3), 3).filter(_ < 0).isEmpty()) + assert(!sc.parallelize(Seq(1,2,3), 3).filter(_ > 1).isEmpty()) + } + test("sample preserves partitioner") { val partitioner = new HashPartitioner(2) val rdd = sc.parallelize(Seq((0, 1), (2, 3))).partitionBy(partitioner) @@ -887,6 +900,23 @@ class RDDSuite extends FunSuite with SharedSparkContext { assert(ancestors6.count(_.isInstanceOf[CyclicalDependencyRDD[_]]) === 3) } + test("task serialization exception should not hang scheduler") { + class BadSerializable extends Serializable { + @throws(classOf[IOException]) + private def writeObject(out: ObjectOutputStream): Unit = throw new KryoException("Bad serialization") + + @throws(classOf[IOException]) + private def readObject(in: ObjectInputStream): Unit = {} + } + // Note that in the original bug, SPARK-4349, that this verifies, the job would only hang if there were + // more threads in the Spark Context than there were number of objects in this sequence. + intercept[Throwable] { + sc.parallelize(Seq(new BadSerializable, new BadSerializable)).collect + } + // Check that the context has not crashed + sc.parallelize(1 to 100).map(x => x*2).collect + } + /** A contrived RDD that allows the manual addition of dependencies after creation. */ private class CyclicalDependencyRDD[T: ClassTag] extends RDD[T](sc, Nil) { private val mutableDependencies: ArrayBuffer[Dependency[_]] = ArrayBuffer.empty 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 d6ec9e129cce..eb116213f69f 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -19,9 +19,8 @@ package org.apache.spark.scheduler import scala.collection.mutable.{ArrayBuffer, HashSet, HashMap, Map} import scala.language.reflectiveCalls +import scala.util.control.NonFatal -import akka.actor._ -import akka.testkit.{ImplicitSender, TestKit, TestActorRef} import org.scalatest.{BeforeAndAfter, FunSuiteLike} import org.scalatest.concurrent.Timeouts import org.scalatest.time.SpanSugar._ @@ -33,10 +32,16 @@ import org.apache.spark.storage.{BlockId, BlockManagerId, BlockManagerMaster} import org.apache.spark.util.CallSite import org.apache.spark.executor.TaskMetrics -class BuggyDAGEventProcessActor extends Actor { - val state = 0 - def receive = { - case _ => throw new SparkException("error") +class DAGSchedulerEventProcessLoopTester(dagScheduler: DAGScheduler) + extends DAGSchedulerEventProcessLoop(dagScheduler) { + + override def post(event: DAGSchedulerEvent): Unit = { + try { + // Forward event to `onReceive` directly to avoid processing event asynchronously. + onReceive(event) + } catch { + case NonFatal(e) => onError(e) + } } } @@ -65,8 +70,7 @@ class MyRDD( class DAGSchedulerSuiteDummyException extends Exception -class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with FunSuiteLike - with ImplicitSender with BeforeAndAfter with LocalSparkContext with Timeouts { +class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSparkContext with Timeouts { val conf = new SparkConf /** Set of TaskSets the DAGScheduler has requested executed. */ @@ -113,7 +117,7 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F var mapOutputTracker: MapOutputTrackerMaster = null var scheduler: DAGScheduler = null - var dagEventProcessTestActor: TestActorRef[DAGSchedulerEventProcessActor] = null + var dagEventProcessLoopTester: DAGSchedulerEventProcessLoop = null /** * Set of cache locations to return from our mock BlockManagerMaster. @@ -167,13 +171,11 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F runLocallyWithinThread(job) } } - dagEventProcessTestActor = TestActorRef[DAGSchedulerEventProcessActor]( - Props(classOf[DAGSchedulerEventProcessActor], scheduler))(system) + dagEventProcessLoopTester = new DAGSchedulerEventProcessLoopTester(scheduler) } override def afterAll() { super.afterAll() - TestKit.shutdownActorSystem(system) } /** @@ -190,7 +192,7 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F * DAGScheduler event loop. */ private def runEvent(event: DAGSchedulerEvent) { - dagEventProcessTestActor.receive(event) + dagEventProcessLoopTester.post(event) } /** @@ -247,6 +249,7 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F test("[SPARK-3353] parent stage should have lower stage id") { sparkListener.stageByOrderOfExecution.clear() sc.parallelize(1 to 10).map(x => (x, x)).reduceByKey(_ + _, 4).count() + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) assert(sparkListener.stageByOrderOfExecution.length === 2) assert(sparkListener.stageByOrderOfExecution(0) < sparkListener.stageByOrderOfExecution(1)) } @@ -396,8 +399,7 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F runLocallyWithinThread(job) } } - dagEventProcessTestActor = TestActorRef[DAGSchedulerEventProcessActor]( - Props(classOf[DAGSchedulerEventProcessActor], noKillScheduler))(system) + dagEventProcessLoopTester = new DAGSchedulerEventProcessLoopTester(noKillScheduler) val jobId = submit(new MyRDD(sc, 1, Nil), Array(0)) cancel(jobId) // Because the job wasn't actually cancelled, we shouldn't have received a failure message. @@ -725,18 +727,6 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F assert(sc.parallelize(1 to 10, 2).first() === 1) } - test("DAGSchedulerActorSupervisor closes the SparkContext when EventProcessActor crashes") { - val actorSystem = ActorSystem("test") - val supervisor = actorSystem.actorOf( - Props(classOf[DAGSchedulerActorSupervisor], scheduler), "dagSupervisor") - supervisor ! Props[BuggyDAGEventProcessActor] - val child = expectMsgType[ActorRef] - watch(child) - child ! "hi" - expectMsgPF(){ case Terminated(child) => () } - assert(scheduler.sc.dagScheduler === null) - } - test("accumulator not calculated for resubmitted result stage") { //just for register val accum = new Accumulator[Int](0, AccumulatorParam.IntAccumulatorParam) 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 1de7e130039a..437d8693c0b1 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -160,7 +160,7 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter with Loggin */ private def testApplicationEventLogging(compressionCodec: Option[String] = None) { val conf = getLoggingConf(testDirPath, compressionCodec) - val sc = new SparkContext("local", "test", conf) + val sc = new SparkContext("local-cluster[2,2,512]", "test", conf) assert(sc.eventLogger.isDefined) val eventLogger = sc.eventLogger.get val expectedLogDir = testDir.toURI().toString() @@ -184,6 +184,7 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter with Loggin val eventSet = mutable.Set( SparkListenerApplicationStart, SparkListenerBlockManagerAdded, + SparkListenerExecutorAdded, SparkListenerEnvironmentUpdate, SparkListenerJobStart, SparkListenerJobEnd, diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/ByteType.java b/core/src/test/scala/org/apache/spark/scheduler/NotSerializableFakeTask.scala similarity index 52% rename from sql/core/src/main/java/org/apache/spark/sql/api/java/ByteType.java rename to core/src/test/scala/org/apache/spark/scheduler/NotSerializableFakeTask.scala index e5cdf06b21bb..6b75c98839e0 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/ByteType.java +++ b/core/src/test/scala/org/apache/spark/scheduler/NotSerializableFakeTask.scala @@ -15,13 +15,26 @@ * limitations under the License. */ -package org.apache.spark.sql.api.java; +package org.apache.spark.scheduler + +import java.io.{ObjectInputStream, ObjectOutputStream, IOException} + +import org.apache.spark.TaskContext /** - * The data type representing byte and Byte values. - * - * {@code ByteType} is represented by the singleton object {@link DataType#ByteType}. + * A Task implementation that fails to serialize. */ -public class ByteType extends DataType { - protected ByteType() {} +private[spark] class NotSerializableFakeTask(myId: Int, stageId: Int) extends Task[Array[Byte]](stageId, 0) { + override def runTask(context: TaskContext): Array[Byte] = Array.empty[Byte] + override def preferredLocations: Seq[TaskLocation] = Seq[TaskLocation]() + + @throws(classOf[IOException]) + private def writeObject(out: ObjectOutputStream): Unit = { + if (stageId == 0) { + throw new IllegalStateException("Cannot serialize") + } + } + + @throws(classOf[IOException]) + private def readObject(in: ObjectInputStream): Unit = {} } diff --git a/core/src/test/scala/org/apache/spark/scheduler/PoolSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/PoolSuite.scala new file mode 100644 index 000000000000..e8f461e2f56c --- /dev/null +++ b/core/src/test/scala/org/apache/spark/scheduler/PoolSuite.scala @@ -0,0 +1,183 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.scheduler + +import java.util.Properties + +import org.scalatest.FunSuite + +import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext} + +/** + * Tests that pools and the associated scheduling algorithms for FIFO and fair scheduling work + * correctly. + */ +class PoolSuite extends FunSuite with LocalSparkContext { + + def createTaskSetManager(stageId: Int, numTasks: Int, taskScheduler: TaskSchedulerImpl) + : TaskSetManager = { + val tasks = Array.tabulate[Task[_]](numTasks) { i => + new FakeTask(i, Nil) + } + new TaskSetManager(taskScheduler, new TaskSet(tasks, stageId, 0, 0, null), 0) + } + + def scheduleTaskAndVerifyId(taskId: Int, rootPool: Pool, expectedStageId: Int) { + val taskSetQueue = rootPool.getSortedTaskSetQueue + val nextTaskSetToSchedule = + taskSetQueue.find(t => (t.runningTasks + t.tasksSuccessful) < t.numTasks) + assert(nextTaskSetToSchedule.isDefined) + nextTaskSetToSchedule.get.addRunningTask(taskId) + assert(nextTaskSetToSchedule.get.stageId === expectedStageId) + } + + test("FIFO Scheduler Test") { + sc = new SparkContext("local", "TaskSchedulerImplSuite") + val taskScheduler = new TaskSchedulerImpl(sc) + + val rootPool = new Pool("", SchedulingMode.FIFO, 0, 0) + val schedulableBuilder = new FIFOSchedulableBuilder(rootPool) + schedulableBuilder.buildPools() + + val taskSetManager0 = createTaskSetManager(0, 2, taskScheduler) + val taskSetManager1 = createTaskSetManager(1, 2, taskScheduler) + val taskSetManager2 = createTaskSetManager(2, 2, taskScheduler) + schedulableBuilder.addTaskSetManager(taskSetManager0, null) + schedulableBuilder.addTaskSetManager(taskSetManager1, null) + schedulableBuilder.addTaskSetManager(taskSetManager2, null) + + scheduleTaskAndVerifyId(0, rootPool, 0) + scheduleTaskAndVerifyId(1, rootPool, 0) + scheduleTaskAndVerifyId(2, rootPool, 1) + scheduleTaskAndVerifyId(3, rootPool, 1) + scheduleTaskAndVerifyId(4, rootPool, 2) + scheduleTaskAndVerifyId(5, rootPool, 2) + } + + /** + * This test creates three scheduling pools, and creates task set managers in the first + * two scheduling pools. The test verifies that as tasks are scheduled, the fair scheduling + * algorithm properly orders the two scheduling pools. + */ + test("Fair Scheduler Test") { + val xmlPath = getClass.getClassLoader.getResource("fairscheduler.xml").getFile() + val conf = new SparkConf().set("spark.scheduler.allocation.file", xmlPath) + sc = new SparkContext("local", "TaskSchedulerImplSuite", conf) + val taskScheduler = new TaskSchedulerImpl(sc) + + val rootPool = new Pool("", SchedulingMode.FAIR, 0, 0) + val schedulableBuilder = new FairSchedulableBuilder(rootPool, sc.conf) + schedulableBuilder.buildPools() + + // Ensure that the XML file was read in correctly. + assert(rootPool.getSchedulableByName("default") != null) + assert(rootPool.getSchedulableByName("1") != null) + assert(rootPool.getSchedulableByName("2") != null) + assert(rootPool.getSchedulableByName("3") != null) + assert(rootPool.getSchedulableByName("1").minShare === 2) + assert(rootPool.getSchedulableByName("1").weight === 1) + assert(rootPool.getSchedulableByName("2").minShare === 3) + assert(rootPool.getSchedulableByName("2").weight === 1) + assert(rootPool.getSchedulableByName("3").minShare === 0) + assert(rootPool.getSchedulableByName("3").weight === 1) + + val properties1 = new Properties() + properties1.setProperty("spark.scheduler.pool","1") + val properties2 = new Properties() + properties2.setProperty("spark.scheduler.pool","2") + + val taskSetManager10 = createTaskSetManager(0, 1, taskScheduler) + val taskSetManager11 = createTaskSetManager(1, 1, taskScheduler) + val taskSetManager12 = createTaskSetManager(2, 2, taskScheduler) + schedulableBuilder.addTaskSetManager(taskSetManager10, properties1) + schedulableBuilder.addTaskSetManager(taskSetManager11, properties1) + schedulableBuilder.addTaskSetManager(taskSetManager12, properties1) + + val taskSetManager23 = createTaskSetManager(3, 2, taskScheduler) + val taskSetManager24 = createTaskSetManager(4, 2, taskScheduler) + schedulableBuilder.addTaskSetManager(taskSetManager23, properties2) + schedulableBuilder.addTaskSetManager(taskSetManager24, properties2) + + // Pool 1 share ratio: 0. Pool 2 share ratio: 0. 1 gets scheduled based on ordering of names. + scheduleTaskAndVerifyId(0, rootPool, 0) + // Pool 1 share ratio: 1/2. Pool 2 share ratio: 0. 2 gets scheduled because ratio is lower. + scheduleTaskAndVerifyId(1, rootPool, 3) + // Pool 1 share ratio: 1/2. Pool 2 share ratio: 1/3. 2 gets scheduled because ratio is lower. + scheduleTaskAndVerifyId(2, rootPool, 3) + // Pool 1 share ratio: 1/2. Pool 2 share ratio: 2/3. 1 gets scheduled because ratio is lower. + scheduleTaskAndVerifyId(3, rootPool, 1) + // Pool 1 share ratio: 1. Pool 2 share ratio: 2/3. 2 gets scheduled because ratio is lower. + scheduleTaskAndVerifyId(4, rootPool, 4) + // Neither pool is needy so ordering is based on number of running tasks. + // Pool 1 running tasks: 2, Pool 2 running tasks: 3. 1 gets scheduled because fewer running + // tasks. + scheduleTaskAndVerifyId(5, rootPool, 2) + // Pool 1 running tasks: 3, Pool 2 running tasks: 3. 1 gets scheduled because of naming + // ordering. + scheduleTaskAndVerifyId(6, rootPool, 2) + // Pool 1 running tasks: 4, Pool 2 running tasks: 3. 2 gets scheduled because fewer running + // tasks. + scheduleTaskAndVerifyId(7, rootPool, 4) + } + + test("Nested Pool Test") { + sc = new SparkContext("local", "TaskSchedulerImplSuite") + val taskScheduler = new TaskSchedulerImpl(sc) + + val rootPool = new Pool("", SchedulingMode.FAIR, 0, 0) + val pool0 = new Pool("0", SchedulingMode.FAIR, 3, 1) + val pool1 = new Pool("1", SchedulingMode.FAIR, 4, 1) + rootPool.addSchedulable(pool0) + rootPool.addSchedulable(pool1) + + val pool00 = new Pool("00", SchedulingMode.FAIR, 2, 2) + val pool01 = new Pool("01", SchedulingMode.FAIR, 1, 1) + pool0.addSchedulable(pool00) + pool0.addSchedulable(pool01) + + val pool10 = new Pool("10", SchedulingMode.FAIR, 2, 2) + val pool11 = new Pool("11", SchedulingMode.FAIR, 2, 1) + pool1.addSchedulable(pool10) + pool1.addSchedulable(pool11) + + val taskSetManager000 = createTaskSetManager(0, 5, taskScheduler) + val taskSetManager001 = createTaskSetManager(1, 5, taskScheduler) + pool00.addSchedulable(taskSetManager000) + pool00.addSchedulable(taskSetManager001) + + val taskSetManager010 = createTaskSetManager(2, 5, taskScheduler) + val taskSetManager011 = createTaskSetManager(3, 5, taskScheduler) + pool01.addSchedulable(taskSetManager010) + pool01.addSchedulable(taskSetManager011) + + val taskSetManager100 = createTaskSetManager(4, 5, taskScheduler) + val taskSetManager101 = createTaskSetManager(5, 5, taskScheduler) + pool10.addSchedulable(taskSetManager100) + pool10.addSchedulable(taskSetManager101) + + val taskSetManager110 = createTaskSetManager(6, 5, taskScheduler) + val taskSetManager111 = createTaskSetManager(7, 5, taskScheduler) + pool11.addSchedulable(taskSetManager110) + pool11.addSchedulable(taskSetManager111) + + scheduleTaskAndVerifyId(0, rootPool, 0) + scheduleTaskAndVerifyId(1, rootPool, 4) + scheduleTaskAndVerifyId(2, rootPool, 6) + scheduleTaskAndVerifyId(3, rootPool, 2) + } +} diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala index b276343cb412..0fb1bdd30d97 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -26,28 +26,27 @@ import org.scalatest.Matchers import org.apache.spark.{LocalSparkContext, SparkContext} import org.apache.spark.executor.TaskMetrics +import org.apache.spark.util.ResetSystemProperties -class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers - with BeforeAndAfter with BeforeAndAfterAll { +class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers with BeforeAndAfter + with BeforeAndAfterAll with ResetSystemProperties { /** Length of time to wait while draining listener events. */ val WAIT_TIMEOUT_MILLIS = 10000 + val jobCompletionTime = 1421191296660L + before { sc = new SparkContext("local", "SparkListenerSuite") } - override def afterAll() { - System.clearProperty("spark.akka.frameSize") - } - test("basic creation and shutdown of LiveListenerBus") { val counter = new BasicJobCounter val bus = new LiveListenerBus bus.addListener(counter) // Listener bus hasn't started yet, so posting events should not increment counter - (1 to 5).foreach { _ => bus.post(SparkListenerJobEnd(0, JobSucceeded)) } + (1 to 5).foreach { _ => bus.post(SparkListenerJobEnd(0, jobCompletionTime, JobSucceeded)) } assert(counter.count === 0) // Starting listener bus should flush all buffered events @@ -57,7 +56,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers // After listener bus has stopped, posting events should not increment counter bus.stop() - (1 to 5).foreach { _ => bus.post(SparkListenerJobEnd(0, JobSucceeded)) } + (1 to 5).foreach { _ => bus.post(SparkListenerJobEnd(0, jobCompletionTime, JobSucceeded)) } assert(counter.count === 5) // Listener bus must not be started twice @@ -102,7 +101,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers bus.addListener(blockingListener) bus.start() - bus.post(SparkListenerJobEnd(0, JobSucceeded)) + bus.post(SparkListenerJobEnd(0, jobCompletionTime, JobSucceeded)) listenerStarted.acquire() // Listener should be blocked after start @@ -348,7 +347,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers bus.start() // Post events to all listeners, and wait until the queue is drained - (1 to 5).foreach { _ => bus.post(SparkListenerJobEnd(0, JobSucceeded)) } + (1 to 5).foreach { _ => bus.post(SparkListenerJobEnd(0, jobCompletionTime, JobSucceeded)) } assert(bus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) // The exception should be caught, and the event should be propagated to other listeners diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerWithClusterSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerWithClusterSuite.scala new file mode 100644 index 000000000000..623a687c359a --- /dev/null +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerWithClusterSuite.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.scheduler + +import org.apache.spark.scheduler.cluster.ExecutorInfo +import org.apache.spark.{SparkContext, LocalSparkContext} + +import org.scalatest.{FunSuite, BeforeAndAfter, BeforeAndAfterAll} + +import scala.collection.mutable + +/** + * Unit tests for SparkListener that require a local cluster. + */ +class SparkListenerWithClusterSuite extends FunSuite with LocalSparkContext + with BeforeAndAfter with BeforeAndAfterAll { + + /** Length of time to wait while draining listener events. */ + val WAIT_TIMEOUT_MILLIS = 10000 + + before { + sc = new SparkContext("local-cluster[2,1,512]", "SparkListenerSuite") + } + + test("SparkListener sends executor added message") { + val listener = new SaveExecutorInfo + sc.addSparkListener(listener) + + val rdd1 = sc.parallelize(1 to 100, 4) + val rdd2 = rdd1.map(_.toString) + rdd2.setName("Target RDD") + rdd2.count() + + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + assert(listener.addedExecutorInfo.size == 2) + assert(listener.addedExecutorInfo("0").totalCores == 1) + assert(listener.addedExecutorInfo("1").totalCores == 1) + } + + private class SaveExecutorInfo extends SparkListener { + val addedExecutorInfo = mutable.Map[String, ExecutorInfo]() + + override def onExecutorAdded(executor: SparkListenerExecutorAdded) { + addedExecutorInfo(executor.executorId) = executor.executorInfo + } + } +} diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala index 561a5e9cd90c..057e22691602 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala @@ -45,13 +45,13 @@ class TaskContextSuite extends FunSuite with BeforeAndAfter with LocalSparkConte val task = new ResultTask[String, String]( 0, sc.broadcast(closureSerializer.serialize((rdd, func)).array), rdd.partitions(0), Seq(), 0) intercept[RuntimeException] { - task.run(0) + task.run(0, 0) } assert(TaskContextSuite.completed === true) } test("all TaskCompletionListeners should be called even if some fail") { - val context = new TaskContextImpl(0, 0, 0) + val context = new TaskContextImpl(0, 0, 0, 0) val listener = mock(classOf[TaskCompletionListener]) context.addTaskCompletionListener(_ => throw new Exception("blah")) context.addTaskCompletionListener(listener) @@ -63,6 +63,33 @@ class TaskContextSuite extends FunSuite with BeforeAndAfter with LocalSparkConte verify(listener, times(1)).onTaskCompletion(any()) } + + test("TaskContext.attemptNumber should return attempt number, not task id (SPARK-4014)") { + sc = new SparkContext("local[1,2]", "test") // use maxRetries = 2 because we test failed tasks + // Check that attemptIds are 0 for all tasks' initial attempts + val attemptIds = sc.parallelize(Seq(1, 2), 2).mapPartitions { iter => + Seq(TaskContext.get().attemptNumber).iterator + }.collect() + assert(attemptIds.toSet === Set(0)) + + // Test a job with failed tasks + val attemptIdsWithFailedTask = sc.parallelize(Seq(1, 2), 2).mapPartitions { iter => + val attemptId = TaskContext.get().attemptNumber + if (iter.next() == 1 && attemptId == 0) { + throw new Exception("First execution of task failed") + } + Seq(attemptId).iterator + }.collect() + assert(attemptIdsWithFailedTask.toSet === Set(0, 1)) + } + + test("TaskContext.attemptId returns taskAttemptId for backwards-compatibility (SPARK-4014)") { + sc = new SparkContext("local", "test") + val attemptIds = sc.parallelize(Seq(1, 2, 3, 4), 4).mapPartitions { iter => + Seq(TaskContext.get().attemptId).iterator + }.collect() + assert(attemptIds.toSet === Set(0, 1, 2, 3)) + } } private object TaskContextSuite { diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala index 5768a3a733f0..e3a3803e6483 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala @@ -19,9 +19,14 @@ package org.apache.spark.scheduler import java.nio.ByteBuffer -import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSuite} +import scala.concurrent.duration._ +import scala.language.postfixOps +import scala.util.control.NonFatal -import org.apache.spark.{LocalSparkContext, SparkContext, SparkEnv} +import org.scalatest.{BeforeAndAfter, FunSuite} +import org.scalatest.concurrent.Eventually._ + +import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkEnv} import org.apache.spark.storage.TaskResultBlockId /** @@ -34,6 +39,8 @@ class ResultDeletingTaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedule extends TaskResultGetter(sparkEnv, scheduler) { var removedResult = false + @volatile var removeBlockSuccessfully = false + override def enqueueSuccessfulTask( taskSetManager: TaskSetManager, tid: Long, serializedData: ByteBuffer) { if (!removedResult) { @@ -42,6 +49,15 @@ class ResultDeletingTaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedule serializer.get().deserialize[TaskResult[_]](serializedData) match { case IndirectTaskResult(blockId, size) => sparkEnv.blockManager.master.removeBlock(blockId) + // removeBlock is asynchronous. Need to wait it's removed successfully + try { + eventually(timeout(3 seconds), interval(200 milliseconds)) { + assert(!sparkEnv.blockManager.master.contains(blockId)) + } + removeBlockSuccessfully = true + } catch { + case NonFatal(e) => removeBlockSuccessfully = false + } case directResult: DirectTaskResult[_] => taskSetManager.abort("Internal error: expect only indirect results") } @@ -55,27 +71,20 @@ class ResultDeletingTaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedule /** * Tests related to handling task results (both direct and indirect). */ -class TaskResultGetterSuite extends FunSuite with BeforeAndAfter with BeforeAndAfterAll - with LocalSparkContext { +class TaskResultGetterSuite extends FunSuite with BeforeAndAfter with LocalSparkContext { - override def beforeAll { - // Set the Akka frame size to be as small as possible (it must be an integer, so 1 is as small - // as we can make it) so the tests don't take too long. - System.setProperty("spark.akka.frameSize", "1") - } - - override def afterAll { - System.clearProperty("spark.akka.frameSize") - } + // Set the Akka frame size to be as small as possible (it must be an integer, so 1 is as small + // as we can make it) so the tests don't take too long. + def conf: SparkConf = new SparkConf().set("spark.akka.frameSize", "1") test("handling results smaller than Akka frame size") { - sc = new SparkContext("local", "test") + sc = new SparkContext("local", "test", conf) val result = sc.parallelize(Seq(1), 1).map(x => 2 * x).reduce((x, y) => x) assert(result === 2) } test("handling results larger than Akka frame size") { - sc = new SparkContext("local", "test") + sc = new SparkContext("local", "test", conf) val akkaFrameSize = sc.env.actorSystem.settings.config.getBytes("akka.remote.netty.tcp.maximum-frame-size").toInt val result = sc.parallelize(Seq(1), 1).map(x => 1.to(akkaFrameSize).toArray).reduce((x, y) => x) @@ -89,7 +98,7 @@ class TaskResultGetterSuite extends FunSuite with BeforeAndAfter with BeforeAndA test("task retried if result missing from block manager") { // Set the maximum number of task failures to > 0, so that the task set isn't aborted // after the result is missing. - sc = new SparkContext("local[1,2]", "test") + sc = new SparkContext("local[1,2]", "test", conf) // If this test hangs, it's probably because no resource offers were made after the task // failed. val scheduler: TaskSchedulerImpl = sc.taskScheduler match { @@ -99,10 +108,12 @@ class TaskResultGetterSuite extends FunSuite with BeforeAndAfter with BeforeAndA assert(false, "Expect local cluster to use TaskSchedulerImpl") throw new ClassCastException } - scheduler.taskResultGetter = new ResultDeletingTaskResultGetter(sc.env, scheduler) + val resultGetter = new ResultDeletingTaskResultGetter(sc.env, scheduler) + scheduler.taskResultGetter = resultGetter val akkaFrameSize = sc.env.actorSystem.settings.config.getBytes("akka.remote.netty.tcp.maximum-frame-size").toInt val result = sc.parallelize(Seq(1), 1).map(x => 1.to(akkaFrameSize).toArray).reduce((x, y) => x) + assert(resultGetter.removeBlockSuccessfully) assert(result === 1.to(akkaFrameSize).toArray) // Make sure two tasks were run (one failed one, and a second retried one). diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala index 7532da88c606..add13f5b2176 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -30,238 +30,8 @@ class FakeSchedulerBackend extends SchedulerBackend { def defaultParallelism() = 1 } -class FakeTaskSetManager( - initPriority: Int, - initStageId: Int, - initNumTasks: Int, - taskScheduler: TaskSchedulerImpl, - taskSet: TaskSet) - extends TaskSetManager(taskScheduler, taskSet, 0) { - - parent = null - weight = 1 - minShare = 2 - priority = initPriority - stageId = initStageId - name = "TaskSet_"+stageId - override val numTasks = initNumTasks - tasksSuccessful = 0 - - var numRunningTasks = 0 - override def runningTasks = numRunningTasks - - def increaseRunningTasks(taskNum: Int) { - numRunningTasks += taskNum - if (parent != null) { - parent.increaseRunningTasks(taskNum) - } - } - - def decreaseRunningTasks(taskNum: Int) { - numRunningTasks -= taskNum - if (parent != null) { - parent.decreaseRunningTasks(taskNum) - } - } - - override def addSchedulable(schedulable: Schedulable) { - } - - override def removeSchedulable(schedulable: Schedulable) { - } - - override def getSchedulableByName(name: String): Schedulable = { - null - } - - override def executorLost(executorId: String, host: String): Unit = { - } - - override def resourceOffer( - execId: String, - host: String, - maxLocality: TaskLocality.TaskLocality) - : Option[TaskDescription] = - { - if (tasksSuccessful + numRunningTasks < numTasks) { - increaseRunningTasks(1) - Some(new TaskDescription(0, execId, "task 0:0", 0, null)) - } else { - None - } - } - - override def checkSpeculatableTasks(): Boolean = { - true - } - - def taskFinished() { - decreaseRunningTasks(1) - tasksSuccessful +=1 - if (tasksSuccessful == numTasks) { - parent.removeSchedulable(this) - } - } - - def abort() { - decreaseRunningTasks(numRunningTasks) - parent.removeSchedulable(this) - } -} - class TaskSchedulerImplSuite extends FunSuite with LocalSparkContext with Logging { - def createDummyTaskSetManager(priority: Int, stage: Int, numTasks: Int, cs: TaskSchedulerImpl, - taskSet: TaskSet): FakeTaskSetManager = { - new FakeTaskSetManager(priority, stage, numTasks, cs , taskSet) - } - - def resourceOffer(rootPool: Pool): Int = { - val taskSetQueue = rootPool.getSortedTaskSetQueue - /* Just for Test*/ - for (manager <- taskSetQueue) { - logInfo("parentName:%s, parent running tasks:%d, name:%s,runningTasks:%d".format( - manager.parent.name, manager.parent.runningTasks, manager.name, manager.runningTasks)) - } - for (taskSet <- taskSetQueue) { - taskSet.resourceOffer("execId_1", "hostname_1", TaskLocality.ANY) match { - case Some(task) => - return taskSet.stageId - case None => {} - } - } - -1 - } - - def checkTaskSetId(rootPool: Pool, expectedTaskSetId: Int) { - assert(resourceOffer(rootPool) === expectedTaskSetId) - } - - test("FIFO Scheduler Test") { - sc = new SparkContext("local", "TaskSchedulerImplSuite") - val taskScheduler = new TaskSchedulerImpl(sc) - val taskSet = FakeTask.createTaskSet(1) - - val rootPool = new Pool("", SchedulingMode.FIFO, 0, 0) - val schedulableBuilder = new FIFOSchedulableBuilder(rootPool) - schedulableBuilder.buildPools() - - val taskSetManager0 = createDummyTaskSetManager(0, 0, 2, taskScheduler, taskSet) - val taskSetManager1 = createDummyTaskSetManager(0, 1, 2, taskScheduler, taskSet) - val taskSetManager2 = createDummyTaskSetManager(0, 2, 2, taskScheduler, taskSet) - schedulableBuilder.addTaskSetManager(taskSetManager0, null) - schedulableBuilder.addTaskSetManager(taskSetManager1, null) - schedulableBuilder.addTaskSetManager(taskSetManager2, null) - - checkTaskSetId(rootPool, 0) - resourceOffer(rootPool) - checkTaskSetId(rootPool, 1) - resourceOffer(rootPool) - taskSetManager1.abort() - checkTaskSetId(rootPool, 2) - } - - test("Fair Scheduler Test") { - sc = new SparkContext("local", "TaskSchedulerImplSuite") - val taskScheduler = new TaskSchedulerImpl(sc) - val taskSet = FakeTask.createTaskSet(1) - - val xmlPath = getClass.getClassLoader.getResource("fairscheduler.xml").getFile() - System.setProperty("spark.scheduler.allocation.file", xmlPath) - val rootPool = new Pool("", SchedulingMode.FAIR, 0, 0) - val schedulableBuilder = new FairSchedulableBuilder(rootPool, sc.conf) - schedulableBuilder.buildPools() - - assert(rootPool.getSchedulableByName("default") != null) - assert(rootPool.getSchedulableByName("1") != null) - assert(rootPool.getSchedulableByName("2") != null) - assert(rootPool.getSchedulableByName("3") != null) - assert(rootPool.getSchedulableByName("1").minShare === 2) - assert(rootPool.getSchedulableByName("1").weight === 1) - assert(rootPool.getSchedulableByName("2").minShare === 3) - assert(rootPool.getSchedulableByName("2").weight === 1) - assert(rootPool.getSchedulableByName("3").minShare === 0) - assert(rootPool.getSchedulableByName("3").weight === 1) - - val properties1 = new Properties() - properties1.setProperty("spark.scheduler.pool","1") - val properties2 = new Properties() - properties2.setProperty("spark.scheduler.pool","2") - - val taskSetManager10 = createDummyTaskSetManager(1, 0, 1, taskScheduler, taskSet) - val taskSetManager11 = createDummyTaskSetManager(1, 1, 1, taskScheduler, taskSet) - val taskSetManager12 = createDummyTaskSetManager(1, 2, 2, taskScheduler, taskSet) - schedulableBuilder.addTaskSetManager(taskSetManager10, properties1) - schedulableBuilder.addTaskSetManager(taskSetManager11, properties1) - schedulableBuilder.addTaskSetManager(taskSetManager12, properties1) - - val taskSetManager23 = createDummyTaskSetManager(2, 3, 2, taskScheduler, taskSet) - val taskSetManager24 = createDummyTaskSetManager(2, 4, 2, taskScheduler, taskSet) - schedulableBuilder.addTaskSetManager(taskSetManager23, properties2) - schedulableBuilder.addTaskSetManager(taskSetManager24, properties2) - - checkTaskSetId(rootPool, 0) - checkTaskSetId(rootPool, 3) - checkTaskSetId(rootPool, 3) - checkTaskSetId(rootPool, 1) - checkTaskSetId(rootPool, 4) - checkTaskSetId(rootPool, 2) - checkTaskSetId(rootPool, 2) - checkTaskSetId(rootPool, 4) - - taskSetManager12.taskFinished() - assert(rootPool.getSchedulableByName("1").runningTasks === 3) - taskSetManager24.abort() - assert(rootPool.getSchedulableByName("2").runningTasks === 2) - } - - test("Nested Pool Test") { - sc = new SparkContext("local", "TaskSchedulerImplSuite") - val taskScheduler = new TaskSchedulerImpl(sc) - val taskSet = FakeTask.createTaskSet(1) - - val rootPool = new Pool("", SchedulingMode.FAIR, 0, 0) - val pool0 = new Pool("0", SchedulingMode.FAIR, 3, 1) - val pool1 = new Pool("1", SchedulingMode.FAIR, 4, 1) - rootPool.addSchedulable(pool0) - rootPool.addSchedulable(pool1) - - val pool00 = new Pool("00", SchedulingMode.FAIR, 2, 2) - val pool01 = new Pool("01", SchedulingMode.FAIR, 1, 1) - pool0.addSchedulable(pool00) - pool0.addSchedulable(pool01) - - val pool10 = new Pool("10", SchedulingMode.FAIR, 2, 2) - val pool11 = new Pool("11", SchedulingMode.FAIR, 2, 1) - pool1.addSchedulable(pool10) - pool1.addSchedulable(pool11) - - val taskSetManager000 = createDummyTaskSetManager(0, 0, 5, taskScheduler, taskSet) - val taskSetManager001 = createDummyTaskSetManager(0, 1, 5, taskScheduler, taskSet) - pool00.addSchedulable(taskSetManager000) - pool00.addSchedulable(taskSetManager001) - - val taskSetManager010 = createDummyTaskSetManager(1, 2, 5, taskScheduler, taskSet) - val taskSetManager011 = createDummyTaskSetManager(1, 3, 5, taskScheduler, taskSet) - pool01.addSchedulable(taskSetManager010) - pool01.addSchedulable(taskSetManager011) - - val taskSetManager100 = createDummyTaskSetManager(2, 4, 5, taskScheduler, taskSet) - val taskSetManager101 = createDummyTaskSetManager(2, 5, 5, taskScheduler, taskSet) - pool10.addSchedulable(taskSetManager100) - pool10.addSchedulable(taskSetManager101) - - val taskSetManager110 = createDummyTaskSetManager(3, 6, 5, taskScheduler, taskSet) - val taskSetManager111 = createDummyTaskSetManager(3, 7, 5, taskScheduler, taskSet) - pool11.addSchedulable(taskSetManager110) - pool11.addSchedulable(taskSetManager111) - - checkTaskSetId(rootPool, 0) - checkTaskSetId(rootPool, 4) - checkTaskSetId(rootPool, 6) - checkTaskSetId(rootPool, 2) - } - test("Scheduler does not always schedule tasks on the same workers") { sc = new SparkContext("local", "TaskSchedulerImplSuite") val taskScheduler = new TaskSchedulerImpl(sc) @@ -305,7 +75,6 @@ class TaskSchedulerImplSuite extends FunSuite with LocalSparkContext with Loggin override def taskStarted(task: Task[_], taskInfo: TaskInfo) {} override def executorAdded(execId: String, host: String) {} } - taskScheduler.setDAGScheduler(dagScheduler) // Give zero core offers. Should not generate any tasks val zeroCoreWorkerOffers = Seq(new WorkerOffer("executor0", "host0", 0), new WorkerOffer("executor1", "host1", 0)) @@ -331,4 +100,34 @@ class TaskSchedulerImplSuite extends FunSuite with LocalSparkContext with Loggin assert(1 === taskDescriptions.length) assert("executor0" === taskDescriptions(0).executorId) } + + test("Scheduler does not crash when tasks are not serializable") { + sc = new SparkContext("local", "TaskSchedulerImplSuite") + val taskCpus = 2 + + sc.conf.set("spark.task.cpus", taskCpus.toString) + val taskScheduler = new TaskSchedulerImpl(sc) + taskScheduler.initialize(new FakeSchedulerBackend) + // Need to initialize a DAGScheduler for the taskScheduler to use for callbacks. + val dagScheduler = new DAGScheduler(sc, taskScheduler) { + override def taskStarted(task: Task[_], taskInfo: TaskInfo) {} + override def executorAdded(execId: String, host: String) {} + } + val numFreeCores = 1 + taskScheduler.setDAGScheduler(dagScheduler) + var taskSet = new TaskSet(Array(new NotSerializableFakeTask(1, 0), new NotSerializableFakeTask(0, 1)), 0, 0, 0, null) + val multiCoreWorkerOffers = Seq(new WorkerOffer("executor0", "host0", taskCpus), + new WorkerOffer("executor1", "host1", numFreeCores)) + taskScheduler.submitTasks(taskSet) + var taskDescriptions = taskScheduler.resourceOffers(multiCoreWorkerOffers).flatten + assert(0 === taskDescriptions.length) + + // Now check that we can still submit tasks + // Even if one of the tasks has not-serializable tasks, the other task set should still be processed without error + taskScheduler.submitTasks(taskSet) + taskScheduler.submitTasks(FakeTask.createTaskSet(1)) + taskDescriptions = taskScheduler.resourceOffers(multiCoreWorkerOffers).flatten + assert(taskDescriptions.map(_.executorId) === Seq("executor0")) + } + } 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 472191551a01..84b9b788237b 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.scheduler +import java.io.{ObjectInputStream, ObjectOutputStream, IOException} import java.util.Random import scala.collection.mutable.ArrayBuffer @@ -563,6 +564,19 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { assert(manager.emittedTaskSizeWarning) } + test("Not serializable exception thrown if the task cannot be serialized") { + sc = new SparkContext("local", "test") + val sched = new FakeTaskScheduler(sc, ("exec1", "host1")) + + val taskSet = new TaskSet(Array(new NotSerializableFakeTask(1, 0), new NotSerializableFakeTask(0, 1)), 0, 0, 0, null) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES) + + intercept[TaskNotSerializableException] { + manager.resourceOffer("exec1", "host1", ANY) + } + assert(manager.isZombie) + } + test("abort the job if total size of results is too large") { val conf = new SparkConf().set("spark.driver.maxResultSize", "2m") sc = new SparkContext("local", "test", conf) diff --git a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala index e60e70afd321..073814c127ed 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala @@ -17,23 +17,58 @@ package org.apache.spark.scheduler.mesos +import org.apache.spark.executor.MesosExecutorBackend import org.scalatest.FunSuite -import org.apache.spark.{scheduler, SparkConf, SparkContext, LocalSparkContext} -import org.apache.spark.scheduler.{TaskDescription, WorkerOffer, TaskSchedulerImpl} +import org.apache.spark.{SparkConf, SparkContext, LocalSparkContext} +import org.apache.spark.scheduler.{SparkListenerExecutorAdded, LiveListenerBus, + TaskDescription, WorkerOffer, TaskSchedulerImpl} +import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.scheduler.cluster.mesos.{MemoryUtils, MesosSchedulerBackend} import org.apache.mesos.SchedulerDriver -import org.apache.mesos.Protos._ -import org.scalatest.mock.EasyMockSugar +import org.apache.mesos.Protos.{ExecutorInfo => MesosExecutorInfo, _} import org.apache.mesos.Protos.Value.Scalar import org.easymock.{Capture, EasyMock} import java.nio.ByteBuffer import java.util.Collections import java.util +import org.scalatest.mock.EasyMockSugar + import scala.collection.mutable import scala.collection.mutable.ArrayBuffer class MesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with EasyMockSugar { + test("check spark-class location correctly") { + val conf = new SparkConf + conf.set("spark.mesos.executor.home" , "/mesos-home") + + val listenerBus = EasyMock.createMock(classOf[LiveListenerBus]) + listenerBus.post(SparkListenerExecutorAdded("s1", new ExecutorInfo("host1", 2))) + EasyMock.replay(listenerBus) + + val sc = EasyMock.createMock(classOf[SparkContext]) + EasyMock.expect(sc.getSparkHome()).andReturn(Option("/spark-home")).anyTimes() + EasyMock.expect(sc.conf).andReturn(conf).anyTimes() + EasyMock.expect(sc.executorEnvs).andReturn(new mutable.HashMap).anyTimes() + EasyMock.expect(sc.executorMemory).andReturn(100).anyTimes() + EasyMock.expect(sc.listenerBus).andReturn(listenerBus) + EasyMock.replay(sc) + val taskScheduler = EasyMock.createMock(classOf[TaskSchedulerImpl]) + EasyMock.expect(taskScheduler.CPUS_PER_TASK).andReturn(2).anyTimes() + EasyMock.replay(taskScheduler) + + val mesosSchedulerBackend = new MesosSchedulerBackend(taskScheduler, sc, "master") + + // uri is null. + val executorInfo = mesosSchedulerBackend.createExecutorInfo("test-id") + assert(executorInfo.getCommand.getValue === s" /mesos-home/bin/spark-class ${classOf[MesosExecutorBackend].getName}") + + // uri exists. + conf.set("spark.executor.uri", "hdfs:///test-app-1.0.0.tgz") + val executorInfo1 = mesosSchedulerBackend.createExecutorInfo("test-id") + assert(executorInfo1.getCommand.getValue === s"cd test-app-1*; ./bin/spark-class ${classOf[MesosExecutorBackend].getName}") + } + test("mesos resource offers result in launching tasks") { def createOffer(id: Int, mem: Int, cpu: Int) = { val builder = Offer.newBuilder() @@ -52,11 +87,16 @@ class MesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with Ea val driver = EasyMock.createMock(classOf[SchedulerDriver]) val taskScheduler = EasyMock.createMock(classOf[TaskSchedulerImpl]) + val listenerBus = EasyMock.createMock(classOf[LiveListenerBus]) + listenerBus.post(SparkListenerExecutorAdded("s1", new ExecutorInfo("host1", 2))) + EasyMock.replay(listenerBus) + val sc = EasyMock.createMock(classOf[SparkContext]) EasyMock.expect(sc.executorMemory).andReturn(100).anyTimes() EasyMock.expect(sc.getSparkHome()).andReturn(Option("/path")).anyTimes() EasyMock.expect(sc.executorEnvs).andReturn(new mutable.HashMap).anyTimes() EasyMock.expect(sc.conf).andReturn(new SparkConf).anyTimes() + EasyMock.expect(sc.listenerBus).andReturn(listenerBus) EasyMock.replay(sc) val minMem = MemoryUtils.calculateTotalMemory(sc).toInt @@ -80,7 +120,7 @@ class MesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with Ea mesosOffers.get(2).getHostname, 2 )) - val taskDesc = new TaskDescription(1L, "s1", "n1", 0, ByteBuffer.wrap(new Array[Byte](0))) + val taskDesc = new TaskDescription(1L, 0, "s1", "n1", 0, ByteBuffer.wrap(new Array[Byte](0))) EasyMock.expect(taskScheduler.resourceOffers(EasyMock.eq(expectedWorkerOffers))).andReturn(Seq(Seq(taskDesc))) EasyMock.expect(taskScheduler.CPUS_PER_TASK).andReturn(2).anyTimes() EasyMock.replay(taskScheduler) diff --git a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosTaskLaunchDataSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosTaskLaunchDataSuite.scala new file mode 100644 index 000000000000..86a42a7398e4 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosTaskLaunchDataSuite.scala @@ -0,0 +1,38 @@ +/* + * 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.mesos + +import java.nio.ByteBuffer + +import org.scalatest.FunSuite + +import org.apache.spark.scheduler.cluster.mesos.MesosTaskLaunchData + +class MesosTaskLaunchDataSuite extends FunSuite { + test("serialize and deserialize data must be same") { + val serializedTask = ByteBuffer.allocate(40) + (Range(100, 110).map(serializedTask.putInt(_))) + serializedTask.rewind + val attemptNumber = 100 + val byteString = MesosTaskLaunchData(serializedTask, attemptNumber).toByteString + serializedTask.rewind + val mesosTaskLaunchData = MesosTaskLaunchData.fromByteString(byteString) + assert(mesosTaskLaunchData.attemptNumber == attemptNumber) + assert(mesosTaskLaunchData.serializedTask.equals(serializedTask)) + } +} 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 5554efbcbadf..ffe6f039145e 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -33,7 +33,7 @@ import akka.util.Timeout import org.mockito.Mockito.{mock, when} -import org.scalatest.{BeforeAndAfter, FunSuite, Matchers, PrivateMethodTester} +import org.scalatest._ import org.scalatest.concurrent.Eventually._ import org.scalatest.concurrent.Timeouts._ @@ -44,18 +44,17 @@ import org.apache.spark.scheduler.LiveListenerBus import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} import org.apache.spark.shuffle.hash.HashShuffleManager import org.apache.spark.storage.BlockManagerMessages.BlockManagerHeartbeat -import org.apache.spark.util.{AkkaUtils, ByteBufferInputStream, SizeEstimator, Utils} +import org.apache.spark.util._ -class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter - with PrivateMethodTester { +class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfterEach + with PrivateMethodTester with ResetSystemProperties { private val conf = new SparkConf(false) var store: BlockManager = null var store2: BlockManager = null var actorSystem: ActorSystem = null var master: BlockManagerMaster = null - var oldArch: String = null conf.set("spark.authenticate", "false") val securityMgr = new SecurityManager(conf) val mapOutputTracker = new MapOutputTrackerMaster(conf) @@ -79,13 +78,13 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter manager } - before { + override def beforeEach(): Unit = { val (actorSystem, boundPort) = AkkaUtils.createActorSystem( "test", "localhost", 0, conf = conf, securityManager = securityMgr) this.actorSystem = actorSystem // Set the arch to 64-bit and compressedOops to true to get a deterministic test-case - oldArch = System.setProperty("os.arch", "amd64") + System.setProperty("os.arch", "amd64") conf.set("os.arch", "amd64") conf.set("spark.test.useCompressedOops", "true") conf.set("spark.driver.port", boundPort.toString) @@ -100,7 +99,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter SizeEstimator invokePrivate initialize() } - after { + override def afterEach(): Unit = { if (store != null) { store.stop() store = null @@ -113,14 +112,6 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter actorSystem.awaitTermination() actorSystem = null master = null - - if (oldArch != null) { - conf.set("os.arch", oldArch) - } else { - System.clearProperty("os.arch") - } - - System.clearProperty("spark.test.useCompressedOops") } test("StorageLevel object caching") { diff --git a/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala b/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala index 1eaabb93adbe..37b593b2c5f7 100644 --- a/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala @@ -89,7 +89,7 @@ class ShuffleBlockFetcherIteratorSuite extends FunSuite { ) val iterator = new ShuffleBlockFetcherIterator( - new TaskContextImpl(0, 0, 0), + new TaskContextImpl(0, 0, 0, 0), transfer, blockManager, blocksByAddress, @@ -154,7 +154,7 @@ class ShuffleBlockFetcherIteratorSuite extends FunSuite { val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( (remoteBmId, blocks.keys.map(blockId => (blockId, 1.asInstanceOf[Long])).toSeq)) - val taskContext = new TaskContextImpl(0, 0, 0) + val taskContext = new TaskContextImpl(0, 0, 0, 0) val iterator = new ShuffleBlockFetcherIterator( taskContext, transfer, @@ -217,7 +217,7 @@ class ShuffleBlockFetcherIteratorSuite extends FunSuite { val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( (remoteBmId, blocks.keys.map(blockId => (blockId, 1.asInstanceOf[Long])).toSeq)) - val taskContext = new TaskContextImpl(0, 0, 0) + val taskContext = new TaskContextImpl(0, 0, 0, 0) val iterator = new ShuffleBlockFetcherIterator( taskContext, transfer, diff --git a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala index 787f4c2b5a8b..e85a436cdba1 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala @@ -173,7 +173,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers { // Simulate fetch failures: val mappedData = data.map { x => val taskContext = TaskContext.get - if (taskContext.attemptId() == 1) { // Cause this stage to fail on its first attempt. + if (taskContext.attemptNumber == 0) { // Cause this stage to fail on its first attempt. val env = SparkEnv.get val bmAddress = env.blockManager.blockManagerId val shuffleId = shuffleHandle.shuffleId diff --git a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala index 12af60caf7d5..68074ae32a67 100644 --- a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala @@ -28,6 +28,8 @@ import org.apache.spark.util.Utils class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matchers { + val jobSubmissionTime = 1421191042750L + val jobCompletionTime = 1421191296660L private def createStageStartEvent(stageId: Int) = { val stageInfo = new StageInfo(stageId, 0, stageId.toString, 0, null, "") @@ -46,12 +48,12 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc val stageInfos = stageIds.map { stageId => new StageInfo(stageId, 0, stageId.toString, 0, null, "") } - SparkListenerJobStart(jobId, stageInfos) + SparkListenerJobStart(jobId, jobSubmissionTime, stageInfos) } private def createJobEndEvent(jobId: Int, failed: Boolean = false) = { val result = if (failed) JobFailed(new Exception("dummy failure")) else JobSucceeded - SparkListenerJobEnd(jobId, result) + SparkListenerJobEnd(jobId, jobCompletionTime, result) } private def runJob(listener: SparkListener, jobId: Int, shouldFail: Boolean = false) { @@ -138,7 +140,7 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc assert(listener.stageIdToData.size === 0) // finish this task, should get updated shuffleRead - shuffleReadMetrics.remoteBytesRead = 1000 + shuffleReadMetrics.incRemoteBytesRead(1000) taskMetrics.setShuffleReadMetrics(Some(shuffleReadMetrics)) var taskInfo = new TaskInfo(1234L, 0, 1, 0L, "exe-1", "host1", TaskLocality.NODE_LOCAL, false) taskInfo.finishTime = 1 @@ -224,18 +226,18 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc val shuffleWriteMetrics = new ShuffleWriteMetrics() taskMetrics.setShuffleReadMetrics(Some(shuffleReadMetrics)) taskMetrics.shuffleWriteMetrics = Some(shuffleWriteMetrics) - shuffleReadMetrics.remoteBytesRead = base + 1 - shuffleReadMetrics.remoteBlocksFetched = base + 2 - shuffleWriteMetrics.shuffleBytesWritten = base + 3 - taskMetrics.executorRunTime = base + 4 - taskMetrics.diskBytesSpilled = base + 5 - taskMetrics.memoryBytesSpilled = base + 6 + shuffleReadMetrics.incRemoteBytesRead(base + 1) + shuffleReadMetrics.incRemoteBlocksFetched(base + 2) + shuffleWriteMetrics.incShuffleBytesWritten(base + 3) + taskMetrics.setExecutorRunTime(base + 4) + taskMetrics.incDiskBytesSpilled(base + 5) + taskMetrics.incMemoryBytesSpilled(base + 6) val inputMetrics = new InputMetrics(DataReadMethod.Hadoop) - taskMetrics.inputMetrics = Some(inputMetrics) - inputMetrics.bytesRead = base + 7 + taskMetrics.setInputMetrics(Some(inputMetrics)) + inputMetrics.addBytesRead(base + 7) val outputMetrics = new OutputMetrics(DataWriteMethod.Hadoop) taskMetrics.outputMetrics = Some(outputMetrics) - outputMetrics.bytesWritten = base + 8 + outputMetrics.setBytesWritten(base + 8) taskMetrics } 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 7bca1711ae22..6bbf72e929dc 100644 --- a/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala @@ -31,7 +31,7 @@ import org.apache.spark.storage.BlockManagerId /** * Test the AkkaUtils with various security settings. */ -class AkkaUtilsSuite extends FunSuite with LocalSparkContext { +class AkkaUtilsSuite extends FunSuite with LocalSparkContext with ResetSystemProperties { test("remote fetch security bad password") { val conf = new SparkConf diff --git a/core/src/test/scala/org/apache/spark/util/EventLoopSuite.scala b/core/src/test/scala/org/apache/spark/util/EventLoopSuite.scala new file mode 100644 index 000000000000..10541f878476 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/util/EventLoopSuite.scala @@ -0,0 +1,188 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util + +import java.util.concurrent.CountDownLatch + +import scala.collection.mutable +import scala.concurrent.duration._ +import scala.language.postfixOps + +import org.scalatest.concurrent.Eventually._ +import org.scalatest.concurrent.Timeouts +import org.scalatest.FunSuite + +class EventLoopSuite extends FunSuite with Timeouts { + + test("EventLoop") { + val buffer = new mutable.ArrayBuffer[Int] with mutable.SynchronizedBuffer[Int] + val eventLoop = new EventLoop[Int]("test") { + + override def onReceive(event: Int): Unit = { + buffer += event + } + + override def onError(e: Throwable): Unit = {} + } + eventLoop.start() + (1 to 100).foreach(eventLoop.post) + eventually(timeout(5 seconds), interval(200 millis)) { + assert((1 to 100) === buffer.toSeq) + } + eventLoop.stop() + } + + test("EventLoop: start and stop") { + val eventLoop = new EventLoop[Int]("test") { + + override def onReceive(event: Int): Unit = {} + + override def onError(e: Throwable): Unit = {} + } + assert(false === eventLoop.isActive) + eventLoop.start() + assert(true === eventLoop.isActive) + eventLoop.stop() + assert(false === eventLoop.isActive) + } + + test("EventLoop: onError") { + val e = new RuntimeException("Oops") + @volatile var receivedError: Throwable = null + val eventLoop = new EventLoop[Int]("test") { + + override def onReceive(event: Int): Unit = { + throw e + } + + override def onError(e: Throwable): Unit = { + receivedError = e + } + } + eventLoop.start() + eventLoop.post(1) + eventually(timeout(5 seconds), interval(200 millis)) { + assert(e === receivedError) + } + eventLoop.stop() + } + + test("EventLoop: error thrown from onError should not crash the event thread") { + val e = new RuntimeException("Oops") + @volatile var receivedError: Throwable = null + val eventLoop = new EventLoop[Int]("test") { + + override def onReceive(event: Int): Unit = { + throw e + } + + override def onError(e: Throwable): Unit = { + receivedError = e + throw new RuntimeException("Oops") + } + } + eventLoop.start() + eventLoop.post(1) + eventually(timeout(5 seconds), interval(200 millis)) { + assert(e === receivedError) + assert(eventLoop.isActive) + } + eventLoop.stop() + } + + test("EventLoop: calling stop multiple times should only call onStop once") { + var onStopTimes = 0 + val eventLoop = new EventLoop[Int]("test") { + + override def onReceive(event: Int): Unit = { + } + + override def onError(e: Throwable): Unit = { + } + + override def onStop(): Unit = { + onStopTimes += 1 + } + } + + eventLoop.start() + + eventLoop.stop() + eventLoop.stop() + eventLoop.stop() + + assert(1 === onStopTimes) + } + + test("EventLoop: post event in multiple threads") { + @volatile var receivedEventsCount = 0 + val eventLoop = new EventLoop[Int]("test") { + + override def onReceive(event: Int): Unit = { + receivedEventsCount += 1 + } + + override def onError(e: Throwable): Unit = { + } + + } + eventLoop.start() + + val threadNum = 5 + val eventsFromEachThread = 100 + (1 to threadNum).foreach { _ => + new Thread() { + override def run(): Unit = { + (1 to eventsFromEachThread).foreach(eventLoop.post) + } + }.start() + } + + eventually(timeout(5 seconds), interval(200 millis)) { + assert(threadNum * eventsFromEachThread === receivedEventsCount) + } + eventLoop.stop() + } + + test("EventLoop: onReceive swallows InterruptException") { + val onReceiveLatch = new CountDownLatch(1) + val eventLoop = new EventLoop[Int]("test") { + + override def onReceive(event: Int): Unit = { + onReceiveLatch.countDown() + try { + Thread.sleep(5000) + } catch { + case ie: InterruptedException => // swallow + } + } + + override def onError(e: Throwable): Unit = { + } + + } + eventLoop.start() + eventLoop.post(1) + failAfter(5 seconds) { + // Wait until we enter `onReceive` + onReceiveLatch.await() + eventLoop.stop() + } + assert(false === eventLoop.isActive) + } +} 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 593d6dd8c379..0357fc6ce278 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.util import java.util.Properties +import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.shuffle.MetadataFetchFailedException import scala.collection.Map @@ -33,6 +34,9 @@ import org.apache.spark.storage._ class JsonProtocolSuite extends FunSuite { + val jobSubmissionTime = 1421191042750L + val jobCompletionTime = 1421191296660L + test("SparkListenerEvent") { val stageSubmitted = SparkListenerStageSubmitted(makeStageInfo(100, 200, 300, 400L, 500L), properties) @@ -53,9 +57,9 @@ class JsonProtocolSuite extends FunSuite { val stageIds = Seq[Int](1, 2, 3, 4) val stageInfos = stageIds.map(x => makeStageInfo(x, x * 200, x * 300, x * 400L, x * 500L)) - SparkListenerJobStart(10, stageInfos, properties) + SparkListenerJobStart(10, jobSubmissionTime, stageInfos, properties) } - val jobEnd = SparkListenerJobEnd(20, JobSucceeded) + val jobEnd = SparkListenerJobEnd(20, jobCompletionTime, JobSucceeded) val environmentUpdate = SparkListenerEnvironmentUpdate(Map[String, Seq[(String, String)]]( "JVM Information" -> Seq(("GC speed", "9999 objects/s"), ("Java home", "Land of coffee")), "Spark Properties" -> Seq(("Job throughput", "80000 jobs/s, regardless of job type")), @@ -69,6 +73,9 @@ class JsonProtocolSuite extends FunSuite { val unpersistRdd = SparkListenerUnpersistRDD(12345) val applicationStart = SparkListenerApplicationStart("The winner of all", None, 42L, "Garfield") val applicationEnd = SparkListenerApplicationEnd(42L) + val executorAdded = SparkListenerExecutorAdded("exec1", + new ExecutorInfo("Hostee.awesome.com", 11)) + val executorRemoved = SparkListenerExecutorRemoved("exec2") testEvent(stageSubmitted, stageSubmittedJsonString) testEvent(stageCompleted, stageCompletedJsonString) @@ -85,6 +92,8 @@ class JsonProtocolSuite extends FunSuite { testEvent(unpersistRdd, unpersistRDDJsonString) testEvent(applicationStart, applicationStartJsonString) testEvent(applicationEnd, applicationEndJsonString) + testEvent(executorAdded, executorAddedJsonString) + testEvent(executorRemoved, executorRemovedJsonString) } test("Dependent Classes") { @@ -94,6 +103,7 @@ class JsonProtocolSuite extends FunSuite { testTaskMetrics(makeTaskMetrics( 33333L, 44444L, 55555L, 66666L, 7, 8, hasHadoopInput = false, hasOutput = false)) testBlockManagerId(BlockManagerId("Hong", "Kong", 500)) + testExecutorInfo(new ExecutorInfo("host", 43)) // StorageLevel testStorageLevel(StorageLevel.NONE) @@ -240,13 +250,31 @@ class JsonProtocolSuite extends FunSuite { val stageInfos = stageIds.map(x => makeStageInfo(x, x * 200, x * 300, x * 400, x * 500)) val dummyStageInfos = stageIds.map(id => new StageInfo(id, 0, "unknown", 0, Seq.empty, "unknown")) - val jobStart = SparkListenerJobStart(10, stageInfos, properties) + val jobStart = SparkListenerJobStart(10, jobSubmissionTime, stageInfos, properties) val oldEvent = JsonProtocol.jobStartToJson(jobStart).removeField({_._1 == "Stage Infos"}) val expectedJobStart = - SparkListenerJobStart(10, dummyStageInfos, properties) + SparkListenerJobStart(10, jobSubmissionTime, dummyStageInfos, properties) assertEquals(expectedJobStart, JsonProtocol.jobStartFromJson(oldEvent)) } + test("SparkListenerJobStart and SparkListenerJobEnd backward compatibility") { + // Prior to Spark 1.3.0, SparkListenerJobStart did not have a "Submission Time" property. + // Also, SparkListenerJobEnd did not have a "Completion Time" property. + val stageIds = Seq[Int](1, 2, 3, 4) + val stageInfos = stageIds.map(x => makeStageInfo(x * 10, x * 20, x * 30, x * 40, x * 50)) + val jobStart = SparkListenerJobStart(11, jobSubmissionTime, stageInfos, properties) + val oldStartEvent = JsonProtocol.jobStartToJson(jobStart) + .removeField({ _._1 == "Submission Time"}) + val expectedJobStart = SparkListenerJobStart(11, -1, stageInfos, properties) + assertEquals(expectedJobStart, JsonProtocol.jobStartFromJson(oldStartEvent)) + + val jobEnd = SparkListenerJobEnd(11, jobCompletionTime, JobSucceeded) + val oldEndEvent = JsonProtocol.jobEndToJson(jobEnd) + .removeField({ _._1 == "Completion Time"}) + val expectedJobEnd = SparkListenerJobEnd(11, -1, JobSucceeded) + assertEquals(expectedJobEnd, JsonProtocol.jobEndFromJson(oldEndEvent)) + } + /** -------------------------- * | Helper test running methods | * --------------------------- */ @@ -280,7 +308,7 @@ class JsonProtocolSuite extends FunSuite { private def testBlockManagerId(id: BlockManagerId) { val newId = JsonProtocol.blockManagerIdFromJson(JsonProtocol.blockManagerIdToJson(id)) - assertEquals(id, newId) + assert(id === newId) } private def testTaskInfo(info: TaskInfo) { @@ -303,6 +331,10 @@ class JsonProtocolSuite extends FunSuite { assert(blockId === newBlockId) } + private def testExecutorInfo(info: ExecutorInfo) { + val newInfo = JsonProtocol.executorInfoFromJson(JsonProtocol.executorInfoToJson(info)) + assertEquals(info, newInfo) + } /** -------------------------------- * | Util methods for comparing events | @@ -335,22 +367,13 @@ class JsonProtocolSuite extends FunSuite { assertEquals(e1.jobResult, e2.jobResult) case (e1: SparkListenerEnvironmentUpdate, e2: SparkListenerEnvironmentUpdate) => assertEquals(e1.environmentDetails, e2.environmentDetails) - case (e1: SparkListenerBlockManagerAdded, e2: SparkListenerBlockManagerAdded) => - assert(e1.maxMem === e2.maxMem) - assert(e1.time === e2.time) - assertEquals(e1.blockManagerId, e2.blockManagerId) - case (e1: SparkListenerBlockManagerRemoved, e2: SparkListenerBlockManagerRemoved) => - assert(e1.time === e2.time) - assertEquals(e1.blockManagerId, e2.blockManagerId) - case (e1: SparkListenerUnpersistRDD, e2: SparkListenerUnpersistRDD) => - assert(e1.rddId == e2.rddId) - case (e1: SparkListenerApplicationStart, e2: SparkListenerApplicationStart) => - assert(e1.appName == e2.appName) - assert(e1.time == e2.time) - assert(e1.sparkUser == e2.sparkUser) - case (e1: SparkListenerApplicationEnd, e2: SparkListenerApplicationEnd) => - assert(e1.time == e2.time) - case (SparkListenerShutdown, SparkListenerShutdown) => + case (e1: SparkListenerExecutorAdded, e2: SparkListenerExecutorAdded) => + assert(e1.executorId == e1.executorId) + assertEquals(e1.executorInfo, e2.executorInfo) + case (e1: SparkListenerExecutorRemoved, e2: SparkListenerExecutorRemoved) => + assert(e1.executorId == e1.executorId) + case (e1, e2) => + assert(e1 === e2) case _ => fail("Events don't match in types!") } } @@ -401,6 +424,11 @@ class JsonProtocolSuite extends FunSuite { assert(info1.accumulables === info2.accumulables) } + private def assertEquals(info1: ExecutorInfo, info2: ExecutorInfo) { + assert(info1.executorHost == info2.executorHost) + assert(info1.totalCores == info2.totalCores) + } + private def assertEquals(metrics1: TaskMetrics, metrics2: TaskMetrics) { assert(metrics1.hostname === metrics2.hostname) assert(metrics1.executorDeserializeTime === metrics2.executorDeserializeTime) @@ -435,16 +463,6 @@ class JsonProtocolSuite extends FunSuite { assert(metrics1.bytesRead === metrics2.bytesRead) } - private def assertEquals(bm1: BlockManagerId, bm2: BlockManagerId) { - if (bm1 == null || bm2 == null) { - assert(bm1 === bm2) - } else { - assert(bm1.executorId === bm2.executorId) - assert(bm1.host === bm2.host) - assert(bm1.port === bm2.port) - } - } - private def assertEquals(result1: JobResult, result2: JobResult) { (result1, result2) match { case (JobSucceeded, JobSucceeded) => @@ -462,7 +480,7 @@ class JsonProtocolSuite extends FunSuite { assert(r1.shuffleId === r2.shuffleId) assert(r1.mapId === r2.mapId) assert(r1.reduceId === r2.reduceId) - assertEquals(r1.bmAddress, r2.bmAddress) + assert(r1.bmAddress === r2.bmAddress) assert(r1.message === r2.message) case (r1: ExceptionFailure, r2: ExceptionFailure) => assert(r1.className === r2.className) @@ -623,34 +641,34 @@ class JsonProtocolSuite extends FunSuite { hasHadoopInput: Boolean, hasOutput: Boolean) = { val t = new TaskMetrics - t.hostname = "localhost" - t.executorDeserializeTime = a - t.executorRunTime = b - t.resultSize = c - t.jvmGCTime = d - t.resultSerializationTime = a + b - t.memoryBytesSpilled = a + c + t.setHostname("localhost") + t.setExecutorDeserializeTime(a) + t.setExecutorRunTime(b) + t.setResultSize(c) + t.setJvmGCTime(d) + t.setResultSerializationTime(a + b) + t.incMemoryBytesSpilled(a + c) if (hasHadoopInput) { val inputMetrics = new InputMetrics(DataReadMethod.Hadoop) - inputMetrics.bytesRead = d + e + f - t.inputMetrics = Some(inputMetrics) + inputMetrics.addBytesRead(d + e + f) + t.setInputMetrics(Some(inputMetrics)) } else { val sr = new ShuffleReadMetrics - sr.remoteBytesRead = b + d - sr.localBlocksFetched = e - sr.fetchWaitTime = a + d - sr.remoteBlocksFetched = f + sr.incRemoteBytesRead(b + d) + sr.incLocalBlocksFetched(e) + sr.incFetchWaitTime(a + d) + sr.incRemoteBlocksFetched(f) t.setShuffleReadMetrics(Some(sr)) } if (hasOutput) { val outputMetrics = new OutputMetrics(DataWriteMethod.Hadoop) - outputMetrics.bytesWritten = a + b + c + outputMetrics.setBytesWritten(a + b + c) t.outputMetrics = Some(outputMetrics) } else { val sw = new ShuffleWriteMetrics - sw.shuffleBytesWritten = a + b + c - sw.shuffleWriteTime = b + c + d + sw.incShuffleBytesWritten(a + b + c) + sw.incShuffleWriteTime(b + c + d) t.shuffleWriteMetrics = Some(sw) } // Make at most 6 blocks @@ -1078,6 +1096,7 @@ class JsonProtocolSuite extends FunSuite { |{ | "Event": "SparkListenerJobStart", | "Job ID": 10, + | "Submission Time": 1421191042750, | "Stage Infos": [ | { | "Stage ID": 1, @@ -1352,6 +1371,7 @@ class JsonProtocolSuite extends FunSuite { |{ | "Event": "SparkListenerJobEnd", | "Job ID": 20, + | "Completion Time": 1421191296660, | "Job Result": { | "Result": "JobSucceeded" | } @@ -1431,4 +1451,24 @@ class JsonProtocolSuite extends FunSuite { | "Timestamp": 42 |} """ + + private val executorAddedJsonString = + """ + |{ + | "Event": "SparkListenerExecutorAdded", + | "Executor ID": "exec1", + | "Executor Info": { + | "Host": "Hostee.awesome.com", + | "Total Cores": 11 + | } + |} + """ + + private val executorRemovedJsonString = + """ + |{ + | "Event": "SparkListenerExecutorRemoved", + | "Executor ID": "exec2" + |} + """ } diff --git a/core/src/test/scala/org/apache/spark/util/ResetSystemProperties.scala b/core/src/test/scala/org/apache/spark/util/ResetSystemProperties.scala new file mode 100644 index 000000000000..d4b92f33dd9e --- /dev/null +++ b/core/src/test/scala/org/apache/spark/util/ResetSystemProperties.scala @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util + +import java.util.Properties + +import org.scalatest.{BeforeAndAfterEach, Suite} + +/** + * Mixin for automatically resetting system properties that are modified in ScalaTest tests. + * This resets the properties after each individual test. + * + * The order in which fixtures are mixed in affects the order in which they are invoked by tests. + * If we have a suite `MySuite extends FunSuite with Foo with Bar`, then + * Bar's `super` is Foo, so Bar's beforeEach() will and afterEach() methods will be invoked first + * by the rest runner. + * + * This means that ResetSystemProperties should appear as the last trait in test suites that it's + * mixed into in order to ensure that the system properties snapshot occurs as early as possible. + * ResetSystemProperties calls super.afterEach() before performing its own cleanup, ensuring that + * the old properties are restored as late as possible. + * + * See the "Composing fixtures by stacking traits" section at + * http://www.scalatest.org/user_guide/sharing_fixtures for more details about this pattern. + */ +private[spark] trait ResetSystemProperties extends BeforeAndAfterEach { this: Suite => + var oldProperties: Properties = null + + override def beforeEach(): Unit = { + oldProperties = new Properties(System.getProperties) + super.beforeEach() + } + + override def afterEach(): Unit = { + try { + super.afterEach() + } finally { + System.setProperties(oldProperties) + oldProperties = null + } + } +} diff --git a/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala b/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala index 0ea2d13a8350..7424c2e91d4f 100644 --- a/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala @@ -17,9 +17,7 @@ package org.apache.spark.util -import org.scalatest.BeforeAndAfterAll -import org.scalatest.FunSuite -import org.scalatest.PrivateMethodTester +import org.scalatest.{BeforeAndAfterEach, BeforeAndAfterAll, FunSuite, PrivateMethodTester} class DummyClass1 {} @@ -46,20 +44,12 @@ class DummyString(val arr: Array[Char]) { } class SizeEstimatorSuite - extends FunSuite with BeforeAndAfterAll with PrivateMethodTester { + extends FunSuite with BeforeAndAfterEach with PrivateMethodTester with ResetSystemProperties { - var oldArch: String = _ - var oldOops: String = _ - - override def beforeAll() { + override def beforeEach() { // Set the arch to 64-bit and compressedOops to true to get a deterministic test-case - oldArch = System.setProperty("os.arch", "amd64") - oldOops = System.setProperty("spark.test.useCompressedOops", "true") - } - - override def afterAll() { - resetOrClear("os.arch", oldArch) - resetOrClear("spark.test.useCompressedOops", oldOops) + System.setProperty("os.arch", "amd64") + System.setProperty("spark.test.useCompressedOops", "true") } test("simple classes") { @@ -122,7 +112,7 @@ class SizeEstimatorSuite } test("32-bit arch") { - val arch = System.setProperty("os.arch", "x86") + System.setProperty("os.arch", "x86") val initialize = PrivateMethod[Unit]('initialize) SizeEstimator invokePrivate initialize() @@ -131,14 +121,13 @@ class SizeEstimatorSuite assertResult(48)(SizeEstimator.estimate(DummyString("a"))) assertResult(48)(SizeEstimator.estimate(DummyString("ab"))) assertResult(56)(SizeEstimator.estimate(DummyString("abcdefgh"))) - resetOrClear("os.arch", arch) } // NOTE: The String class definition varies across JDK versions (1.6 vs. 1.7) and vendors // (Sun vs IBM). Use a DummyString class to make tests deterministic. test("64-bit arch with no compressed oops") { - val arch = System.setProperty("os.arch", "amd64") - val oops = System.setProperty("spark.test.useCompressedOops", "false") + System.setProperty("os.arch", "amd64") + System.setProperty("spark.test.useCompressedOops", "false") val initialize = PrivateMethod[Unit]('initialize) SizeEstimator invokePrivate initialize() @@ -146,16 +135,5 @@ class SizeEstimatorSuite assertResult(64)(SizeEstimator.estimate(DummyString("a"))) assertResult(64)(SizeEstimator.estimate(DummyString("ab"))) assertResult(72)(SizeEstimator.estimate(DummyString("abcdefgh"))) - - resetOrClear("os.arch", arch) - resetOrClear("spark.test.useCompressedOops", oops) - } - - def resetOrClear(prop: String, oldValue: String) { - if (oldValue != null) { - System.setProperty(prop, oldValue) - } else { - System.clearProperty(prop) - } } } 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 f9d4bea823f7..4544382094f9 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -31,7 +31,7 @@ import org.scalatest.FunSuite import org.apache.spark.SparkConf -class UtilsSuite extends FunSuite { +class UtilsSuite extends FunSuite with ResetSystemProperties { test("bytesToString") { assert(Utils.bytesToString(10) === "10.0 B") diff --git a/dev/run-tests b/dev/run-tests index 20603fc08923..2257a566bb1b 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -21,8 +21,10 @@ FWDIR="$(cd "`dirname $0`"/..; pwd)" cd "$FWDIR" -# Remove work directory +# Clean up work directory and caches rm -rf ./work +rm -rf ~/.ivy2/local/org.apache.spark +rm -rf ~/.ivy2/cache/org.apache.spark source "$FWDIR/dev/run-tests-codes.sh" diff --git a/docs/_config.yml b/docs/_config.yml index a96a76dd9ab5..e2db274e1f61 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -17,6 +17,6 @@ SPARK_VERSION: 1.3.0-SNAPSHOT SPARK_VERSION_SHORT: 1.3.0 SCALA_BINARY_VERSION: "2.10" SCALA_VERSION: "2.10.4" -MESOS_VERSION: 0.18.1 +MESOS_VERSION: 0.21.0 SPARK_ISSUE_TRACKER_URL: https://issues.apache.org/jira/browse/SPARK SPARK_GITHUB_URL: https://github.com/apache/spark diff --git a/docs/building-spark.md b/docs/building-spark.md index c1bcd91b5b85..fb93017861ed 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -151,9 +151,10 @@ Thus, the full flow for running continuous-compilation of the `core` submodule m $ mvn scala:cc ``` -# Using With IntelliJ IDEA +# Building Spark with IntelliJ IDEA or Eclipse -This setup works fine in IntelliJ IDEA 11.1.4. After opening the project via the pom.xml file in the project root folder, you only need to activate either the hadoop1 or hadoop2 profile in the "Maven Properties" popout. We have not tried Eclipse/Scala IDE with this. +For help in setting up IntelliJ IDEA or Eclipse for Spark development, and troubleshooting, refer to the +[wiki page for IDE setup](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark#ContributingtoSpark-IDESetup). # Building Spark Debian Packages diff --git a/docs/configuration.md b/docs/configuration.md index fa9d311f8506..efbab4085317 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -102,11 +102,10 @@ of the most common options to set are: - spark.executor.memory - 512m + spark.driver.cores + 1 - Amount of memory to use per executor process, in the same format as JVM memory strings - (e.g. 512m, 2g). + Number of cores to use for the driver process, only in cluster mode. @@ -117,6 +116,14 @@ of the most common options to set are: (e.g. 512m, 2g). + + spark.executor.memory + 512m + + Amount of memory to use per executor process, in the same format as JVM memory strings + (e.g. 512m, 2g). + + spark.driver.maxResultSize 1g @@ -678,7 +685,7 @@ Apart from these, the following properties are also available, and may be useful spark.storage.memoryMapThreshold - 8192 + 2097152 Size of a block, in bytes, above which Spark memory maps when reading a block from disk. This prevents Spark from memory mapping very small blocks. In general, memory @@ -709,7 +716,9 @@ Apart from these, the following properties are also available, and may be useful If set to true, validates the output specification (e.g. checking if the output directory already exists) used in saveAsHadoopFile and other variants. This can be disabled to silence exceptions due to pre-existing output directories. We recommend that users do not disable this except if trying to achieve compatibility with - previous versions of Spark. Simply use Hadoop's FileSystem API to delete output directories by hand. + previous versions of Spark. Simply use Hadoop's FileSystem API to delete output directories by hand. + This setting is ignored for jobs generated through Spark Streaming's StreamingContext, since + data may need to be rewritten to pre-existing output directories during checkpoint recovery. spark.hadoop.cloneConf @@ -816,6 +825,16 @@ Apart from these, the following properties are also available, and may be useful Communication timeout between Spark nodes, in seconds. + + spark.network.timeout + 120 + + Default timeout for all network interactions, in seconds. This config will be used in + place of spark.core.connection.ack.wait.timeout, spark.akka.timeout, + spark.storage.blockManagerSlaveTimeoutMs or + spark.shuffle.io.connectionTimeout, if they are not configured. + + spark.akka.heartbeat.pauses 6000 @@ -1216,7 +1235,7 @@ Apart from these, the following properties are also available, and may be useful - spark.streaming.receiver.writeAheadLogs.enable + spark.streaming.receiver.writeAheadLog.enable false Enable write ahead logs for receivers. All the input data received through receivers diff --git a/docs/ml-guide.md b/docs/ml-guide.md index 1c2e27341473..be178d7689fd 100644 --- a/docs/ml-guide.md +++ b/docs/ml-guide.md @@ -3,13 +3,16 @@ layout: global title: Spark ML Programming Guide --- -Spark ML is Spark's new machine learning package. It is currently an alpha component but is potentially a successor to [MLlib](mllib-guide.html). The `spark.ml` package aims to replace the old APIs with a cleaner, more uniform set of APIs which will help users create full machine learning pipelines. +`spark.ml` is a new package introduced in Spark 1.2, which aims to provide a uniform set of +high-level APIs that help users create and tune practical machine learning pipelines. +It is currently an alpha component, and we would like to hear back from the community about +how it fits real-world use cases and how it could be improved. -MLlib vs. Spark ML: - -* Users can use algorithms from either of the two packages, but APIs may differ. Currently, `spark.ml` offers a subset of the algorithms from `spark.mllib`. Since Spark ML is an alpha component, its API may change in future releases. -* Developers should contribute new algorithms to `spark.mllib` and can optionally contribute to `spark.ml`. See below for more details. -* Spark ML only has Scala and Java APIs, whereas MLlib also has a Python API. +Note that we will keep supporting and adding features to `spark.mllib` along with the +development of `spark.ml`. +Users should be comfortable using `spark.mllib` features and expect more features coming. +Developers should contribute new algorithms to `spark.mllib` and can optionally contribute +to `spark.ml`. **Table of Contents** @@ -686,17 +689,3 @@ Spark ML currently depends on MLlib and has the same dependencies. Please see the [MLlib Dependencies guide](mllib-guide.html#Dependencies) for more info. Spark ML also depends upon Spark SQL, but the relevant parts of Spark SQL do not bring additional dependencies. - -# Developers - -**Development plan** - -If all goes well, `spark.ml` will become the primary ML package at the time of the Spark 1.3 release. Initially, simple wrappers will be used to port algorithms to `spark.ml`, but eventually, code will be moved to `spark.ml` and `spark.mllib` will be deprecated. - -**Advice to developers** - -During the next development cycle, new algorithms should be contributed to `spark.mllib`, but we welcome patches sent to either package. If an algorithm is best expressed using the new API (e.g., feature transformers), we may ask for developers to use the new `spark.ml` API. -Wrappers for old and new algorithms can be contributed to `spark.ml`. - -Users will be able to use algorithms from either of the two packages. The main difficulty will be the differences in APIs between the two packages. - diff --git a/docs/mllib-guide.md b/docs/mllib-guide.md index efd7dda31071..39c64d06926b 100644 --- a/docs/mllib-guide.md +++ b/docs/mllib-guide.md @@ -35,16 +35,20 @@ MLlib is under active development. The APIs marked `Experimental`/`DeveloperApi` may change in future releases, and the migration guide below will explain all changes between releases. -# spark.ml: The New ML Package +# spark.ml: high-level APIs for ML pipelines -Spark 1.2 includes a new machine learning package called `spark.ml`, currently an alpha component but potentially a successor to `spark.mllib`. The `spark.ml` package aims to replace the old APIs with a cleaner, more uniform set of APIs which will help users create full machine learning pipelines. +Spark 1.2 includes a new package called `spark.ml`, which aims to provide a uniform set of +high-level APIs that help users create and tune practical machine learning pipelines. +It is currently an alpha component, and we would like to hear back from the community about +how it fits real-world use cases and how it could be improved. -See the **[spark.ml programming guide](ml-guide.html)** for more information on this package. - -Users can use algorithms from either of the two packages, but APIs may differ. Currently, `spark.ml` offers a subset of the algorithms from `spark.mllib`. +Note that we will keep supporting and adding features to `spark.mllib` along with the +development of `spark.ml`. +Users should be comfortable using `spark.mllib` features and expect more features coming. +Developers should contribute new algorithms to `spark.mllib` and can optionally contribute +to `spark.ml`. -Developers should contribute new algorithms to `spark.mllib` and can optionally contribute to `spark.ml`. -See the `spark.ml` programming guide linked above for more details. +See the **[spark.ml programming guide](ml-guide.html)** for more information on this package. # Dependencies diff --git a/docs/programming-guide.md b/docs/programming-guide.md index 5e0d5c15d706..2443fc29b470 100644 --- a/docs/programming-guide.md +++ b/docs/programming-guide.md @@ -913,7 +913,7 @@ for details. cogroup(otherDataset, [numTasks]) - When called on datasets of type (K, V) and (K, W), returns a dataset of (K, Iterable<V>, Iterable<W>) tuples. This operation is also called groupWith. + When called on datasets of type (K, V) and (K, W), returns a dataset of (K, (Iterable<V>, Iterable<W>)) tuples. This operation is also called groupWith. cartesian(otherDataset) @@ -1316,7 +1316,35 @@ For accumulator updates performed inside actions only, Spark guarantees t will only be applied once, i.e. restarted tasks will not update the value. In transformations, users should be aware of that each task's update may be applied more than once if tasks or job stages are re-executed. +Accumulators do not change the lazy evaluation model of Spark. If they are being updated within an operation on an RDD, their value is only updated once that RDD is computed as part of an action. Consequently, accumulator updates are not guaranteed to be executed when made within a lazy transformation like `map()`. The below code fragment demonstrates this property: +
    + +
    +{% highlight scala %} +val acc = sc.accumulator(0) +data.map(x => acc += x; f(x)) +// Here, acc is still 0 because no actions have cause the `map` to be computed. +{% endhighlight %} +
    + +
    +{% highlight java %} +Accumulator accum = sc.accumulator(0); +data.map(x -> accum.add(x); f(x);); +// Here, accum is still 0 because no actions have cause the `map` to be computed. +{% endhighlight %} +
    + +
    +{% highlight python %} +accum = sc.accumulator(0) +data.map(lambda x => acc.add(x); f(x)) +# Here, acc is still 0 because no actions have cause the `map` to be computed. +{% endhighlight %} +
    + +
    # Deploying to a Cluster diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index da1c8e8aa866..68ab127bcf08 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -21,6 +21,31 @@ Most of the configs are the same for Spark on YARN as for other deployment modes + + + + + + + + + + + + + + + @@ -90,7 +115,14 @@ Most of the configs are the same for Spark on YARN as for other deployment modes + + + + + @@ -145,10 +177,18 @@ Most of the configs are the same for Spark on YARN as for other deployment modes + + + + +
    Property NameDefaultMeaning
    spark.yarn.am.memory512m + Amount of memory to use for the YARN Application Master in client mode, in the same format as JVM memory strings (e.g. 512m, 2g). + In cluster mode, use spark.driver.memory instead. +
    spark.driver.cores1 + Number of cores used by the driver in YARN cluster mode. + Since the driver is run in the same JVM as the YARN Application Master in cluster mode, this also controls the cores used by the YARN AM. + In client mode, use spark.yarn.am.cores to control the number of cores used by the YARN AM instead. +
    spark.yarn.am.cores1 + Number of cores to use for the YARN Application Master in client mode. + In cluster mode, use spark.driver.cores instead. +
    spark.yarn.am.waitTime 100000spark.yarn.driver.memoryOverhead 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. This tends to grow with the container size (typically 6-10%). + The amount of off heap memory (in megabytes) to be allocated per driver in cluster mode. 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%). +
    spark.yarn.am.memoryOverheadAM memory * 0.07, with minimum of 384 + Same as spark.yarn.driver.memoryOverhead, but for the Application Master in client mode.
    spark.yarn.am.extraJavaOptions (none) - A string of extra JVM options to pass to the Yarn ApplicationMaster in client mode. + A string of extra JVM options to pass to the YARN Application Master in client mode. In cluster mode, use spark.driver.extraJavaOptions instead.
    spark.yarn.maxAppAttemptsyarn.resourcemanager.am.max-attempts in YARN + The maximum number of attempts that will be made to submit the application. + It should be no larger than the global number of max attempts in the YARN configuration. +
    # Launching Spark on YARN diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 2aea8a8aedaf..be8c5c2c1522 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -831,13 +831,10 @@ turning on some experimental options. ## Caching Data In Memory -Spark SQL can cache tables using an in-memory columnar format by calling `sqlContext.cacheTable("tableName")`. +Spark SQL can cache tables using an in-memory columnar format by calling `sqlContext.cacheTable("tableName")` or `schemaRDD.cache()`. Then Spark SQL will scan only required columns and will automatically tune compression to minimize memory usage and GC pressure. You can call `sqlContext.uncacheTable("tableName")` to remove the table from memory. -Note that if you call `schemaRDD.cache()` rather than `sqlContext.cacheTable(...)`, tables will _not_ be cached using -the in-memory columnar format, and therefore `sqlContext.cacheTable(...)` is strongly recommended for this use case. - Configuration of in-memory caching can be done using the `setConf` method on SQLContext or by running `SET key=value` commands using SQL. @@ -1010,12 +1007,11 @@ let user control table caching explicitly: CACHE TABLE logs_last_month; UNCACHE TABLE logs_last_month; -**NOTE:** `CACHE TABLE tbl` is lazy, similar to `.cache` on an RDD. This command only marks `tbl` to ensure that -partitions are cached when calculated but doesn't actually cache it until a query that touches `tbl` is executed. -To force the table to be cached, you may simply count the table immediately after executing `CACHE TABLE`: +**NOTE:** `CACHE TABLE tbl` is now __eager__ by default not __lazy__. Don’t need to trigger cache materialization manually anymore. - CACHE TABLE logs_last_month; - SELECT COUNT(1) FROM logs_last_month; +Spark SQL newly introduced a statement to let user control table caching whether or not lazy since Spark 1.2.0: + + CACHE [LAZY] TABLE [AS SELECT] ... Several caching related features are not supported yet: @@ -1337,9 +1333,9 @@ import org.apache.spark.sql._
    All data types of Spark SQL are located in the package of -`org.apache.spark.sql.api.java`. To access or create a data type, +`org.apache.spark.sql.types`. To access or create a data type, please use factory methods provided in -`org.apache.spark.sql.api.java.DataType`. +`org.apache.spark.sql.types.DataTypes`. @@ -1350,109 +1346,110 @@ please use factory methods provided in @@ -1462,7 +1459,7 @@ please use factory methods provided in
    ByteType byte or Byte - DataType.ByteType + DataTypes.ByteType
    ShortType short or Short - DataType.ShortType + DataTypes.ShortType
    IntegerType int or Integer - DataType.IntegerType + DataTypes.IntegerType
    LongType long or Long - DataType.LongType + DataTypes.LongType
    FloatType float or Float - DataType.FloatType + DataTypes.FloatType
    DoubleType double or Double - DataType.DoubleType + DataTypes.DoubleType
    DecimalType java.math.BigDecimal - DataType.DecimalType + DataTypes.createDecimalType()
    + DataTypes.createDecimalType(precision, scale).
    StringType String - DataType.StringType + DataTypes.StringType
    BinaryType byte[] - DataType.BinaryType + DataTypes.BinaryType
    BooleanType boolean or Boolean - DataType.BooleanType + DataTypes.BooleanType
    TimestampType java.sql.Timestamp - DataType.TimestampType + DataTypes.TimestampType
    DateType java.sql.Date - DataType.DateType + DataTypes.DateType
    ArrayType java.util.List - DataType.createArrayType(elementType)
    + DataTypes.createArrayType(elementType)
    Note: The value of containsNull will be true
    - DataType.createArrayType(elementType, containsNull). + DataTypes.createArrayType(elementType, containsNull).
    MapType java.util.Map - DataType.createMapType(keyType, valueType)
    + DataTypes.createMapType(keyType, valueType)
    Note: The value of valueContainsNull will be true.
    - DataType.createMapType(keyType, valueType, valueContainsNull)
    + DataTypes.createMapType(keyType, valueType, valueContainsNull)
    StructType org.apache.spark.sql.api.java.Row - DataType.createStructType(fields)
    + DataTypes.createStructType(fields)
    Note: fields is a List or an array of StructFields. Also, two fields with the same name are not allowed.
    The value type in Java of the data type of this field (For example, int for a StructField with the data type IntegerType) - DataType.createStructField(name, dataType, nullable) + DataTypes.createStructField(name, dataType, nullable)
    diff --git a/docs/streaming-kafka-integration.md b/docs/streaming-kafka-integration.md index 1c956fcb40da..0e38fe2144e9 100644 --- a/docs/streaming-kafka-integration.md +++ b/docs/streaming-kafka-integration.md @@ -4,7 +4,7 @@ title: Spark Streaming + Kafka Integration Guide --- [Apache Kafka](http://kafka.apache.org/) is publish-subscribe messaging rethought as a distributed, partitioned, replicated commit log service. Here we explain how to configure Spark Streaming to receive data from Kafka. -1. **Linking:** In your SBT/Maven projrect definition, link your streaming application against the following artifact (see [Linking section](streaming-programming-guide.html#linking) in the main programming guide for further information). +1. **Linking:** In your SBT/Maven project definition, link your streaming application against the following artifact (see [Linking section](streaming-programming-guide.html#linking) in the main programming guide for further information). groupId = org.apache.spark artifactId = spark-streaming-kafka_{{site.SCALA_BINARY_VERSION}} @@ -20,7 +20,7 @@ title: Spark Streaming + Kafka Integration Guide streamingContext, [zookeeperQuorum], [group id of the consumer], [per-topic number of Kafka partitions to consume]) See the [API docs](api/scala/index.html#org.apache.spark.streaming.kafka.KafkaUtils$) - and the [example]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala). + and the [example]({{site.SPARK_GITHUB_URL}}/blob/master/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala).
    import org.apache.spark.streaming.kafka.*; diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 01450efe35e5..e37a2bb37b9a 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -1574,7 +1574,7 @@ To run a Spark Streaming applications, you need to have the following. recovery, thus ensuring zero data loss (discussed in detail in the [Fault-tolerance Semantics](#fault-tolerance-semantics) section). This can be enabled by setting the [configuration parameter](configuration.html#spark-streaming) - `spark.streaming.receiver.writeAheadLogs.enable` to `true`. However, these stronger semantics may + `spark.streaming.receiver.writeAheadLog.enable` to `true`. However, these stronger semantics may come at the cost of the receiving throughput of individual receivers. This can be corrected by running [more receivers in parallel](#level-of-parallelism-in-data-receiving) to increase aggregate throughput. Additionally, it is recommended that the replication of the diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 485eea4f5e68..abab209a05ba 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -39,10 +39,26 @@ from optparse import OptionParser from sys import stderr +VALID_SPARK_VERSIONS = set([ + "0.7.3", + "0.8.0", + "0.8.1", + "0.9.0", + "0.9.1", + "0.9.2", + "1.0.0", + "1.0.1", + "1.0.2", + "1.1.0", + "1.1.1", + "1.2.0", +]) + DEFAULT_SPARK_VERSION = "1.2.0" +DEFAULT_SPARK_GITHUB_REPO = "https://github.com/apache/spark" SPARK_EC2_DIR = os.path.dirname(os.path.realpath(__file__)) - MESOS_SPARK_EC2_BRANCH = "branch-1.3" + # A URL prefix from which to fetch AMI information AMI_PREFIX = "https://raw.github.com/mesos/spark-ec2/{b}/ami-list".format(b=MESOS_SPARK_EC2_BRANCH) @@ -126,8 +142,8 @@ def parse_args(): help="Version of Spark to use: 'X.Y.Z' or a specific git hash (default: %default)") parser.add_option( "--spark-git-repo", - default="https://github.com/apache/spark", - help="Github repo from which to checkout supplied commit hash") + default=DEFAULT_SPARK_GITHUB_REPO, + help="Github repo from which to checkout supplied commit hash (default: %default)") parser.add_option( "--hadoop-major-version", default="1", help="Major version of Hadoop (default: %default)") @@ -236,6 +252,26 @@ def get_or_make_group(conn, name, vpc_id): return conn.create_security_group(name, "Spark EC2 group", vpc_id) +def get_validate_spark_version(version, repo): + if "." in version: + version = version.replace("v", "") + if version not in VALID_SPARK_VERSIONS: + print >> stderr, "Don't know about Spark version: {v}".format(v=version) + sys.exit(1) + return version + else: + github_commit_url = "{repo}/commit/{commit_hash}".format(repo=repo, commit_hash=version) + request = urllib2.Request(github_commit_url) + request.get_method = lambda: 'HEAD' + try: + response = urllib2.urlopen(request) + except urllib2.HTTPError, e: + print >> stderr, "Couldn't validate Spark commit: {url}".format(url=github_commit_url) + print >> stderr, "Received HTTP response code of {code}.".format(code=e.code) + sys.exit(1) + return version + + # Check whether a given EC2 instance object is in a state we consider active, # i.e. not terminating or terminated. We count both stopping and stopped as # active since we can restart stopped clusters. @@ -243,29 +279,6 @@ def is_active(instance): return (instance.state in ['pending', 'running', 'stopping', 'stopped']) -# 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", - # These are dummy versions (no Shark versions after this) - "1.0.0": "1.0.0", - "1.0.1": "1.0.1", - "1.0.2": "1.0.2", - "1.1.0": "1.1.0", - "1.1.1": "1.1.1", - "1.2.0": "1.2.0", - } - version = opts.spark_version.replace("v", "") - if version not in spark_shark_map: - print >> stderr, "Don't know about Spark version: %s" % version - sys.exit(1) - return (version, spark_shark_map[version]) - - # 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 @@ -619,7 +632,7 @@ def setup_cluster(conn, master_nodes, slave_nodes, opts, deploy_ssh_key): print slave.public_dns_name ssh_write(slave.public_dns_name, opts, ['tar', 'x'], dot_ssh_tar) - modules = ['spark', 'shark', 'ephemeral-hdfs', 'persistent-hdfs', + modules = ['spark', 'ephemeral-hdfs', 'persistent-hdfs', 'mapreduce', 'spark-standalone', 'tachyon'] if opts.hadoop_major_version == "1": @@ -706,9 +719,7 @@ def wait_for_cluster_state(conn, opts, cluster_instances, cluster_state): sys.stdout.flush() start_time = datetime.now() - num_attempts = 0 - conn = ec2.connect_to_region(opts.region) while True: time.sleep(5 * num_attempts) # seconds @@ -815,13 +826,11 @@ def deploy_files(conn, root_dir, opts, master_nodes, slave_nodes, modules): cluster_url = "%s:7077" % active_master if "." in opts.spark_version: - # Pre-built spark & shark deploy - (spark_v, shark_v) = get_spark_shark_version(opts) + # Pre-built Spark deploy + spark_v = get_validate_spark_version(opts.spark_version, opts.spark_git_repo) else: # Spark-only custom deploy spark_v = "%s|%s" % (opts.spark_git_repo, opts.spark_version) - shark_v = "" - modules = filter(lambda x: x != "shark", modules) template_vars = { "master_list": '\n'.join([i.public_dns_name for i in master_nodes]), @@ -834,7 +843,6 @@ def deploy_files(conn, root_dir, opts, master_nodes, slave_nodes, modules): "swap": str(opts.swap), "modules": '\n'.join(modules), "spark_version": spark_v, - "shark_version": shark_v, "hadoop_major_version": opts.hadoop_major_version, "spark_worker_instances": "%d" % opts.worker_instances, "spark_master_opts": opts.master_opts @@ -983,6 +991,8 @@ def real_main(): (opts, action, cluster_name) = parse_args() # Input parameter validation + get_validate_spark_version(opts.spark_version, opts.spark_git_repo) + if opts.wait is not None: # NOTE: DeprecationWarnings are silent in 2.7+ by default. # To show them, run Python with the -Wdefault switch. diff --git a/examples/pom.xml b/examples/pom.xml index 8713230e1e8e..4b92147725f6 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -98,143 +98,145 @@ ${project.version} - org.eclipse.jetty - jetty-server + org.apache.hbase + hbase-testing-util + ${hbase.version} + ${hbase.deps.scope} + + + + org.apache.hbase + hbase-annotations + + + org.jruby + jruby-complete + + + + + org.apache.hbase + hbase-protocol + ${hbase.version} + ${hbase.deps.scope} + + + org.apache.hbase + hbase-common + ${hbase.version} + ${hbase.deps.scope} + + + + org.apache.hbase + hbase-annotations + + + + + org.apache.hbase + hbase-client + ${hbase.version} + ${hbase.deps.scope} + + + + org.apache.hbase + hbase-annotations + + + io.netty + netty + + + + + org.apache.hbase + hbase-server + ${hbase.version} + ${hbase.deps.scope} + + + + org.apache.hbase + hbase-annotations + + + org.apache.hadoop + hadoop-core + + + org.apache.hadoop + hadoop-client + + + org.apache.hadoop + hadoop-mapreduce-client-jobclient + + + org.apache.hadoop + hadoop-mapreduce-client-core + + + org.apache.hadoop + hadoop-auth + + + org.apache.hadoop + hadoop-annotations + + + org.apache.hadoop + hadoop-hdfs + + + org.apache.hbase + hbase-hadoop1-compat + + + org.apache.commons + commons-math + + + com.sun.jersey + jersey-core + + + org.slf4j + slf4j-api + + + com.sun.jersey + jersey-server + + + com.sun.jersey + jersey-core + + + com.sun.jersey + jersey-json + + + + commons-io + commons-io + + + + + org.apache.hbase + hbase-hadoop-compat + ${hbase.version} + ${hbase.deps.scope} + + + org.apache.hbase + hbase-hadoop-compat + ${hbase.version} + test-jar + test - - org.apache.hbase - hbase-testing-util - ${hbase.version} - - - - org.apache.hbase - hbase-annotations - - - org.jruby - jruby-complete - - - - - org.apache.hbase - hbase-protocol - ${hbase.version} - - - org.apache.hbase - hbase-common - ${hbase.version} - - - - org.apache.hbase - hbase-annotations - - - - - org.apache.hbase - hbase-client - ${hbase.version} - - - - org.apache.hbase - hbase-annotations - - - io.netty - netty - - - - - org.apache.hbase - hbase-server - ${hbase.version} - - - org.apache.hadoop - hadoop-core - - - org.apache.hadoop - hadoop-client - - - org.apache.hadoop - hadoop-mapreduce-client-jobclient - - - org.apache.hadoop - hadoop-mapreduce-client-core - - - org.apache.hadoop - hadoop-auth - - - - org.apache.hbase - hbase-annotations - - - org.apache.hadoop - hadoop-annotations - - - org.apache.hadoop - hadoop-hdfs - - - org.apache.hbase - hbase-hadoop1-compat - - - org.apache.commons - commons-math - - - com.sun.jersey - jersey-core - - - org.slf4j - slf4j-api - - - com.sun.jersey - jersey-server - - - com.sun.jersey - jersey-core - - - com.sun.jersey - jersey-json - - - - commons-io - commons-io - - - - - org.apache.hbase - hbase-hadoop-compat - ${hbase.version} - - - org.apache.hbase - hbase-hadoop-compat - ${hbase.version} - test-jar - test - org.apache.commons commons-math3 @@ -244,11 +246,6 @@ algebird-core_${scala.binary.version} 0.8.1 - - org.scalatest - scalatest_${scala.binary.version} - test - org.scalacheck scalacheck_${scala.binary.version} @@ -313,31 +310,6 @@ org.apache.maven.plugins maven-shade-plugin - - false - ${project.build.directory}/scala-${scala.binary.version}/spark-examples-${project.version}-hadoop${hadoop.version}.jar - - - *:* - - - - - com.google.guava:guava - - com/google/common/base/Optional* - - - - *:* - - META-INF/*.SF - META-INF/*.DSA - META-INF/*.RSA - - - - package @@ -345,6 +317,34 @@ shade + false + ${project.build.directory}/scala-${scala.binary.version}/spark-examples-${project.version}-hadoop${hadoop.version}.jar + + + *:* + + + + + com.google.guava:guava + + + ** + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + com.google @@ -393,30 +393,7 @@ - hbase-hadoop2 - - - hbase.profile - hadoop2 - - - - 0.98.7-hadoop2 - - - - hbase-hadoop1 - - - !hbase.profile - - - - 0.98.7-hadoop1 - - - - scala-2.10 @@ -454,5 +431,37 @@ + + + + flume-provided + + provided + + + + hadoop-provided + + provided + + + + hbase-provided + + provided + + + + hive-provided + + provided + + + + parquet-provided + + provided + + diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaCrossValidatorExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaCrossValidatorExample.java index f4b4f8d8c7b2..247d2a5e31a8 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaCrossValidatorExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaCrossValidatorExample.java @@ -33,9 +33,9 @@ import org.apache.spark.ml.tuning.CrossValidator; import org.apache.spark.ml.tuning.CrossValidatorModel; import org.apache.spark.ml.tuning.ParamGridBuilder; -import org.apache.spark.sql.api.java.JavaSQLContext; -import org.apache.spark.sql.api.java.JavaSchemaRDD; -import org.apache.spark.sql.api.java.Row; +import org.apache.spark.sql.SchemaRDD; +import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.Row; /** * A simple example demonstrating model selection using CrossValidator. @@ -55,7 +55,7 @@ public class JavaCrossValidatorExample { public static void main(String[] args) { SparkConf conf = new SparkConf().setAppName("JavaCrossValidatorExample"); JavaSparkContext jsc = new JavaSparkContext(conf); - JavaSQLContext jsql = new JavaSQLContext(jsc); + SQLContext jsql = new SQLContext(jsc); // Prepare training documents, which are labeled. List localTraining = Lists.newArrayList( @@ -71,8 +71,7 @@ public static void main(String[] args) { new LabeledDocument(9L, "a e c l", 0.0), new LabeledDocument(10L, "spark compile", 1.0), new LabeledDocument(11L, "hadoop software", 0.0)); - JavaSchemaRDD training = - jsql.applySchema(jsc.parallelize(localTraining), LabeledDocument.class); + SchemaRDD training = jsql.applySchema(jsc.parallelize(localTraining), LabeledDocument.class); // Configure an ML pipeline, which consists of three stages: tokenizer, hashingTF, and lr. Tokenizer tokenizer = new Tokenizer() @@ -113,11 +112,11 @@ public static void main(String[] args) { new Document(5L, "l m n"), new Document(6L, "mapreduce spark"), new Document(7L, "apache hadoop")); - JavaSchemaRDD test = jsql.applySchema(jsc.parallelize(localTest), Document.class); + SchemaRDD test = jsql.applySchema(jsc.parallelize(localTest), Document.class); // Make predictions on test documents. cvModel uses the best model found (lrModel). cvModel.transform(test).registerAsTable("prediction"); - JavaSchemaRDD predictions = jsql.sql("SELECT id, text, score, prediction FROM prediction"); + SchemaRDD predictions = jsql.sql("SELECT id, text, score, prediction FROM prediction"); for (Row r: predictions.collect()) { System.out.println("(" + r.get(0) + ", " + r.get(1) + ") --> score=" + r.get(2) + ", prediction=" + r.get(3)); diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java index e25b271777ed..5b92655e2e83 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java @@ -28,9 +28,9 @@ import org.apache.spark.ml.classification.LogisticRegression; import org.apache.spark.mllib.linalg.Vectors; import org.apache.spark.mllib.regression.LabeledPoint; -import org.apache.spark.sql.api.java.JavaSQLContext; -import org.apache.spark.sql.api.java.JavaSchemaRDD; -import org.apache.spark.sql.api.java.Row; +import org.apache.spark.sql.SchemaRDD; +import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.Row; /** * A simple example demonstrating ways to specify parameters for Estimators and Transformers. @@ -44,7 +44,7 @@ public class JavaSimpleParamsExample { public static void main(String[] args) { SparkConf conf = new SparkConf().setAppName("JavaSimpleParamsExample"); JavaSparkContext jsc = new JavaSparkContext(conf); - JavaSQLContext jsql = new JavaSQLContext(jsc); + SQLContext jsql = new SQLContext(jsc); // Prepare training data. // We use LabeledPoint, which is a JavaBean. Spark SQL can convert RDDs of JavaBeans @@ -54,7 +54,7 @@ public static void main(String[] args) { new LabeledPoint(0.0, Vectors.dense(2.0, 1.0, -1.0)), new LabeledPoint(0.0, Vectors.dense(2.0, 1.3, 1.0)), new LabeledPoint(1.0, Vectors.dense(0.0, 1.2, -0.5))); - JavaSchemaRDD training = jsql.applySchema(jsc.parallelize(localTraining), LabeledPoint.class); + SchemaRDD training = jsql.applySchema(jsc.parallelize(localTraining), LabeledPoint.class); // Create a LogisticRegression instance. This instance is an Estimator. LogisticRegression lr = new LogisticRegression(); @@ -94,14 +94,14 @@ public static void main(String[] args) { new LabeledPoint(1.0, Vectors.dense(-1.0, 1.5, 1.3)), new LabeledPoint(0.0, Vectors.dense(3.0, 2.0, -0.1)), new LabeledPoint(1.0, Vectors.dense(0.0, 2.2, -1.5))); - JavaSchemaRDD test = jsql.applySchema(jsc.parallelize(localTest), LabeledPoint.class); + SchemaRDD test = jsql.applySchema(jsc.parallelize(localTest), LabeledPoint.class); // Make predictions on test documents using the Transformer.transform() method. // LogisticRegression.transform will only use the 'features' column. // Note that model2.transform() outputs a 'probability' column instead of the usual 'score' // column since we renamed the lr.scoreCol parameter previously. model2.transform(test).registerAsTable("results"); - JavaSchemaRDD results = + SchemaRDD results = jsql.sql("SELECT features, label, probability, prediction FROM results"); for (Row r: results.collect()) { System.out.println("(" + r.get(0) + ", " + r.get(1) + ") -> prob=" + r.get(2) diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleTextClassificationPipeline.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleTextClassificationPipeline.java index 54f18014e4b2..74db449fada7 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleTextClassificationPipeline.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleTextClassificationPipeline.java @@ -21,6 +21,7 @@ import com.google.common.collect.Lists; +import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.ml.Pipeline; import org.apache.spark.ml.PipelineModel; @@ -28,10 +29,9 @@ import org.apache.spark.ml.classification.LogisticRegression; import org.apache.spark.ml.feature.HashingTF; import org.apache.spark.ml.feature.Tokenizer; -import org.apache.spark.sql.api.java.JavaSQLContext; -import org.apache.spark.sql.api.java.JavaSchemaRDD; -import org.apache.spark.sql.api.java.Row; -import org.apache.spark.SparkConf; +import org.apache.spark.sql.SchemaRDD; +import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.Row; /** * A simple text classification pipeline that recognizes "spark" from input text. It uses the Java @@ -46,7 +46,7 @@ public class JavaSimpleTextClassificationPipeline { public static void main(String[] args) { SparkConf conf = new SparkConf().setAppName("JavaSimpleTextClassificationPipeline"); JavaSparkContext jsc = new JavaSparkContext(conf); - JavaSQLContext jsql = new JavaSQLContext(jsc); + SQLContext jsql = new SQLContext(jsc); // Prepare training documents, which are labeled. List localTraining = Lists.newArrayList( @@ -54,8 +54,7 @@ public static void main(String[] args) { new LabeledDocument(1L, "b d", 0.0), new LabeledDocument(2L, "spark f g h", 1.0), new LabeledDocument(3L, "hadoop mapreduce", 0.0)); - JavaSchemaRDD training = - jsql.applySchema(jsc.parallelize(localTraining), LabeledDocument.class); + SchemaRDD training = jsql.applySchema(jsc.parallelize(localTraining), LabeledDocument.class); // Configure an ML pipeline, which consists of three stages: tokenizer, hashingTF, and lr. Tokenizer tokenizer = new Tokenizer() @@ -80,11 +79,11 @@ public static void main(String[] args) { new Document(5L, "l m n"), new Document(6L, "mapreduce spark"), new Document(7L, "apache hadoop")); - JavaSchemaRDD test = jsql.applySchema(jsc.parallelize(localTest), Document.class); + SchemaRDD test = jsql.applySchema(jsc.parallelize(localTest), Document.class); // Make predictions on test documents. model.transform(test).registerAsTable("prediction"); - JavaSchemaRDD predictions = jsql.sql("SELECT id, text, score, prediction FROM prediction"); + SchemaRDD predictions = jsql.sql("SELECT id, text, score, prediction FROM prediction"); for (Row r: predictions.collect()) { System.out.println("(" + r.get(0) + ", " + r.get(1) + ") --> score=" + r.get(2) + ", prediction=" + r.get(3)); 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 01c77bd44337..b70804635d5c 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 @@ -26,9 +26,9 @@ import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.Function; -import org.apache.spark.sql.api.java.JavaSQLContext; -import org.apache.spark.sql.api.java.JavaSchemaRDD; -import org.apache.spark.sql.api.java.Row; +import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SchemaRDD; +import org.apache.spark.sql.Row; public class JavaSparkSQL { public static class Person implements Serializable { @@ -55,7 +55,7 @@ public void setAge(int age) { public static void main(String[] args) throws Exception { SparkConf sparkConf = new SparkConf().setAppName("JavaSparkSQL"); JavaSparkContext ctx = new JavaSparkContext(sparkConf); - JavaSQLContext sqlCtx = new JavaSQLContext(ctx); + SQLContext sqlCtx = new SQLContext(ctx); System.out.println("=== Data source: RDD ==="); // Load a text file and convert each line to a Java Bean. @@ -74,15 +74,15 @@ public Person call(String line) { }); // Apply a schema to an RDD of Java Beans and register it as a table. - JavaSchemaRDD schemaPeople = sqlCtx.applySchema(people, Person.class); + SchemaRDD schemaPeople = sqlCtx.applySchema(people, Person.class); schemaPeople.registerTempTable("people"); // SQL can be run over RDDs that have been registered as tables. - JavaSchemaRDD teenagers = sqlCtx.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19"); + SchemaRDD teenagers = sqlCtx.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19"); // 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() { + List teenagerNames = teenagers.toJavaRDD().map(new Function() { @Override public String call(Row row) { return "Name: " + row.getString(0); @@ -99,13 +99,13 @@ public String call(Row row) { // Read in the parquet file created above. // Parquet files are self-describing so the schema is preserved. // The result of loading a parquet file is also a JavaSchemaRDD. - JavaSchemaRDD parquetFile = sqlCtx.parquetFile("people.parquet"); + SchemaRDD parquetFile = sqlCtx.parquetFile("people.parquet"); //Parquet files can also be registered as tables and then used in SQL statements. parquetFile.registerTempTable("parquetFile"); - JavaSchemaRDD teenagers2 = + SchemaRDD teenagers2 = sqlCtx.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19"); - teenagerNames = teenagers2.map(new Function() { + teenagerNames = teenagers2.toJavaRDD().map(new Function() { @Override public String call(Row row) { return "Name: " + row.getString(0); @@ -120,7 +120,7 @@ public String call(Row row) { // The path can be either a single text file or a directory storing text files. String path = "examples/src/main/resources/people.json"; // Create a JavaSchemaRDD from the file(s) pointed by path - JavaSchemaRDD peopleFromJsonFile = sqlCtx.jsonFile(path); + SchemaRDD peopleFromJsonFile = sqlCtx.jsonFile(path); // Because the schema of a JSON dataset is automatically inferred, to write queries, // it is better to take a look at what is the schema. @@ -134,11 +134,11 @@ public String call(Row row) { peopleFromJsonFile.registerTempTable("people"); // SQL statements can be run by using the sql methods provided by sqlCtx. - JavaSchemaRDD teenagers3 = sqlCtx.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19"); + SchemaRDD teenagers3 = sqlCtx.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19"); // 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() { + teenagerNames = teenagers3.toJavaRDD().map(new Function() { @Override public String call(Row row) { return "Name: " + row.getString(0); } }).collect(); @@ -151,7 +151,7 @@ public String call(Row row) { List jsonData = Arrays.asList( "{\"name\":\"Yin\",\"address\":{\"city\":\"Columbus\",\"state\":\"Ohio\"}}"); JavaRDD anotherPeopleRDD = ctx.parallelize(jsonData); - JavaSchemaRDD peopleFromJsonRDD = sqlCtx.jsonRDD(anotherPeopleRDD); + SchemaRDD peopleFromJsonRDD = sqlCtx.jsonRDD(anotherPeopleRDD.rdd()); // Take a look at the schema of this new JavaSchemaRDD. peopleFromJsonRDD.printSchema(); @@ -164,8 +164,8 @@ public String call(Row row) { peopleFromJsonRDD.registerTempTable("people2"); - JavaSchemaRDD peopleWithCity = sqlCtx.sql("SELECT name, address.city FROM people2"); - List nameAndCity = peopleWithCity.map(new Function() { + SchemaRDD peopleWithCity = sqlCtx.sql("SELECT name, address.city FROM people2"); + List nameAndCity = peopleWithCity.toJavaRDD().map(new Function() { @Override public String call(Row row) { return "Name: " + row.getString(0) + ", City: " + row.getString(1); diff --git a/examples/src/main/python/streaming/kafka_wordcount.py b/examples/src/main/python/streaming/kafka_wordcount.py new file mode 100644 index 000000000000..dad760aa4db5 --- /dev/null +++ b/examples/src/main/python/streaming/kafka_wordcount.py @@ -0,0 +1,57 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +""" + Counts words in UTF8 encoded, '\n' delimited text received from the network every second. + Usage: network_wordcount.py + + To run this on your local machine, you need to setup Kafka and create a producer first + $ bin/zookeeper-server-start.sh config/zookeeper.properties + $ bin/kafka-server-start.sh config/server.properties + $ bin/kafka-topics.sh --create --zookeeper localhost:2181 --partitions 1 --topic test + $ bin/kafka-console-producer.sh --broker-list localhost:9092 --topic test + + and then run the example + `$ bin/spark-submit --driver-class-path external/kafka-assembly/target/scala-*/\ + spark-streaming-kafka-assembly-*.jar examples/src/main/python/streaming/kafka_wordcount.py \ + localhost:2181 test` +""" + +import sys + +from pyspark import SparkContext +from pyspark.streaming import StreamingContext +from pyspark.streaming.kafka import KafkaUtils + +if __name__ == "__main__": + if len(sys.argv) != 3: + print >> sys.stderr, "Usage: network_wordcount.py " + exit(-1) + + sc = SparkContext(appName="PythonStreamingKafkaWordCount") + ssc = StreamingContext(sc, 1) + + zkQuorum, topic = sys.argv[1:] + kvs = KafkaUtils.createStream(ssc, zkQuorum, "spark-streaming-consumer", {topic: 1}) + lines = kvs.map(lambda x: x[1]) + counts = lines.flatMap(lambda line: line.split(" ")) \ + .map(lambda word: (word, 1)) \ + .reduceByKey(lambda a, b: a+b) + counts.pprint() + + ssc.start() + ssc.awaitTermination() diff --git a/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala index adecd934358c..1b53f3edbe92 100644 --- a/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala @@ -28,11 +28,9 @@ object BroadcastTest { val bcName = if (args.length > 2) args(2) else "Http" val blockSize = if (args.length > 3) args(3) else "4096" - System.setProperty("spark.broadcast.factory", "org.apache.spark.broadcast." + bcName + - "BroadcastFactory") - System.setProperty("spark.broadcast.blockSize", blockSize) val sparkConf = new SparkConf().setAppName("Broadcast Test") - + .set("spark.broadcast.factory", s"org.apache.spark.broadcast.${bcName}BroaddcastFactory") + .set("spark.broadcast.blockSize", blockSize) val sc = new SparkContext(sparkConf) val slices = if (args.length > 0) args(0).toInt else 2 diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala b/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala index 9fbb0a800d73..35b8dd6c29b6 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala @@ -27,8 +27,8 @@ object SparkPi { val conf = new SparkConf().setAppName("Spark Pi") val spark = new SparkContext(conf) val slices = if (args.length > 0) args(0).toInt else 2 - val n = 100000 * slices - val count = spark.parallelize(1 to n, slices).map { i => + val n = math.min(100000L * slices, Int.MaxValue).toInt // avoid overflow + val count = spark.parallelize(1 until n, slices).map { i => val x = random * 2 - 1 val y = random * 2 - 1 if (x*x + y*y < 1) 1 else 0 diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/CrossValidatorExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/CrossValidatorExample.scala index ce6bc066bd70..d8c7ef38ee46 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/CrossValidatorExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/CrossValidatorExample.scala @@ -106,5 +106,7 @@ object CrossValidatorExample { .foreach { case Row(id: Long, text: String, score: Double, prediction: Double) => println("(" + id + ", " + text + ") --> score=" + score + ", prediction=" + prediction) } + + sc.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala index 44d5b084c269..e8a2adff929c 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala @@ -97,5 +97,7 @@ object SimpleParamsExample { .foreach { case Row(features: Vector, label: Double, prob: Double, prediction: Double) => println("(" + features + ", " + label + ") -> prob=" + prob + ", prediction=" + prediction) } + + sc.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala index 92895a05e479..b9a6ef0229de 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala @@ -85,5 +85,7 @@ object SimpleTextClassificationPipeline { .foreach { case Row(id: Long, text: String, score: Double, prediction: Double) => println("(" + id + ", " + text + ") --> score=" + score + ", prediction=" + prediction) } + + sc.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/DenseGmmEM.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/DenseGmmEM.scala new file mode 100644 index 000000000000..de58be38c7bf --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/DenseGmmEM.scala @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.mllib + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.mllib.clustering.GaussianMixtureEM +import org.apache.spark.mllib.linalg.Vectors + +/** + * An example Gaussian Mixture Model EM app. Run with + * {{{ + * ./bin/run-example org.apache.spark.examples.mllib.DenseGmmEM + * }}} + * If you use it as a template to create your own app, please use `spark-submit` to submit your app. + */ +object DenseGmmEM { + def main(args: Array[String]): Unit = { + if (args.length < 3) { + println("usage: DenseGmmEM [maxIterations]") + } else { + val maxIterations = if (args.length > 3) args(3).toInt else 100 + run(args(0), args(1).toInt, args(2).toDouble, maxIterations) + } + } + + private def run(inputFile: String, k: Int, convergenceTol: Double, maxIterations: Int) { + val conf = new SparkConf().setAppName("Gaussian Mixture Model EM example") + val ctx = new SparkContext(conf) + + val data = ctx.textFile(inputFile).map { line => + Vectors.dense(line.trim.split(' ').map(_.toDouble)) + }.cache() + + val clusters = new GaussianMixtureEM() + .setK(k) + .setConvergenceTol(convergenceTol) + .setMaxIterations(maxIterations) + .run(data) + + for (i <- 0 until clusters.k) { + println("weight=%f\nmu=%s\nsigma=\n%s\n" format + (clusters.weights(i), clusters.gaussians(i).mu, clusters.gaussians(i).sigma)) + } + + println("Cluster labels (first <= 100):") + val clusterLabels = clusters.predict(data) + clusterLabels.take(100).foreach { x => + print(" " + x) + } + println() + } +} diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index 72618b6515f8..0706f1ebf66e 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -38,37 +38,10 @@ org.apache.flume flume-ng-sdk - ${flume.version} - - - io.netty - netty - - - org.apache.thrift - libthrift - - org.apache.flume flume-ng-core - ${flume.version} - - - io.netty - netty - - - org.apache.thrift - libthrift - - - - - org.scalatest - scalatest_${scala.binary.version} - test org.scala-lang @@ -91,10 +64,6 @@ target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes - - org.scalatest - scalatest-maven-plugin - org.apache.avro avro-maven-plugin diff --git a/external/flume-sink/src/test/resources/log4j.properties b/external/flume-sink/src/test/resources/log4j.properties index 4411d6e20c52..2a58e9981722 100644 --- a/external/flume-sink/src/test/resources/log4j.properties +++ b/external/flume-sink/src/test/resources/log4j.properties @@ -17,9 +17,8 @@ # Set everything to be logged to the file streaming/target/unit-tests.log log4j.rootCategory=INFO, file -# log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file=org.apache.log4j.FileAppender -log4j.appender.file.append=false +log4j.appender.file.append=true 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 diff --git a/external/flume/pom.xml b/external/flume/pom.xml index a682f0e8471d..1f2681394c58 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -48,23 +48,11 @@ org.apache.flume - flume-ng-sdk - ${flume.version} - - - io.netty - netty - - - org.apache.thrift - libthrift - - + flume-ng-core - org.scalatest - scalatest_${scala.binary.version} - test + org.apache.flume + flume-ng-sdk org.scalacheck @@ -85,11 +73,5 @@ target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes - - - org.scalatest - scalatest-maven-plugin - - diff --git a/external/flume/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java b/external/flume/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java index 6e1f01900071..1e24da7f5f60 100644 --- a/external/flume/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java +++ b/external/flume/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java @@ -17,6 +17,7 @@ package org.apache.spark.streaming; +import org.apache.spark.SparkConf; import org.apache.spark.streaming.api.java.JavaStreamingContext; import org.junit.After; import org.junit.Before; @@ -27,8 +28,11 @@ public abstract class LocalJavaStreamingContext { @Before public void setUp() { - System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock"); - ssc = new JavaStreamingContext("local[2]", "test", new Duration(1000)); + SparkConf conf = new SparkConf() + .setMaster("local[2]") + .setAppName("test") + .set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock"); + ssc = new JavaStreamingContext(conf, new Duration(1000)); ssc.checkpoint("checkpoint"); } diff --git a/external/flume/src/test/resources/log4j.properties b/external/flume/src/test/resources/log4j.properties index 4411d6e20c52..9697237bfa1a 100644 --- a/external/flume/src/test/resources/log4j.properties +++ b/external/flume/src/test/resources/log4j.properties @@ -15,11 +15,10 @@ # limitations under the License. # -# Set everything to be logged to the file streaming/target/unit-tests.log +# Set everything to be logged to the file target/unit-tests.log log4j.rootCategory=INFO, file -# log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file=org.apache.log4j.FileAppender -log4j.appender.file.append=false +log4j.appender.file.append=true 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 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 13943ed5442b..f333e3891b5f 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 @@ -80,7 +80,7 @@ class FlumeStreamSuite extends FunSuite with BeforeAndAfter with Matchers with L val socket = new ServerSocket(trialPort) socket.close() (null, trialPort) - })._2 + }, conf)._2 } /** Setup and start the streaming context */ diff --git a/external/kafka-assembly/pom.xml b/external/kafka-assembly/pom.xml new file mode 100644 index 000000000000..503fc129dc4f --- /dev/null +++ b/external/kafka-assembly/pom.xml @@ -0,0 +1,106 @@ + + + + + 4.0.0 + + org.apache.spark + spark-parent + 1.3.0-SNAPSHOT + ../../pom.xml + + + org.apache.spark + spark-streaming-kafka-assembly_2.10 + jar + Spark Project External Kafka Assembly + http://spark.apache.org/ + + + streaming-kafka-assembly + scala-${scala.binary.version} + spark-streaming-kafka-assembly-${project.version}.jar + ${project.build.directory}/${spark.jar.dir}/${spark.jar.basename} + + + + + org.apache.spark + spark-streaming-kafka_${scala.binary.version} + ${project.version} + + + org.apache.spark + spark-streaming_${scala.binary.version} + ${project.version} + provided + + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + org.apache.maven.plugins + maven-shade-plugin + + false + ${spark.jar} + + + *:* + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + package + + shade + + + + + + reference.conf + + + log4j.properties + + + + + + + + + + + + diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml index b3f44471cd32..b29b0509656b 100644 --- a/external/kafka/pom.xml +++ b/external/kafka/pom.xml @@ -74,11 +74,6 @@ 3.2 test - - org.scalatest - scalatest_${scala.binary.version} - test - org.scalacheck scalacheck_${scala.binary.version} @@ -98,11 +93,5 @@ target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes - - - org.scalatest - scalatest-maven-plugin - - diff --git a/external/kafka/src/test/resources/log4j.properties b/external/kafka/src/test/resources/log4j.properties index 4411d6e20c52..9697237bfa1a 100644 --- a/external/kafka/src/test/resources/log4j.properties +++ b/external/kafka/src/test/resources/log4j.properties @@ -15,11 +15,10 @@ # limitations under the License. # -# Set everything to be logged to the file streaming/target/unit-tests.log +# Set everything to be logged to the file target/unit-tests.log log4j.rootCategory=INFO, file -# log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file=org.apache.log4j.FileAppender -log4j.appender.file.append=false +log4j.appender.file.append=true 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 diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index 9025915f4447..560c8b9d1827 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -46,11 +46,6 @@ org.eclipse.paho.client.mqttv3 1.0.1 - - org.scalatest - scalatest_${scala.binary.version} - test - org.scalacheck scalacheck_${scala.binary.version} @@ -66,15 +61,15 @@ junit-interface test + + org.apache.activemq + activemq-core + 5.7.0 + test + target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes - - - org.scalatest - scalatest-maven-plugin - - diff --git a/external/mqtt/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java b/external/mqtt/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java index 6e1f01900071..1e24da7f5f60 100644 --- a/external/mqtt/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java +++ b/external/mqtt/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java @@ -17,6 +17,7 @@ package org.apache.spark.streaming; +import org.apache.spark.SparkConf; import org.apache.spark.streaming.api.java.JavaStreamingContext; import org.junit.After; import org.junit.Before; @@ -27,8 +28,11 @@ public abstract class LocalJavaStreamingContext { @Before public void setUp() { - System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock"); - ssc = new JavaStreamingContext("local[2]", "test", new Duration(1000)); + SparkConf conf = new SparkConf() + .setMaster("local[2]") + .setAppName("test") + .set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock"); + ssc = new JavaStreamingContext(conf, new Duration(1000)); ssc.checkpoint("checkpoint"); } diff --git a/external/mqtt/src/test/resources/log4j.properties b/external/mqtt/src/test/resources/log4j.properties index 4411d6e20c52..9697237bfa1a 100644 --- a/external/mqtt/src/test/resources/log4j.properties +++ b/external/mqtt/src/test/resources/log4j.properties @@ -15,11 +15,10 @@ # limitations under the License. # -# Set everything to be logged to the file streaming/target/unit-tests.log +# Set everything to be logged to the file target/unit-tests.log log4j.rootCategory=INFO, file -# log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file=org.apache.log4j.FileAppender -log4j.appender.file.append=false +log4j.appender.file.append=true 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 diff --git a/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala b/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala index 84595acf45cc..fe53a29cba0c 100644 --- a/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala +++ b/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala @@ -17,31 +17,118 @@ package org.apache.spark.streaming.mqtt -import org.scalatest.FunSuite +import java.net.{URI, ServerSocket} -import org.apache.spark.streaming.{Seconds, StreamingContext} +import scala.concurrent.duration._ +import scala.language.postfixOps + +import org.apache.activemq.broker.{TransportConnector, BrokerService} +import org.eclipse.paho.client.mqttv3._ +import org.eclipse.paho.client.mqttv3.persist.MqttDefaultFilePersistence + +import org.scalatest.{BeforeAndAfter, FunSuite} +import org.scalatest.concurrent.Eventually + +import org.apache.spark.streaming.{Milliseconds, StreamingContext} import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.dstream.ReceiverInputDStream +import org.apache.spark.SparkConf +import org.apache.spark.util.Utils -class MQTTStreamSuite extends FunSuite { - - val batchDuration = Seconds(1) +class MQTTStreamSuite extends FunSuite with Eventually with BeforeAndAfter { + private val batchDuration = Milliseconds(500) private val master: String = "local[2]" - private val framework: String = this.getClass.getSimpleName + private val freePort = findFreePort() + private val brokerUri = "//localhost:" + freePort + private val topic = "def" + private val persistenceDir = Utils.createTempDir() - test("mqtt input stream") { - val ssc = new StreamingContext(master, framework, batchDuration) - val brokerUrl = "abc" - val topic = "def" + private var ssc: StreamingContext = _ + private var broker: BrokerService = _ + private var connector: TransportConnector = _ + + before { + ssc = new StreamingContext(master, framework, batchDuration) + setupMQTT() + } - // tests the API, does not actually test data receiving - val test1: ReceiverInputDStream[String] = MQTTUtils.createStream(ssc, brokerUrl, topic) - val test2: ReceiverInputDStream[String] = - MQTTUtils.createStream(ssc, brokerUrl, topic, StorageLevel.MEMORY_AND_DISK_SER_2) + after { + if (ssc != null) { + ssc.stop() + ssc = null + } + Utils.deleteRecursively(persistenceDir) + tearDownMQTT() + } - // TODO: Actually test receiving data + test("mqtt input stream") { + val sendMessage = "MQTT demo for spark streaming" + val receiveStream: ReceiverInputDStream[String] = + MQTTUtils.createStream(ssc, "tcp:" + brokerUri, topic, StorageLevel.MEMORY_ONLY) + var receiveMessage: List[String] = List() + receiveStream.foreachRDD { rdd => + if (rdd.collect.length > 0) { + receiveMessage = receiveMessage ::: List(rdd.first) + receiveMessage + } + } + ssc.start() + publishData(sendMessage) + eventually(timeout(10000 milliseconds), interval(100 milliseconds)) { + assert(sendMessage.equals(receiveMessage(0))) + } ssc.stop() } + + private def setupMQTT() { + broker = new BrokerService() + connector = new TransportConnector() + connector.setName("mqtt") + connector.setUri(new URI("mqtt:" + brokerUri)) + broker.addConnector(connector) + broker.start() + } + + private def tearDownMQTT() { + if (broker != null) { + broker.stop() + broker = null + } + if (connector != null) { + connector.stop() + connector = null + } + } + + private def findFreePort(): Int = { + Utils.startServiceOnPort(23456, (trialPort: Int) => { + val socket = new ServerSocket(trialPort) + socket.close() + (null, trialPort) + }, new SparkConf())._2 + } + + def publishData(data: String): Unit = { + var client: MqttClient = null + try { + val persistence: MqttClientPersistence = new MqttDefaultFilePersistence(persistenceDir.getAbsolutePath) + client = new MqttClient("tcp:" + brokerUri, MqttClient.generateClientId(), persistence) + client.connect() + if (client.isConnected) { + val msgTopic: MqttTopic = client.getTopic(topic) + val message: MqttMessage = new MqttMessage(data.getBytes("utf-8")) + message.setQos(1) + message.setRetained(true) + for (i <- 0 to 100) { + msgTopic.publish(message) + } + } + } finally { + client.disconnect() + client.close() + client = null + } + } } diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml index 000ace1446e5..da6ffe7662f6 100644 --- a/external/twitter/pom.xml +++ b/external/twitter/pom.xml @@ -46,11 +46,6 @@ twitter4j-stream 3.0.3 - - org.scalatest - scalatest_${scala.binary.version} - test - org.scalacheck scalacheck_${scala.binary.version} @@ -70,11 +65,5 @@ target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes - - - org.scalatest - scalatest-maven-plugin - - diff --git a/external/twitter/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java b/external/twitter/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java index 6e1f01900071..1e24da7f5f60 100644 --- a/external/twitter/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java +++ b/external/twitter/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java @@ -17,6 +17,7 @@ package org.apache.spark.streaming; +import org.apache.spark.SparkConf; import org.apache.spark.streaming.api.java.JavaStreamingContext; import org.junit.After; import org.junit.Before; @@ -27,8 +28,11 @@ public abstract class LocalJavaStreamingContext { @Before public void setUp() { - System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock"); - ssc = new JavaStreamingContext("local[2]", "test", new Duration(1000)); + SparkConf conf = new SparkConf() + .setMaster("local[2]") + .setAppName("test") + .set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock"); + ssc = new JavaStreamingContext(conf, new Duration(1000)); ssc.checkpoint("checkpoint"); } diff --git a/external/twitter/src/test/resources/log4j.properties b/external/twitter/src/test/resources/log4j.properties index 4411d6e20c52..64bfc5745088 100644 --- a/external/twitter/src/test/resources/log4j.properties +++ b/external/twitter/src/test/resources/log4j.properties @@ -15,11 +15,10 @@ # limitations under the License. # -# Set everything to be logged to the file streaming/target/unit-tests.log +# Set everything to be logged to the filetarget/unit-tests.log log4j.rootCategory=INFO, file -# log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file=org.apache.log4j.FileAppender -log4j.appender.file.append=false +log4j.appender.file.append=true 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 diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml index 29c452093502..e919c2c9b19e 100644 --- a/external/zeromq/pom.xml +++ b/external/zeromq/pom.xml @@ -44,12 +44,6 @@ ${akka.group} akka-zeromq_${scala.binary.version} - ${akka.version} - - - org.scalatest - scalatest_${scala.binary.version} - test org.scalacheck @@ -70,11 +64,5 @@ target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes - - - org.scalatest - scalatest-maven-plugin - - diff --git a/external/zeromq/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java b/external/zeromq/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java index 6e1f01900071..1e24da7f5f60 100644 --- a/external/zeromq/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java +++ b/external/zeromq/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java @@ -17,6 +17,7 @@ package org.apache.spark.streaming; +import org.apache.spark.SparkConf; import org.apache.spark.streaming.api.java.JavaStreamingContext; import org.junit.After; import org.junit.Before; @@ -27,8 +28,11 @@ public abstract class LocalJavaStreamingContext { @Before public void setUp() { - System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock"); - ssc = new JavaStreamingContext("local[2]", "test", new Duration(1000)); + SparkConf conf = new SparkConf() + .setMaster("local[2]") + .setAppName("test") + .set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock"); + ssc = new JavaStreamingContext(conf, new Duration(1000)); ssc.checkpoint("checkpoint"); } diff --git a/external/zeromq/src/test/resources/log4j.properties b/external/zeromq/src/test/resources/log4j.properties index 4411d6e20c52..9697237bfa1a 100644 --- a/external/zeromq/src/test/resources/log4j.properties +++ b/external/zeromq/src/test/resources/log4j.properties @@ -15,11 +15,10 @@ # limitations under the License. # -# Set everything to be logged to the file streaming/target/unit-tests.log +# Set everything to be logged to the file target/unit-tests.log log4j.rootCategory=INFO, file -# log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file=org.apache.log4j.FileAppender -log4j.appender.file.append=false +log4j.appender.file.append=true 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 diff --git a/extras/java8-tests/pom.xml b/extras/java8-tests/pom.xml index c8477a656631..0fb431808bac 100644 --- a/extras/java8-tests/pom.xml +++ b/extras/java8-tests/pom.xml @@ -60,11 +60,6 @@ junit-interface test - - org.scalatest - scalatest_${scala.binary.version} - test - @@ -159,16 +154,6 @@ - - org.scalatest - scalatest-maven-plugin - - - test - none - - - diff --git a/extras/java8-tests/src/test/resources/log4j.properties b/extras/java8-tests/src/test/resources/log4j.properties index bb0ab319a008..287c8e356350 100644 --- a/extras/java8-tests/src/test/resources/log4j.properties +++ b/extras/java8-tests/src/test/resources/log4j.properties @@ -18,7 +18,7 @@ # Set everything to be logged to the file target/unit-tests.log log4j.rootCategory=INFO, file log4j.appender.file=org.apache.log4j.FileAppender -log4j.appender.file.append=false +log4j.appender.file.append=true 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 diff --git a/extras/kinesis-asl/pom.xml b/extras/kinesis-asl/pom.xml index c0d3a6111911..c815eda52bda 100644 --- a/extras/kinesis-asl/pom.xml +++ b/extras/kinesis-asl/pom.xml @@ -57,11 +57,6 @@ aws-java-sdk ${aws.java.sdk.version} - - org.scalatest - scalatest_${scala.binary.version} - test - org.mockito mockito-all @@ -86,11 +81,5 @@ target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes - - - org.scalatest - scalatest-maven-plugin - - diff --git a/extras/kinesis-asl/src/test/resources/log4j.properties b/extras/kinesis-asl/src/test/resources/log4j.properties index d9d08f68687d..853ef0ed2986 100644 --- a/extras/kinesis-asl/src/test/resources/log4j.properties +++ b/extras/kinesis-asl/src/test/resources/log4j.properties @@ -14,10 +14,11 @@ # See the License for the specific language governing permissions and # limitations under the License. # + +# Set everything to be logged to the file target/unit-tests.log log4j.rootCategory=INFO, file -# log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file=org.apache.log4j.FileAppender -log4j.appender.file.append=false +log4j.appender.file.append=true 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 diff --git a/graphx/pom.xml b/graphx/pom.xml index 9982b36f9b62..72374aae6da9 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -45,15 +45,6 @@ jblas ${jblas.version} - - org.eclipse.jetty - jetty-server - - - org.scalatest - scalatest_${scala.binary.version} - test - org.scalacheck scalacheck_${scala.binary.version} @@ -63,11 +54,5 @@ target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes - - - org.scalatest - scalatest-maven-plugin - - 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 116d1ea70017..dc8b4789c4b6 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala @@ -278,6 +278,32 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Seriali retVal } + /** + * Convert bi-directional edges into uni-directional ones. + * Some graph algorithms (e.g., TriangleCount) assume that an input graph + * has its edges in canonical direction. + * This function rewrites the vertex ids of edges so that srcIds are bigger + * than dstIds, and merges the duplicated edges. + * + * @param mergeFunc the user defined reduce function which should + * be commutative and associative and is used to combine the output + * of the map phase + * + * @return the resulting graph with canonical edges + */ + def convertToCanonicalEdges( + mergeFunc: (ED, ED) => ED = (e1, e2) => e1): Graph[VD, ED] = { + val newEdges = + graph.edges + .map { + case e if e.srcId < e.dstId => ((e.srcId, e.dstId), e.attr) + case e => ((e.dstId, e.srcId), e.attr) + } + .reduceByKey(mergeFunc) + .map(e => new Edge(e._1._1, e._1._2, e._2)) + Graph(graph.vertices, newEdges) + } + /** * Execute a Pregel-like iterative vertex-parallel abstraction. The * user-defined vertex-program `vprog` is executed in parallel on diff --git a/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala b/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala index 13033fee0e6b..7372dfbd9fe9 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala @@ -32,9 +32,9 @@ trait PartitionStrategy extends Serializable { object PartitionStrategy { /** * Assigns edges to partitions using a 2D partitioning of the sparse edge adjacency matrix, - * guaranteeing a `2 * sqrt(numParts)` bound on vertex replication. + * guaranteeing a `2 * sqrt(numParts) - 1` bound on vertex replication. * - * Suppose we have a graph with 11 vertices that we want to partition + * Suppose we have a graph with 12 vertices that we want to partition * over 9 machines. We can use the following sparse matrix representation: * *
    @@ -61,7 +61,7 @@ object PartitionStrategy {
        * that edges adjacent to `v11` can only be in the first column of blocks `(P0, P3,
        * P6)` or the last
        * row of blocks `(P6, P7, P8)`.  As a consequence we can guarantee that `v11` will need to be
    -   * replicated to at most `2 * sqrt(numParts)` machines.
    +   * replicated to at most `2 * sqrt(numParts) - 1` machines.
        *
        * Notice that `P0` has many edges and as a consequence this partitioning would lead to poor work
        * balance.  To improve balance we first multiply each vertex id by a large prime to shuffle the
    diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala
    index 409cf60977f6..906d42328fcb 100644
    --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala
    +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala
    @@ -129,44 +129,45 @@ private[impl] case class EdgeWithLocalIds[@specialized ED](
         srcId: VertexId, dstId: VertexId, localSrcId: Int, localDstId: Int, attr: ED)
     
     private[impl] object EdgeWithLocalIds {
    -  implicit def lexicographicOrdering[ED] = new Ordering[EdgeWithLocalIds[ED]] {
    -    override def compare(a: EdgeWithLocalIds[ED], b: EdgeWithLocalIds[ED]): Int = {
    -      if (a.srcId == b.srcId) {
    -        if (a.dstId == b.dstId) 0
    -        else if (a.dstId < b.dstId) -1
    +  implicit def lexicographicOrdering[ED]: Ordering[EdgeWithLocalIds[ED]] =
    +    new Ordering[EdgeWithLocalIds[ED]] {
    +      override def compare(a: EdgeWithLocalIds[ED], b: EdgeWithLocalIds[ED]): Int = {
    +        if (a.srcId == b.srcId) {
    +          if (a.dstId == b.dstId) 0
    +          else if (a.dstId < b.dstId) -1
    +          else 1
    +        } else if (a.srcId < b.srcId) -1
             else 1
    -      } else if (a.srcId < b.srcId) -1
    -      else 1
    +      }
         }
    -  }
     
    -  private[graphx] def edgeArraySortDataFormat[ED]
    -      = new SortDataFormat[EdgeWithLocalIds[ED], Array[EdgeWithLocalIds[ED]]] {
    -    override def getKey(
    -        data: Array[EdgeWithLocalIds[ED]], pos: Int): EdgeWithLocalIds[ED] = {
    -      data(pos)
    -    }
    +  private[graphx] def edgeArraySortDataFormat[ED] = {
    +    new SortDataFormat[EdgeWithLocalIds[ED], Array[EdgeWithLocalIds[ED]]] {
    +      override def getKey(data: Array[EdgeWithLocalIds[ED]], pos: Int): EdgeWithLocalIds[ED] = {
    +        data(pos)
    +      }
     
    -    override def swap(data: Array[EdgeWithLocalIds[ED]], pos0: Int, pos1: Int): Unit = {
    -      val tmp = data(pos0)
    -      data(pos0) = data(pos1)
    -      data(pos1) = tmp
    -    }
    +      override def swap(data: Array[EdgeWithLocalIds[ED]], pos0: Int, pos1: Int): Unit = {
    +        val tmp = data(pos0)
    +        data(pos0) = data(pos1)
    +        data(pos1) = tmp
    +      }
     
    -    override def copyElement(
    -        src: Array[EdgeWithLocalIds[ED]], srcPos: Int,
    -        dst: Array[EdgeWithLocalIds[ED]], dstPos: Int) {
    -      dst(dstPos) = src(srcPos)
    -    }
    +      override def copyElement(
    +          src: Array[EdgeWithLocalIds[ED]], srcPos: Int,
    +          dst: Array[EdgeWithLocalIds[ED]], dstPos: Int) {
    +        dst(dstPos) = src(srcPos)
    +      }
     
    -    override def copyRange(
    -        src: Array[EdgeWithLocalIds[ED]], srcPos: Int,
    -        dst: Array[EdgeWithLocalIds[ED]], dstPos: Int, length: Int) {
    -      System.arraycopy(src, srcPos, dst, dstPos, length)
    -    }
    +      override def copyRange(
    +          src: Array[EdgeWithLocalIds[ED]], srcPos: Int,
    +          dst: Array[EdgeWithLocalIds[ED]], dstPos: Int, length: Int) {
    +        System.arraycopy(src, srcPos, dst, dstPos, length)
    +      }
     
    -    override def allocate(length: Int): Array[EdgeWithLocalIds[ED]] = {
    -      new Array[EdgeWithLocalIds[ED]](length)
    +      override def allocate(length: Int): Array[EdgeWithLocalIds[ED]] = {
    +        new Array[EdgeWithLocalIds[ED]](length)
    +      }
         }
       }
     }
    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 5412d720475d..aa320088f208 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
    @@ -74,8 +74,8 @@ object ShippableVertexPartition {
        * Implicit conversion to allow invoking `VertexPartitionBase` operations directly on a
        * `ShippableVertexPartition`.
        */
    -  implicit def shippablePartitionToOps[VD: ClassTag](partition: ShippableVertexPartition[VD]) =
    -    new ShippableVertexPartitionOps(partition)
    +  implicit def shippablePartitionToOps[VD: ClassTag](partition: ShippableVertexPartition[VD])
    +    : ShippableVertexPartitionOps[VD] = new ShippableVertexPartitionOps(partition)
     
       /**
        * Implicit evidence that `ShippableVertexPartition` is a member of the
    diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala
    index 55c7a19d1bda..fbe53acfc32a 100644
    --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala
    +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala
    @@ -38,8 +38,8 @@ private[graphx] object VertexPartition {
        * Implicit conversion to allow invoking `VertexPartitionBase` operations directly on a
        * `VertexPartition`.
        */
    -  implicit def partitionToOps[VD: ClassTag](partition: VertexPartition[VD]) =
    -    new VertexPartitionOps(partition)
    +  implicit def partitionToOps[VD: ClassTag](partition: VertexPartition[VD])
    +    : VertexPartitionOps[VD] = new VertexPartitionOps(partition)
     
       /**
        * Implicit evidence that `VertexPartition` is a member of the `VertexPartitionBaseOpsConstructor`
    diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala
    index b40aa1b417a0..4fd2548b7faf 100644
    --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala
    +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartitionBaseOps.scala
    @@ -238,8 +238,8 @@ private[graphx] abstract class VertexPartitionBaseOps
        * because these methods return a `Self` and this implicit conversion re-wraps that in a
        * `VertexPartitionBaseOps`. This relies on the context bound on `Self`.
        */
    -  private implicit def toOps[VD2: ClassTag](
    -      partition: Self[VD2]): VertexPartitionBaseOps[VD2, Self] = {
    +  private implicit def toOps[VD2: ClassTag](partition: Self[VD2])
    +    : VertexPartitionBaseOps[VD2, Self] = {
         implicitly[VertexPartitionBaseOpsConstructor[Self]].toOps(partition)
       }
     }
    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 8a13c7422154..2d6a825b6172 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
    @@ -133,6 +133,12 @@ object GraphGenerators {
         // This ensures that the 4 quadrants are the same size at all recursion levels
         val numVertices = math.round(
           math.pow(2.0, math.ceil(math.log(requestedNumVertices) / math.log(2.0)))).toInt
    +    val numEdgesUpperBound =
    +      math.pow(2.0, 2 * ((math.log(numVertices) / math.log(2.0)) - 1)).toInt
    +    if (numEdgesUpperBound < numEdges) {
    +      throw new IllegalArgumentException(
    +        s"numEdges must be <= $numEdgesUpperBound but was $numEdges")
    +    }
         var edges: Set[Edge[Int]] = Set()
         while (edges.size < numEdges) {
           if (edges.size % 100 == 0) {
    diff --git a/graphx/src/test/resources/log4j.properties b/graphx/src/test/resources/log4j.properties
    index 9dd05f17f012..287c8e356350 100644
    --- a/graphx/src/test/resources/log4j.properties
    +++ b/graphx/src/test/resources/log4j.properties
    @@ -15,10 +15,10 @@
     # limitations under the License.
     #
     
    -# Set everything to be logged to the file core/target/unit-tests.log
    +# Set everything to be logged to the file target/unit-tests.log
     log4j.rootCategory=INFO, file
     log4j.appender.file=org.apache.log4j.FileAppender
    -log4j.appender.file.append=false
    +log4j.appender.file.append=true
     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
    diff --git a/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala
    index ea94d4accb63..9bc8007ce49c 100644
    --- a/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala
    +++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala
    @@ -79,6 +79,21 @@ class GraphOpsSuite extends FunSuite with LocalSparkContext {
         }
       }
     
    +  test ("convertToCanonicalEdges") {
    +    withSpark { sc =>
    +      val vertices =
    +        sc.parallelize(Seq[(VertexId, String)]((1, "one"), (2, "two"), (3, "three")), 2)
    +      val edges =
    +        sc.parallelize(Seq(Edge(1, 2, 1), Edge(2, 1, 1), Edge(3, 2, 2)))
    +      val g: Graph[String, Int] = Graph(vertices, edges)
    +
    +      val g1 = g.convertToCanonicalEdges()
    +
    +      val e = g1.edges.collect().toSet
    +      assert(e === Set(Edge(1, 2, 1), Edge(2, 3, 2)))
    +    }
    +  }
    +
       test("collectEdgesCycleDirectionOut") {
         withSpark { sc =>
           val graph = getCycleGraph(sc, 100)
    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 3abefbe52fa8..8d9c8ddccbb3 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
    @@ -110,4 +110,14 @@ class GraphGeneratorsSuite extends FunSuite with LocalSparkContext {
         }
       }
     
    +  test("SPARK-5064 GraphGenerators.rmatGraph numEdges upper bound") {
    +    withSpark { sc =>
    +      val g1 = GraphGenerators.rmatGraph(sc, 4, 4)
    +      assert(g1.edges.count() === 4)
    +      intercept[IllegalArgumentException] {
    +        val g2 = GraphGenerators.rmatGraph(sc, 4, 8)
    +      }
    +    }
    +  }
    +
     }
    diff --git a/make-distribution.sh b/make-distribution.sh
    index 45c99e42e5a5..4b979fbe1170 100755
    --- a/make-distribution.sh
    +++ b/make-distribution.sh
    @@ -28,18 +28,20 @@ set -o pipefail
     set -e
     
     # Figure out where the Spark framework is installed
    -FWDIR="$(cd "`dirname "$0"`"; pwd)"
    -DISTDIR="$FWDIR/dist"
    +SPARK_HOME="$(cd "`dirname "$0"`"; pwd)"
    +DISTDIR="$SPARK_HOME/dist"
     
     SPARK_TACHYON=false
     MAKE_TGZ=false
     NAME=none
    +MVN="$SPARK_HOME/build/mvn"
     
     function exit_with_usage {
       echo "make-distribution.sh - tool for making binary distributions of Spark"
       echo ""
       echo "usage:"
    -  echo "./make-distribution.sh [--name] [--tgz] [--with-tachyon] "
    +  cl_options="[--name] [--tgz] [--mvn ] [--with-tachyon]"
    +  echo "./make-distribution.sh $cl_options "
       echo "See Spark's \"Building Spark\" doc for correct Maven options."
       echo ""
       exit 1
    @@ -71,6 +73,10 @@ while (( "$#" )); do
         --tgz)
           MAKE_TGZ=true
           ;;
    +    --mvn)
    +      MVN="$2"
    +      shift
    +      ;;
         --name)
           NAME="$2"
           shift
    @@ -109,9 +115,9 @@ if which git &>/dev/null; then
         unset GITREV
     fi
     
    -if ! which mvn &>/dev/null; then
    -    echo -e "You need Maven installed to build Spark."
    -    echo -e "Download Maven from https://maven.apache.org/"
    +if ! which $MVN &>/dev/null; then
    +    echo -e "Could not locate Maven command: '$MVN'."
    +    echo -e "Specify the Maven command with the --mvn flag"
         exit -1;
     fi
     
    @@ -119,7 +125,7 @@ VERSION=$(mvn help:evaluate -Dexpression=project.version 2>/dev/null | grep -v "
     SPARK_HADOOP_VERSION=$(mvn help:evaluate -Dexpression=hadoop.version $@ 2>/dev/null\
         | grep -v "INFO"\
         | tail -n 1)
    -SPARK_HIVE=$(mvn help:evaluate -Dexpression=project.activeProfiles -pl sql/hive $@ 2>/dev/null\
    +SPARK_HIVE=$($MVN help:evaluate -Dexpression=project.activeProfiles -pl sql/hive $@ 2>/dev/null\
         | grep -v "INFO"\
         | fgrep --count "hive";\
         # Reset exit status to 0, otherwise the script stops here if the last grep finds nothing\
    @@ -161,11 +167,11 @@ else
     fi
     
     # Build uber fat JAR
    -cd "$FWDIR"
    +cd "$SPARK_HOME"
     
     export MAVEN_OPTS="-Xmx2g -XX:MaxPermSize=512M -XX:ReservedCodeCacheSize=512m"
     
    -BUILD_COMMAND="mvn clean package -DskipTests $@"
    +BUILD_COMMAND="$MVN clean package -DskipTests $@"
     
     # Actually build the jar
     echo -e "\nBuilding with..."
    @@ -177,41 +183,43 @@ ${BUILD_COMMAND}
     rm -rf "$DISTDIR"
     mkdir -p "$DISTDIR/lib"
     echo "Spark $VERSION$GITREVSTRING built for Hadoop $SPARK_HADOOP_VERSION" > "$DISTDIR/RELEASE"
    +echo "Build flags: $@" >> "$DISTDIR/RELEASE"
     
     # Copy jars
    -cp "$FWDIR"/assembly/target/scala*/*assembly*hadoop*.jar "$DISTDIR/lib/"
    -cp "$FWDIR"/examples/target/scala*/spark-examples*.jar "$DISTDIR/lib/"
    +cp "$SPARK_HOME"/assembly/target/scala*/*assembly*hadoop*.jar "$DISTDIR/lib/"
    +cp "$SPARK_HOME"/examples/target/scala*/spark-examples*.jar "$DISTDIR/lib/"
    +cp "$SPARK_HOME"/external/kafka/scala*/*kafka*assembly*.jar "$DISTDIR/lib/"
     # This will fail if the -Pyarn profile is not provided
     # In this case, silence the error and ignore the return code of this command
    -cp "$FWDIR"/network/yarn/target/scala*/spark-*-yarn-shuffle.jar "$DISTDIR/lib/" &> /dev/null || :
    +cp "$SPARK_HOME"/network/yarn/target/scala*/spark-*-yarn-shuffle.jar "$DISTDIR/lib/" &> /dev/null || :
     
     # Copy example sources (needed for python and SQL)
     mkdir -p "$DISTDIR/examples/src/main"
    -cp -r "$FWDIR"/examples/src/main "$DISTDIR/examples/src/"
    +cp -r "$SPARK_HOME"/examples/src/main "$DISTDIR/examples/src/"
     
     if [ "$SPARK_HIVE" == "1" ]; then
    -  cp "$FWDIR"/lib_managed/jars/datanucleus*.jar "$DISTDIR/lib/"
    +  cp "$SPARK_HOME"/lib_managed/jars/datanucleus*.jar "$DISTDIR/lib/"
     fi
     
     # Copy license and ASF files
    -cp "$FWDIR/LICENSE" "$DISTDIR"
    -cp "$FWDIR/NOTICE" "$DISTDIR"
    +cp "$SPARK_HOME/LICENSE" "$DISTDIR"
    +cp "$SPARK_HOME/NOTICE" "$DISTDIR"
     
    -if [ -e "$FWDIR"/CHANGES.txt ]; then
    -  cp "$FWDIR/CHANGES.txt" "$DISTDIR"
    +if [ -e "$SPARK_HOME"/CHANGES.txt ]; then
    +  cp "$SPARK_HOME/CHANGES.txt" "$DISTDIR"
     fi
     
     # Copy data files
    -cp -r "$FWDIR/data" "$DISTDIR"
    +cp -r "$SPARK_HOME/data" "$DISTDIR"
     
     # Copy other things
     mkdir "$DISTDIR"/conf
    -cp "$FWDIR"/conf/*.template "$DISTDIR"/conf
    -cp "$FWDIR/README.md" "$DISTDIR"
    -cp -r "$FWDIR/bin" "$DISTDIR"
    -cp -r "$FWDIR/python" "$DISTDIR"
    -cp -r "$FWDIR/sbin" "$DISTDIR"
    -cp -r "$FWDIR/ec2" "$DISTDIR"
    +cp "$SPARK_HOME"/conf/*.template "$DISTDIR"/conf
    +cp "$SPARK_HOME/README.md" "$DISTDIR"
    +cp -r "$SPARK_HOME/bin" "$DISTDIR"
    +cp -r "$SPARK_HOME/python" "$DISTDIR"
    +cp -r "$SPARK_HOME/sbin" "$DISTDIR"
    +cp -r "$SPARK_HOME/ec2" "$DISTDIR"
     
     # Download and copy in tachyon, if requested
     if [ "$SPARK_TACHYON" == "true" ]; then
    @@ -243,9 +251,9 @@ fi
     
     if [ "$MAKE_TGZ" == "true" ]; then
       TARDIR_NAME=spark-$VERSION-bin-$NAME
    -  TARDIR="$FWDIR/$TARDIR_NAME"
    +  TARDIR="$SPARK_HOME/$TARDIR_NAME"
       rm -rf "$TARDIR"
       cp -r "$DISTDIR" "$TARDIR"
    -  tar czf "spark-$VERSION-bin-$NAME.tgz" -C "$FWDIR" "$TARDIR_NAME"
    +  tar czf "spark-$VERSION-bin-$NAME.tgz" -C "$SPARK_HOME" "$TARDIR_NAME"
       rm -rf "$TARDIR"
     fi
    diff --git a/mllib/pom.xml b/mllib/pom.xml
    index 0a6dda0ab8c8..a0bda89ccaa7 100644
    --- a/mllib/pom.xml
    +++ b/mllib/pom.xml
    @@ -29,7 +29,7 @@
       spark-mllib_2.10
       
         mllib
    -    
    +  
       jar
       Spark Project ML Library
       http://spark.apache.org/
    @@ -50,10 +50,6 @@
           spark-sql_${scala.binary.version}
           ${project.version}
         
    -    
    -      org.eclipse.jetty
    -      jetty-server
    -    
         
           org.jblas
           jblas
    @@ -80,11 +76,6 @@
           org.apache.commons
           commons-math3
         
    -    
    -      org.scalatest
    -      scalatest_${scala.binary.version}
    -      test
    -    
         
           org.scalacheck
           scalacheck_${scala.binary.version}
    @@ -129,12 +120,6 @@
       
         target/scala-${scala.binary.version}/classes
         target/scala-${scala.binary.version}/test-classes
    -    
    -      
    -        org.scalatest
    -        scalatest-maven-plugin
    -      
    -    
         
           
             ../python
    diff --git a/mllib/src/main/scala/org/apache/spark/ml/Estimator.scala b/mllib/src/main/scala/org/apache/spark/ml/Estimator.scala
    index fdbee743e817..77d230eb4a12 100644
    --- a/mllib/src/main/scala/org/apache/spark/ml/Estimator.scala
    +++ b/mllib/src/main/scala/org/apache/spark/ml/Estimator.scala
    @@ -18,12 +18,10 @@
     package org.apache.spark.ml
     
     import scala.annotation.varargs
    -import scala.collection.JavaConverters._
     
     import org.apache.spark.annotation.AlphaComponent
     import org.apache.spark.ml.param.{ParamMap, ParamPair, Params}
     import org.apache.spark.sql.SchemaRDD
    -import org.apache.spark.sql.api.java.JavaSchemaRDD
     
     /**
      * :: AlphaComponent ::
    @@ -66,40 +64,4 @@ abstract class Estimator[M <: Model[M]] extends PipelineStage with Params {
       def fit(dataset: SchemaRDD, paramMaps: Array[ParamMap]): Seq[M] = {
         paramMaps.map(fit(dataset, _))
       }
    -
    -  // Java-friendly versions of fit.
    -
    -  /**
    -   * Fits a single model to the input data with optional parameters.
    -   *
    -   * @param dataset input dataset
    -   * @param paramPairs optional list of param pairs (overwrite embedded params)
    -   * @return fitted model
    -   */
    -  @varargs
    -  def fit(dataset: JavaSchemaRDD, paramPairs: ParamPair[_]*): M = {
    -    fit(dataset.schemaRDD, paramPairs: _*)
    -  }
    -
    -  /**
    -   * Fits a single model to the  input data with provided parameter map.
    -   *
    -   * @param dataset input dataset
    -   * @param paramMap parameter map
    -   * @return fitted model
    -   */
    -  def fit(dataset: JavaSchemaRDD, paramMap: ParamMap): M = {
    -    fit(dataset.schemaRDD, paramMap)
    -  }
    -
    -  /**
    -   * Fits multiple models to the input data with multiple sets of parameters.
    -   *
    -   * @param dataset input dataset
    -   * @param paramMaps an array of parameter maps
    -   * @return fitted models, matching the input parameter maps
    -   */
    -  def fit(dataset: JavaSchemaRDD, paramMaps: Array[ParamMap]): java.util.List[M] = {
    -    fit(dataset.schemaRDD, paramMaps).asJava
    -  }
     }
    diff --git a/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala b/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala
    index 081a574beea5..ad6fed178fae 100644
    --- a/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala
    +++ b/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala
    @@ -21,8 +21,9 @@ import scala.collection.mutable.ListBuffer
     
     import org.apache.spark.Logging
     import org.apache.spark.annotation.AlphaComponent
    -import org.apache.spark.ml.param.{Params, Param, ParamMap}
    -import org.apache.spark.sql.{SchemaRDD, StructType}
    +import org.apache.spark.ml.param.{Param, ParamMap}
    +import org.apache.spark.sql.SchemaRDD
    +import org.apache.spark.sql.types.StructType
     
     /**
      * :: AlphaComponent ::
    diff --git a/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala b/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala
    index 23fbd228d01c..af56f9c43535 100644
    --- a/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala
    +++ b/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala
    @@ -23,10 +23,9 @@ import org.apache.spark.Logging
     import org.apache.spark.annotation.AlphaComponent
     import org.apache.spark.ml.param._
     import org.apache.spark.sql.SchemaRDD
    -import org.apache.spark.sql.api.java.JavaSchemaRDD
     import org.apache.spark.sql.catalyst.analysis.Star
     import org.apache.spark.sql.catalyst.expressions.ScalaUdf
    -import org.apache.spark.sql.catalyst.types._
    +import org.apache.spark.sql.types._
     
     /**
      * :: AlphaComponent ::
    @@ -55,29 +54,6 @@ abstract class Transformer extends PipelineStage with Params {
        * @return transformed dataset
        */
       def transform(dataset: SchemaRDD, paramMap: ParamMap): SchemaRDD
    -
    -  // Java-friendly versions of transform.
    -
    -  /**
    -   * Transforms the dataset with optional parameters.
    -   * @param dataset input datset
    -   * @param paramPairs optional list of param pairs, overwrite embedded params
    -   * @return transformed dataset
    -   */
    -  @varargs
    -  def transform(dataset: JavaSchemaRDD, paramPairs: ParamPair[_]*): JavaSchemaRDD = {
    -    transform(dataset.schemaRDD, paramPairs: _*).toJavaSchemaRDD
    -  }
    -
    -  /**
    -   * Transforms the dataset with provided parameter map as additional parameters.
    -   * @param dataset input dataset
    -   * @param paramMap additional parameters, overwrite embedded params
    -   * @return transformed dataset
    -   */
    -  def transform(dataset: JavaSchemaRDD, paramMap: ParamMap): JavaSchemaRDD = {
    -    transform(dataset.schemaRDD, paramMap).toJavaSchemaRDD
    -  }
     }
     
     /**
    diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala
    index 85b8899636ca..8c570812f831 100644
    --- a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala
    +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala
    @@ -26,6 +26,7 @@ import org.apache.spark.mllib.regression.LabeledPoint
     import org.apache.spark.sql._
     import org.apache.spark.sql.catalyst.analysis.Star
     import org.apache.spark.sql.catalyst.dsl._
    +import org.apache.spark.sql.types.{DoubleType, StructField, StructType}
     import org.apache.spark.storage.StorageLevel
     
     /**
    diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala
    index 0b0504e036ec..12473cb2b571 100644
    --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala
    +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala
    @@ -21,7 +21,8 @@ import org.apache.spark.annotation.AlphaComponent
     import org.apache.spark.ml._
     import org.apache.spark.ml.param._
     import org.apache.spark.mllib.evaluation.BinaryClassificationMetrics
    -import org.apache.spark.sql.{DoubleType, Row, SchemaRDD}
    +import org.apache.spark.sql.{Row, SchemaRDD}
    +import org.apache.spark.sql.types.DoubleType
     
     /**
      * :: AlphaComponent ::
    diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala
    index e0bfb1e484a2..0956062643f2 100644
    --- a/mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala
    +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala
    @@ -22,7 +22,7 @@ import org.apache.spark.ml.UnaryTransformer
     import org.apache.spark.ml.param.{IntParam, ParamMap}
     import org.apache.spark.mllib.feature
     import org.apache.spark.mllib.linalg.{VectorUDT, Vector}
    -import org.apache.spark.sql.catalyst.types.DataType
    +import org.apache.spark.sql.types.DataType
     
     /**
      * :: AlphaComponent ::
    diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala
    index 896a6b83b67b..72825f6e0218 100644
    --- a/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala
    +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala
    @@ -25,6 +25,7 @@ import org.apache.spark.mllib.linalg.{Vector, VectorUDT}
     import org.apache.spark.sql._
     import org.apache.spark.sql.catalyst.analysis.Star
     import org.apache.spark.sql.catalyst.dsl._
    +import org.apache.spark.sql.types.{StructField, StructType}
     
     /**
      * Params for [[StandardScaler]] and [[StandardScalerModel]].
    diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala
    index 9352f40f372d..e622a5cf9e6f 100644
    --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala
    +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala
    @@ -20,7 +20,7 @@ package org.apache.spark.ml.feature
     import org.apache.spark.annotation.AlphaComponent
     import org.apache.spark.ml.UnaryTransformer
     import org.apache.spark.ml.param.ParamMap
    -import org.apache.spark.sql.{DataType, StringType, ArrayType}
    +import org.apache.spark.sql.types.{DataType, StringType, ArrayType}
     
     /**
      * :: AlphaComponent ::
    diff --git a/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala b/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala
    index 194b9bfd9a9e..08fe99176424 100644
    --- a/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala
    +++ b/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala
    @@ -24,7 +24,8 @@ import org.apache.spark.annotation.AlphaComponent
     import org.apache.spark.ml._
     import org.apache.spark.ml.param.{IntParam, Param, ParamMap, Params}
     import org.apache.spark.mllib.util.MLUtils
    -import org.apache.spark.sql.{SchemaRDD, StructType}
    +import org.apache.spark.sql.SchemaRDD
    +import org.apache.spark.sql.types.StructType
     
     /**
      * Params for [[CrossValidator]] and [[CrossValidatorModel]].
    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 c4e5fd8e461f..430d763ef7ca 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
    @@ -266,12 +266,16 @@ class PythonMLLibAPI extends Serializable {
           k: Int,
           maxIterations: Int,
           runs: Int,
    -      initializationMode: String): KMeansModel = {
    +      initializationMode: String,
    +      seed: java.lang.Long): KMeansModel = {
         val kMeansAlg = new KMeans()
           .setK(k)
           .setMaxIterations(maxIterations)
           .setRuns(runs)
           .setInitializationMode(initializationMode)
    +
    +    if (seed != null) kMeansAlg.setSeed(seed)
    +
         try {
           kMeansAlg.run(data.rdd.persist(StorageLevel.MEMORY_AND_DISK))
         } finally {
    @@ -624,6 +628,21 @@ class PythonMLLibAPI extends Serializable {
         RG.normalRDD(jsc.sc, size, parts, s)
       }
     
    +  /**
    +   * Java stub for Python mllib RandomRDDGenerators.logNormalRDD()
    +   */
    +  def logNormalRDD(jsc: JavaSparkContext,
    +      mean: Double,
    +      std: Double,
    +      size: Long,
    +      numPartitions: java.lang.Integer,
    +      seed: java.lang.Long): JavaRDD[Double] = {
    +    val parts = getNumPartitionsOrDefault(numPartitions, jsc)
    +    val s = getSeedOrDefault(seed)
    +    RG.logNormalRDD(jsc.sc, mean, std, size, parts, s)
    +  }
    +
    +
       /**
        * Java stub for Python mllib RandomRDDGenerators.poissonRDD()
        */
    @@ -637,6 +656,33 @@ class PythonMLLibAPI extends Serializable {
         RG.poissonRDD(jsc.sc, mean, size, parts, s)
       }
     
    +  /**
    +   * Java stub for Python mllib RandomRDDGenerators.exponentialRDD()
    +   */
    +  def exponentialRDD(jsc: JavaSparkContext,
    +      mean: Double,
    +      size: Long,
    +      numPartitions: java.lang.Integer,
    +      seed: java.lang.Long): JavaRDD[Double] = {
    +    val parts = getNumPartitionsOrDefault(numPartitions, jsc)
    +    val s = getSeedOrDefault(seed)
    +    RG.exponentialRDD(jsc.sc, mean, size, parts, s)
    +  }
    +
    +  /**
    +   * Java stub for Python mllib RandomRDDGenerators.gammaRDD()
    +   */
    +  def gammaRDD(jsc: JavaSparkContext,
    +      shape: Double,
    +      scale: Double,
    +      size: Long,
    +      numPartitions: java.lang.Integer,
    +      seed: java.lang.Long): JavaRDD[Double] = {
    +    val parts = getNumPartitionsOrDefault(numPartitions, jsc)
    +    val s = getSeedOrDefault(seed)
    +    RG.gammaRDD(jsc.sc, shape, scale, size, parts, s)
    +  }
    +
       /**
        * Java stub for Python mllib RandomRDDGenerators.uniformVectorRDD()
        */
    @@ -663,6 +709,22 @@ class PythonMLLibAPI extends Serializable {
         RG.normalVectorRDD(jsc.sc, numRows, numCols, parts, s)
       }
     
    +  /**
    +   * Java stub for Python mllib RandomRDDGenerators.logNormalVectorRDD()
    +   */
    +  def logNormalVectorRDD(jsc: JavaSparkContext,
    +      mean: Double,
    +      std: Double,
    +      numRows: Long,
    +      numCols: Int,
    +      numPartitions: java.lang.Integer,
    +      seed: java.lang.Long): JavaRDD[Vector] = {
    +    val parts = getNumPartitionsOrDefault(numPartitions, jsc)
    +    val s = getSeedOrDefault(seed)
    +    RG.logNormalVectorRDD(jsc.sc, mean, std, numRows, numCols, parts, s)
    +  }
    +
    +
       /**
        * Java stub for Python mllib RandomRDDGenerators.poissonVectorRDD()
        */
    @@ -677,6 +739,36 @@ class PythonMLLibAPI extends Serializable {
         RG.poissonVectorRDD(jsc.sc, mean, numRows, numCols, parts, s)
       }
     
    +  /**
    +   * Java stub for Python mllib RandomRDDGenerators.exponentialVectorRDD()
    +   */
    +  def exponentialVectorRDD(jsc: JavaSparkContext,
    +      mean: Double,
    +      numRows: Long,
    +      numCols: Int,
    +      numPartitions: java.lang.Integer,
    +      seed: java.lang.Long): JavaRDD[Vector] = {
    +    val parts = getNumPartitionsOrDefault(numPartitions, jsc)
    +    val s = getSeedOrDefault(seed)
    +    RG.exponentialVectorRDD(jsc.sc, mean, numRows, numCols, parts, s)
    +  }
    +
    +  /**
    +   * Java stub for Python mllib RandomRDDGenerators.gammaVectorRDD()
    +   */
    +  def gammaVectorRDD(jsc: JavaSparkContext,
    +      shape: Double,
    +      scale: Double,
    +      numRows: Long,
    +      numCols: Int,
    +      numPartitions: java.lang.Integer,
    +      seed: java.lang.Long): JavaRDD[Vector] = {
    +    val parts = getNumPartitionsOrDefault(numPartitions, jsc)
    +    val s = getSeedOrDefault(seed)
    +    RG.gammaVectorRDD(jsc.sc, shape, scale, numRows, numCols, parts, s)
    +  }
    +
    +
     }
     
     /**
    diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala
    index 8c8e4a161aa5..a967df857bed 100644
    --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala
    +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala
    @@ -93,10 +93,10 @@ class NaiveBayes private (private var lambda: Double) extends Serializable with
       def run(data: RDD[LabeledPoint]) = {
         val requireNonnegativeValues: Vector => Unit = (v: Vector) => {
           val values = v match {
    -        case sv: SparseVector =>
    -          sv.values
    -        case dv: DenseVector =>
    -          dv.values
    +        case SparseVector(size, indices, values) =>
    +          values
    +        case DenseVector(values) =>
    +          values
           }
           if (!values.forall(_ >= 0.0)) {
             throw new SparkException(s"Naive Bayes requires nonnegative feature values but found $v.")
    diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureEM.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureEM.scala
    new file mode 100644
    index 000000000000..899fe5e9e9cf
    --- /dev/null
    +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureEM.scala
    @@ -0,0 +1,251 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.mllib.clustering
    +
    +import scala.collection.mutable.IndexedSeq
    +
    +import breeze.linalg.{DenseVector => BreezeVector, DenseMatrix => BreezeMatrix, diag, Transpose}
    +
    +import org.apache.spark.mllib.linalg.{Matrices, Vector, Vectors, DenseVector, DenseMatrix, BLAS}
    +import org.apache.spark.mllib.stat.distribution.MultivariateGaussian
    +import org.apache.spark.mllib.util.MLUtils
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.util.Utils
    +
    +/**
    + * This class performs expectation maximization for multivariate Gaussian
    + * Mixture Models (GMMs).  A GMM represents a composite distribution of
    + * independent Gaussian distributions with associated "mixing" weights
    + * specifying each's contribution to the composite.
    + *
    + * Given a set of sample points, this class will maximize the log-likelihood 
    + * for a mixture of k Gaussians, iterating until the log-likelihood changes by 
    + * less than convergenceTol, or until it has reached the max number of iterations.
    + * While this process is generally guaranteed to converge, it is not guaranteed
    + * to find a global optimum.  
    + * 
    + * @param k The number of independent Gaussians in the mixture model
    + * @param convergenceTol The maximum change in log-likelihood at which convergence
    + * is considered to have occurred.
    + * @param maxIterations The maximum number of iterations to perform
    + */
    +class GaussianMixtureEM private (
    +    private var k: Int, 
    +    private var convergenceTol: Double, 
    +    private var maxIterations: Int,
    +    private var seed: Long) extends Serializable {
    +  
    +  /** A default instance, 2 Gaussians, 100 iterations, 0.01 log-likelihood threshold */
    +  def this() = this(2, 0.01, 100, Utils.random.nextLong())
    +  
    +  // number of samples per cluster to use when initializing Gaussians
    +  private val nSamples = 5
    +  
    +  // an initializing GMM can be provided rather than using the 
    +  // default random starting point
    +  private var initialModel: Option[GaussianMixtureModel] = None
    +  
    +  /** Set the initial GMM starting point, bypassing the random initialization.
    +   *  You must call setK() prior to calling this method, and the condition
    +   *  (model.k == this.k) must be met; failure will result in an IllegalArgumentException
    +   */
    +  def setInitialModel(model: GaussianMixtureModel): this.type = {
    +    if (model.k == k) {
    +      initialModel = Some(model)
    +    } else {
    +      throw new IllegalArgumentException("mismatched cluster count (model.k != k)")
    +    }
    +    this
    +  }
    +  
    +  /** Return the user supplied initial GMM, if supplied */
    +  def getInitialModel: Option[GaussianMixtureModel] = initialModel
    +  
    +  /** Set the number of Gaussians in the mixture model.  Default: 2 */
    +  def setK(k: Int): this.type = {
    +    this.k = k
    +    this
    +  }
    +  
    +  /** Return the number of Gaussians in the mixture model */
    +  def getK: Int = k
    +  
    +  /** Set the maximum number of iterations to run. Default: 100 */
    +  def setMaxIterations(maxIterations: Int): this.type = {
    +    this.maxIterations = maxIterations
    +    this
    +  }
    +  
    +  /** Return the maximum number of iterations to run */
    +  def getMaxIterations: Int = maxIterations
    +  
    +  /**
    +   * Set the largest change in log-likelihood at which convergence is 
    +   * considered to have occurred.
    +   */
    +  def setConvergenceTol(convergenceTol: Double): this.type = {
    +    this.convergenceTol = convergenceTol
    +    this
    +  }
    +  
    +  /**
    +   * Return the largest change in log-likelihood at which convergence is
    +   * considered to have occurred.
    +   */
    +  def getConvergenceTol: Double = convergenceTol
    +
    +  /** Set the random seed */
    +  def setSeed(seed: Long): this.type = {
    +    this.seed = seed
    +    this
    +  }
    +
    +  /** Return the random seed */
    +  def getSeed: Long = seed
    +
    +  /** Perform expectation maximization */
    +  def run(data: RDD[Vector]): GaussianMixtureModel = {
    +    val sc = data.sparkContext
    +    
    +    // we will operate on the data as breeze data
    +    val breezeData = data.map(u => u.toBreeze.toDenseVector).cache()
    +    
    +    // Get length of the input vectors
    +    val d = breezeData.first().length
    +    
    +    // Determine initial weights and corresponding Gaussians.
    +    // If the user supplied an initial GMM, we use those values, otherwise
    +    // we start with uniform weights, a random mean from the data, and
    +    // diagonal covariance matrices using component variances
    +    // derived from the samples    
    +    val (weights, gaussians) = initialModel match {
    +      case Some(gmm) => (gmm.weights, gmm.gaussians)
    +      
    +      case None => {
    +        val samples = breezeData.takeSample(withReplacement = true, k * nSamples, seed)
    +        (Array.fill(k)(1.0 / k), Array.tabulate(k) { i => 
    +          val slice = samples.view(i * nSamples, (i + 1) * nSamples)
    +          new MultivariateGaussian(vectorMean(slice), initCovariance(slice)) 
    +        })  
    +      }
    +    }
    +    
    +    var llh = Double.MinValue // current log-likelihood 
    +    var llhp = 0.0            // previous log-likelihood
    +    
    +    var iter = 0
    +    while(iter < maxIterations && Math.abs(llh-llhp) > convergenceTol) {
    +      // create and broadcast curried cluster contribution function
    +      val compute = sc.broadcast(ExpectationSum.add(weights, gaussians)_)
    +      
    +      // aggregate the cluster contribution for all sample points
    +      val sums = breezeData.aggregate(ExpectationSum.zero(k, d))(compute.value, _ += _)
    +      
    +      // Create new distributions based on the partial assignments
    +      // (often referred to as the "M" step in literature)
    +      val sumWeights = sums.weights.sum
    +      var i = 0
    +      while (i < k) {
    +        val mu = sums.means(i) / sums.weights(i)
    +        BLAS.syr(-sums.weights(i), Vectors.fromBreeze(mu).asInstanceOf[DenseVector],
    +          Matrices.fromBreeze(sums.sigmas(i)).asInstanceOf[DenseMatrix])
    +        weights(i) = sums.weights(i) / sumWeights
    +        gaussians(i) = new MultivariateGaussian(mu, sums.sigmas(i) / sums.weights(i))
    +        i = i + 1
    +      }
    +   
    +      llhp = llh // current becomes previous
    +      llh = sums.logLikelihood // this is the freshly computed log-likelihood
    +      iter += 1
    +    } 
    +    
    +    new GaussianMixtureModel(weights, gaussians)
    +  }
    +    
    +  /** Average of dense breeze vectors */
    +  private def vectorMean(x: IndexedSeq[BreezeVector[Double]]): BreezeVector[Double] = {
    +    val v = BreezeVector.zeros[Double](x(0).length)
    +    x.foreach(xi => v += xi)
    +    v / x.length.toDouble 
    +  }
    +  
    +  /**
    +   * Construct matrix where diagonal entries are element-wise
    +   * variance of input vectors (computes biased variance)
    +   */
    +  private def initCovariance(x: IndexedSeq[BreezeVector[Double]]): BreezeMatrix[Double] = {
    +    val mu = vectorMean(x)
    +    val ss = BreezeVector.zeros[Double](x(0).length)
    +    x.map(xi => (xi - mu) :^ 2.0).foreach(u => ss += u)
    +    diag(ss / x.length.toDouble)
    +  }
    +}
    +
    +// companion class to provide zero constructor for ExpectationSum
    +private object ExpectationSum {
    +  def zero(k: Int, d: Int): ExpectationSum = {
    +    new ExpectationSum(0.0, Array.fill(k)(0.0), 
    +      Array.fill(k)(BreezeVector.zeros(d)), Array.fill(k)(BreezeMatrix.zeros(d,d)))
    +  }
    +  
    +  // compute cluster contributions for each input point
    +  // (U, T) => U for aggregation
    +  def add(
    +      weights: Array[Double], 
    +      dists: Array[MultivariateGaussian])
    +      (sums: ExpectationSum, x: BreezeVector[Double]): ExpectationSum = {
    +    val p = weights.zip(dists).map {
    +      case (weight, dist) => MLUtils.EPSILON + weight * dist.pdf(x)
    +    }
    +    val pSum = p.sum
    +    sums.logLikelihood += math.log(pSum)
    +    val xxt = x * new Transpose(x)
    +    var i = 0
    +    while (i < sums.k) {
    +      p(i) /= pSum
    +      sums.weights(i) += p(i)
    +      sums.means(i) += x * p(i)
    +      BLAS.syr(p(i), Vectors.fromBreeze(x).asInstanceOf[DenseVector],
    +        Matrices.fromBreeze(sums.sigmas(i)).asInstanceOf[DenseMatrix])
    +      i = i + 1
    +    }
    +    sums
    +  }  
    +}
    +
    +// Aggregation class for partial expectation results
    +private class ExpectationSum(
    +    var logLikelihood: Double,
    +    val weights: Array[Double],
    +    val means: Array[BreezeVector[Double]],
    +    val sigmas: Array[BreezeMatrix[Double]]) extends Serializable {
    +  
    +  val k = weights.length
    +  
    +  def +=(x: ExpectationSum): ExpectationSum = {
    +    var i = 0
    +    while (i < k) {
    +      weights(i) += x.weights(i)
    +      means(i) += x.means(i)
    +      sigmas(i) += x.sigmas(i)
    +      i = i + 1
    +    }
    +    logLikelihood += x.logLikelihood
    +    this
    +  }  
    +}
    diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala
    new file mode 100644
    index 000000000000..1a2178ee7f71
    --- /dev/null
    +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala
    @@ -0,0 +1,83 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.mllib.clustering
    +
    +import breeze.linalg.{DenseVector => BreezeVector}
    +
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.mllib.linalg.Vector
    +import org.apache.spark.mllib.stat.distribution.MultivariateGaussian
    +import org.apache.spark.mllib.util.MLUtils
    +
    +/**
    + * Multivariate Gaussian Mixture Model (GMM) consisting of k Gaussians, where points 
    + * are drawn from each Gaussian i=1..k with probability w(i); mu(i) and sigma(i) are 
    + * the respective mean and covariance for each Gaussian distribution i=1..k. 
    + * 
    + * @param weight Weights for each Gaussian distribution in the mixture, where weight(i) is
    + *               the weight for Gaussian i, and weight.sum == 1
    + * @param mu Means for each Gaussian in the mixture, where mu(i) is the mean for Gaussian i
    + * @param sigma Covariance maxtrix for each Gaussian in the mixture, where sigma(i) is the
    + *              covariance matrix for Gaussian i
    + */
    +class GaussianMixtureModel(
    +  val weights: Array[Double], 
    +  val gaussians: Array[MultivariateGaussian]) extends Serializable {
    +  
    +  require(weights.length == gaussians.length, "Length of weight and Gaussian arrays must match")
    +  
    +  /** Number of gaussians in mixture */
    +  def k: Int = weights.length
    +
    +  /** Maps given points to their cluster indices. */
    +  def predict(points: RDD[Vector]): RDD[Int] = {
    +    val responsibilityMatrix = predictSoft(points)
    +    responsibilityMatrix.map(r => r.indexOf(r.max))
    +  }
    +  
    +  /**
    +   * Given the input vectors, return the membership value of each vector
    +   * to all mixture components. 
    +   */
    +  def predictSoft(points: RDD[Vector]): RDD[Array[Double]] = {
    +    val sc = points.sparkContext
    +    val bcDists = sc.broadcast(gaussians)
    +    val bcWeights = sc.broadcast(weights)
    +    points.map { x => 
    +      computeSoftAssignments(x.toBreeze.toDenseVector, bcDists.value, bcWeights.value, k)
    +    }
    +  }
    +  
    +  /**
    +   * Compute the partial assignments for each vector
    +   */
    +  private def computeSoftAssignments(
    +      pt: BreezeVector[Double],
    +      dists: Array[MultivariateGaussian],
    +      weights: Array[Double],
    +      k: Int): Array[Double] = {
    +    val p = weights.zip(dists).map {
    +      case (weight, dist) => MLUtils.EPSILON + weight * dist.pdf(pt)
    +    }
    +    val pSum = p.sum 
    +    for (i <- 0 until k) {
    +      p(i) /= pSum
    +    }
    +    p
    +  }  
    +}
    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 54c301d3e9e1..fc46da3a9342 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
    @@ -19,14 +19,14 @@ package org.apache.spark.mllib.clustering
     
     import scala.collection.mutable.ArrayBuffer
     
    -import org.apache.spark.annotation.Experimental
     import org.apache.spark.Logging
    -import org.apache.spark.SparkContext._
    +import org.apache.spark.annotation.Experimental
     import org.apache.spark.mllib.linalg.{Vector, Vectors}
     import org.apache.spark.mllib.linalg.BLAS.{axpy, scal}
     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
     import org.apache.spark.util.random.XORShiftRandom
     
     /**
    @@ -43,13 +43,14 @@ class KMeans private (
         private var runs: Int,
         private var initializationMode: String,
         private var initializationSteps: Int,
    -    private var epsilon: Double) extends Serializable with Logging {
    +    private var epsilon: Double,
    +    private var seed: Long) extends Serializable with Logging {
     
       /**
        * Constructs a KMeans instance with default parameters: {k: 2, maxIterations: 20, runs: 1,
    -   * initializationMode: "k-means||", initializationSteps: 5, epsilon: 1e-4}.
    +   * initializationMode: "k-means||", initializationSteps: 5, epsilon: 1e-4, seed: random}.
        */
    -  def this() = this(2, 20, 1, KMeans.K_MEANS_PARALLEL, 5, 1e-4)
    +  def this() = this(2, 20, 1, KMeans.K_MEANS_PARALLEL, 5, 1e-4, Utils.random.nextLong())
     
       /** Set the number of clusters to create (k). Default: 2. */
       def setK(k: Int): this.type = {
    @@ -112,6 +113,12 @@ class KMeans private (
         this
       }
     
    +  /** Set the random seed for cluster initialization. */
    +  def setSeed(seed: Long): this.type = {
    +    this.seed = seed
    +    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.
    @@ -255,7 +262,7 @@ class KMeans private (
       private def initRandom(data: RDD[VectorWithNorm])
       : Array[Array[VectorWithNorm]] = {
         // Sample all the cluster centers in one pass to avoid repeated scans
    -    val sample = data.takeSample(true, runs * k, new XORShiftRandom().nextInt()).toSeq
    +    val sample = data.takeSample(true, runs * k, new XORShiftRandom(this.seed).nextInt()).toSeq
         Array.tabulate(runs)(r => sample.slice(r * k, (r + 1) * k).map { v =>
           new VectorWithNorm(Vectors.dense(v.vector.toArray), v.norm)
         }.toArray)
    @@ -272,45 +279,80 @@ class KMeans private (
        */
       private def initKMeansParallel(data: RDD[VectorWithNorm])
       : Array[Array[VectorWithNorm]] = {
    -    // Initialize each run's center to a random point
    -    val seed = new XORShiftRandom().nextInt()
    +    // Initialize empty centers and point costs.
    +    val centers = Array.tabulate(runs)(r => ArrayBuffer.empty[VectorWithNorm])
    +    var costs = data.map(_ => Vectors.dense(Array.fill(runs)(Double.PositiveInfinity))).cache()
    +
    +    // Initialize each run's first center to a random point.
    +    val seed = new XORShiftRandom(this.seed).nextInt()
         val sample = data.takeSample(true, runs, seed).toSeq
    -    val centers = Array.tabulate(runs)(r => ArrayBuffer(sample(r).toDense))
    +    val newCenters = Array.tabulate(runs)(r => ArrayBuffer(sample(r).toDense))
    +
    +    /** Merges new centers to centers. */
    +    def mergeNewCenters(): Unit = {
    +      var r = 0
    +      while (r < runs) {
    +        centers(r) ++= newCenters(r)
    +        newCenters(r).clear()
    +        r += 1
    +      }
    +    }
     
         // On each step, sample 2 * k points on average for each run with probability proportional
    -    // to their squared distance from that run's current centers
    +    // to their squared distance from that run's centers. Note that only distances between points
    +    // and new centers are computed in each iteration.
         var step = 0
         while (step < initializationSteps) {
    -      val bcCenters = data.context.broadcast(centers)
    -      val sumCosts = data.flatMap { point =>
    -        (0 until runs).map { r =>
    -          (r, KMeans.pointCost(bcCenters.value(r), point))
    -        }
    -      }.reduceByKey(_ + _).collectAsMap()
    -      val chosen = data.mapPartitionsWithIndex { (index, points) =>
    +      val bcNewCenters = data.context.broadcast(newCenters)
    +      val preCosts = costs
    +      costs = data.zip(preCosts).map { case (point, cost) =>
    +        Vectors.dense(
    +          Array.tabulate(runs) { r =>
    +            math.min(KMeans.pointCost(bcNewCenters.value(r), point), cost(r))
    +          })
    +      }.cache()
    +      val sumCosts = costs
    +        .aggregate(Vectors.zeros(runs))(
    +          seqOp = (s, v) => {
    +            // s += v
    +            axpy(1.0, v, s)
    +            s
    +          },
    +          combOp = (s0, s1) => {
    +            // s0 += s1
    +            axpy(1.0, s1, s0)
    +            s0
    +          }
    +        )
    +      preCosts.unpersist(blocking = false)
    +      val chosen = data.zip(costs).mapPartitionsWithIndex { (index, pointsWithCosts) =>
             val rand = new XORShiftRandom(seed ^ (step << 16) ^ index)
    -        points.flatMap { p =>
    +        pointsWithCosts.flatMap { case (p, c) =>
               (0 until runs).filter { r =>
    -            rand.nextDouble() < 2.0 * KMeans.pointCost(bcCenters.value(r), p) * k / sumCosts(r)
    +            rand.nextDouble() < 2.0 * c(r) * k / sumCosts(r)
               }.map((_, p))
             }
           }.collect()
    +      mergeNewCenters()
           chosen.foreach { case (r, p) =>
    -        centers(r) += p.toDense
    +        newCenters(r) += p.toDense
           }
           step += 1
         }
     
    +    mergeNewCenters()
    +    costs.unpersist(blocking = false)
    +
         // Finally, we might have a set of more than k candidate centers for each run; weigh each
         // candidate by the number of points in the dataset mapping to it and run a local k-means++
         // on the weighted centers to pick just k of them
         val bcCenters = data.context.broadcast(centers)
         val weightMap = data.flatMap { p =>
    -      (0 until runs).map { r =>
    +      Iterator.tabulate(runs) { r =>
             ((r, KMeans.findClosest(bcCenters.value(r), p)._1), 1.0)
           }
         }.reduceByKey(_ + _).collectAsMap()
    -    val finalCenters = (0 until runs).map { r =>
    +    val finalCenters = (0 until runs).par.map { r =>
           val myCenters = centers(r).toArray
           val myWeights = (0 until myCenters.length).map(i => weightMap.getOrElse((r, i), 0.0)).toArray
           LocalKMeans.kMeansPlusPlus(r, myCenters, myWeights, k, 30)
    @@ -333,7 +375,32 @@ object KMeans {
       /**
        * Trains a k-means model using the given set of parameters.
        *
    -   * @param data training points stored as `RDD[Array[Double]]`
    +   * @param data training points stored as `RDD[Vector]`
    +   * @param k number of clusters
    +   * @param maxIterations max number of iterations
    +   * @param runs number of parallel runs, defaults to 1. The best model is returned.
    +   * @param initializationMode initialization model, either "random" or "k-means||" (default).
    +   * @param seed random seed value for cluster initialization
    +   */
    +  def train(
    +      data: RDD[Vector],
    +      k: Int,
    +      maxIterations: Int,
    +      runs: Int,
    +      initializationMode: String,
    +      seed: Long): KMeansModel = {
    +    new KMeans().setK(k)
    +      .setMaxIterations(maxIterations)
    +      .setRuns(runs)
    +      .setInitializationMode(initializationMode)
    +      .setSeed(seed)
    +      .run(data)
    +  }
    +
    +  /**
    +   * Trains a k-means model using the given set of parameters.
    +   *
    +   * @param data training points stored as `RDD[Vector]`
        * @param k number of clusters
        * @param maxIterations max number of iterations
        * @param runs number of parallel runs, defaults to 1. The best model is returned.
    diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetrics.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetrics.scala
    index 1af40de2c7fc..ced042e2f96c 100644
    --- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetrics.scala
    +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetrics.scala
    @@ -28,9 +28,30 @@ import org.apache.spark.rdd.{RDD, UnionRDD}
      * Evaluator for binary classification.
      *
      * @param scoreAndLabels an RDD of (score, label) pairs.
    + * @param numBins if greater than 0, then the curves (ROC curve, PR curve) computed internally
    + *                will be down-sampled to this many "bins". If 0, no down-sampling will occur.
    + *                This is useful because the curve contains a point for each distinct score
    + *                in the input, and this could be as large as the input itself -- millions of
    + *                points or more, when thousands may be entirely sufficient to summarize
    + *                the curve. After down-sampling, the curves will instead be made of approximately
    + *                `numBins` points instead. Points are made from bins of equal numbers of
    + *                consecutive points. The size of each bin is
    + *                `floor(scoreAndLabels.count() / numBins)`, which means the resulting number
    + *                of bins may not exactly equal numBins. The last bin in each partition may
    + *                be smaller as a result, meaning there may be an extra sample at
    + *                partition boundaries.
      */
     @Experimental
    -class BinaryClassificationMetrics(scoreAndLabels: RDD[(Double, Double)]) extends Logging {
    +class BinaryClassificationMetrics(
    +    val scoreAndLabels: RDD[(Double, Double)],
    +    val numBins: Int) extends Logging {
    +
    +  require(numBins >= 0, "numBins must be nonnegative")
    +
    +  /**
    +   * Defaults `numBins` to 0.
    +   */
    +  def this(scoreAndLabels: RDD[(Double, Double)]) = this(scoreAndLabels, 0)
     
       /** Unpersist intermediate RDDs used in the computation. */
       def unpersist() {
    @@ -103,7 +124,39 @@ class BinaryClassificationMetrics(scoreAndLabels: RDD[(Double, Double)]) extends
           mergeValue = (c: BinaryLabelCounter, label: Double) => c += label,
           mergeCombiners = (c1: BinaryLabelCounter, c2: BinaryLabelCounter) => c1 += c2
         ).sortByKey(ascending = false)
    -    val agg = counts.values.mapPartitions { iter =>
    +
    +    val binnedCounts =
    +      // Only down-sample if bins is > 0
    +      if (numBins == 0) {
    +        // Use original directly
    +        counts
    +      } else {
    +        val countsSize = counts.count()
    +        // Group the iterator into chunks of about countsSize / numBins points,
    +        // so that the resulting number of bins is about numBins
    +        var grouping = countsSize / numBins
    +        if (grouping < 2) {
    +          // numBins was more than half of the size; no real point in down-sampling to bins
    +          logInfo(s"Curve is too small ($countsSize) for $numBins bins to be useful")
    +          counts
    +        } else {
    +          if (grouping >= Int.MaxValue) {
    +            logWarning(
    +              s"Curve too large ($countsSize) for $numBins bins; capping at ${Int.MaxValue}")
    +            grouping = Int.MaxValue
    +          }
    +          counts.mapPartitions(_.grouped(grouping.toInt).map { pairs =>
    +            // The score of the combined point will be just the first one's score
    +            val firstScore = pairs.head._1
    +            // The point will contain all counts in this chunk
    +            val agg = new BinaryLabelCounter()
    +            pairs.foreach(pair => agg += pair._2)
    +            (firstScore, agg)
    +          })
    +        }
    +      }
    +
    +    val agg = binnedCounts.values.mapPartitions { iter =>
           val agg = new BinaryLabelCounter()
           iter.foreach(agg += _)
           Iterator(agg)
    @@ -113,7 +166,7 @@ class BinaryClassificationMetrics(scoreAndLabels: RDD[(Double, Double)]) extends
             (agg: BinaryLabelCounter, c: BinaryLabelCounter) => agg.clone() += c)
         val totalCount = partitionwiseCumulativeCounts.last
         logInfo(s"Total counts: $totalCount")
    -    val cumulativeCounts = counts.mapPartitionsWithIndex(
    +    val cumulativeCounts = binnedCounts.mapPartitionsWithIndex(
           (index: Int, iter: Iterator[(Double, BinaryLabelCounter)]) => {
             val cumCount = partitionwiseCumulativeCounts(index)
             iter.map { case (score, c) =>
    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 19120e1e8af1..3260f27513c7 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
    @@ -86,20 +86,20 @@ private object IDF {
             df = BDV.zeros(doc.size)
           }
           doc match {
    -        case sv: SparseVector =>
    -          val nnz = sv.indices.size
    +        case SparseVector(size, indices, values) =>
    +          val nnz = indices.size
               var k = 0
               while (k < nnz) {
    -            if (sv.values(k) > 0) {
    -              df(sv.indices(k)) += 1L
    +            if (values(k) > 0) {
    +              df(indices(k)) += 1L
                 }
                 k += 1
               }
    -        case dv: DenseVector =>
    -          val n = dv.size
    +        case DenseVector(values) =>
    +          val n = values.size
               var j = 0
               while (j < n) {
    -            if (dv.values(j) > 0.0) {
    +            if (values(j) > 0.0) {
                   df(j) += 1L
                 }
                 j += 1
    @@ -207,20 +207,20 @@ private object IDFModel {
       def transform(idf: Vector, v: Vector): Vector = {
         val n = v.size
         v match {
    -      case sv: SparseVector =>
    -        val nnz = sv.indices.size
    +      case SparseVector(size, indices, values) =>
    +        val nnz = indices.size
             val newValues = new Array[Double](nnz)
             var k = 0
             while (k < nnz) {
    -          newValues(k) = sv.values(k) * idf(sv.indices(k))
    +          newValues(k) = values(k) * idf(indices(k))
               k += 1
             }
    -        Vectors.sparse(n, sv.indices, newValues)
    -      case dv: DenseVector =>
    +        Vectors.sparse(n, indices, newValues)
    +      case DenseVector(values) =>
             val newValues = new Array[Double](n)
             var j = 0
             while (j < n) {
    -          newValues(j) = dv.values(j) * idf(j)
    +          newValues(j) = values(j) * idf(j)
               j += 1
             }
             Vectors.dense(newValues)
    diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala
    index 1ced26a9b70a..32848e039eb8 100644
    --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala
    +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala
    @@ -52,8 +52,8 @@ class Normalizer(p: Double) extends VectorTransformer {
           // However, for sparse vector, the `index` array will not be changed,
           // so we can re-use it to save memory.
           vector match {
    -        case dv: DenseVector =>
    -          val values = dv.values.clone()
    +        case DenseVector(vs) =>
    +          val values = vs.clone()
               val size = values.size
               var i = 0
               while (i < size) {
    @@ -61,15 +61,15 @@ class Normalizer(p: Double) extends VectorTransformer {
                 i += 1
               }
               Vectors.dense(values)
    -        case sv: SparseVector =>
    -          val values = sv.values.clone()
    +        case SparseVector(size, ids, vs) =>
    +          val values = vs.clone()
               val nnz = values.size
               var i = 0
               while (i < nnz) {
                 values(i) /= norm
                 i += 1
               }
    -          Vectors.sparse(sv.size, sv.indices, values)
    +          Vectors.sparse(size, ids, values)
             case v => throw new IllegalArgumentException("Do not support vector type " + v.getClass)
           }
         } else {
    diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala
    index 8c4c5db5258d..3c2091732f9b 100644
    --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala
    +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala
    @@ -105,8 +105,8 @@ class StandardScalerModel private[mllib] (
           // This can be avoid by having a local reference of `shift`.
           val localShift = shift
           vector match {
    -        case dv: DenseVector =>
    -          val values = dv.values.clone()
    +        case DenseVector(vs) =>
    +          val values = vs.clone()
               val size = values.size
               if (withStd) {
                 // Having a local reference of `factor` to avoid overhead as the comment before.
    @@ -130,8 +130,8 @@ class StandardScalerModel private[mllib] (
           // Having a local reference of `factor` to avoid overhead as the comment before.
           val localFactor = factor
           vector match {
    -        case dv: DenseVector =>
    -          val values = dv.values.clone()
    +        case DenseVector(vs) =>
    +          val values = vs.clone()
               val size = values.size
               var i = 0
               while(i < size) {
    @@ -139,18 +139,17 @@ class StandardScalerModel private[mllib] (
                 i += 1
               }
               Vectors.dense(values)
    -        case sv: SparseVector =>
    +        case SparseVector(size, indices, vs) =>
               // For sparse vector, the `index` array inside sparse vector object will not be changed,
               // so we can re-use it to save memory.
    -          val indices = sv.indices
    -          val values = sv.values.clone()
    +          val values = vs.clone()
               val nnz = values.size
               var i = 0
               while (i < nnz) {
                 values(i) *= localFactor(indices(i))
                 i += 1
               }
    -          Vectors.sparse(sv.size, indices, values)
    +          Vectors.sparse(size, indices, values)
             case v => throw new IllegalArgumentException("Do not support vector type " + v.getClass)
           }
         } else {
    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 7960f3cab576..d25a7cd5b439 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
    @@ -71,7 +71,8 @@ class Word2Vec extends Serializable with Logging {
       private var numPartitions = 1
       private var numIterations = 1
       private var seed = Utils.random.nextLong()
    -
    +  private var minCount = 5
    +  
       /**
        * Sets vector size (default: 100).
        */
    @@ -114,6 +115,15 @@ class Word2Vec extends Serializable with Logging {
         this
       }
     
    +  /** 
    +   * Sets minCount, the minimum number of times a token must appear to be included in the word2vec 
    +   * model's vocabulary (default: 5).
    +   */
    +  def setMinCount(minCount: Int): this.type = {
    +    this.minCount = minCount
    +    this
    +  }
    +  
       private val EXP_TABLE_SIZE = 1000
       private val MAX_EXP = 6
       private val MAX_CODE_LENGTH = 40
    @@ -122,9 +132,6 @@ class Word2Vec extends Serializable with Logging {
       /** context words from [-window, window] */
       private val window = 5
     
    -  /** minimum frequency to consider a vocabulary word */
    -  private val minCount = 5
    -
       private var trainWordsCount = 0
       private var vocabSize = 0
       private var vocab: Array[VocabWord] = null
    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 9fed513becdd..3414daccd7ca 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
    @@ -228,6 +228,32 @@ private[spark] object BLAS extends Serializable with Logging {
         }
         _nativeBLAS
       }
    + 
    +  /**
    +   * A := alpha * x * x^T^ + A
    +   * @param alpha a real scalar that will be multiplied to x * x^T^.
    +   * @param x the vector x that contains the n elements.
    +   * @param A the symmetric matrix A. Size of n x n.
    +   */
    +  def syr(alpha: Double, x: DenseVector, A: DenseMatrix) {
    +    val mA = A.numRows
    +    val nA = A.numCols
    +    require(mA == nA, s"A is not a symmetric matrix. A: $mA x $nA")
    +    require(mA == x.size, s"The size of x doesn't match the rank of A. A: $mA x $nA, x: ${x.size}")
    +
    +    nativeBLAS.dsyr("U", x.size, alpha, x.values, 1, A.values, nA)
    +
    +    // Fill lower triangular part of A
    +    var i = 0
    +    while (i < mA) {
    +      var j = i + 1
    +      while (j < nA) {
    +        A(j, i) = A(i, j)
    +        j += 1
    +      }
    +      i += 1
    +    }    
    +  }
     
       /**
        * C := alpha * A * B + beta * C
    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 327366a1a3a8..5a7281ec6dc3 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,9 +17,11 @@
     
     package org.apache.spark.mllib.linalg
     
    -import java.util.{Random, Arrays}
    +import java.util.{Arrays, Random}
     
    -import breeze.linalg.{Matrix => BM, DenseMatrix => BDM, CSCMatrix => BSM}
    +import scala.collection.mutable.{ArrayBuilder => MArrayBuilder, HashSet => MHashSet, ArrayBuffer}
    +
    +import breeze.linalg.{CSCMatrix => BSM, DenseMatrix => BDM, Matrix => BM}
     
     /**
      * Trait for a local matrix.
    @@ -80,6 +82,16 @@ sealed trait Matrix extends Serializable {
     
       /** A human readable representation of the matrix */
       override def toString: String = toBreeze.toString()
    +
    +  /** Map the values of this matrix using a function. Generates a new matrix. Performs the
    +    * function on only the backing array. For example, an operation such as addition or
    +    * subtraction will only be performed on the non-zero values in a `SparseMatrix`. */
    +  private[mllib] def map(f: Double => Double): Matrix
    +
    +  /** Update all the values of this matrix using the function f. Performed in-place on the
    +    * backing array. For example, an operation such as addition or subtraction will only be
    +    * performed on the non-zero values in a `SparseMatrix`. */
    +  private[mllib] def update(f: Double => Double): Matrix
     }
     
     /**
    @@ -123,6 +135,122 @@ class DenseMatrix(val numRows: Int, val numCols: Int, val values: Array[Double])
       }
     
       override def copy = new DenseMatrix(numRows, numCols, values.clone())
    +
    +  private[mllib] def map(f: Double => Double) = new DenseMatrix(numRows, numCols, values.map(f))
    +
    +  private[mllib] def update(f: Double => Double): DenseMatrix = {
    +    val len = values.length
    +    var i = 0
    +    while (i < len) {
    +      values(i) = f(values(i))
    +      i += 1
    +    }
    +    this
    +  }
    +
    +  /** Generate a `SparseMatrix` from the given `DenseMatrix`. */
    +  def toSparse(): SparseMatrix = {
    +    val spVals: MArrayBuilder[Double] = new MArrayBuilder.ofDouble
    +    val colPtrs: Array[Int] = new Array[Int](numCols + 1)
    +    val rowIndices: MArrayBuilder[Int] = new MArrayBuilder.ofInt
    +    var nnz = 0
    +    var j = 0
    +    while (j < numCols) {
    +      var i = 0
    +      val indStart = j * numRows
    +      while (i < numRows) {
    +        val v = values(indStart + i)
    +        if (v != 0.0) {
    +          rowIndices += i
    +          spVals += v
    +          nnz += 1
    +        }
    +        i += 1
    +      }
    +      j += 1
    +      colPtrs(j) = nnz
    +    }
    +    new SparseMatrix(numRows, numCols, colPtrs, rowIndices.result(), spVals.result())
    +  }
    +}
    +
    +/**
    + * Factory methods for [[org.apache.spark.mllib.linalg.DenseMatrix]].
    + */
    +object DenseMatrix {
    +
    +  /**
    +   * 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): DenseMatrix =
    +    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): DenseMatrix =
    +    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): DenseMatrix = {
    +    val identity = DenseMatrix.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
    +   * @param rng a random number generator
    +   * @return `DenseMatrix` with size `numRows` x `numCols` and values in U(0, 1)
    +   */
    +  def rand(numRows: Int, numCols: Int, rng: Random): DenseMatrix = {
    +    new DenseMatrix(numRows, numCols, Array.fill(numRows * numCols)(rng.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
    +   * @param rng a random number generator
    +   * @return `DenseMatrix` with size `numRows` x `numCols` and values in N(0, 1)
    +   */
    +  def randn(numRows: Int, numCols: Int, rng: Random): DenseMatrix = {
    +    new DenseMatrix(numRows, numCols, Array.fill(numRows * numCols)(rng.nextGaussian()))
    +  }
    +
    +  /**
    +   * Generate a diagonal matrix in `DenseMatrix` format from the supplied values.
    +   * @param vector a `Vector` that 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): DenseMatrix = {
    +    val n = vector.size
    +    val matrix = DenseMatrix.zeros(n, n)
    +    val values = vector.toArray
    +    var i = 0
    +    while (i < n) {
    +      matrix.update(i, i, values(i))
    +      i += 1
    +    }
    +    matrix
    +  }
     }
     
     /**
    @@ -156,6 +284,8 @@ class SparseMatrix(
       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")
    +  require(values.length == colPtrs.last, "The last value of colPtrs must equal the number of " +
    +    s"elements. values.length: ${values.length}, colPtrs.last: ${colPtrs.last}")
     
       override def toArray: Array[Double] = {
         val arr = new Array[Double](numRows * numCols)
    @@ -188,7 +318,7 @@ class SparseMatrix(
     
       private[mllib] def update(i: Int, j: Int, v: Double): Unit = {
         val ind = index(i, j)
    -    if (ind == -1){
    +    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 {
    @@ -197,6 +327,192 @@ class SparseMatrix(
       }
     
       override def copy = new SparseMatrix(numRows, numCols, colPtrs, rowIndices, values.clone())
    +
    +  private[mllib] def map(f: Double => Double) =
    +    new SparseMatrix(numRows, numCols, colPtrs, rowIndices, values.map(f))
    +
    +  private[mllib] def update(f: Double => Double): SparseMatrix = {
    +    val len = values.length
    +    var i = 0
    +    while (i < len) {
    +      values(i) = f(values(i))
    +      i += 1
    +    }
    +    this
    +  }
    +
    +  /** Generate a `DenseMatrix` from the given `SparseMatrix`. */
    +  def toDense(): DenseMatrix = {
    +    new DenseMatrix(numRows, numCols, toArray)
    +  }
    +}
    +
    +/**
    + * Factory methods for [[org.apache.spark.mllib.linalg.SparseMatrix]].
    + */
    +object SparseMatrix {
    +
    +  /**
    +   * Generate a `SparseMatrix` from Coordinate List (COO) format. Input must be an array of
    +   * (i, j, value) tuples. Entries that have duplicate values of i and j are
    +   * added together. Tuples where value is equal to zero will be omitted.
    +   * @param numRows number of rows of the matrix
    +   * @param numCols number of columns of the matrix
    +   * @param entries Array of (i, j, value) tuples
    +   * @return The corresponding `SparseMatrix`
    +   */
    +  def fromCOO(numRows: Int, numCols: Int, entries: Iterable[(Int, Int, Double)]): SparseMatrix = {
    +    val sortedEntries = entries.toSeq.sortBy(v => (v._2, v._1))
    +    val numEntries = sortedEntries.size
    +    if (sortedEntries.nonEmpty) {
    +      // Since the entries are sorted by column index, we only need to check the first and the last.
    +      for (col <- Seq(sortedEntries.head._2, sortedEntries.last._2)) {
    +        require(col >= 0 && col < numCols, s"Column index out of range [0, $numCols): $col.")
    +      }
    +    }
    +    val colPtrs = new Array[Int](numCols + 1)
    +    val rowIndices = MArrayBuilder.make[Int]
    +    rowIndices.sizeHint(numEntries)
    +    val values = MArrayBuilder.make[Double]
    +    values.sizeHint(numEntries)
    +    var nnz = 0
    +    var prevCol = 0
    +    var prevRow = -1
    +    var prevVal = 0.0
    +    // Append a dummy entry to include the last one at the end of the loop.
    +    (sortedEntries.view :+ (numRows, numCols, 1.0)).foreach { case (i, j, v) =>
    +      if (v != 0) {
    +        if (i == prevRow && j == prevCol) {
    +          prevVal += v
    +        } else {
    +          if (prevVal != 0) {
    +            require(prevRow >= 0 && prevRow < numRows,
    +              s"Row index out of range [0, $numRows): $prevRow.")
    +            nnz += 1
    +            rowIndices += prevRow
    +            values += prevVal
    +          }
    +          prevRow = i
    +          prevVal = v
    +          while (prevCol < j) {
    +            colPtrs(prevCol + 1) = nnz
    +            prevCol += 1
    +          }
    +        }
    +      }
    +    }
    +    new SparseMatrix(numRows, numCols, colPtrs, rowIndices.result(), values.result())
    +  }
    +
    +  /**
    +   * Generate an Identity Matrix in `SparseMatrix` format.
    +   * @param n number of rows and columns of the matrix
    +   * @return `SparseMatrix` with size `n` x `n` and values of ones on the diagonal
    +   */
    +  def speye(n: Int): SparseMatrix = {
    +    new SparseMatrix(n, n, (0 to n).toArray, (0 until n).toArray, Array.fill(n)(1.0))
    +  }
    +
    +  /**
    +   * Generates the skeleton of a random `SparseMatrix` with a given random number generator.
    +   * The values of the matrix returned are undefined.
    +   */
    +  private def genRandMatrix(
    +      numRows: Int,
    +      numCols: Int,
    +      density: Double,
    +      rng: Random): SparseMatrix = {
    +    require(numRows > 0, s"numRows must be greater than 0 but got $numRows")
    +    require(numCols > 0, s"numCols must be greater than 0 but got $numCols")
    +    require(density >= 0.0 && density <= 1.0,
    +      s"density must be a double in the range 0.0 <= d <= 1.0. Currently, density: $density")
    +    val size = numRows.toLong * numCols
    +    val expected = size * density
    +    assert(expected < Int.MaxValue,
    +      "The expected number of nonzeros cannot be greater than Int.MaxValue.")
    +    val nnz = math.ceil(expected).toInt
    +    if (density == 0.0) {
    +      new SparseMatrix(numRows, numCols, new Array[Int](numCols + 1), Array[Int](), Array[Double]())
    +    } else if (density == 1.0) {
    +      val colPtrs = Array.tabulate(numCols + 1)(j => j * numRows)
    +      val rowIndices = Array.tabulate(size.toInt)(idx => idx % numRows)
    +      new SparseMatrix(numRows, numCols, colPtrs, rowIndices, new Array[Double](numRows * numCols))
    +    } else if (density < 0.34) {
    +      // draw-by-draw, expected number of iterations is less than 1.5 * nnz
    +      val entries = MHashSet[(Int, Int)]()
    +      while (entries.size < nnz) {
    +        entries += ((rng.nextInt(numRows), rng.nextInt(numCols)))
    +      }
    +      SparseMatrix.fromCOO(numRows, numCols, entries.map(v => (v._1, v._2, 1.0)))
    +    } else {
    +      // selection-rejection method
    +      var idx = 0L
    +      var numSelected = 0
    +      var j = 0
    +      val colPtrs = new Array[Int](numCols + 1)
    +      val rowIndices = new Array[Int](nnz)
    +      while (j < numCols && numSelected < nnz) {
    +        var i = 0
    +        while (i < numRows && numSelected < nnz) {
    +          if (rng.nextDouble() < 1.0 * (nnz - numSelected) / (size - idx)) {
    +            rowIndices(numSelected) = i
    +            numSelected += 1
    +          }
    +          i += 1
    +          idx += 1
    +        }
    +        colPtrs(j + 1) = numSelected
    +        j += 1
    +      }
    +      new SparseMatrix(numRows, numCols, colPtrs, rowIndices, new Array[Double](nnz))
    +    }
    +  }
    +
    +  /**
    +   * Generate a `SparseMatrix` consisting of i.i.d. uniform random numbers. The number of non-zero
    +   * elements equal the ceiling of `numRows` x `numCols` x `density`
    +   *
    +   * @param numRows number of rows of the matrix
    +   * @param numCols number of columns of the matrix
    +   * @param density the desired density for the matrix
    +   * @param rng a random number generator
    +   * @return `SparseMatrix` with size `numRows` x `numCols` and values in U(0, 1)
    +   */
    +  def sprand(numRows: Int, numCols: Int, density: Double, rng: Random): SparseMatrix = {
    +    val mat = genRandMatrix(numRows, numCols, density, rng)
    +    mat.update(i => rng.nextDouble())
    +  }
    +
    +  /**
    +   * Generate a `SparseMatrix` consisting of i.i.d. gaussian random numbers.
    +   * @param numRows number of rows of the matrix
    +   * @param numCols number of columns of the matrix
    +   * @param density the desired density for the matrix
    +   * @param rng a random number generator
    +   * @return `SparseMatrix` with size `numRows` x `numCols` and values in N(0, 1)
    +   */
    +  def sprandn(numRows: Int, numCols: Int, density: Double, rng: Random): SparseMatrix = {
    +    val mat = genRandMatrix(numRows, numCols, density, rng)
    +    mat.update(i => rng.nextGaussian())
    +  }
    +
    +  /**
    +   * Generate a diagonal matrix in `SparseMatrix` format from the supplied values.
    +   * @param vector a `Vector` that will form the values on the diagonal of the matrix
    +   * @return Square `SparseMatrix` with size `values.length` x `values.length` and non-zero
    +   *         `values` on the diagonal
    +   */
    +  def diag(vector: Vector): SparseMatrix = {
    +    val n = vector.size
    +    vector match {
    +      case sVec: SparseVector =>
    +        SparseMatrix.fromCOO(n, n, sVec.indices.zip(sVec.values).map(v => (v._1, v._1, v._2)))
    +      case dVec: DenseVector =>
    +        val entries = dVec.values.zipWithIndex
    +        val nnzVals = entries.filter(v => v._1 != 0.0)
    +        SparseMatrix.fromCOO(n, n, nnzVals.map(v => (v._2, v._2, v._1)))
    +    }
    +  }
     }
     
     /**
    @@ -256,72 +572,250 @@ object Matrices {
        * 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
    +   * @return `Matrix` 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))
    +  def zeros(numRows: Int, numCols: Int): Matrix = DenseMatrix.zeros(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
    +   * @return `Matrix` 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))
    +  def ones(numRows: Int, numCols: Int): Matrix = DenseMatrix.ones(numRows, numCols)
     
       /**
    -   * Generate an Identity Matrix in `DenseMatrix` format.
    +   * Generate a dense Identity Matrix in `Matrix` 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
    +   * @return `Matrix` 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
    -  }
    +  def eye(n: Int): Matrix = DenseMatrix.eye(n)
    +
    +  /**
    +   * Generate a sparse Identity Matrix in `Matrix` format.
    +   * @param n number of rows and columns of the matrix
    +   * @return `Matrix` with size `n` x `n` and values of ones on the diagonal
    +   */
    +  def speye(n: Int): Matrix = SparseMatrix.speye(n)
     
       /**
        * 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
        * @param rng a random number generator
    -   * @return `DenseMatrix` with size `numRows` x `numCols` and values in U(0, 1)
    +   * @return `Matrix` with size `numRows` x `numCols` and values in U(0, 1)
        */
    -  def rand(numRows: Int, numCols: Int, rng: Random): Matrix = {
    -    new DenseMatrix(numRows, numCols, Array.fill(numRows * numCols)(rng.nextDouble()))
    -  }
    +  def rand(numRows: Int, numCols: Int, rng: Random): Matrix =
    +    DenseMatrix.rand(numRows, numCols, rng)
    +
    +  /**
    +   * Generate a `SparseMatrix` consisting of i.i.d. gaussian random numbers.
    +   * @param numRows number of rows of the matrix
    +   * @param numCols number of columns of the matrix
    +   * @param density the desired density for the matrix
    +   * @param rng a random number generator
    +   * @return `Matrix` with size `numRows` x `numCols` and values in U(0, 1)
    +   */
    +  def sprand(numRows: Int, numCols: Int, density: Double, rng: Random): Matrix =
    +    SparseMatrix.sprand(numRows, numCols, density, rng)
     
       /**
        * 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
        * @param rng a random number generator
    -   * @return `DenseMatrix` with size `numRows` x `numCols` and values in N(0, 1)
    +   * @return `Matrix` with size `numRows` x `numCols` and values in N(0, 1)
        */
    -  def randn(numRows: Int, numCols: Int, rng: Random): Matrix = {
    -    new DenseMatrix(numRows, numCols, Array.fill(numRows * numCols)(rng.nextGaussian()))
    -  }
    +  def randn(numRows: Int, numCols: Int, rng: Random): Matrix =
    +    DenseMatrix.randn(numRows, numCols, rng)
    +
    +  /**
    +   * Generate a `SparseMatrix` consisting of i.i.d. gaussian random numbers.
    +   * @param numRows number of rows of the matrix
    +   * @param numCols number of columns of the matrix
    +   * @param density the desired density for the matrix
    +   * @param rng a random number generator
    +   * @return `Matrix` with size `numRows` x `numCols` and values in N(0, 1)
    +   */
    +  def sprandn(numRows: Int, numCols: Int, density: Double, rng: Random): Matrix =
    +    SparseMatrix.sprandn(numRows, numCols, density, rng)
     
       /**
        * 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`
    +   * @return Square `Matrix` 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
    +  def diag(vector: Vector): Matrix = DenseMatrix.diag(vector)
    +
    +  /**
    +   * Horizontally concatenate a sequence of matrices. The returned matrix will be in the format
    +   * the matrices are supplied in. Supplying a mix of dense and sparse matrices will result in
    +   * a sparse matrix. If the Array is empty, an empty `DenseMatrix` will be returned.
    +   * @param matrices array of matrices
    +   * @return a single `Matrix` composed of the matrices that were horizontally concatenated
    +   */
    +  def horzcat(matrices: Array[Matrix]): Matrix = {
    +    if (matrices.isEmpty) {
    +      return new DenseMatrix(0, 0, Array[Double]())
    +    } else if (matrices.size == 1) {
    +      return matrices(0)
    +    }
    +    val numRows = matrices(0).numRows
    +    var hasSparse = false
    +    var numCols = 0
    +    matrices.foreach { mat =>
    +      require(numRows == mat.numRows, "The number of rows of the matrices in this sequence, " +
    +        "don't match!")
    +      mat match {
    +        case sparse: SparseMatrix => hasSparse = true
    +        case dense: DenseMatrix => // empty on purpose
    +        case _ => throw new IllegalArgumentException("Unsupported matrix format. Expected " +
    +          s"SparseMatrix or DenseMatrix. Instead got: ${mat.getClass}")
    +      }
    +      numCols += mat.numCols
    +    }
    +    if (!hasSparse) {
    +      new DenseMatrix(numRows, numCols, matrices.flatMap(_.toArray))
    +    } else {
    +      var startCol = 0
    +      val entries: Array[(Int, Int, Double)] = matrices.flatMap {
    +        case spMat: SparseMatrix =>
    +          var j = 0
    +          val colPtrs = spMat.colPtrs
    +          val rowIndices = spMat.rowIndices
    +          val values = spMat.values
    +          val data = new Array[(Int, Int, Double)](values.length)
    +          val nCols = spMat.numCols
    +          while (j < nCols) {
    +            var idx = colPtrs(j)
    +            while (idx < colPtrs(j + 1)) {
    +              val i = rowIndices(idx)
    +              val v = values(idx)
    +              data(idx) = (i, j + startCol, v)
    +              idx += 1
    +            }
    +            j += 1
    +          }
    +          startCol += nCols
    +          data
    +        case dnMat: DenseMatrix =>
    +          val data = new ArrayBuffer[(Int, Int, Double)]()
    +          var j = 0
    +          val nCols = dnMat.numCols
    +          val nRows = dnMat.numRows
    +          val values = dnMat.values
    +          while (j < nCols) {
    +            var i = 0
    +            val indStart = j * nRows
    +            while (i < nRows) {
    +              val v = values(indStart + i)
    +              if (v != 0.0) {
    +                data.append((i, j + startCol, v))
    +              }
    +              i += 1
    +            }
    +            j += 1
    +          }
    +          startCol += nCols
    +          data
    +      }
    +      SparseMatrix.fromCOO(numRows, numCols, entries)
    +    }
    +  }
    +
    +  /**
    +   * Vertically concatenate a sequence of matrices. The returned matrix will be in the format
    +   * the matrices are supplied in. Supplying a mix of dense and sparse matrices will result in
    +   * a sparse matrix. If the Array is empty, an empty `DenseMatrix` will be returned.
    +   * @param matrices array of matrices
    +   * @return a single `Matrix` composed of the matrices that were vertically concatenated
    +   */
    +  def vertcat(matrices: Array[Matrix]): Matrix = {
    +    if (matrices.isEmpty) {
    +      return new DenseMatrix(0, 0, Array[Double]())
    +    } else if (matrices.size == 1) {
    +      return matrices(0)
    +    }
    +    val numCols = matrices(0).numCols
    +    var hasSparse = false
    +    var numRows = 0
    +    matrices.foreach { mat =>
    +      require(numCols == mat.numCols, "The number of rows of the matrices in this sequence, " +
    +        "don't match!")
    +      mat match {
    +        case sparse: SparseMatrix =>
    +          hasSparse = true
    +        case dense: DenseMatrix =>
    +        case _ => throw new IllegalArgumentException("Unsupported matrix format. Expected " +
    +          s"SparseMatrix or DenseMatrix. Instead got: ${mat.getClass}")
    +      }
    +      numRows += mat.numRows
    +
    +    }
    +    if (!hasSparse) {
    +      val allValues = new Array[Double](numRows * numCols)
    +      var startRow = 0
    +      matrices.foreach { mat =>
    +        var j = 0
    +        val nRows = mat.numRows
    +        val values = mat.toArray
    +        while (j < numCols) {
    +          var i = 0
    +          val indStart = j * numRows + startRow
    +          val subMatStart = j * nRows
    +          while (i < nRows) {
    +            allValues(indStart + i) = values(subMatStart + i)
    +            i += 1
    +          }
    +          j += 1
    +        }
    +        startRow += nRows
    +      }
    +      new DenseMatrix(numRows, numCols, allValues)
    +    } else {
    +      var startRow = 0
    +      val entries: Array[(Int, Int, Double)] = matrices.flatMap {
    +        case spMat: SparseMatrix =>
    +          var j = 0
    +          val colPtrs = spMat.colPtrs
    +          val rowIndices = spMat.rowIndices
    +          val values = spMat.values
    +          val data = new Array[(Int, Int, Double)](values.length)
    +          while (j < numCols) {
    +            var idx = colPtrs(j)
    +            while (idx < colPtrs(j + 1)) {
    +              val i = rowIndices(idx)
    +              val v = values(idx)
    +              data(idx) = (i + startRow, j, v)
    +              idx += 1
    +            }
    +            j += 1
    +          }
    +          startRow += spMat.numRows
    +          data
    +        case dnMat: DenseMatrix =>
    +          val data = new ArrayBuffer[(Int, Int, Double)]()
    +          var j = 0
    +          val nCols = dnMat.numCols
    +          val nRows = dnMat.numRows
    +          val values = dnMat.values
    +          while (j < nCols) {
    +            var i = 0
    +            val indStart = j * nRows
    +            while (i < nRows) {
    +              val v = values(indStart + i)
    +              if (v != 0.0) {
    +                data.append((i + startRow, j, v))
    +              }
    +              i += 1
    +            }
    +            j += 1
    +          }
    +          startRow += nRows
    +          data
    +      }
    +      SparseMatrix.fromCOO(numRows, numCols, entries)
         }
    -    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 47d1a76fa361..7ee0224ad466 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
    @@ -27,9 +27,8 @@ import breeze.linalg.{DenseVector => BDV, SparseVector => BSV, Vector => BV}
     
     import org.apache.spark.SparkException
     import org.apache.spark.mllib.util.NumericParser
    -import org.apache.spark.sql.catalyst.annotation.SQLUserDefinedType
     import org.apache.spark.sql.catalyst.expressions.{GenericMutableRow, Row}
    -import org.apache.spark.sql.catalyst.types._
    +import org.apache.spark.sql.types._
     
     /**
      * Represents a numeric vector, whose index type is Int and value type is Double.
    @@ -51,13 +50,35 @@ sealed trait Vector extends Serializable {
     
       override def equals(other: Any): Boolean = {
         other match {
    -      case v: Vector =>
    -        util.Arrays.equals(this.toArray, v.toArray)
    +      case v2: Vector => {
    +        if (this.size != v2.size) return false
    +        (this, v2) match {
    +          case (s1: SparseVector, s2: SparseVector) =>
    +            Vectors.equals(s1.indices, s1.values, s2.indices, s2.values)
    +          case (s1: SparseVector, d1: DenseVector) =>
    +            Vectors.equals(s1.indices, s1.values, 0 until d1.size, d1.values)
    +          case (d1: DenseVector, s1: SparseVector) =>
    +            Vectors.equals(0 until d1.size, d1.values, s1.indices, s1.values)
    +          case (_, _) => util.Arrays.equals(this.toArray, v2.toArray)
    +        }
    +      }
           case _ => false
         }
       }
     
    -  override def hashCode(): Int = util.Arrays.hashCode(this.toArray)
    +  override def hashCode(): Int = {
    +    var result: Int = size + 31
    +    this.foreachActive { case (index, value) =>
    +      // ignore explict 0 for comparison between sparse and dense
    +      if (value != 0) {
    +        result = 31 * result + index
    +        // refer to {@link java.util.Arrays.equals} for hash algorithm
    +        val bits = java.lang.Double.doubleToLongBits(value)
    +        result = 31 * result + (bits ^ (bits >>> 32)).toInt
    +      }
    +    }
    +    return result
    +  }
     
       /**
        * Converts the instance to a breeze vector.
    @@ -108,16 +129,16 @@ private[spark] class VectorUDT extends UserDefinedType[Vector] {
       override def serialize(obj: Any): Row = {
         val row = new GenericMutableRow(4)
         obj match {
    -      case sv: SparseVector =>
    +      case SparseVector(size, indices, values) =>
             row.setByte(0, 0)
    -        row.setInt(1, sv.size)
    -        row.update(2, sv.indices.toSeq)
    -        row.update(3, sv.values.toSeq)
    -      case dv: DenseVector =>
    +        row.setInt(1, size)
    +        row.update(2, indices.toSeq)
    +        row.update(3, values.toSeq)
    +      case DenseVector(values) =>
             row.setByte(0, 1)
             row.setNullAt(1)
             row.setNullAt(2)
    -        row.update(3, dv.values.toSeq)
    +        row.update(3, values.toSeq)
         }
         row
       }
    @@ -268,11 +289,11 @@ object Vectors {
        * @param p norm.
        * @return norm in L^p^ space.
        */
    -  private[spark] def norm(vector: Vector, p: Double): Double = {
    +  def norm(vector: Vector, p: Double): Double = {
         require(p >= 1.0)
         val values = vector match {
    -      case dv: DenseVector => dv.values
    -      case sv: SparseVector => sv.values
    +      case DenseVector(vs) => vs
    +      case SparseVector(n, ids, vs) => vs
           case v => throw new IllegalArgumentException("Do not support vector type " + v.getClass)
         }
         val size = values.size
    @@ -312,6 +333,114 @@ object Vectors {
           math.pow(sum, 1.0 / p)
         }
       }
    + 
    +  /**
    +   * Returns the squared distance between two Vectors.
    +   * @param v1 first Vector.
    +   * @param v2 second Vector.
    +   * @return squared distance between two Vectors.
    +   */
    +  def sqdist(v1: Vector, v2: Vector): Double = {
    +    var squaredDistance = 0.0
    +    (v1, v2) match { 
    +      case (v1: SparseVector, v2: SparseVector) =>
    +        val v1Values = v1.values
    +        val v1Indices = v1.indices
    +        val v2Values = v2.values
    +        val v2Indices = v2.indices
    +        val nnzv1 = v1Indices.size
    +        val nnzv2 = v2Indices.size
    +        
    +        var kv1 = 0
    +        var kv2 = 0
    +        while (kv1 < nnzv1 || kv2 < nnzv2) {
    +          var score = 0.0
    + 
    +          if (kv2 >= nnzv2 || (kv1 < nnzv1 && v1Indices(kv1) < v2Indices(kv2))) {
    +            score = v1Values(kv1)
    +            kv1 += 1
    +          } else if (kv1 >= nnzv1 || (kv2 < nnzv2 && v2Indices(kv2) < v1Indices(kv1))) {
    +            score = v2Values(kv2)
    +            kv2 += 1
    +          } else {
    +            score = v1Values(kv1) - v2Values(kv2)
    +            kv1 += 1
    +            kv2 += 1
    +          }
    +          squaredDistance += score * score
    +        }
    +
    +      case (v1: SparseVector, v2: DenseVector) if v1.indices.length / v1.size < 0.5 =>
    +        squaredDistance = sqdist(v1, v2)
    +
    +      case (v1: DenseVector, v2: SparseVector) if v2.indices.length / v2.size < 0.5 =>
    +        squaredDistance = sqdist(v2, v1)
    +
    +      // When a SparseVector is approximately dense, we treat it as a DenseVector
    +      case (v1, v2) =>
    +        squaredDistance = v1.toArray.zip(v2.toArray).foldLeft(0.0){ (distance, elems) =>
    +          val score = elems._1 - elems._2
    +          distance + score * score
    +        }
    +    }
    +    squaredDistance
    +  }
    +
    +  /**
    +   * Returns the squared distance between DenseVector and SparseVector.
    +   */
    +  private[mllib] def sqdist(v1: SparseVector, v2: DenseVector): Double = {
    +    var kv1 = 0
    +    var kv2 = 0
    +    val indices = v1.indices
    +    var squaredDistance = 0.0
    +    val nnzv1 = indices.size
    +    val nnzv2 = v2.size
    +    var iv1 = if (nnzv1 > 0) indices(kv1) else -1
    +   
    +    while (kv2 < nnzv2) {
    +      var score = 0.0
    +      if (kv2 != iv1) {
    +        score = v2(kv2)
    +      } else {
    +        score = v1.values(kv1) - v2(kv2)
    +        if (kv1 < nnzv1 - 1) {
    +          kv1 += 1
    +          iv1 = indices(kv1)
    +        }
    +      }
    +      squaredDistance += score * score
    +      kv2 += 1
    +    }
    +    squaredDistance
    +  }
    +
    +  /**
    +   * Check equality between sparse/dense vectors
    +   */
    +  private[mllib] def equals(
    +      v1Indices: IndexedSeq[Int],
    +      v1Values: Array[Double],
    +      v2Indices: IndexedSeq[Int],
    +      v2Values: Array[Double]): Boolean = {
    +    val v1Size = v1Values.size
    +    val v2Size = v2Values.size
    +    var k1 = 0
    +    var k2 = 0
    +    var allEqual = true
    +    while (allEqual) {
    +      while (k1 < v1Size && v1Values(k1) == 0) k1 += 1
    +      while (k2 < v2Size && v2Values(k2) == 0) k2 += 1
    +
    +      if (k1 >= v1Size || k2 >= v2Size) {
    +        return k1 >= v1Size && k2 >= v2Size // check end alignment
    +      }
    +      allEqual = v1Indices(k1) == v2Indices(k2) && v1Values(k1) == v2Values(k2)
    +      k1 += 1
    +      k2 += 1
    +    }
    +    allEqual
    +  }
     }
     
     /**
    @@ -346,6 +475,10 @@ class DenseVector(val values: Array[Double]) extends Vector {
       }
     }
     
    +object DenseVector {
    +  def unapply(dv: DenseVector): Option[Array[Double]] = Some(dv.values)
    +}
    +
     /**
      * A sparse vector represented by an index array and an value array.
      *
    @@ -393,3 +526,8 @@ class SparseVector(
         }
       }
     }
    +
    +object SparseVector {
    +  def unapply(sv: SparseVector): Option[(Int, Array[Int], Array[Double])] =
    +    Some((sv.size, sv.indices, sv.values))
    +}
    diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala
    index 06d8915f3bfa..b60559c853a5 100644
    --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala
    +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala
    @@ -69,6 +69,11 @@ class CoordinateMatrix(
         nRows
       }
     
    +  /** Transposes this CoordinateMatrix. */
    +  def transpose(): CoordinateMatrix = {
    +    new CoordinateMatrix(entries.map(x => MatrixEntry(x.j, x.i, x.value)), numCols(), numRows())
    +  }
    +
       /** Converts to IndexedRowMatrix. The number of columns must be within the integer range. */
       def toIndexedRowMatrix(): IndexedRowMatrix = {
         val nl = numCols()
    diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala
    index 5c1acca0ec53..c518271f0472 100644
    --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala
    +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala
    @@ -75,6 +75,23 @@ class IndexedRowMatrix(
         new RowMatrix(rows.map(_.vector), 0L, nCols)
       }
     
    +  /**
    +   * Converts this matrix to a
    +   * [[org.apache.spark.mllib.linalg.distributed.CoordinateMatrix]].
    +   */
    +  def toCoordinateMatrix(): CoordinateMatrix = {
    +    val entries = rows.flatMap { row =>
    +      val rowIndex = row.index
    +      row.vector match {
    +        case SparseVector(size, indices, values) =>
    +          Iterator.tabulate(indices.size)(i => MatrixEntry(rowIndex, indices(i), values(i)))
    +        case DenseVector(values) =>
    +          Iterator.tabulate(values.size)(i => MatrixEntry(rowIndex, i, values(i)))
    +      }
    +    }
    +    new CoordinateMatrix(entries, numRows(), numCols())
    +  }
    +
       /**
        * Computes the singular value decomposition of this IndexedRowMatrix.
        * Denote this matrix by A (m x n), this will compute matrices U, S, V such that A = U * S * V'.
    @@ -102,6 +119,9 @@ class IndexedRowMatrix(
           k: Int,
           computeU: Boolean = false,
           rCond: Double = 1e-9): SingularValueDecomposition[IndexedRowMatrix, Matrix] = {
    +
    +    val n = numCols().toInt
    +    require(k > 0 && k <= n, s"Requested k singular values but got k=$k and numCols=$n.")
         val indices = rows.map(_.index)
         val svd = toRowMatrix().computeSVD(k, computeU, rCond)
         val U = if (computeU) {
    @@ -142,7 +162,7 @@ class IndexedRowMatrix(
         val mat = BDM.zeros[Double](m, n)
         rows.collect().foreach { case IndexedRow(rowIndex, vector) =>
           val i = rowIndex.toInt
    -      vector.toBreeze.activeIterator.foreach { case (j, v) =>
    +      vector.foreachActive { case (j, v) =>
             mat(i, j) = v
           }
         }
    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 10a515af8880..02075edbabf8 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
    @@ -131,8 +131,8 @@ class RowMatrix(
           throw new IllegalArgumentException(s"Argument with more than 65535 cols: $cols")
         }
         if (cols > 10000) {
    -      val mem = cols * cols * 8
    -      logWarning(s"$cols columns will require at least $mem bytes of memory!")
    +      val memMB = (cols.toLong * cols) / 125000
    +      logWarning(s"$cols columns will require at least $memMB megabytes of memory!")
         }
       }
     
    @@ -212,7 +212,7 @@ class RowMatrix(
           tol: Double,
           mode: String): SingularValueDecomposition[RowMatrix, Matrix] = {
         val n = numCols().toInt
    -    require(k > 0 && k <= n, s"Request up to n singular values but got k=$k and n=$n.")
    +    require(k > 0 && k <= n, s"Requested k singular values but got k=$k and numCols=$n.")
     
         object SVDMode extends Enumeration {
           val LocalARPACK, LocalLAPACK, DistARPACK = Value
    @@ -528,21 +528,21 @@ class RowMatrix(
           iter.flatMap { row =>
             val buf = new ListBuffer[((Int, Int), Double)]()
             row match {
    -          case sv: SparseVector =>
    -            val nnz = sv.indices.size
    +          case SparseVector(size, indices, values) =>
    +            val nnz = indices.size
                 var k = 0
                 while (k < nnz) {
    -              scaled(k) = sv.values(k) / q(sv.indices(k))
    +              scaled(k) = values(k) / q(indices(k))
                   k += 1
                 }
                 k = 0
                 while (k < nnz) {
    -              val i = sv.indices(k)
    +              val i = 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 j = indices(l)
                       val jVal = scaled(l)
                       if (jVal != 0 && rand.nextDouble() < p(j)) {
                         buf += (((i, j), iVal * jVal))
    @@ -552,11 +552,11 @@ class RowMatrix(
                   }
                   k += 1
                 }
    -          case dv: DenseVector =>
    -            val n = dv.values.size
    +          case DenseVector(values) =>
    +            val n = values.size
                 var i = 0
                 while (i < n) {
    -              scaled(i) = dv.values(i) / q(i)
    +              scaled(i) = values(i) / q(i)
                   i += 1
                 }
                 i = 0
    @@ -588,8 +588,8 @@ class RowMatrix(
         val n = numCols().toInt
         val mat = BDM.zeros[Double](m, n)
         var i = 0
    -    rows.collect().foreach { v =>
    -      v.toBreeze.activeIterator.foreach { case (j, v) =>
    +    rows.collect().foreach { vector =>
    +      vector.foreachActive { case (j, v) =>
             mat(i, j) = v
           }
           i += 1
    @@ -620,11 +620,9 @@ object RowMatrix {
         // TODO: Find a better home (breeze?) for this method.
         val n = v.size
         v match {
    -      case dv: DenseVector =>
    -        blas.dspr("U", n, alpha, dv.values, 1, U)
    -      case sv: SparseVector =>
    -        val indices = sv.indices
    -        val values = sv.values
    +      case DenseVector(values) =>
    +        blas.dspr("U", n, alpha, values, 1, U)
    +      case SparseVector(size, indices, values) =>
             val nnz = indices.length
             var colStartIdx = 0
             var prevCol = 0
    diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala
    index 5a419d164029..1ca0f36c6ac3 100644
    --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala
    +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala
    @@ -20,6 +20,7 @@ package org.apache.spark.mllib.optimization
     import org.apache.spark.annotation.DeveloperApi
     import org.apache.spark.mllib.linalg.{Vector, Vectors}
     import org.apache.spark.mllib.linalg.BLAS.{axpy, dot, scal}
    +import org.apache.spark.mllib.util.MLUtils
     
     /**
      * :: DeveloperApi ::
    @@ -66,9 +67,10 @@ class LogisticGradient extends Gradient {
         scal(gradientMultiplier, gradient)
         val loss =
           if (label > 0) {
    -        math.log1p(math.exp(margin)) // log1p is log(1+p) but more accurate for small p
    +        // The following is equivalent to log(1 + exp(margin)) but more numerically stable.
    +        MLUtils.log1pExp(margin)
           } else {
    -        math.log1p(math.exp(margin)) - margin
    +        MLUtils.log1pExp(margin) - margin
           }
     
         (gradient, loss)
    @@ -83,9 +85,10 @@ class LogisticGradient extends Gradient {
         val gradientMultiplier = (1.0 / (1.0 + math.exp(margin))) - label
         axpy(gradientMultiplier, data, cumGradient)
         if (label > 0) {
    -      math.log1p(math.exp(margin))
    +      // The following is equivalent to log(1 + exp(margin)) but more numerically stable.
    +      MLUtils.log1pExp(margin)
         } else {
    -      math.log1p(math.exp(margin)) - margin
    +      MLUtils.log1pExp(margin) - margin
         }
       }
     }
    diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala
    index 90ac25222600..bee951a2e5e2 100644
    --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala
    +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala
    @@ -116,6 +116,7 @@ class ALS private (
     
       /** storage level for user/product in/out links */
       private var intermediateRDDStorageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK
    +  private var finalRDDStorageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK
     
       /**
        * Set the number of blocks for both user blocks and product blocks to parallelize the computation
    @@ -204,6 +205,19 @@ class ALS private (
         this
       }
     
    +  /**
    +   * :: DeveloperApi ::
    +   * Sets storage level for final RDDs (user/product used in MatrixFactorizationModel). The default
    +   * value is `MEMORY_AND_DISK`. Users can change it to a serialized storage, e.g. 
    +   * `MEMORY_AND_DISK_SER` and set `spark.rdd.compress` to `true` to reduce the space requirement,
    +   * at the cost of speed.
    +   */
    +  @DeveloperApi
    +  def setFinalRDDStorageLevel(storageLevel: StorageLevel): this.type = {
    +    this.finalRDDStorageLevel = storageLevel
    +    this
    +  }
    +
       /**
        * Run ALS with the configured parameters on an input RDD of (user, product, rating) triples.
        * Returns a MatrixFactorizationModel with feature vectors for each user and product.
    @@ -307,8 +321,8 @@ class ALS private (
         val usersOut = unblockFactors(users, userOutLinks)
         val productsOut = unblockFactors(products, productOutLinks)
     
    -    usersOut.setName("usersOut").persist(StorageLevel.MEMORY_AND_DISK)
    -    productsOut.setName("productsOut").persist(StorageLevel.MEMORY_AND_DISK)
    +    usersOut.setName("usersOut").persist(finalRDDStorageLevel)
    +    productsOut.setName("productsOut").persist(finalRDDStorageLevel)
     
         // Materialize usersOut and productsOut.
         usersOut.count()
    diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala
    index f9791c657178..8ecd5c6ad93c 100644
    --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala
    +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala
    @@ -45,7 +45,7 @@ class LassoModel (
     /**
      * Train a regression model with L1-regularization using Stochastic Gradient Descent.
      * This solves the l1-regularized least squares regression formulation
    - *          f(weights) = 1/n ||A weights-y||^2  + regParam ||weights||_1
    + *          f(weights) = 1/2n ||A weights-y||^2  + regParam ||weights||_1
      * Here the data matrix has n rows, and the input RDD holds the set of rows of A, each with
      * its corresponding right hand side label y.
      * See also the documentation for the precise formulation.
    diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala
    index c8cad773f5ef..076ba35051c9 100644
    --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala
    +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala
    @@ -45,7 +45,7 @@ class RidgeRegressionModel (
     /**
      * Train a regression model with L2-regularization using Stochastic Gradient Descent.
      * This solves the l1-regularized least squares regression formulation
    - *          f(weights) = 1/n ||A weights-y||^2  + regParam/2 ||weights||^2
    + *          f(weights) = 1/2n ||A weights-y||^2  + regParam/2 ||weights||^2
      * Here the data matrix has n rows, and the input RDD holds the set of rows of A, each with
      * its corresponding right hand side label y.
      * See also the documentation for the precise formulation.
    diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussian.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussian.scala
    new file mode 100644
    index 000000000000..fd186b5ee6f7
    --- /dev/null
    +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussian.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.mllib.stat.distribution
    +
    +import breeze.linalg.{DenseVector => DBV, DenseMatrix => DBM, diag, max, eigSym}
    +
    +import org.apache.spark.annotation.DeveloperApi;
    +import org.apache.spark.mllib.linalg.{Vectors, Vector, Matrices, Matrix}
    +import org.apache.spark.mllib.util.MLUtils
    +
    +/**
    + * :: DeveloperApi ::
    + * This class provides basic functionality for a Multivariate Gaussian (Normal) Distribution. In
    + * the event that the covariance matrix is singular, the density will be computed in a
    + * reduced dimensional subspace under which the distribution is supported.
    + * (see [[http://en.wikipedia.org/wiki/Multivariate_normal_distribution#Degenerate_case]])
    + * 
    + * @param mu The mean vector of the distribution
    + * @param sigma The covariance matrix of the distribution
    + */
    +@DeveloperApi
    +class MultivariateGaussian (
    +    val mu: Vector, 
    +    val sigma: Matrix) extends Serializable {
    +
    +  require(sigma.numCols == sigma.numRows, "Covariance matrix must be square")
    +  require(mu.size == sigma.numCols, "Mean vector length must match covariance matrix size")
    +  
    +  private val breezeMu = mu.toBreeze.toDenseVector
    +  
    +  /**
    +   * private[mllib] constructor
    +   * 
    +   * @param mu The mean vector of the distribution
    +   * @param sigma The covariance matrix of the distribution
    +   */
    +  private[mllib] def this(mu: DBV[Double], sigma: DBM[Double]) = {
    +    this(Vectors.fromBreeze(mu), Matrices.fromBreeze(sigma))
    +  }
    +  
    +  /**
    +   * Compute distribution dependent constants:
    +   *    rootSigmaInv = D^(-1/2)^ * U, where sigma = U * D * U.t
    +   *    u = log((2*pi)^(-k/2)^ * det(sigma)^(-1/2)^) 
    +   */
    +  private val (rootSigmaInv: DBM[Double], u: Double) = calculateCovarianceConstants
    +  
    +  /** Returns density of this multivariate Gaussian at given point, x */
    +  def pdf(x: Vector): Double = {
    +    pdf(x.toBreeze.toDenseVector)
    +  }
    +  
    +  /** Returns the log-density of this multivariate Gaussian at given point, x */
    +  def logpdf(x: Vector): Double = {
    +    logpdf(x.toBreeze.toDenseVector)
    +  }
    +  
    +  /** Returns density of this multivariate Gaussian at given point, x */
    +  private[mllib] def pdf(x: DBV[Double]): Double = {
    +    math.exp(logpdf(x))
    +  }
    +  
    +  /** Returns the log-density of this multivariate Gaussian at given point, x */
    +  private[mllib] def logpdf(x: DBV[Double]): Double = {
    +    val delta = x - breezeMu
    +    val v = rootSigmaInv * delta
    +    u + v.t * v * -0.5
    +  }
    +  
    +  /**
    +   * Calculate distribution dependent components used for the density function:
    +   *    pdf(x) = (2*pi)^(-k/2)^ * det(sigma)^(-1/2)^ * exp((-1/2) * (x-mu).t * inv(sigma) * (x-mu))
    +   * where k is length of the mean vector.
    +   * 
    +   * We here compute distribution-fixed parts 
    +   *  log((2*pi)^(-k/2)^ * det(sigma)^(-1/2)^)
    +   * and
    +   *  D^(-1/2)^ * U, where sigma = U * D * U.t
    +   *  
    +   * Both the determinant and the inverse can be computed from the singular value decomposition
    +   * of sigma.  Noting that covariance matrices are always symmetric and positive semi-definite,
    +   * we can use the eigendecomposition. We also do not compute the inverse directly; noting
    +   * that 
    +   * 
    +   *    sigma = U * D * U.t
    +   *    inv(Sigma) = U * inv(D) * U.t 
    +   *               = (D^{-1/2}^ * U).t * (D^{-1/2}^ * U)
    +   * 
    +   * and thus
    +   * 
    +   *    -0.5 * (x-mu).t * inv(Sigma) * (x-mu) = -0.5 * norm(D^{-1/2}^ * U  * (x-mu))^2^
    +   *  
    +   * To guard against singular covariance matrices, this method computes both the 
    +   * pseudo-determinant and the pseudo-inverse (Moore-Penrose).  Singular values are considered
    +   * to be non-zero only if they exceed a tolerance based on machine precision, matrix size, and
    +   * relation to the maximum singular value (same tolerance used by, e.g., Octave).
    +   */
    +  private def calculateCovarianceConstants: (DBM[Double], Double) = {
    +    val eigSym.EigSym(d, u) = eigSym(sigma.toBreeze.toDenseMatrix) // sigma = u * diag(d) * u.t
    +    
    +    // For numerical stability, values are considered to be non-zero only if they exceed tol.
    +    // This prevents any inverted value from exceeding (eps * n * max(d))^-1
    +    val tol = MLUtils.EPSILON * max(d) * d.length
    +    
    +    try {
    +      // log(pseudo-determinant) is sum of the logs of all non-zero singular values
    +      val logPseudoDetSigma = d.activeValuesIterator.filter(_ > tol).map(math.log).sum
    +      
    +      // calculate the root-pseudo-inverse of the diagonal matrix of singular values 
    +      // by inverting the square root of all non-zero values
    +      val pinvS = diag(new DBV(d.map(v => if (v > tol) math.sqrt(1.0 / v) else 0.0).toArray))
    +    
    +      (pinvS * u, -0.5 * (mu.size * math.log(2.0 * math.Pi) + logPseudoDetSigma))
    +    } catch {
    +      case uex: UnsupportedOperationException =>
    +        throw new IllegalArgumentException("Covariance matrix has no non-zero singular values")
    +    }
    +  }
    +}
    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 73e7e32c6db3..b3e8ed9af8c5 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
    @@ -64,13 +64,6 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo
         val rfModel = rf.run(input)
         rfModel.trees(0)
       }
    -
    -  /**
    -   * Trains a decision tree model over an RDD. This is deprecated because it hides the static
    -   * methods with the same name in Java.
    -   */
    -  @deprecated("Please use DecisionTree.run instead.", "1.2.0")
    -  def train(input: RDD[LabeledPoint]): DecisionTreeModel = run(input)
     }
     
     object DecisionTree extends Serializable with Logging {
    diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/BoostingStrategy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/BoostingStrategy.scala
    index cf51d041c65a..ed8e6a796f8c 100644
    --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/BoostingStrategy.scala
    +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/BoostingStrategy.scala
    @@ -68,6 +68,15 @@ case class BoostingStrategy(
     @Experimental
     object BoostingStrategy {
     
    +  /**
    +   * Returns default configuration for the boosting algorithm
    +   * @param algo Learning goal.  Supported: "Classification" or "Regression"
    +   * @return Configuration for boosting algorithm
    +   */
    +  def defaultParams(algo: String): BoostingStrategy = {
    +    defaultParams(Algo.fromString(algo))
    +  }
    +
       /**
        * Returns default configuration for the boosting algorithm
        * @param algo Learning goal.  Supported:
    @@ -75,15 +84,15 @@ object BoostingStrategy {
        *             [[org.apache.spark.mllib.tree.configuration.Algo.Regression]]
        * @return Configuration for boosting algorithm
        */
    -  def defaultParams(algo: String): BoostingStrategy = {
    -    val treeStrategy = Strategy.defaultStrategy(algo)
    -    treeStrategy.maxDepth = 3
    +  def defaultParams(algo: Algo): BoostingStrategy = {
    +    val treeStragtegy = Strategy.defaultStategy(algo)
    +    treeStragtegy.maxDepth = 3
         algo match {
    -      case "Classification" =>
    -        treeStrategy.numClasses = 2
    -        new BoostingStrategy(treeStrategy, LogLoss)
    -      case "Regression" =>
    -        new BoostingStrategy(treeStrategy, SquaredError)
    +      case Algo.Classification =>
    +        treeStragtegy.numClasses = 2
    +        new BoostingStrategy(treeStragtegy, LogLoss)
    +      case Algo.Regression =>
    +        new BoostingStrategy(treeStragtegy, SquaredError)
           case _ =>
             throw new IllegalArgumentException(s"$algo is not supported by boosting.")
         }
    diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala
    index d5cd89ab94e8..972959885f39 100644
    --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala
    +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala
    @@ -173,11 +173,19 @@ object Strategy {
        * Construct a default set of parameters for [[org.apache.spark.mllib.tree.DecisionTree]]
        * @param algo  "Classification" or "Regression"
        */
    -  def defaultStrategy(algo: String): Strategy = algo match {
    -    case "Classification" =>
    +  def defaultStrategy(algo: String): Strategy = {
    +    defaultStategy(Algo.fromString(algo))
    +  }
    +
    +  /**
    +   * Construct a default set of parameters for [[org.apache.spark.mllib.tree.DecisionTree]]
    +   * @param algo Algo.Classification or Algo.Regression
    +   */
    +  def defaultStategy(algo: Algo): Strategy = algo match {
    +    case Algo.Classification =>
           new Strategy(algo = Classification, impurity = Gini, maxDepth = 10,
             numClasses = 2)
    -    case "Regression" =>
    +    case Algo.Regression =>
           new Strategy(algo = Regression, impurity = Variance, maxDepth = 10,
             numClasses = 0)
       }
    diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/LogLoss.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/LogLoss.scala
    index 7ce9fa6f86c4..55213e695638 100644
    --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/LogLoss.scala
    +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/LogLoss.scala
    @@ -20,6 +20,7 @@ package org.apache.spark.mllib.tree.loss
     import org.apache.spark.annotation.DeveloperApi
     import org.apache.spark.mllib.regression.LabeledPoint
     import org.apache.spark.mllib.tree.model.TreeEnsembleModel
    +import org.apache.spark.mllib.util.MLUtils
     import org.apache.spark.rdd.RDD
     
     /**
    @@ -61,13 +62,8 @@ object LogLoss extends Loss {
         data.map { case point =>
           val prediction = model.predict(point.features)
           val margin = 2.0 * point.label * prediction
    -      // The following are equivalent to 2.0 * log(1 + exp(-margin)) but are more numerically
    -      // stable.
    -      if (margin >= 0) {
    -        2.0 * math.log1p(math.exp(-margin))
    -      } else {
    -        2.0 * (-margin + math.log1p(math.exp(margin)))
    -      }
    +      // The following is equivalent to 2.0 * log(1 + exp(-margin)) but more numerically stable.
    +      2.0 * MLUtils.log1pExp(-margin)
         }.mean()
       }
     }
    diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala
    index b0d05ae33e1b..5d6ddd47f67d 100644
    --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala
    +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala
    @@ -19,8 +19,7 @@ package org.apache.spark.mllib.util
     
     import scala.reflect.ClassTag
     
    -import breeze.linalg.{DenseVector => BDV, SparseVector => BSV,
    -  squaredDistance => breezeSquaredDistance}
    +import breeze.linalg.{DenseVector => BDV, SparseVector => BSV}
     
     import org.apache.spark.annotation.Experimental
     import org.apache.spark.SparkContext
    @@ -28,7 +27,7 @@ import org.apache.spark.rdd.RDD
     import org.apache.spark.rdd.PartitionwiseSampledRDD
     import org.apache.spark.util.random.BernoulliCellSampler
     import org.apache.spark.mllib.regression.LabeledPoint
    -import org.apache.spark.mllib.linalg.{SparseVector, Vector, Vectors}
    +import org.apache.spark.mllib.linalg.{SparseVector, DenseVector, Vector, Vectors}
     import org.apache.spark.mllib.linalg.BLAS.dot
     import org.apache.spark.storage.StorageLevel
     import org.apache.spark.streaming.StreamingContext
    @@ -39,7 +38,7 @@ import org.apache.spark.streaming.dstream.DStream
      */
     object MLUtils {
     
    -  private[util] lazy val EPSILON = {
    +  private[mllib] lazy val EPSILON = {
         var eps = 1.0
         while ((1.0 + (eps / 2.0)) != 1.0) {
           eps /= 2.0
    @@ -154,10 +153,12 @@ object MLUtils {
       def saveAsLibSVMFile(data: RDD[LabeledPoint], dir: String) {
         // TODO: allow to specify label precision and feature precision.
         val dataStr = data.map { case LabeledPoint(label, features) =>
    -      val featureStrings = features.toBreeze.activeIterator.map { case (i, v) =>
    -        s"${i + 1}:$v"
    +      val sb = new StringBuilder(label.toString)
    +      features.foreachActive { case (i, v) =>
    +        sb += ' '
    +        sb ++= s"${i + 1}:$v"
           }
    -      (Iterator(label) ++ featureStrings).mkString(" ")
    +      sb.mkString
         }
         dataStr.saveAsTextFile(dir)
       }
    @@ -264,7 +265,7 @@ object MLUtils {
         }
         Vectors.fromBreeze(vector1)
       }
    -
    + 
       /**
        * Returns the squared Euclidean distance between two vectors. The following formula will be used
        * if it does not introduce too much numerical error:
    @@ -314,13 +315,27 @@ object MLUtils {
           val precisionBound2 = EPSILON * (sumSquaredNorm + 2.0 * math.abs(dotValue)) /
             (sqDist + EPSILON)
           if (precisionBound2 > precision) {
    -        // TODO: breezeSquaredDistance is slow,
    -        // so we should replace it with our own implementation.
    -        sqDist = breezeSquaredDistance(v1.toBreeze, v2.toBreeze)
    +        sqDist = Vectors.sqdist(v1, v2)
           }
         } else {
    -      sqDist = breezeSquaredDistance(v1.toBreeze, v2.toBreeze)
    +      sqDist = Vectors.sqdist(v1, v2)
         }
         sqDist
       }
    +
    +  /**
    +   * When `x` is positive and large, computing `math.log(1 + math.exp(x))` will lead to arithmetic
    +   * overflow. This will happen when `x > 709.78` which is not a very large number.
    +   * It can be addressed by rewriting the formula into `x + math.log1p(math.exp(-x))` when `x > 0`.
    +   *
    +   * @param x a floating-point value as input.
    +   * @return the result of `math.log(1 + math.exp(x))`.
    +   */
    +  private[mllib] def log1pExp(x: Double): Double = {
    +    if (x > 0) {
    +      x + math.log1p(math.exp(-x))
    +    } else {
    +      math.log1p(math.exp(x))
    +    }
    +  }
     }
    diff --git a/mllib/src/test/java/org/apache/spark/ml/JavaPipelineSuite.java b/mllib/src/test/java/org/apache/spark/ml/JavaPipelineSuite.java
    index 42846677ed28..47f1f46c6c26 100644
    --- a/mllib/src/test/java/org/apache/spark/ml/JavaPipelineSuite.java
    +++ b/mllib/src/test/java/org/apache/spark/ml/JavaPipelineSuite.java
    @@ -26,10 +26,9 @@
     import org.apache.spark.mllib.regression.LabeledPoint;
     import org.apache.spark.ml.classification.LogisticRegression;
     import org.apache.spark.ml.feature.StandardScaler;
    -import org.apache.spark.sql.api.java.JavaSQLContext;
    -import org.apache.spark.sql.api.java.JavaSchemaRDD;
    -import static org.apache.spark.mllib.classification.LogisticRegressionSuite
    -  .generateLogisticInputAsList;
    +import org.apache.spark.sql.SchemaRDD;
    +import org.apache.spark.sql.SQLContext;
    +import static org.apache.spark.mllib.classification.LogisticRegressionSuite.generateLogisticInputAsList;
     
     /**
      * Test Pipeline construction and fitting in Java.
    @@ -37,13 +36,13 @@
     public class JavaPipelineSuite {
     
       private transient JavaSparkContext jsc;
    -  private transient JavaSQLContext jsql;
    -  private transient JavaSchemaRDD dataset;
    +  private transient SQLContext jsql;
    +  private transient SchemaRDD dataset;
     
       @Before
       public void setUp() {
         jsc = new JavaSparkContext("local", "JavaPipelineSuite");
    -    jsql = new JavaSQLContext(jsc);
    +    jsql = new SQLContext(jsc);
         JavaRDD points =
           jsc.parallelize(generateLogisticInputAsList(1.0, 1.0, 100, 42), 2);
         dataset = jsql.applySchema(points, LabeledPoint.class);
    @@ -66,7 +65,7 @@ public void pipeline() {
           .setStages(new PipelineStage[] {scaler, lr});
         PipelineModel model = pipeline.fit(dataset);
         model.transform(dataset).registerTempTable("prediction");
    -    JavaSchemaRDD predictions = jsql.sql("SELECT label, score, prediction FROM prediction");
    -    predictions.collect();
    +    SchemaRDD predictions = jsql.sql("SELECT label, score, prediction FROM prediction");
    +    predictions.collectAsList();
       }
     }
    diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java b/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java
    index 76eb7f00329f..2eba83335bb5 100644
    --- a/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java
    +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java
    @@ -26,21 +26,20 @@
     
     import org.apache.spark.api.java.JavaSparkContext;
     import org.apache.spark.mllib.regression.LabeledPoint;
    -import org.apache.spark.sql.api.java.JavaSQLContext;
    -import org.apache.spark.sql.api.java.JavaSchemaRDD;
    -import static org.apache.spark.mllib.classification.LogisticRegressionSuite
    -  .generateLogisticInputAsList;
    +import org.apache.spark.sql.SchemaRDD;
    +import org.apache.spark.sql.SQLContext;
    +import static org.apache.spark.mllib.classification.LogisticRegressionSuite.generateLogisticInputAsList;
     
     public class JavaLogisticRegressionSuite implements Serializable {
     
       private transient JavaSparkContext jsc;
    -  private transient JavaSQLContext jsql;
    -  private transient JavaSchemaRDD dataset;
    +  private transient SQLContext jsql;
    +  private transient SchemaRDD dataset;
     
       @Before
       public void setUp() {
         jsc = new JavaSparkContext("local", "JavaLogisticRegressionSuite");
    -    jsql = new JavaSQLContext(jsc);
    +    jsql = new SQLContext(jsc);
         List points = generateLogisticInputAsList(1.0, 1.0, 100, 42);
         dataset = jsql.applySchema(jsc.parallelize(points, 2), LabeledPoint.class);
       }
    @@ -56,8 +55,8 @@ public void logisticRegression() {
         LogisticRegression lr = new LogisticRegression();
         LogisticRegressionModel model = lr.fit(dataset);
         model.transform(dataset).registerTempTable("prediction");
    -    JavaSchemaRDD predictions = jsql.sql("SELECT label, score, prediction FROM prediction");
    -    predictions.collect();
    +    SchemaRDD predictions = jsql.sql("SELECT label, score, prediction FROM prediction");
    +    predictions.collectAsList();
       }
     
       @Test
    @@ -68,8 +67,8 @@ public void logisticRegressionWithSetters() {
         LogisticRegressionModel model = lr.fit(dataset);
         model.transform(dataset, model.threshold().w(0.8)) // overwrite threshold
           .registerTempTable("prediction");
    -    JavaSchemaRDD predictions = jsql.sql("SELECT label, score, prediction FROM prediction");
    -    predictions.collect();
    +    SchemaRDD predictions = jsql.sql("SELECT label, score, prediction FROM prediction");
    +    predictions.collectAsList();
       }
     
       @Test
    diff --git a/mllib/src/test/java/org/apache/spark/ml/tuning/JavaCrossValidatorSuite.java b/mllib/src/test/java/org/apache/spark/ml/tuning/JavaCrossValidatorSuite.java
    index a266ebd2071a..a9f1c4a2c3ca 100644
    --- a/mllib/src/test/java/org/apache/spark/ml/tuning/JavaCrossValidatorSuite.java
    +++ b/mllib/src/test/java/org/apache/spark/ml/tuning/JavaCrossValidatorSuite.java
    @@ -30,21 +30,20 @@
     import org.apache.spark.ml.evaluation.BinaryClassificationEvaluator;
     import org.apache.spark.ml.param.ParamMap;
     import org.apache.spark.mllib.regression.LabeledPoint;
    -import org.apache.spark.sql.api.java.JavaSQLContext;
    -import org.apache.spark.sql.api.java.JavaSchemaRDD;
    -import static org.apache.spark.mllib.classification.LogisticRegressionSuite
    -  .generateLogisticInputAsList;
    +import org.apache.spark.sql.SchemaRDD;
    +import org.apache.spark.sql.SQLContext;
    +import static org.apache.spark.mllib.classification.LogisticRegressionSuite.generateLogisticInputAsList;
     
     public class JavaCrossValidatorSuite implements Serializable {
     
       private transient JavaSparkContext jsc;
    -  private transient JavaSQLContext jsql;
    -  private transient JavaSchemaRDD dataset;
    +  private transient SQLContext jsql;
    +  private transient SchemaRDD dataset;
     
       @Before
       public void setUp() {
         jsc = new JavaSparkContext("local", "JavaCrossValidatorSuite");
    -    jsql = new JavaSQLContext(jsc);
    +    jsql = new SQLContext(jsc);
         List points = generateLogisticInputAsList(1.0, 1.0, 100, 42);
         dataset = jsql.applySchema(jsc.parallelize(points, 2), LabeledPoint.class);
       }
    diff --git a/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaMatricesSuite.java b/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaMatricesSuite.java
    new file mode 100644
    index 000000000000..704d484d0b58
    --- /dev/null
    +++ b/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaMatricesSuite.java
    @@ -0,0 +1,163 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.mllib.linalg;
    +
    +import static org.junit.Assert.*;
    +import org.junit.Test;
    +
    +import java.io.Serializable;
    +import java.util.Random;
    +
    +public class JavaMatricesSuite implements Serializable {
    +
    +    @Test
    +    public void randMatrixConstruction() {
    +        Random rng = new Random(24);
    +        Matrix r = Matrices.rand(3, 4, rng);
    +        rng.setSeed(24);
    +        DenseMatrix dr = DenseMatrix.rand(3, 4, rng);
    +        assertArrayEquals(r.toArray(), dr.toArray(), 0.0);
    +
    +        rng.setSeed(24);
    +        Matrix rn = Matrices.randn(3, 4, rng);
    +        rng.setSeed(24);
    +        DenseMatrix drn = DenseMatrix.randn(3, 4, rng);
    +        assertArrayEquals(rn.toArray(), drn.toArray(), 0.0);
    +
    +        rng.setSeed(24);
    +        Matrix s = Matrices.sprand(3, 4, 0.5, rng);
    +        rng.setSeed(24);
    +        SparseMatrix sr = SparseMatrix.sprand(3, 4, 0.5, rng);
    +        assertArrayEquals(s.toArray(), sr.toArray(), 0.0);
    +
    +        rng.setSeed(24);
    +        Matrix sn = Matrices.sprandn(3, 4, 0.5, rng);
    +        rng.setSeed(24);
    +        SparseMatrix srn = SparseMatrix.sprandn(3, 4, 0.5, rng);
    +        assertArrayEquals(sn.toArray(), srn.toArray(), 0.0);
    +    }
    +
    +    @Test
    +    public void identityMatrixConstruction() {
    +        Matrix r = Matrices.eye(2);
    +        DenseMatrix dr = DenseMatrix.eye(2);
    +        SparseMatrix sr = SparseMatrix.speye(2);
    +        assertArrayEquals(r.toArray(), dr.toArray(), 0.0);
    +        assertArrayEquals(sr.toArray(), dr.toArray(), 0.0);
    +        assertArrayEquals(r.toArray(), new double[]{1.0, 0.0, 0.0, 1.0}, 0.0);
    +    }
    +
    +    @Test
    +    public void diagonalMatrixConstruction() {
    +        Vector v = Vectors.dense(1.0, 0.0, 2.0);
    +        Vector sv = Vectors.sparse(3, new int[]{0, 2}, new double[]{1.0, 2.0});
    +
    +        Matrix m = Matrices.diag(v);
    +        Matrix sm = Matrices.diag(sv);
    +        DenseMatrix d = DenseMatrix.diag(v);
    +        DenseMatrix sd = DenseMatrix.diag(sv);
    +        SparseMatrix s = SparseMatrix.diag(v);
    +        SparseMatrix ss = SparseMatrix.diag(sv);
    +
    +        assertArrayEquals(m.toArray(), sm.toArray(), 0.0);
    +        assertArrayEquals(d.toArray(), sm.toArray(), 0.0);
    +        assertArrayEquals(d.toArray(), sd.toArray(), 0.0);
    +        assertArrayEquals(sd.toArray(), s.toArray(), 0.0);
    +        assertArrayEquals(s.toArray(), ss.toArray(), 0.0);
    +        assertArrayEquals(s.values(), ss.values(), 0.0);
    +        assert(s.values().length == 2);
    +        assert(ss.values().length == 2);
    +        assert(s.colPtrs().length == 4);
    +        assert(ss.colPtrs().length == 4);
    +    }
    +
    +    @Test
    +    public void zerosMatrixConstruction() {
    +        Matrix z = Matrices.zeros(2, 2);
    +        Matrix one = Matrices.ones(2, 2);
    +        DenseMatrix dz = DenseMatrix.zeros(2, 2);
    +        DenseMatrix done = DenseMatrix.ones(2, 2);
    +
    +        assertArrayEquals(z.toArray(), new double[]{0.0, 0.0, 0.0, 0.0}, 0.0);
    +        assertArrayEquals(dz.toArray(), new double[]{0.0, 0.0, 0.0, 0.0}, 0.0);
    +        assertArrayEquals(one.toArray(), new double[]{1.0, 1.0, 1.0, 1.0}, 0.0);
    +        assertArrayEquals(done.toArray(), new double[]{1.0, 1.0, 1.0, 1.0}, 0.0);
    +    }
    +
    +    @Test
    +    public void sparseDenseConversion() {
    +        int m = 3;
    +        int n = 2;
    +        double[] values = new double[]{1.0, 2.0, 4.0, 5.0};
    +        double[] allValues = new double[]{1.0, 2.0, 0.0, 0.0, 4.0, 5.0};
    +        int[] colPtrs = new int[]{0, 2, 4};
    +        int[] rowIndices = new int[]{0, 1, 1, 2};
    +
    +        SparseMatrix spMat1 = new SparseMatrix(m, n, colPtrs, rowIndices, values);
    +        DenseMatrix deMat1 = new DenseMatrix(m, n, allValues);
    +
    +        SparseMatrix spMat2 = deMat1.toSparse();
    +        DenseMatrix deMat2 = spMat1.toDense();
    +
    +        assertArrayEquals(spMat1.toArray(), spMat2.toArray(), 0.0);
    +        assertArrayEquals(deMat1.toArray(), deMat2.toArray(), 0.0);
    +    }
    +
    +    @Test
    +    public void concatenateMatrices() {
    +        int m = 3;
    +        int n = 2;
    +
    +        Random rng = new Random(42);
    +        SparseMatrix spMat1 = SparseMatrix.sprand(m, n, 0.5, rng);
    +        rng.setSeed(42);
    +        DenseMatrix deMat1 = DenseMatrix.rand(m, n, rng);
    +        Matrix deMat2 = Matrices.eye(3);
    +        Matrix spMat2 = Matrices.speye(3);
    +        Matrix deMat3 = Matrices.eye(2);
    +        Matrix spMat3 = Matrices.speye(2);
    +
    +        Matrix spHorz = Matrices.horzcat(new Matrix[]{spMat1, spMat2});
    +        Matrix deHorz1 = Matrices.horzcat(new Matrix[]{deMat1, deMat2});
    +        Matrix deHorz2 = Matrices.horzcat(new Matrix[]{spMat1, deMat2});
    +        Matrix deHorz3 = Matrices.horzcat(new Matrix[]{deMat1, spMat2});
    +
    +        assert(deHorz1.numRows() == 3);
    +        assert(deHorz2.numRows() == 3);
    +        assert(deHorz3.numRows() == 3);
    +        assert(spHorz.numRows() == 3);
    +        assert(deHorz1.numCols() == 5);
    +        assert(deHorz2.numCols() == 5);
    +        assert(deHorz3.numCols() == 5);
    +        assert(spHorz.numCols() == 5);
    +
    +        Matrix spVert = Matrices.vertcat(new Matrix[]{spMat1, spMat3});
    +        Matrix deVert1 = Matrices.vertcat(new Matrix[]{deMat1, deMat3});
    +        Matrix deVert2 = Matrices.vertcat(new Matrix[]{spMat1, deMat3});
    +        Matrix deVert3 = Matrices.vertcat(new Matrix[]{deMat1, spMat3});
    +
    +        assert(deVert1.numRows() == 5);
    +        assert(deVert2.numRows() == 5);
    +        assert(deVert3.numRows() == 5);
    +        assert(spVert.numRows() == 5);
    +        assert(deVert1.numCols() == 2);
    +        assert(deVert2.numCols() == 2);
    +        assert(deVert3.numCols() == 2);
    +        assert(spVert.numCols() == 2);
    +    }
    +}
    diff --git a/mllib/src/test/resources/log4j.properties b/mllib/src/test/resources/log4j.properties
    index a469badf603c..9697237bfa1a 100644
    --- a/mllib/src/test/resources/log4j.properties
    +++ b/mllib/src/test/resources/log4j.properties
    @@ -15,10 +15,10 @@
     # limitations under the License.
     #
     
    -# Set everything to be logged to the file core/target/unit-tests.log
    +# Set everything to be logged to the file target/unit-tests.log
     log4j.rootCategory=INFO, file
     log4j.appender.file=org.apache.log4j.FileAppender
    -log4j.appender.file.append=false
    +log4j.appender.file.append=true
     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
    diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/GMMExpectationMaximizationSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/GMMExpectationMaximizationSuite.scala
    new file mode 100644
    index 000000000000..198997b5bb2b
    --- /dev/null
    +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/GMMExpectationMaximizationSuite.scala
    @@ -0,0 +1,83 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.mllib.clustering
    +
    +import org.scalatest.FunSuite
    +
    +import org.apache.spark.mllib.linalg.{Vectors, Matrices}
    +import org.apache.spark.mllib.stat.distribution.MultivariateGaussian
    +import org.apache.spark.mllib.util.MLlibTestSparkContext
    +import org.apache.spark.mllib.util.TestingUtils._
    +
    +class GMMExpectationMaximizationSuite extends FunSuite with MLlibTestSparkContext {
    +  test("single cluster") {
    +    val data = sc.parallelize(Array(
    +      Vectors.dense(6.0, 9.0),
    +      Vectors.dense(5.0, 10.0),
    +      Vectors.dense(4.0, 11.0)
    +    ))
    +    
    +    // expectations
    +    val Ew = 1.0
    +    val Emu = Vectors.dense(5.0, 10.0)
    +    val Esigma = Matrices.dense(2, 2, Array(2.0 / 3.0, -2.0 / 3.0, -2.0 / 3.0, 2.0 / 3.0))
    +
    +    val seeds = Array(314589, 29032897, 50181, 494821, 4660)
    +    seeds.foreach { seed =>
    +      val gmm = new GaussianMixtureEM().setK(1).setSeed(seed).run(data)
    +      assert(gmm.weights(0) ~== Ew absTol 1E-5)
    +      assert(gmm.gaussians(0).mu ~== Emu absTol 1E-5)
    +      assert(gmm.gaussians(0).sigma ~== Esigma absTol 1E-5)
    +    }
    +  }
    +  
    +  test("two clusters") {
    +    val data = sc.parallelize(Array(
    +      Vectors.dense(-5.1971), Vectors.dense(-2.5359), Vectors.dense(-3.8220),
    +      Vectors.dense(-5.2211), Vectors.dense(-5.0602), Vectors.dense( 4.7118),
    +      Vectors.dense( 6.8989), Vectors.dense( 3.4592), Vectors.dense( 4.6322),
    +      Vectors.dense( 5.7048), Vectors.dense( 4.6567), Vectors.dense( 5.5026),
    +      Vectors.dense( 4.5605), Vectors.dense( 5.2043), Vectors.dense( 6.2734)
    +    ))
    +  
    +    // we set an initial gaussian to induce expected results
    +    val initialGmm = new GaussianMixtureModel(
    +      Array(0.5, 0.5),
    +      Array(
    +        new MultivariateGaussian(Vectors.dense(-1.0), Matrices.dense(1, 1, Array(1.0))),
    +        new MultivariateGaussian(Vectors.dense(1.0), Matrices.dense(1, 1, Array(1.0)))
    +      )
    +    )
    +    
    +    val Ew = Array(1.0 / 3.0, 2.0 / 3.0)
    +    val Emu = Array(Vectors.dense(-4.3673), Vectors.dense(5.1604))
    +    val Esigma = Array(Matrices.dense(1, 1, Array(1.1098)), Matrices.dense(1, 1, Array(0.86644)))
    +    
    +    val gmm = new GaussianMixtureEM()
    +      .setK(2)
    +      .setInitialModel(initialGmm)
    +      .run(data)
    +      
    +    assert(gmm.weights(0) ~== Ew(0) absTol 1E-3)
    +    assert(gmm.weights(1) ~== Ew(1) absTol 1E-3)
    +    assert(gmm.gaussians(0).mu ~== Emu(0) absTol 1E-3)
    +    assert(gmm.gaussians(1).mu ~== Emu(1) absTol 1E-3)
    +    assert(gmm.gaussians(0).sigma ~== Esigma(0) absTol 1E-3)
    +    assert(gmm.gaussians(1).sigma ~== Esigma(1) absTol 1E-3)
    +  }
    +}
    diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala
    index 9ebef8466c83..caee5917000a 100644
    --- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala
    +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala
    @@ -90,6 +90,27 @@ class KMeansSuite extends FunSuite with MLlibTestSparkContext {
         assert(model.clusterCenters.size === 3)
       }
     
    +  test("deterministic initialization") {
    +    // Create a large-ish set of points for clustering
    +    val points = List.tabulate(1000)(n => Vectors.dense(n, n))
    +    val rdd = sc.parallelize(points, 3)
    +
    +    for (initMode <- Seq(RANDOM, K_MEANS_PARALLEL)) {
    +      // Create three deterministic models and compare cluster means
    +      val model1 = KMeans.train(rdd, k = 10, maxIterations = 2, runs = 1,
    +        initializationMode = initMode, seed = 42)
    +      val centers1 = model1.clusterCenters
    +
    +      val model2 = KMeans.train(rdd, k = 10, maxIterations = 2, runs = 1,
    +        initializationMode = initMode, seed = 42)
    +      val centers2 = model2.clusterCenters
    +
    +      centers1.zip(centers2).foreach { case (c1, c2) =>
    +        assert(c1 ~== c2 absTol 1E-14)
    +      }
    +    }
    +  }
    +
       test("single cluster with big dataset") {
         val smallData = Array(
           Vectors.dense(1.0, 2.0, 6.0),
    diff --git a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetricsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetricsSuite.scala
    index 8a18e2971cab..e0224f960cc4 100644
    --- a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetricsSuite.scala
    +++ b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetricsSuite.scala
    @@ -124,4 +124,40 @@ class BinaryClassificationMetricsSuite extends FunSuite with MLlibTestSparkConte
     
         validateMetrics(metrics, thresholds, rocCurve, prCurve, f1, f2, precisions, recalls)
       }
    +
    +  test("binary evaluation metrics with downsampling") {
    +    val scoreAndLabels = Seq(
    +      (0.1, 0.0), (0.2, 0.0), (0.3, 1.0), (0.4, 0.0), (0.5, 0.0),
    +      (0.6, 1.0), (0.7, 1.0), (0.8, 0.0), (0.9, 1.0))
    +
    +    val scoreAndLabelsRDD = sc.parallelize(scoreAndLabels, 1)
    +
    +    val original = new BinaryClassificationMetrics(scoreAndLabelsRDD)
    +    val originalROC = original.roc().collect().sorted.toList
    +    // Add 2 for (0,0) and (1,1) appended at either end
    +    assert(2 + scoreAndLabels.size == originalROC.size)
    +    assert(
    +      List(
    +        (0.0, 0.0), (0.0, 0.25), (0.2, 0.25), (0.2, 0.5), (0.2, 0.75),
    +        (0.4, 0.75), (0.6, 0.75), (0.6, 1.0), (0.8, 1.0), (1.0, 1.0),
    +        (1.0, 1.0)
    +      ) ==
    +      originalROC)
    +
    +    val numBins = 4
    +
    +    val downsampled = new BinaryClassificationMetrics(scoreAndLabelsRDD, numBins)
    +    val downsampledROC = downsampled.roc().collect().sorted.toList
    +    assert(
    +      // May have to add 1 if the sample factor didn't divide evenly
    +      2 + (numBins + (if (scoreAndLabels.size % numBins == 0) 0 else 1)) ==
    +      downsampledROC.size)
    +    assert(
    +      List(
    +        (0.0, 0.0), (0.2, 0.25), (0.2, 0.75), (0.6, 0.75), (0.8, 1.0),
    +        (1.0, 1.0), (1.0, 1.0)
    +      ) ==
    +      downsampledROC)
    +  }
    +
     }
    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 5d70c914f14b..771878e925ea 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
    @@ -127,6 +127,47 @@ class BLASSuite extends FunSuite {
         }
       }
     
    +  test("syr") {
    +    val dA = new DenseMatrix(4, 4,
    +      Array(0.0, 1.2, 2.2, 3.1, 1.2, 3.2, 5.3, 4.6, 2.2, 5.3, 1.8, 3.0, 3.1, 4.6, 3.0, 0.8))
    +    val x = new DenseVector(Array(0.0, 2.7, 3.5, 2.1))
    +    val alpha = 0.15
    +
    +    val expected = new DenseMatrix(4, 4,
    +      Array(0.0, 1.2, 2.2, 3.1, 1.2, 4.2935, 6.7175, 5.4505, 2.2, 6.7175, 3.6375, 4.1025, 3.1,
    +        5.4505, 4.1025, 1.4615))
    +
    +    syr(alpha, x, dA)
    +
    +    assert(dA ~== expected absTol 1e-15)
    + 
    +    val dB =
    +      new DenseMatrix(3, 4, Array(0.0, 1.2, 2.2, 3.1, 1.2, 3.2, 5.3, 4.6, 2.2, 5.3, 1.8, 3.0))
    +
    +    withClue("Matrix A must be a symmetric Matrix") {
    +      intercept[Exception] {
    +        syr(alpha, x, dB)
    +      }
    +    }
    + 
    +    val dC =
    +      new DenseMatrix(3, 3, Array(0.0, 1.2, 2.2, 1.2, 3.2, 5.3, 2.2, 5.3, 1.8))
    +
    +    withClue("Size of vector must match the rank of matrix") {
    +      intercept[Exception] {
    +        syr(alpha, x, dC)
    +      }
    +    }
    + 
    +    val y = new DenseVector(Array(0.0, 2.7, 3.5, 2.1, 1.5))
    +
    +    withClue("Size of vector must match the rank of matrix") {
    +      intercept[Exception] {
    +        syr(alpha, y, dA)
    +      }
    +    }
    +  }
    +
       test("gemm") {
     
         val dA =
    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 322a0e924291..a35d0fe389fd 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
    @@ -43,9 +43,9 @@ class MatricesSuite extends FunSuite {
     
       test("sparse matrix construction") {
         val m = 3
    -    val n = 2
    +    val n = 4
         val values = Array(1.0, 2.0, 4.0, 5.0)
    -    val colPtrs = Array(0, 2, 4)
    +    val colPtrs = Array(0, 2, 2, 4, 4)
         val rowIndices = Array(1, 2, 1, 2)
         val mat = Matrices.sparse(m, n, colPtrs, rowIndices, values).asInstanceOf[SparseMatrix]
         assert(mat.numRows === m)
    @@ -53,6 +53,13 @@ class MatricesSuite extends FunSuite {
         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")
    +
    +    val entries: Array[(Int, Int, Double)] = Array((2, 2, 3.0), (1, 0, 1.0), (2, 0, 2.0),
    +        (1, 2, 2.0), (2, 2, 2.0), (1, 2, 2.0), (0, 0, 0.0))
    +
    +    val mat2 = SparseMatrix.fromCOO(m, n, entries)
    +    assert(mat.toBreeze === mat2.toBreeze)
    +    assert(mat2.values.length == 4)
       }
     
       test("sparse matrix construction with wrong number of elements") {
    @@ -117,6 +124,142 @@ class MatricesSuite extends FunSuite {
         assert(sparseMat.values(2) === 10.0)
       }
     
    +  test("toSparse, toDense") {
    +    val m = 3
    +    val n = 2
    +    val values = Array(1.0, 2.0, 4.0, 5.0)
    +    val allValues = Array(1.0, 2.0, 0.0, 0.0, 4.0, 5.0)
    +    val colPtrs = Array(0, 2, 4)
    +    val rowIndices = Array(0, 1, 1, 2)
    +
    +    val spMat1 = new SparseMatrix(m, n, colPtrs, rowIndices, values)
    +    val deMat1 = new DenseMatrix(m, n, allValues)
    +
    +    val spMat2 = deMat1.toSparse()
    +    val deMat2 = spMat1.toDense()
    +
    +    assert(spMat1.toBreeze === spMat2.toBreeze)
    +    assert(deMat1.toBreeze === deMat2.toBreeze)
    +  }
    +
    +  test("map, update") {
    +    val m = 3
    +    val n = 2
    +    val values = Array(1.0, 2.0, 4.0, 5.0)
    +    val allValues = Array(1.0, 2.0, 0.0, 0.0, 4.0, 5.0)
    +    val colPtrs = Array(0, 2, 4)
    +    val rowIndices = Array(0, 1, 1, 2)
    +
    +    val spMat1 = new SparseMatrix(m, n, colPtrs, rowIndices, values)
    +    val deMat1 = new DenseMatrix(m, n, allValues)
    +    val deMat2 = deMat1.map(_ * 2)
    +    val spMat2 = spMat1.map(_ * 2)
    +    deMat1.update(_ * 2)
    +    spMat1.update(_ * 2)
    +
    +    assert(spMat1.toArray === spMat2.toArray)
    +    assert(deMat1.toArray === deMat2.toArray)
    +  }
    +
    +  test("horzcat, vertcat, eye, speye") {
    +    val m = 3
    +    val n = 2
    +    val values = Array(1.0, 2.0, 4.0, 5.0)
    +    val allValues = Array(1.0, 2.0, 0.0, 0.0, 4.0, 5.0)
    +    val colPtrs = Array(0, 2, 4)
    +    val rowIndices = Array(0, 1, 1, 2)
    +
    +    val spMat1 = new SparseMatrix(m, n, colPtrs, rowIndices, values)
    +    val deMat1 = new DenseMatrix(m, n, allValues)
    +    val deMat2 = Matrices.eye(3)
    +    val spMat2 = Matrices.speye(3)
    +    val deMat3 = Matrices.eye(2)
    +    val spMat3 = Matrices.speye(2)
    +
    +    val spHorz = Matrices.horzcat(Array(spMat1, spMat2))
    +    val spHorz2 = Matrices.horzcat(Array(spMat1, deMat2))
    +    val spHorz3 = Matrices.horzcat(Array(deMat1, spMat2))
    +    val deHorz1 = Matrices.horzcat(Array(deMat1, deMat2))
    +
    +    val deHorz2 = Matrices.horzcat(Array[Matrix]())
    +
    +    assert(deHorz1.numRows === 3)
    +    assert(spHorz2.numRows === 3)
    +    assert(spHorz3.numRows === 3)
    +    assert(spHorz.numRows === 3)
    +    assert(deHorz1.numCols === 5)
    +    assert(spHorz2.numCols === 5)
    +    assert(spHorz3.numCols === 5)
    +    assert(spHorz.numCols === 5)
    +    assert(deHorz2.numRows === 0)
    +    assert(deHorz2.numCols === 0)
    +    assert(deHorz2.toArray.length === 0)
    +
    +    assert(deHorz1.toBreeze.toDenseMatrix === spHorz2.toBreeze.toDenseMatrix)
    +    assert(spHorz2.toBreeze === spHorz3.toBreeze)
    +    assert(spHorz(0, 0) === 1.0)
    +    assert(spHorz(2, 1) === 5.0)
    +    assert(spHorz(0, 2) === 1.0)
    +    assert(spHorz(1, 2) === 0.0)
    +    assert(spHorz(1, 3) === 1.0)
    +    assert(spHorz(2, 4) === 1.0)
    +    assert(spHorz(1, 4) === 0.0)
    +    assert(deHorz1(0, 0) === 1.0)
    +    assert(deHorz1(2, 1) === 5.0)
    +    assert(deHorz1(0, 2) === 1.0)
    +    assert(deHorz1(1, 2) == 0.0)
    +    assert(deHorz1(1, 3) === 1.0)
    +    assert(deHorz1(2, 4) === 1.0)
    +    assert(deHorz1(1, 4) === 0.0)
    +
    +    intercept[IllegalArgumentException] {
    +      Matrices.horzcat(Array(spMat1, spMat3))
    +    }
    +
    +    intercept[IllegalArgumentException] {
    +      Matrices.horzcat(Array(deMat1, spMat3))
    +    }
    +
    +    val spVert = Matrices.vertcat(Array(spMat1, spMat3))
    +    val deVert1 = Matrices.vertcat(Array(deMat1, deMat3))
    +    val spVert2 = Matrices.vertcat(Array(spMat1, deMat3))
    +    val spVert3 = Matrices.vertcat(Array(deMat1, spMat3))
    +    val deVert2 = Matrices.vertcat(Array[Matrix]())
    +
    +    assert(deVert1.numRows === 5)
    +    assert(spVert2.numRows === 5)
    +    assert(spVert3.numRows === 5)
    +    assert(spVert.numRows === 5)
    +    assert(deVert1.numCols === 2)
    +    assert(spVert2.numCols === 2)
    +    assert(spVert3.numCols === 2)
    +    assert(spVert.numCols === 2)
    +    assert(deVert2.numRows === 0)
    +    assert(deVert2.numCols === 0)
    +    assert(deVert2.toArray.length === 0)
    +
    +    assert(deVert1.toBreeze.toDenseMatrix === spVert2.toBreeze.toDenseMatrix)
    +    assert(spVert2.toBreeze === spVert3.toBreeze)
    +    assert(spVert(0, 0) === 1.0)
    +    assert(spVert(2, 1) === 5.0)
    +    assert(spVert(3, 0) === 1.0)
    +    assert(spVert(3, 1) === 0.0)
    +    assert(spVert(4, 1) === 1.0)
    +    assert(deVert1(0, 0) === 1.0)
    +    assert(deVert1(2, 1) === 5.0)
    +    assert(deVert1(3, 0) === 1.0)
    +    assert(deVert1(3, 1) === 0.0)
    +    assert(deVert1(4, 1) === 1.0)
    +
    +    intercept[IllegalArgumentException] {
    +      Matrices.vertcat(Array(spMat1, spMat2))
    +    }
    +
    +    intercept[IllegalArgumentException] {
    +      Matrices.vertcat(Array(deMat1, spMat2))
    +    }
    +  }
    +
       test("zeros") {
         val mat = Matrices.zeros(2, 3).asInstanceOf[DenseMatrix]
         assert(mat.numRows === 2)
    @@ -162,4 +305,29 @@ class MatricesSuite extends FunSuite {
         assert(mat.numCols === 2)
         assert(mat.values.toSeq === Seq(1.0, 0.0, 0.0, 2.0))
       }
    +
    +  test("sprand") {
    +    val rng = mock[Random]
    +    when(rng.nextInt(4)).thenReturn(0, 1, 1, 3, 2, 2, 0, 1, 3, 0)
    +    when(rng.nextDouble()).thenReturn(1.0, 2.0, 3.0, 4.0, 0.5, 0.6, 0.7, 0.8, 0.9, 1.0)
    +    val mat = SparseMatrix.sprand(4, 4, 0.25, rng)
    +    assert(mat.numRows === 4)
    +    assert(mat.numCols === 4)
    +    assert(mat.rowIndices.toSeq === Seq(3, 0, 2, 1))
    +    assert(mat.values.toSeq === Seq(1.0, 2.0, 3.0, 4.0))
    +    val mat2 = SparseMatrix.sprand(2, 3, 1.0, rng)
    +    assert(mat2.rowIndices.toSeq === Seq(0, 1, 0, 1, 0, 1))
    +    assert(mat2.colPtrs.toSeq === Seq(0, 2, 4, 6))
    +  }
    +
    +  test("sprandn") {
    +    val rng = mock[Random]
    +    when(rng.nextInt(4)).thenReturn(0, 1, 1, 3, 2, 2, 0, 1, 3, 0)
    +    when(rng.nextGaussian()).thenReturn(1.0, 2.0, 3.0, 4.0)
    +    val mat = SparseMatrix.sprandn(4, 4, 0.25, rng)
    +    assert(mat.numRows === 4)
    +    assert(mat.numCols === 4)
    +    assert(mat.rowIndices.toSeq === Seq(3, 0, 2, 1))
    +    assert(mat.values.toSeq === Seq(1.0, 2.0, 3.0, 4.0))
    +  }
     }
    diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala
    index f99f01450992..5def899cea11 100644
    --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala
    +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala
    @@ -17,7 +17,9 @@
     
     package org.apache.spark.mllib.linalg
     
    -import breeze.linalg.{DenseMatrix => BDM}
    +import scala.util.Random
    +
    +import breeze.linalg.{DenseMatrix => BDM, squaredDistance => breezeSquaredDistance}
     import org.scalatest.FunSuite
     
     import org.apache.spark.SparkException
    @@ -87,6 +89,24 @@ class VectorsSuite extends FunSuite {
         }
       }
     
    +  test("vectors equals with explicit 0") {
    +    val dv1 = Vectors.dense(Array(0, 0.9, 0, 0.8, 0))
    +    val sv1 = Vectors.sparse(5, Array(1, 3), Array(0.9, 0.8))
    +    val sv2 = Vectors.sparse(5, Array(0, 1, 2, 3, 4), Array(0, 0.9, 0, 0.8, 0))
    +
    +    val vectors = Seq(dv1, sv1, sv2)
    +    for (v <- vectors; u <- vectors) {
    +      assert(v === u)
    +      assert(v.## === u.##)
    +    }
    +
    +    val another = Vectors.sparse(5, Array(0, 1, 3), Array(0, 0.9, 0.2))
    +    for (v <- vectors) {
    +      assert(v != another)
    +      assert(v.## != another.##)
    +    }
    +  }
    +
       test("indexing dense vectors") {
         val vec = Vectors.dense(1.0, 2.0, 3.0, 4.0)
         assert(vec(0) === 1.0)
    @@ -175,6 +195,33 @@ class VectorsSuite extends FunSuite {
         assert(v.size === x.rows)
       }
     
    +  test("sqdist") {
    +    val random = new Random()
    +    for (m <- 1 until 1000 by 100) {
    +      val nnz = random.nextInt(m)
    +
    +      val indices1 = random.shuffle(0 to m - 1).slice(0, nnz).sorted.toArray
    +      val values1 = Array.fill(nnz)(random.nextDouble)
    +      val sparseVector1 = Vectors.sparse(m, indices1, values1)
    +
    +      val indices2 = random.shuffle(0 to m - 1).slice(0, nnz).sorted.toArray
    +      val values2 = Array.fill(nnz)(random.nextDouble)
    +      val sparseVector2 = Vectors.sparse(m, indices2, values2)
    +
    +      val denseVector1 = Vectors.dense(sparseVector1.toArray)
    +      val denseVector2 = Vectors.dense(sparseVector2.toArray)
    +
    +      val squaredDist = breezeSquaredDistance(sparseVector1.toBreeze, sparseVector2.toBreeze)
    +
    +      // SparseVector vs. SparseVector 
    +      assert(Vectors.sqdist(sparseVector1, sparseVector2) ~== squaredDist relTol 1E-8) 
    +      // DenseVector  vs. SparseVector
    +      assert(Vectors.sqdist(denseVector1, sparseVector2) ~== squaredDist relTol 1E-8)
    +      // DenseVector  vs. DenseVector
    +      assert(Vectors.sqdist(denseVector1, denseVector2) ~== squaredDist relTol 1E-8)
    +    }    
    +  }
    +
       test("foreachActive") {
         val dv = Vectors.dense(0.0, 1.2, 3.1, 0.0)
         val sv = Vectors.sparse(4, Seq((1, 1.2), (2, 3.1), (3, 0.0)))
    diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrixSuite.scala
    index f8709751efce..80bef814ce50 100644
    --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrixSuite.scala
    +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrixSuite.scala
    @@ -73,6 +73,11 @@ class CoordinateMatrixSuite extends FunSuite with MLlibTestSparkContext {
         assert(mat.toBreeze() === expected)
       }
     
    +  test("transpose") {
    +    val transposed = mat.transpose()
    +    assert(mat.toBreeze().t === transposed.toBreeze())
    +  }
    +
       test("toIndexedRowMatrix") {
         val indexedRowMatrix = mat.toIndexedRowMatrix()
         val expected = BDM(
    diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrixSuite.scala
    index e25bc02b06c9..b86c2ca5ff13 100644
    --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrixSuite.scala
    +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrixSuite.scala
    @@ -80,6 +80,14 @@ class IndexedRowMatrixSuite extends FunSuite with MLlibTestSparkContext {
         assert(rowMat.rows.collect().toSeq === data.map(_.vector).toSeq)
       }
     
    +  test("toCoordinateMatrix") {
    +    val idxRowMat = new IndexedRowMatrix(indexedRows)
    +    val coordMat = idxRowMat.toCoordinateMatrix()
    +    assert(coordMat.numRows() === m)
    +    assert(coordMat.numCols() === n)
    +    assert(coordMat.toBreeze() === idxRowMat.toBreeze())
    +  }
    +
       test("multiply a local matrix") {
         val A = new IndexedRowMatrix(indexedRows)
         val B = Matrices.dense(3, 2, Array(0.0, 1.0, 2.0, 3.0, 4.0, 5.0))
    @@ -113,6 +121,13 @@ class IndexedRowMatrixSuite extends FunSuite with MLlibTestSparkContext {
         assert(closeToZero(U * brzDiag(s) * V.t - localA))
       }
     
    +  test("validate k in svd") {
    +    val A = new IndexedRowMatrix(indexedRows)
    +    intercept[IllegalArgumentException] {
    +      A.computeSVD(-1)
    +    }
    +  }
    +
       def closeToZero(G: BDM[Double]): Boolean = {
         G.valuesIterator.map(math.abs).sum < 1e-6
       }
    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 dbf55ff81ca9..3309713e91f8 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
    @@ -171,6 +171,14 @@ class RowMatrixSuite extends FunSuite with MLlibTestSparkContext {
         }
       }
     
    +  test("validate k in svd") {
    +    for (mat <- Seq(denseMat, sparseMat)) {
    +      intercept[IllegalArgumentException] {
    +        mat.computeSVD(-1)
    +      }
    +    }
    +  }
    +
       def closeToZero(G: BDM[Double]): Boolean = {
         G.valuesIterator.map(math.abs).sum < 1e-6
       }
    diff --git a/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala
    index 603d0ad127b8..f3b7bfda788f 100644
    --- a/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala
    +++ b/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala
    @@ -27,6 +27,7 @@ import org.jblas.DoubleMatrix
     import org.apache.spark.SparkContext._
     import org.apache.spark.mllib.util.MLlibTestSparkContext
     import org.apache.spark.mllib.recommendation.ALS.BlockStats
    +import org.apache.spark.storage.StorageLevel
     
     object ALSSuite {
     
    @@ -139,6 +140,32 @@ class ALSSuite extends FunSuite with MLlibTestSparkContext {
         assert(u11 != u2)
       }
     
    +  test("Storage Level for RDDs in model") {
    +    val ratings = sc.parallelize(ALSSuite.generateRatings(10, 20, 5, 0.5, false, false)._1, 2)
    +    var storageLevel = StorageLevel.MEMORY_ONLY
    +    var model = new ALS()
    +      .setRank(5)
    +      .setIterations(1)
    +      .setLambda(1.0)
    +      .setBlocks(2)
    +      .setSeed(1)
    +      .setFinalRDDStorageLevel(storageLevel)
    +      .run(ratings)
    +    assert(model.productFeatures.getStorageLevel == storageLevel);
    +    assert(model.userFeatures.getStorageLevel == storageLevel);
    +    storageLevel = StorageLevel.DISK_ONLY
    +    model = new ALS()
    +      .setRank(5)
    +      .setIterations(1)
    +      .setLambda(1.0)
    +      .setBlocks(2)
    +      .setSeed(1)
    +      .setFinalRDDStorageLevel(storageLevel)
    +      .run(ratings)
    +    assert(model.productFeatures.getStorageLevel == storageLevel);
    +    assert(model.userFeatures.getStorageLevel == storageLevel);
    +  }
    +
       test("negative ids") {
         val data = ALSSuite.generateRatings(50, 50, 2, 0.7, false, false)
         val ratings = sc.parallelize(data._1.map { case Rating(u, p, r) =>
    diff --git a/mllib/src/test/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussianSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussianSuite.scala
    new file mode 100644
    index 000000000000..fac2498e4dcb
    --- /dev/null
    +++ b/mllib/src/test/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussianSuite.scala
    @@ -0,0 +1,69 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.mllib.stat.distribution
    +
    +import org.scalatest.FunSuite
    +
    +import org.apache.spark.mllib.linalg.{ Vectors, Matrices }
    +import org.apache.spark.mllib.util.MLlibTestSparkContext
    +import org.apache.spark.mllib.util.TestingUtils._
    +
    +class MultivariateGaussianSuite extends FunSuite with MLlibTestSparkContext {
    +  test("univariate") {
    +    val x1 = Vectors.dense(0.0)
    +    val x2 = Vectors.dense(1.5)
    +                     
    +    val mu = Vectors.dense(0.0)
    +    val sigma1 = Matrices.dense(1, 1, Array(1.0))
    +    val dist1 = new MultivariateGaussian(mu, sigma1)
    +    assert(dist1.pdf(x1) ~== 0.39894 absTol 1E-5)
    +    assert(dist1.pdf(x2) ~== 0.12952 absTol 1E-5)
    +    
    +    val sigma2 = Matrices.dense(1, 1, Array(4.0))
    +    val dist2 = new MultivariateGaussian(mu, sigma2)
    +    assert(dist2.pdf(x1) ~== 0.19947 absTol 1E-5)
    +    assert(dist2.pdf(x2) ~== 0.15057 absTol 1E-5)
    +  }
    +  
    +  test("multivariate") {
    +    val x1 = Vectors.dense(0.0, 0.0)
    +    val x2 = Vectors.dense(1.0, 1.0)
    +    
    +    val mu = Vectors.dense(0.0, 0.0)
    +    val sigma1 = Matrices.dense(2, 2, Array(1.0, 0.0, 0.0, 1.0))
    +    val dist1 = new MultivariateGaussian(mu, sigma1)
    +    assert(dist1.pdf(x1) ~== 0.15915 absTol 1E-5)
    +    assert(dist1.pdf(x2) ~== 0.05855 absTol 1E-5)
    +    
    +    val sigma2 = Matrices.dense(2, 2, Array(4.0, -1.0, -1.0, 2.0))
    +    val dist2 = new MultivariateGaussian(mu, sigma2)
    +    assert(dist2.pdf(x1) ~== 0.060155 absTol 1E-5)
    +    assert(dist2.pdf(x2) ~== 0.033971 absTol 1E-5)
    +  }
    +  
    +  test("multivariate degenerate") {
    +    val x1 = Vectors.dense(0.0, 0.0)
    +    val x2 = Vectors.dense(1.0, 1.0)
    +    
    +    val mu = Vectors.dense(0.0, 0.0)
    +    val sigma = Matrices.dense(2, 2, Array(1.0, 1.0, 1.0, 1.0))
    +    val dist = new MultivariateGaussian(mu, sigma)
    +    assert(dist.pdf(x1) ~== 0.11254 absTol 1E-5)
    +    assert(dist.pdf(x2) ~== 0.068259 absTol 1E-5)
    +  }
    +}
    diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala
    index df07987093fb..668fc1d43c5d 100644
    --- a/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala
    +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala
    @@ -20,18 +20,17 @@ package org.apache.spark.mllib.util
     import java.io.File
     
     import scala.io.Source
    -import scala.math
     
     import org.scalatest.FunSuite
     
    -import breeze.linalg.{DenseVector => BDV, SparseVector => BSV, norm => breezeNorm,
    -  squaredDistance => breezeSquaredDistance}
    +import breeze.linalg.{squaredDistance => breezeSquaredDistance}
     import com.google.common.base.Charsets
     import com.google.common.io.Files
     
     import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vectors}
     import org.apache.spark.mllib.regression.LabeledPoint
     import org.apache.spark.mllib.util.MLUtils._
    +import org.apache.spark.mllib.util.TestingUtils._
     import org.apache.spark.util.Utils
     
     class MLUtilsSuite extends FunSuite with MLlibTestSparkContext {
    @@ -52,12 +51,27 @@ class MLUtilsSuite extends FunSuite with MLlibTestSparkContext {
           val values = indices.map(i => a(i))
           val v2 = Vectors.sparse(n, indices, values)
           val norm2 = Vectors.norm(v2, 2.0)
    +      val v3 = Vectors.sparse(n, indices, indices.map(i => a(i) + 0.5))
    +      val norm3 = Vectors.norm(v3, 2.0)
           val squaredDist = breezeSquaredDistance(v1.toBreeze, v2.toBreeze)
           val fastSquaredDist1 = fastSquaredDistance(v1, norm1, v2, norm2, precision)
           assert((fastSquaredDist1 - squaredDist) <= precision * squaredDist, s"failed with m = $m")
           val fastSquaredDist2 =
             fastSquaredDistance(v1, norm1, Vectors.dense(v2.toArray), norm2, precision)
           assert((fastSquaredDist2 - squaredDist) <= precision * squaredDist, s"failed with m = $m")
    +      val squaredDist2 = breezeSquaredDistance(v2.toBreeze, v3.toBreeze)
    +      val fastSquaredDist3 =
    +        fastSquaredDistance(v2, norm2, v3, norm3, precision)
    +      assert((fastSquaredDist3 - squaredDist2) <= precision * squaredDist2, s"failed with m = $m")
    +      if (m > 10) { 
    +        val v4 = Vectors.sparse(n, indices.slice(0, m - 10),
    +          indices.map(i => a(i) + 0.5).slice(0, m - 10))
    +        val norm4 = Vectors.norm(v4, 2.0)
    +        val squaredDist = breezeSquaredDistance(v2.toBreeze, v4.toBreeze)
    +        val fastSquaredDist =
    +          fastSquaredDistance(v2, norm2, v4, norm4, precision)
    +        assert((fastSquaredDist - squaredDist) <= precision * squaredDist, s"failed with m = $m")
    +      }
         }
       }
     
    @@ -189,4 +203,12 @@ class MLUtilsSuite extends FunSuite with MLlibTestSparkContext {
         assert(points.collect().toSet === loaded.collect().toSet)
         Utils.deleteRecursively(tempDir)
       }
    +
    +  test("log1pExp") {
    +    assert(log1pExp(76.3) ~== math.log1p(math.exp(76.3)) relTol 1E-10)
    +    assert(log1pExp(87296763.234) ~== 87296763.234 relTol 1E-10)
    +
    +    assert(log1pExp(-13.8) ~== math.log1p(math.exp(-13.8)) absTol 1E-10)
    +    assert(log1pExp(-238423789.865) ~== math.log1p(math.exp(-238423789.865)) absTol 1E-10)
    +  }
     }
    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 30b906aaa3ba..e957fa5d25f4 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
    @@ -178,17 +178,17 @@ object TestingUtils {
       implicit class MatrixWithAlmostEquals(val x: Matrix) {
     
         /**
    -     * When the difference of two vectors are within eps, returns true; otherwise, returns false.
    +     * When the difference of two matrices 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.
    +     * When the difference of two matrices 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;
    +     * Throws exception when the difference of two matrices are NOT within eps;
          * otherwise, returns true.
          */
         def ~==(r: CompareMatrixRightSide): Boolean = {
    diff --git a/network/common/pom.xml b/network/common/pom.xml
    index baca859fa501..245a96b8c403 100644
    --- a/network/common/pom.xml
    +++ b/network/common/pom.xml
    @@ -75,11 +75,6 @@
           mockito-all
           test
         
    -    
    -      org.scalatest
    -      scalatest_${scala.binary.version}
    -      test
    -    
       
     
       
    diff --git a/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java b/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java
    index 7c9adf52af0f..6c9178688693 100644
    --- a/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java
    +++ b/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java
    @@ -37,7 +37,8 @@ public boolean preferDirectBufs() {
     
       /** Connect timeout in milliseconds. Default 120 secs. */
       public int connectionTimeoutMs() {
    -    return conf.getInt("spark.shuffle.io.connectionTimeout", 120) * 1000;
    +    int defaultTimeout = conf.getInt("spark.network.timeout", 120);
    +    return conf.getInt("spark.shuffle.io.connectionTimeout", defaultTimeout) * 1000;
       }
     
       /** Number of concurrent connections between two nodes for fetching data. */
    diff --git a/network/shuffle/pom.xml b/network/shuffle/pom.xml
    index 12468567c3ae..5bfa1ac9c373 100644
    --- a/network/shuffle/pom.xml
    +++ b/network/shuffle/pom.xml
    @@ -83,11 +83,6 @@
           mockito-all
           test
         
    -    
    -      org.scalatest
    -      scalatest_${scala.binary.version}
    -      test
    -    
       
     
       
    diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockFetcherSuite.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockFetcherSuite.java
    index 0191fe529e1b..1ad0d72ae5ec 100644
    --- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockFetcherSuite.java
    +++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockFetcherSuite.java
    @@ -54,13 +54,13 @@ public class RetryingBlockFetcherSuite {
       @Before
       public void beforeEach() {
         System.setProperty("spark.shuffle.io.maxRetries", "2");
    -    System.setProperty("spark.shuffle.io.retryWaitMs", "0");
    +    System.setProperty("spark.shuffle.io.retryWait", "0");
       }
     
       @After
       public void afterEach() {
         System.clearProperty("spark.shuffle.io.maxRetries");
    -    System.clearProperty("spark.shuffle.io.retryWaitMs");
    +    System.clearProperty("spark.shuffle.io.retryWait");
       }
     
       @Test
    diff --git a/pom.xml b/pom.xml
    index a843af2b22d6..815e2ad44c79 100644
    --- a/pom.xml
    +++ b/pom.xml
    @@ -115,16 +115,18 @@
         1.6
         spark
         2.0.1
    -    0.18.1
    +    0.21.0
         shaded-protobuf
         1.7.5
         1.2.17
         1.0.4
         2.4.1
         ${hadoop.version}
    -    0.94.6
    +    0.98.7-hadoop1
    +    hbase
         1.4.0
         3.4.5
    +    org.spark-project.hive
         
         0.13.1a
         
    @@ -143,13 +145,36 @@
         4.2.6
         3.1.1
         ${project.build.directory}/spark-test-classpath.txt
    -    64m
    -    512m
         2.10.4
         2.10
         ${scala.version}
         org.scala-lang
    -    1.8.8
    +    1.8.8
    +    1.1.1.6
    +
    +    
    +    compile
    +    compile
    +    compile
    +    compile
    +    compile
    +
    +    
    +    ${session.executionRootDirectory}
    +
    +    64m
    +    512m
    +    512m
       
     
       
    @@ -244,21 +269,20 @@
           
         
       
    -
       
    -  
    +    
         
           org.spark-project.spark
           unused
           1.0.0
         
         
         
           org.codehaus.groovy
    @@ -266,6 +290,15 @@
           2.3.7
           provided
         
    +    
    +    
    +      org.scalatest
    +      scalatest_${scala.binary.version}
    +      test
    +    
       
       
         
    @@ -360,11 +393,13 @@
             org.slf4j
             slf4j-api
             ${slf4j.version}
    +        ${hadoop.deps.scope}
           
           
             org.slf4j
             slf4j-log4j12
             ${slf4j.version}
    +        ${hadoop.deps.scope}
           
           
             org.slf4j
    @@ -381,6 +416,7 @@
             log4j
             log4j
             ${log4j.version}
    +        ${hadoop.deps.scope}
           
           
             com.ning
    @@ -390,7 +426,8 @@
           
             org.xerial.snappy
             snappy-java
    -        1.1.1.6
    +        ${snappy.version}
    +        ${hadoop.deps.scope}
           
           
             net.jpountz.lz4
    @@ -418,6 +455,7 @@
             com.google.protobuf
             protobuf-java
             ${protobuf.version}
    +        ${hadoop.deps.scope}
           
           
             ${akka.group}
    @@ -439,6 +477,17 @@
             akka-testkit_${scala.binary.version}
             ${akka.version}
           
    +      
    +        ${akka.group}
    +        akka-zeromq_${scala.binary.version}
    +        ${akka.version}
    +        
    +          
    +            ${akka.group}
    +            akka-actor_${scala.binary.version}
    +          
    +        
    +      
           
             org.apache.mesos
             mesos
    @@ -568,6 +617,7 @@
             org.apache.curator
             curator-recipes
             2.4.0
    +        ${hadoop.deps.scope}
             
               
                 org.jboss.netty
    @@ -579,6 +629,7 @@
             org.apache.hadoop
             hadoop-client
             ${hadoop.version}
    +        ${hadoop.deps.scope}
             
               
                 asm
    @@ -614,11 +665,13 @@
             org.apache.avro
             avro
             ${avro.version}
    +        ${hadoop.deps.scope}
           
           
             org.apache.avro
             avro-ipc
             ${avro.version}
    +        ${hadoop.deps.scope}
             
               
                 io.netty
    @@ -647,6 +700,7 @@
             avro-mapred
             ${avro.version}
             ${avro.mapred.classifier}
    +        ${hive.deps.scope}
             
               
                 io.netty
    @@ -675,6 +729,7 @@
             net.java.dev.jets3t
             jets3t
             ${jets3t.version}
    +        ${hadoop.deps.scope}
             
               
                 commons-logging
    @@ -686,6 +741,7 @@
             org.apache.hadoop
             hadoop-yarn-api
             ${yarn.version}
    +        ${hadoop.deps.scope}
             
               
                 javax.servlet
    @@ -713,6 +769,7 @@
             org.apache.hadoop
             hadoop-yarn-common
             ${yarn.version}
    +        ${hadoop.deps.scope}
             
               
                 asm
    @@ -769,6 +826,7 @@
             org.apache.hadoop
             hadoop-yarn-server-web-proxy
             ${yarn.version}
    +        ${hadoop.deps.scope}
             
               
                 asm
    @@ -796,6 +854,7 @@
             org.apache.hadoop
             hadoop-yarn-client
             ${yarn.version}
    +        ${hadoop.deps.scope}
             
               
                 asm
    @@ -820,15 +879,126 @@
             
           
           
    -        
    +        org.apache.zookeeper
    +        zookeeper
    +        ${zookeeper.version}
    +        ${hadoop.deps.scope}
    +      
    +      
             org.codehaus.jackson
    -        jackson-mapper-asl
    -        ${jackson.version}
    +        jackson-core-asl
    +        ${codehaus.jackson.version}
    +        ${hadoop.deps.scope}
           
           
             org.codehaus.jackson
             jackson-mapper-asl
    -        ${jackson.version}
    +        ${codehaus.jackson.version}
    +        ${hadoop.deps.scope}
    +      
    +      
    +        ${hive.group}
    +        hive-beeline
    +        ${hive.version}
    +        ${hive.deps.scope}
    +      
    +      
    +        ${hive.group}
    +        hive-cli
    +        ${hive.version}
    +        ${hive.deps.scope}
    +      
    +      
    +        ${hive.group}
    +        hive-exec
    +        ${hive.version}
    +        ${hive.deps.scope}
    +        
    +          
    +            commons-logging
    +            commons-logging
    +          
    +          
    +            com.esotericsoftware.kryo
    +            kryo
    +          
    +        
    +      
    +      
    +        ${hive.group}
    +        hive-jdbc
    +        ${hive.version}
    +        ${hive.deps.scope}
    +      
    +      
    +        ${hive.group}
    +        hive-metastore
    +        ${hive.version}
    +        ${hive.deps.scope}
    +      
    +      
    +        ${hive.group}
    +        hive-serde
    +        ${hive.version}
    +        ${hive.deps.scope}
    +        
    +          
    +            commons-logging
    +            commons-logging
    +          
    +          
    +            commons-logging
    +            commons-logging-api
    +          
    +        
    +      
    +      
    +        com.twitter
    +        parquet-column
    +        ${parquet.version}
    +        ${parquet.deps.scope}
    +      
    +      
    +        com.twitter
    +        parquet-hadoop
    +        ${parquet.version}
    +        ${parquet.deps.scope}
    +      
    +      
    +        org.apache.flume
    +        flume-ng-core
    +        ${flume.version}
    +        ${flume.deps.scope}
    +        
    +          
    +            io.netty
    +            netty
    +          
    +          
    +            org.apache.thrift
    +            libthrift
    +          
    +          
    +            org.mortbay.jetty
    +            servlet-api
    +          
    +        
    +      
    +      
    +        org.apache.flume
    +        flume-ng-sdk
    +        ${flume.version}
    +        ${flume.deps.scope}
    +        
    +          
    +            io.netty
    +            netty
    +          
    +          
    +            org.apache.thrift
    +            libthrift
    +          
    +        
           
         
       
    @@ -905,6 +1075,7 @@
                   -Xmx1024m
                   -XX:PermSize=${PermGen}
                   -XX:MaxPermSize=${MaxPermGen}
    +              -XX:ReservedCodeCacheSize=${CodeCacheSize}
                 
                 
                   -source
    @@ -935,32 +1106,58 @@
                 true
               
             
    +        
             
               org.apache.maven.plugins
               maven-surefire-plugin
    -          2.17
    +          2.18
    +          
               
    -            
    -            true
    +            
    +              **/Test*.java
    +              **/*Test.java
    +              **/*TestCase.java
    +              **/*Suite.java
    +            
    +            ${project.build.directory}/surefire-reports
    +            -Xmx3g -XX:MaxPermSize=${MaxPermGen} -XX:ReservedCodeCacheSize=512m
    +            
    +              true
    +              ${session.executionRootDirectory}
    +              1
    +              false
    +              false
    +              ${test_classpath}
    +              true
    +            
    +            false
               
             
    +        
             
               org.scalatest
               scalatest-maven-plugin
               1.0
    +          
               
                 ${project.build.directory}/surefire-reports
                 .
                 SparkTestSuite.txt
    -            -ea -Xmx3g -XX:MaxPermSize=${MaxPermGen} -XX:ReservedCodeCacheSize=512m
    +            -ea -Xmx3g -XX:MaxPermSize=${MaxPermGen} -XX:ReservedCodeCacheSize=${CodeCacheSize}
                 
    +            
    +              
    +              ${test_classpath}
    +            
                 
                   true
    -              ${session.executionRootDirectory}
    +              ${spark.test.home}
                   1
                   false
                   false
    -              ${test_classpath}
                   true
                 
               
    @@ -983,11 +1180,6 @@
               maven-antrun-plugin
               1.7
             
    -        
    -          org.apache.maven.plugins
    -          maven-shade-plugin
    -          2.2
    -        
             
               org.apache.maven.plugins
               maven-source-plugin
    @@ -1076,6 +1268,7 @@
           
             org.apache.maven.plugins
             maven-shade-plugin
    +        2.2
             
               false
               
    @@ -1159,6 +1352,15 @@
               
             
           
    +      
    +      
    +        org.apache.maven.plugins
    +        maven-surefire-plugin
    +      
    +      
    +        org.scalatest
    +        scalatest-maven-plugin
    +      
         
       
     
    @@ -1264,6 +1466,7 @@
           
             2.2.0
             2.5.0
    +        0.98.7-hadoop2
             hadoop2
           
         
    @@ -1274,6 +1477,7 @@
             2.3.0
             2.5.0
             0.9.0
    +        0.98.7-hadoop2
             3.1.1
             hadoop2
           
    @@ -1285,6 +1489,7 @@
             2.4.0
             2.5.0
             0.9.0
    +        0.98.7-hadoop2
             3.1.1
             hadoop2
           
    @@ -1302,7 +1507,7 @@
           mapr3
           
             1.0.3-mapr-3.0.3
    -        2.3.0-mapr-4.0.0-FCS
    +        2.4.1-mapr-1408
             0.94.17-mapr-1405
             3.4.5-mapr-1406
           
    @@ -1311,8 +1516,8 @@
         
           mapr4
           
    -        2.3.0-mapr-4.0.0-FCS
    -        2.3.0-mapr-4.0.0-FCS
    +        2.4.1-mapr-1408
    +        2.4.1-mapr-1408
             0.94.17-mapr-1405-4.0.0-FCS
             3.4.5-mapr-1406
           
    @@ -1336,53 +1541,6 @@
           
         
     
    -    
    -    
    -      hadoop-provided
    -      
    -        
    -          org.apache.hadoop
    -          hadoop-client
    -          provided
    -        
    -        
    -          org.apache.hadoop
    -          hadoop-yarn-api
    -          provided
    -        
    -        
    -          org.apache.hadoop
    -          hadoop-yarn-common
    -          provided
    -        
    -        
    -          org.apache.hadoop
    -          hadoop-yarn-server-web-proxy
    -          provided
    -        
    -        
    -          org.apache.hadoop
    -          hadoop-yarn-client
    -          provided
    -        
    -        
    -          org.apache.avro
    -          avro
    -          provided
    -        
    -        
    -          org.apache.avro
    -          avro-ipc
    -          provided
    -        
    -        
    -          org.apache.zookeeper
    -          zookeeper
    -          ${zookeeper.version}
    -          provided
    -        
    -      
    -    
         
           hive-thriftserver
           
    @@ -1419,6 +1577,7 @@
           
           
             external/kafka
    +        external/kafka-assembly
           
         
     
    @@ -1435,5 +1594,25 @@
           
         
     
    +    
    +    
    +      flume-provided
    +    
    +    
    +      hadoop-provided
    +    
    +    
    +      hbase-provided
    +    
    +    
    +      hive-provided
    +    
    +    
    +      parquet-provided
    +    
       
     
    diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala
    index 230239aa4050..127973b65819 100644
    --- a/project/MimaExcludes.scala
    +++ b/project/MimaExcludes.scala
    @@ -36,7 +36,6 @@ object MimaExcludes {
             case v if v.startsWith("1.3") =>
               Seq(
                 MimaBuild.excludeSparkPackage("deploy"),
    -            MimaBuild.excludeSparkPackage("graphx"),
                 // These are needed if checking against the sbt build, since they are part of
                 // the maven-generated artifacts in the 1.2 build.
                 MimaBuild.excludeSparkPackage("unused"),
    @@ -53,6 +52,44 @@ object MimaExcludes {
                   "org.apache.spark.mllib.linalg.Matrices.randn"),
                 ProblemFilters.exclude[MissingMethodProblem](
                   "org.apache.spark.mllib.linalg.Matrices.rand")
    +          ) ++ Seq(
    +            // SPARK-3325
    +            ProblemFilters.exclude[MissingMethodProblem](
    +              "org.apache.spark.streaming.api.java.JavaDStreamLike.print"),
    +            // SPARK-2757
    +            ProblemFilters.exclude[IncompatibleResultTypeProblem](
    +              "org.apache.spark.streaming.flume.sink.SparkAvroCallbackHandler." +
    +                "removeAndGetProcessor")
    +          ) ++ Seq(
    +            // SPARK-5123 (SparkSQL data type change) - alpha component only
    +            ProblemFilters.exclude[IncompatibleResultTypeProblem](
    +              "org.apache.spark.ml.feature.HashingTF.outputDataType"),
    +            ProblemFilters.exclude[IncompatibleResultTypeProblem](
    +              "org.apache.spark.ml.feature.Tokenizer.outputDataType"),
    +            ProblemFilters.exclude[IncompatibleMethTypeProblem](
    +              "org.apache.spark.ml.feature.Tokenizer.validateInputType"),
    +            ProblemFilters.exclude[IncompatibleMethTypeProblem](
    +              "org.apache.spark.ml.classification.LogisticRegressionModel.validateAndTransformSchema"),
    +            ProblemFilters.exclude[IncompatibleMethTypeProblem](
    +              "org.apache.spark.ml.classification.LogisticRegression.validateAndTransformSchema")
    +          ) ++ Seq(
    +            // SPARK-4014
    +            ProblemFilters.exclude[MissingMethodProblem](
    +              "org.apache.spark.TaskContext.taskAttemptId"),
    +            ProblemFilters.exclude[MissingMethodProblem](
    +              "org.apache.spark.TaskContext.attemptNumber")
    +          ) ++ Seq(
    +            // SPARK-5166 Spark SQL API stabilization
    +            ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.Transformer.transform"),
    +            ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.Estimator.fit")
    +          ) ++ Seq(
    +            // SPARK-5270
    +            ProblemFilters.exclude[MissingMethodProblem](
    +              "org.apache.spark.api.java.JavaRDDLike.isEmpty")
    +          ) ++ Seq(
    +            // SPARK-5297 Java FileStream do not work with custom key/values
    +            ProblemFilters.exclude[MissingMethodProblem](
    +              "org.apache.spark.streaming.api.java.JavaStreamingContext.fileStream")
               )
     
             case v if v.startsWith("1.2") =>
    diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
    index c512b62f6137..fbc8983b953b 100644
    --- a/project/SparkBuild.scala
    +++ b/project/SparkBuild.scala
    @@ -44,8 +44,9 @@ object BuildCommons {
         sparkKinesisAsl) = Seq("yarn", "yarn-stable", "java8-tests", "ganglia-lgpl",
         "kinesis-asl").map(ProjectRef(buildLocation, _))
     
    -  val assemblyProjects@Seq(assembly, examples, networkYarn) =
    -    Seq("assembly", "examples", "network-yarn").map(ProjectRef(buildLocation, _))
    +  val assemblyProjects@Seq(assembly, examples, networkYarn, streamingKafkaAssembly) =
    +    Seq("assembly", "examples", "network-yarn", "streaming-kafka-assembly")
    +      .map(ProjectRef(buildLocation, _))
     
       val tools = ProjectRef(buildLocation, "tools")
       // Root project.
    @@ -114,17 +115,6 @@ object SparkBuild extends PomBuild {
     
       override val userPropertiesMap = System.getProperties.toMap
     
    -  // Handle case where hadoop.version is set via profile.
    -  // Needed only because we read back this property in sbt
    -  // when we create the assembly jar.
    -  val pom = loadEffectivePom(new File("pom.xml"),
    -    profiles = profiles,
    -    userProps = userPropertiesMap)
    -  if (System.getProperty("hadoop.version") == null) {
    -    System.setProperty("hadoop.version",
    -      pom.getProperties.get("hadoop.version").asInstanceOf[String])
    -  }
    -
       lazy val MavenCompile = config("m2r") extend(Compile)
       lazy val publishLocalBoth = TaskKey[Unit]("publish-local", "publish local for m2 and ivy")
     
    @@ -166,7 +156,7 @@ object SparkBuild extends PomBuild {
     
       // TODO: Add Sql to mima checks
       allProjects.filterNot(x => Seq(spark, sql, hive, hiveThriftServer, catalyst, repl,
    -    streamingFlumeSink, networkCommon, networkShuffle, networkYarn).contains(x)).foreach {
    +    networkCommon, networkShuffle, networkYarn).contains(x)).foreach {
           x => enable(MimaBuild.mimaSettings(sparkHome, x))(x)
         }
     
    @@ -254,10 +244,10 @@ object SQL {
             |import org.apache.spark.sql.catalyst.expressions._
             |import org.apache.spark.sql.catalyst.plans.logical._
             |import org.apache.spark.sql.catalyst.rules._
    -        |import org.apache.spark.sql.catalyst.types._
             |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.types._
             |import org.apache.spark.sql.parquet.ParquetTestData""".stripMargin,
         cleanupCommands in console := "sparkContext.stop()"
       )
    @@ -284,11 +274,11 @@ object Hive {
             |import org.apache.spark.sql.catalyst.expressions._
             |import org.apache.spark.sql.catalyst.plans.logical._
             |import org.apache.spark.sql.catalyst.rules._
    -        |import org.apache.spark.sql.catalyst.types._
             |import org.apache.spark.sql.catalyst.util._
             |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.types._
             |import org.apache.spark.sql.parquet.ParquetTestData""".stripMargin,
         cleanupCommands in console := "sparkContext.stop()",
         // Some of our log4j jars make it impossible to submit jobs from this JVM to Hive Map/Reduce
    @@ -303,14 +293,20 @@ object Assembly {
       import sbtassembly.Plugin._
       import AssemblyKeys._
     
    +  val hadoopVersion = taskKey[String]("The version of hadoop that spark is compiled against.")
    +
       lazy val settings = assemblySettings ++ Seq(
         test in assembly := {},
    -    jarName in assembly <<= (version, moduleName) map { (v, mName) =>
    -      if (mName.contains("network-yarn")) {
    -        // This must match the same name used in maven (see network/yarn/pom.xml)
    -        "spark-" + v + "-yarn-shuffle.jar"
    +    hadoopVersion := {
    +      sys.props.get("hadoop.version")
    +        .getOrElse(SbtPomKeys.effectivePom.value.getProperties.get("hadoop.version").asInstanceOf[String])
    +    },
    +    jarName in assembly <<= (version, moduleName, hadoopVersion) map { (v, mName, hv) =>
    +      if (mName.contains("streaming-kafka-assembly")) {
    +        // This must match the same name used in maven (see external/kafka-assembly/pom.xml)
    +        s"${mName}-${v}.jar"
           } else {
    -        mName + "-" + v + "-hadoop" + System.getProperty("hadoop.version") + ".jar"
    +        s"${mName}-${v}-hadoop${hv}.jar"
           }
         },
         mergeStrategy in assembly := {
    @@ -323,7 +319,6 @@ object Assembly {
           case _                                                   => MergeStrategy.first
         }
       )
    -
     }
     
     object Unidoc {
    diff --git a/python/pyspark/context.py b/python/pyspark/context.py
    index 593d74bca5ff..64f6a3ca6bf4 100644
    --- a/python/pyspark/context.py
    +++ b/python/pyspark/context.py
    @@ -319,7 +319,7 @@ def f(split, iterator):
             # Make sure we distribute data evenly if it's smaller than self.batchSize
             if "__len__" not in dir(c):
                 c = list(c)    # Make it a list so we can compute its length
    -        batchSize = max(1, min(len(c) // numSlices, self._batchSize))
    +        batchSize = max(1, min(len(c) // numSlices, self._batchSize or 1024))
             serializer = BatchedSerializer(self._unbatched_serializer, batchSize)
             serializer.dump_stream(c, tempFile)
             tempFile.close()
    diff --git a/python/pyspark/mllib/clustering.py b/python/pyspark/mllib/clustering.py
    index e2492eef5bd6..6b713aa39374 100644
    --- a/python/pyspark/mllib/clustering.py
    +++ b/python/pyspark/mllib/clustering.py
    @@ -78,10 +78,10 @@ def predict(self, x):
     class KMeans(object):
     
         @classmethod
    -    def train(cls, rdd, k, maxIterations=100, runs=1, initializationMode="k-means||"):
    +    def train(cls, rdd, k, maxIterations=100, runs=1, initializationMode="k-means||", seed=None):
             """Train a k-means clustering model."""
             model = callMLlibFunc("trainKMeansModel", rdd.map(_convert_to_vector), k, maxIterations,
    -                              runs, initializationMode)
    +                              runs, initializationMode, seed)
             centers = callJavaFunc(rdd.context, model.clusterCenters)
             return KMeansModel([c.toArray() for c in centers])
     
    diff --git a/python/pyspark/mllib/common.py b/python/pyspark/mllib/common.py
    index 33c49e239990..3c5ee66cd8b6 100644
    --- a/python/pyspark/mllib/common.py
    +++ b/python/pyspark/mllib/common.py
    @@ -18,7 +18,7 @@
     import py4j.protocol
     from py4j.protocol import Py4JJavaError
     from py4j.java_gateway import JavaObject
    -from py4j.java_collections import MapConverter, ListConverter, JavaArray, JavaList
    +from py4j.java_collections import ListConverter, JavaArray, JavaList
     
     from pyspark import RDD, SparkContext
     from pyspark.serializers import PickleSerializer, AutoBatchedSerializer
    @@ -70,9 +70,7 @@ def _py2java(sc, obj):
             obj = _to_java_object_rdd(obj)
         elif isinstance(obj, SparkContext):
             obj = obj._jsc
    -    elif isinstance(obj, dict):
    -        obj = MapConverter().convert(obj, sc._gateway._gateway_client)
    -    elif isinstance(obj, (list, tuple)):
    +    elif isinstance(obj, list) and (obj or isinstance(obj[0], JavaObject)):
             obj = ListConverter().convert(obj, sc._gateway._gateway_client)
         elif isinstance(obj, JavaObject):
             pass
    diff --git a/python/pyspark/mllib/linalg.py b/python/pyspark/mllib/linalg.py
    index f7aa2b0cb04b..7f21190ed8c2 100644
    --- a/python/pyspark/mllib/linalg.py
    +++ b/python/pyspark/mllib/linalg.py
    @@ -178,7 +178,7 @@ def __init__(self, ar):
             elif not isinstance(ar, np.ndarray):
                 ar = np.array(ar, dtype=np.float64)
             if ar.dtype != np.float64:
    -            ar.astype(np.float64)
    +            ar = ar.astype(np.float64)
             self.array = ar
     
         def __reduce__(self):
    @@ -510,6 +510,23 @@ def __eq__(self, other):
                     and np.array_equal(other.indices, self.indices)
                     and np.array_equal(other.values, self.values))
     
    +    def __getitem__(self, index):
    +        inds = self.indices
    +        vals = self.values
    +        if not isinstance(index, int):
    +            raise ValueError(
    +                "Indices must be of type integer, got type %s" % type(index))
    +        if index < 0:
    +            index += self.size
    +        if index >= self.size or index < 0:
    +            raise ValueError("Index %d out of bounds." % index)
    +
    +        insert_index = np.searchsorted(inds, index)
    +        row_ind = inds[insert_index]
    +        if row_ind == index:
    +            return vals[insert_index]
    +        return 0.
    +
         def __ne__(self, other):
             return not self.__eq__(other)
     
    diff --git a/python/pyspark/mllib/rand.py b/python/pyspark/mllib/rand.py
    index cb4304f92152..20ee9d78bf5b 100644
    --- a/python/pyspark/mllib/rand.py
    +++ b/python/pyspark/mllib/rand.py
    @@ -99,6 +99,38 @@ def normalRDD(sc, size, numPartitions=None, seed=None):
             """
             return callMLlibFunc("normalRDD", sc._jsc, size, numPartitions, seed)
     
    +    @staticmethod
    +    def logNormalRDD(sc, mean, std, size, numPartitions=None, seed=None):
    +        """
    +        Generates an RDD comprised of i.i.d. samples from the log normal
    +        distribution with the input mean and standard distribution.
    +
    +        :param sc: SparkContext used to create the RDD.
    +        :param mean: mean for the log Normal distribution
    +        :param std: std for the log Normal distribution
    +        :param size: Size of the RDD.
    +        :param numPartitions: Number of partitions in the RDD (default: `sc.defaultParallelism`).
    +        :param seed: Random seed (default: a random long integer).
    +        :return: RDD of float comprised of i.i.d. samples ~ log N(mean, std).
    +
    +        >>> from math import sqrt, exp
    +        >>> mean = 0.0
    +        >>> std = 1.0
    +        >>> expMean = exp(mean + 0.5 * std * std)
    +        >>> expStd = sqrt((exp(std * std) - 1.0) * exp(2.0 * mean + std * std))
    +        >>> x = RandomRDDs.logNormalRDD(sc, mean, std, 1000, seed=2L)
    +        >>> stats = x.stats()
    +        >>> stats.count()
    +        1000L
    +        >>> abs(stats.mean() - expMean) < 0.5
    +        True
    +        >>> from math import sqrt
    +        >>> abs(stats.stdev() - expStd) < 0.5
    +        True
    +        """
    +        return callMLlibFunc("logNormalRDD", sc._jsc, float(mean), float(std),
    +                             size, numPartitions, seed)
    +
         @staticmethod
         def poissonRDD(sc, mean, size, numPartitions=None, seed=None):
             """
    @@ -125,6 +157,63 @@ def poissonRDD(sc, mean, size, numPartitions=None, seed=None):
             """
             return callMLlibFunc("poissonRDD", sc._jsc, float(mean), size, numPartitions, seed)
     
    +    @staticmethod
    +    def exponentialRDD(sc, mean, size, numPartitions=None, seed=None):
    +        """
    +        Generates an RDD comprised of i.i.d. samples from the Exponential
    +        distribution with the input mean.
    +
    +        :param sc: SparkContext used to create the RDD.
    +        :param mean: Mean, or 1 / lambda, for the Exponential distribution.
    +        :param size: Size of the RDD.
    +        :param numPartitions: Number of partitions in the RDD (default: `sc.defaultParallelism`).
    +        :param seed: Random seed (default: a random long integer).
    +        :return: RDD of float comprised of i.i.d. samples ~ Exp(mean).
    +
    +        >>> mean = 2.0
    +        >>> x = RandomRDDs.exponentialRDD(sc, mean, 1000, seed=2L)
    +        >>> stats = x.stats()
    +        >>> stats.count()
    +        1000L
    +        >>> abs(stats.mean() - mean) < 0.5
    +        True
    +        >>> from math import sqrt
    +        >>> abs(stats.stdev() - sqrt(mean)) < 0.5
    +        True
    +        """
    +        return callMLlibFunc("exponentialRDD", sc._jsc, float(mean), size, numPartitions, seed)
    +
    +    @staticmethod
    +    def gammaRDD(sc, shape, scale, size, numPartitions=None, seed=None):
    +        """
    +        Generates an RDD comprised of i.i.d. samples from the Gamma
    +        distribution with the input shape and scale.
    +
    +        :param sc: SparkContext used to create the RDD.
    +        :param shape: shape (> 0) parameter for the Gamma distribution
    +        :param scale: scale (> 0) parameter for the Gamma distribution
    +        :param size: Size of the RDD.
    +        :param numPartitions: Number of partitions in the RDD (default: `sc.defaultParallelism`).
    +        :param seed: Random seed (default: a random long integer).
    +        :return: RDD of float comprised of i.i.d. samples ~ Gamma(shape, scale).
    +
    +        >>> from math import sqrt
    +        >>> shape = 1.0
    +        >>> scale = 2.0
    +        >>> expMean = shape * scale
    +        >>> expStd = sqrt(shape * scale * scale)
    +        >>> x = RandomRDDs.gammaRDD(sc, shape, scale, 1000, seed=2L)
    +        >>> stats = x.stats()
    +        >>> stats.count()
    +        1000L
    +        >>> abs(stats.mean() - expMean) < 0.5
    +        True
    +        >>> abs(stats.stdev() - expStd) < 0.5
    +        True
    +        """
    +        return callMLlibFunc("gammaRDD", sc._jsc, float(shape),
    +                             float(scale), size, numPartitions, seed)
    +
         @staticmethod
         @toArray
         def uniformVectorRDD(sc, numRows, numCols, numPartitions=None, seed=None):
    @@ -175,6 +264,40 @@ def normalVectorRDD(sc, numRows, numCols, numPartitions=None, seed=None):
             """
             return callMLlibFunc("normalVectorRDD", sc._jsc, numRows, numCols, numPartitions, seed)
     
    +    @staticmethod
    +    @toArray
    +    def logNormalVectorRDD(sc, mean, std, numRows, numCols, numPartitions=None, seed=None):
    +        """
    +        Generates an RDD comprised of vectors containing i.i.d. samples drawn
    +        from the log normal distribution.
    +
    +        :param sc: SparkContext used to create the RDD.
    +        :param mean: Mean of the log normal distribution
    +        :param std: Standard Deviation of the log normal distribution
    +        :param numRows: Number of Vectors in the RDD.
    +        :param numCols: Number of elements in each Vector.
    +        :param numPartitions: Number of partitions in the RDD (default: `sc.defaultParallelism`).
    +        :param seed: Random seed (default: a random long integer).
    +        :return: RDD of Vector with vectors containing i.i.d. samples ~ log `N(mean, std)`.
    +
    +        >>> import numpy as np
    +        >>> from math import sqrt, exp
    +        >>> mean = 0.0
    +        >>> std = 1.0
    +        >>> expMean = exp(mean + 0.5 * std * std)
    +        >>> expStd = sqrt((exp(std * std) - 1.0) * exp(2.0 * mean + std * std))
    +        >>> mat = np.matrix(RandomRDDs.logNormalVectorRDD(sc, mean, std, \
    +                               100, 100, seed=1L).collect())
    +        >>> mat.shape
    +        (100, 100)
    +        >>> abs(mat.mean() - expMean) < 0.1
    +        True
    +        >>> abs(mat.std() - expStd) < 0.1
    +        True
    +        """
    +        return callMLlibFunc("logNormalVectorRDD", sc._jsc, float(mean), float(std),
    +                             numRows, numCols, numPartitions, seed)
    +
         @staticmethod
         @toArray
         def poissonVectorRDD(sc, mean, numRows, numCols, numPartitions=None, seed=None):
    @@ -205,6 +328,70 @@ def poissonVectorRDD(sc, mean, numRows, numCols, numPartitions=None, seed=None):
             return callMLlibFunc("poissonVectorRDD", sc._jsc, float(mean), numRows, numCols,
                                  numPartitions, seed)
     
    +    @staticmethod
    +    @toArray
    +    def exponentialVectorRDD(sc, mean, numRows, numCols, numPartitions=None, seed=None):
    +        """
    +        Generates an RDD comprised of vectors containing i.i.d. samples drawn
    +        from the Exponential distribution with the input mean.
    +
    +        :param sc: SparkContext used to create the RDD.
    +        :param mean: Mean, or 1 / lambda, for the Exponential distribution.
    +        :param numRows: Number of Vectors in the RDD.
    +        :param numCols: Number of elements in each Vector.
    +        :param numPartitions: Number of partitions in the RDD (default: `sc.defaultParallelism`)
    +        :param seed: Random seed (default: a random long integer).
    +        :return: RDD of Vector with vectors containing i.i.d. samples ~ Exp(mean).
    +
    +        >>> import numpy as np
    +        >>> mean = 0.5
    +        >>> rdd = RandomRDDs.exponentialVectorRDD(sc, mean, 100, 100, seed=1L)
    +        >>> mat = np.mat(rdd.collect())
    +        >>> mat.shape
    +        (100, 100)
    +        >>> abs(mat.mean() - mean) < 0.5
    +        True
    +        >>> from math import sqrt
    +        >>> abs(mat.std() - sqrt(mean)) < 0.5
    +        True
    +        """
    +        return callMLlibFunc("exponentialVectorRDD", sc._jsc, float(mean), numRows, numCols,
    +                             numPartitions, seed)
    +
    +    @staticmethod
    +    @toArray
    +    def gammaVectorRDD(sc, shape, scale, numRows, numCols, numPartitions=None, seed=None):
    +        """
    +        Generates an RDD comprised of vectors containing i.i.d. samples drawn
    +        from the Gamma distribution.
    +
    +        :param sc: SparkContext used to create the RDD.
    +        :param shape: Shape (> 0) of the Gamma distribution
    +        :param scale: Scale (> 0) of the Gamma distribution
    +        :param numRows: Number of Vectors in the RDD.
    +        :param numCols: Number of elements in each Vector.
    +        :param numPartitions: Number of partitions in the RDD (default: `sc.defaultParallelism`).
    +        :param seed: Random seed (default: a random long integer).
    +        :return: RDD of Vector with vectors containing i.i.d. samples ~ Gamma(shape, scale).
    +
    +        >>> import numpy as np
    +        >>> from math import sqrt
    +        >>> shape = 1.0
    +        >>> scale = 2.0
    +        >>> expMean = shape * scale
    +        >>> expStd = sqrt(shape * scale * scale)
    +        >>> mat = np.matrix(RandomRDDs.gammaVectorRDD(sc, shape, scale, \
    +                       100, 100, seed=1L).collect())
    +        >>> mat.shape
    +        (100, 100)
    +        >>> abs(mat.mean() - expMean) < 0.1
    +        True
    +        >>> abs(mat.std() - expStd) < 0.1
    +        True
    +        """
    +        return callMLlibFunc("gammaVectorRDD", sc._jsc, float(shape), float(scale),
    +                             numRows, numCols, numPartitions, seed)
    +
     
     def _test():
         import doctest
    diff --git a/python/pyspark/mllib/tests.py b/python/pyspark/mllib/tests.py
    index 5034f229e824..f48e3d6dacb4 100644
    --- a/python/pyspark/mllib/tests.py
    +++ b/python/pyspark/mllib/tests.py
    @@ -110,6 +110,28 @@ def test_squared_distance(self):
             self.assertEquals(0.0, _squared_distance(dv, dv))
             self.assertEquals(0.0, _squared_distance(lst, lst))
     
    +    def test_conversion(self):
    +        # numpy arrays should be automatically upcast to float64
    +        # tests for fix of [SPARK-5089]
    +        v = array([1, 2, 3, 4], dtype='float64')
    +        dv = DenseVector(v)
    +        self.assertTrue(dv.array.dtype == 'float64')
    +        v = array([1, 2, 3, 4], dtype='float32')
    +        dv = DenseVector(v)
    +        self.assertTrue(dv.array.dtype == 'float64')
    +
    +    def test_sparse_vector_indexing(self):
    +        sv = SparseVector(4, {1: 1, 3: 2})
    +        self.assertEquals(sv[0], 0.)
    +        self.assertEquals(sv[3], 2.)
    +        self.assertEquals(sv[1], 1.)
    +        self.assertEquals(sv[2], 0.)
    +        self.assertEquals(sv[-1], 2)
    +        self.assertEquals(sv[-2], 0)
    +        self.assertEquals(sv[-4], 0)
    +        for ind in [4, -5, 7.8]:
    +            self.assertRaises(ValueError, sv.__getitem__, ind)
    +
     
     class ListTests(PySparkTestCase):
     
    @@ -118,7 +140,7 @@ class ListTests(PySparkTestCase):
         as NumPy arrays.
         """
     
    -    def test_clustering(self):
    +    def test_kmeans(self):
             from pyspark.mllib.clustering import KMeans
             data = [
                 [0, 1.1],
    @@ -130,6 +152,21 @@ def test_clustering(self):
             self.assertEquals(clusters.predict(data[0]), clusters.predict(data[1]))
             self.assertEquals(clusters.predict(data[2]), clusters.predict(data[3]))
     
    +    def test_kmeans_deterministic(self):
    +        from pyspark.mllib.clustering import KMeans
    +        X = range(0, 100, 10)
    +        Y = range(0, 100, 10)
    +        data = [[x, y] for x, y in zip(X, Y)]
    +        clusters1 = KMeans.train(self.sc.parallelize(data),
    +                                 3, initializationMode="k-means||", seed=42)
    +        clusters2 = KMeans.train(self.sc.parallelize(data),
    +                                 3, initializationMode="k-means||", seed=42)
    +        centers1 = clusters1.centers
    +        centers2 = clusters2.centers
    +        for c1, c2 in zip(centers1, centers2):
    +            # TODO: Allow small numeric difference.
    +            self.assertTrue(array_equal(c1, c2))
    +
         def test_classification(self):
             from pyspark.mllib.classification import LogisticRegressionWithSGD, SVMWithSGD, NaiveBayes
             from pyspark.mllib.tree import DecisionTree
    diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py
    index c1120cf781e5..4977400ac1c0 100644
    --- a/python/pyspark/rdd.py
    +++ b/python/pyspark/rdd.py
    @@ -1130,6 +1130,18 @@ def first(self):
                 return rs[0]
             raise ValueError("RDD is empty")
     
    +    def isEmpty(self):
    +        """
    +        Returns true if and only if the RDD contains no elements at all. Note that an RDD
    +        may be empty even when it has at least 1 partition.
    +
    +        >>> sc.parallelize([]).isEmpty()
    +        True
    +        >>> sc.parallelize([1]).isEmpty()
    +        False
    +        """
    +        return self._jrdd.partitions().size() == 0 or len(self.take(1)) == 0
    +
         def saveAsNewAPIHadoopDataset(self, conf, keyConverter=None, valueConverter=None):
             """
             Output a Python RDD of key-value pairs (of form C{RDD[(K, V)]}) to any Hadoop file
    diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py
    index bd08c9a6d20d..df95ce962257 100644
    --- a/python/pyspark/serializers.py
    +++ b/python/pyspark/serializers.py
    @@ -70,6 +70,7 @@ class SpecialLengths(object):
         PYTHON_EXCEPTION_THROWN = -2
         TIMING_DATA = -3
         END_OF_STREAM = -4
    +    NULL = -5
     
     
     class Serializer(object):
    @@ -145,8 +146,10 @@ def _read_with_length(self, stream):
             length = read_int(stream)
             if length == SpecialLengths.END_OF_DATA_SECTION:
                 raise EOFError
    +        if length == SpecialLengths.NULL:
    +            return None
             obj = stream.read(length)
    -        if obj == "":
    +        if len(obj) < length:
                 raise EOFError
             return self.loads(obj)
     
    @@ -181,6 +184,10 @@ def __init__(self, serializer, batchSize=UNLIMITED_BATCH_SIZE):
         def _batched(self, iterator):
             if self.batchSize == self.UNLIMITED_BATCH_SIZE:
                 yield list(iterator)
    +        elif hasattr(iterator, "__len__") and hasattr(iterator, "__getslice__"):
    +            n = len(iterator)
    +            for i in xrange(0, n, self.batchSize):
    +                yield iterator[i: i + self.batchSize]
             else:
                 items = []
                 count = 0
    @@ -480,6 +487,8 @@ def loads(self, stream):
             length = read_int(stream)
             if length == SpecialLengths.END_OF_DATA_SECTION:
                 raise EOFError
    +        if length == SpecialLengths.NULL:
    +            return None
             s = stream.read(length)
             return s.decode("utf-8") if self.use_unicode else s
     
    diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py
    index 0e8b398fc6b9..1990323249cf 100644
    --- a/python/pyspark/sql.py
    +++ b/python/pyspark/sql.py
    @@ -807,14 +807,14 @@ def convert_struct(obj):
                 return
     
             if isinstance(obj, tuple):
    -            if hasattr(obj, "fields"):
    -                d = dict(zip(obj.fields, obj))
    -            if hasattr(obj, "__FIELDS__"):
    +            if hasattr(obj, "_fields"):
    +                d = dict(zip(obj._fields, obj))
    +            elif hasattr(obj, "__FIELDS__"):
                     d = dict(zip(obj.__FIELDS__, obj))
                 elif all(isinstance(x, tuple) and len(x) == 2 for x in obj):
                     d = dict(obj)
                 else:
    -                raise ValueError("unexpected tuple: %s" % obj)
    +                raise ValueError("unexpected tuple: %s" % str(obj))
     
             elif isinstance(obj, dict):
                 d = obj
    @@ -1281,14 +1281,14 @@ def registerFunction(self, name, f, returnType=StringType()):
                                          self._sc._gateway._gateway_client)
             includes = ListConverter().convert(self._sc._python_includes,
                                                self._sc._gateway._gateway_client)
    -        self._ssql_ctx.registerPython(name,
    -                                      bytearray(pickled_command),
    -                                      env,
    -                                      includes,
    -                                      self._sc.pythonExec,
    -                                      broadcast_vars,
    -                                      self._sc._javaAccumulator,
    -                                      returnType.json())
    +        self._ssql_ctx.udf().registerPython(name,
    +                                            bytearray(pickled_command),
    +                                            env,
    +                                            includes,
    +                                            self._sc.pythonExec,
    +                                            broadcast_vars,
    +                                            self._sc._javaAccumulator,
    +                                            returnType.json())
     
         def inferSchema(self, rdd, samplingRatio=None):
             """Infer and apply a schema to an RDD of L{Row}.
    @@ -1327,6 +1327,16 @@ def inferSchema(self, rdd, samplingRatio=None):
             >>> srdd = sqlCtx.inferSchema(nestedRdd2)
             >>> srdd.collect()
             [Row(f1=[[1, 2], [2, 3]], f2=[1, 2]), ..., f2=[2, 3])]
    +
    +        >>> from collections import namedtuple
    +        >>> CustomRow = namedtuple('CustomRow', 'field1 field2')
    +        >>> rdd = sc.parallelize(
    +        ...     [CustomRow(field1=1, field2="row1"),
    +        ...      CustomRow(field1=2, field2="row2"),
    +        ...      CustomRow(field1=3, field2="row3")])
    +        >>> srdd = sqlCtx.inferSchema(rdd)
    +        >>> srdd.collect()[0]
    +        Row(field1=1, field2=u'row1')
             """
     
             if isinstance(rdd, SchemaRDD):
    @@ -1448,7 +1458,7 @@ def applySchema(self, rdd, schema):
     
             jrdd = self._jvm.SerDeUtil.toJavaArray(rdd._to_java_object_rdd())
             srdd = self._ssql_ctx.applySchemaToPythonRDD(jrdd.rdd(), schema.json())
    -        return SchemaRDD(srdd.toJavaSchemaRDD(), self)
    +        return SchemaRDD(srdd, self)
     
         def registerRDDAsTable(self, rdd, tableName):
             """Registers the given RDD as a temporary table in the catalog.
    @@ -1477,7 +1487,7 @@ def parquetFile(self, path):
             >>> sorted(srdd.collect()) == sorted(srdd2.collect())
             True
             """
    -        jschema_rdd = self._ssql_ctx.parquetFile(path).toJavaSchemaRDD()
    +        jschema_rdd = self._ssql_ctx.parquetFile(path)
             return SchemaRDD(jschema_rdd, self)
     
         def jsonFile(self, path, schema=None, samplingRatio=1.0):
    @@ -1539,7 +1549,7 @@ def jsonFile(self, path, schema=None, samplingRatio=1.0):
             else:
                 scala_datatype = self._ssql_ctx.parseDataType(schema.json())
                 srdd = self._ssql_ctx.jsonFile(path, scala_datatype)
    -        return SchemaRDD(srdd.toJavaSchemaRDD(), self)
    +        return SchemaRDD(srdd, self)
     
         def jsonRDD(self, rdd, schema=None, samplingRatio=1.0):
             """Loads an RDD storing one JSON object per string as a L{SchemaRDD}.
    @@ -1609,7 +1619,7 @@ def func(iterator):
             else:
                 scala_datatype = self._ssql_ctx.parseDataType(schema.json())
                 srdd = self._ssql_ctx.jsonRDD(jrdd.rdd(), scala_datatype)
    -        return SchemaRDD(srdd.toJavaSchemaRDD(), self)
    +        return SchemaRDD(srdd, self)
     
         def sql(self, sqlQuery):
             """Return a L{SchemaRDD} representing the result of the given query.
    @@ -1620,7 +1630,7 @@ def sql(self, sqlQuery):
             >>> srdd2.collect()
             [Row(f1=1, f2=u'row1'), Row(f1=2, f2=u'row2'), Row(f1=3, f2=u'row3')]
             """
    -        return SchemaRDD(self._ssql_ctx.sql(sqlQuery).toJavaSchemaRDD(), self)
    +        return SchemaRDD(self._ssql_ctx.sql(sqlQuery), self)
     
         def table(self, tableName):
             """Returns the specified table as a L{SchemaRDD}.
    @@ -1631,7 +1641,7 @@ def table(self, tableName):
             >>> sorted(srdd.collect()) == sorted(srdd2.collect())
             True
             """
    -        return SchemaRDD(self._ssql_ctx.table(tableName).toJavaSchemaRDD(), self)
    +        return SchemaRDD(self._ssql_ctx.table(tableName), self)
     
         def cacheTable(self, tableName):
             """Caches the specified table in-memory."""
    @@ -1676,24 +1686,6 @@ def _ssql_ctx(self):
         def _get_hive_ctx(self):
             return self._jvm.HiveContext(self._jsc.sc())
     
    -    def hiveql(self, hqlQuery):
    -        """
    -        DEPRECATED: Use sql()
    -        """
    -        warnings.warn("hiveql() is deprecated as the sql function now parses using HiveQL by" +
    -                      "default. The SQL dialect for parsing can be set using 'spark.sql.dialect'",
    -                      DeprecationWarning)
    -        return SchemaRDD(self._ssql_ctx.hiveql(hqlQuery).toJavaSchemaRDD(), self)
    -
    -    def hql(self, hqlQuery):
    -        """
    -        DEPRECATED: Use sql()
    -        """
    -        warnings.warn("hql() is deprecated as the sql function now parses using HiveQL by" +
    -                      "default. The SQL dialect for parsing can be set using 'spark.sql.dialect'",
    -                      DeprecationWarning)
    -        return self.hiveql(hqlQuery)
    -
     
     class LocalHiveContext(HiveContext):
     
    @@ -1706,12 +1698,6 @@ def _get_hive_ctx(self):
             return self._jvm.LocalHiveContext(self._jsc.sc())
     
     
    -class TestHiveContext(HiveContext):
    -
    -    def _get_hive_ctx(self):
    -        return self._jvm.TestHiveContext(self._jsc.sc())
    -
    -
     def _create_row(fields, values):
         row = Row(*values)
         row.__FIELDS__ = fields
    @@ -1836,7 +1822,7 @@ def __init__(self, jschema_rdd, sql_ctx):
             self.sql_ctx = sql_ctx
             self._sc = sql_ctx._sc
             clsName = jschema_rdd.getClass().getName()
    -        assert clsName.endswith("JavaSchemaRDD"), "jschema_rdd must be JavaSchemaRDD"
    +        assert clsName.endswith("SchemaRDD"), "jschema_rdd must be SchemaRDD"
             self._jschema_rdd = jschema_rdd
             self._id = None
             self.is_cached = False
    @@ -1870,7 +1856,7 @@ def limit(self, num):
             >>> srdd.limit(0).collect()
             []
             """
    -        rdd = self._jschema_rdd.baseSchemaRDD().limit(num).toJavaSchemaRDD()
    +        rdd = self._jschema_rdd.baseSchemaRDD().limit(num)
             return SchemaRDD(rdd, self.sql_ctx)
     
         def toJSON(self, use_unicode=False):
    @@ -2049,18 +2035,18 @@ def isCheckpointed(self):
     
         def getCheckpointFile(self):
             checkpointFile = self._jschema_rdd.getCheckpointFile()
    -        if checkpointFile.isPresent():
    +        if checkpointFile.isDefined():
                 return checkpointFile.get()
     
         def coalesce(self, numPartitions, shuffle=False):
    -        rdd = self._jschema_rdd.coalesce(numPartitions, shuffle)
    +        rdd = self._jschema_rdd.coalesce(numPartitions, shuffle, None)
             return SchemaRDD(rdd, self.sql_ctx)
     
         def distinct(self, numPartitions=None):
             if numPartitions is None:
                 rdd = self._jschema_rdd.distinct()
             else:
    -            rdd = self._jschema_rdd.distinct(numPartitions)
    +            rdd = self._jschema_rdd.distinct(numPartitions, None)
             return SchemaRDD(rdd, self.sql_ctx)
     
         def intersection(self, other):
    @@ -2071,7 +2057,7 @@ def intersection(self, other):
                 raise ValueError("Can only intersect with another SchemaRDD")
     
         def repartition(self, numPartitions):
    -        rdd = self._jschema_rdd.repartition(numPartitions)
    +        rdd = self._jschema_rdd.repartition(numPartitions, None)
             return SchemaRDD(rdd, self.sql_ctx)
     
         def subtract(self, other, numPartitions=None):
    diff --git a/python/pyspark/streaming/dstream.py b/python/pyspark/streaming/dstream.py
    index 0826ddc56e84..2fe39392ff08 100644
    --- a/python/pyspark/streaming/dstream.py
    +++ b/python/pyspark/streaming/dstream.py
    @@ -157,18 +157,20 @@ def foreachRDD(self, func):
             api = self._ssc._jvm.PythonDStream
             api.callForeachRDD(self._jdstream, jfunc)
     
    -    def pprint(self):
    +    def pprint(self, num=10):
             """
    -        Print the first ten elements of each RDD generated in this DStream.
    +        Print the first num elements of each RDD generated in this DStream.
    +
    +        @param num: the number of elements from the first will be printed.
             """
             def takeAndPrint(time, rdd):
    -            taken = rdd.take(11)
    +            taken = rdd.take(num + 1)
                 print "-------------------------------------------"
                 print "Time: %s" % time
                 print "-------------------------------------------"
    -            for record in taken[:10]:
    +            for record in taken[:num]:
                     print record
    -            if len(taken) > 10:
    +            if len(taken) > num:
                     print "..."
                 print
     
    diff --git a/python/pyspark/streaming/kafka.py b/python/pyspark/streaming/kafka.py
    new file mode 100644
    index 000000000000..2e898c06fcf8
    --- /dev/null
    +++ b/python/pyspark/streaming/kafka.py
    @@ -0,0 +1,82 @@
    +#
    +# Licensed to the Apache Software Foundation (ASF) under one or more
    +# contributor license agreements.  See the NOTICE file distributed with
    +# this work for additional information regarding copyright ownership.
    +# The ASF licenses this file to You under the Apache License, Version 2.0
    +# (the "License"); you may not use this file except in compliance with
    +# the License.  You may obtain a copy of the License at
    +#
    +#    http://www.apache.org/licenses/LICENSE-2.0
    +#
    +# Unless required by applicable law or agreed to in writing, software
    +# distributed under the License is distributed on an "AS IS" BASIS,
    +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +# See the License for the specific language governing permissions and
    +# limitations under the License.
    +#
    +
    +from py4j.java_collections import MapConverter
    +from py4j.java_gateway import java_import, Py4JError
    +
    +from pyspark.storagelevel import StorageLevel
    +from pyspark.serializers import PairDeserializer, NoOpSerializer
    +from pyspark.streaming import DStream
    +
    +__all__ = ['KafkaUtils', 'utf8_decoder']
    +
    +
    +def utf8_decoder(s):
    +    """ Decode the unicode as UTF-8 """
    +    return s and s.decode('utf-8')
    +
    +
    +class KafkaUtils(object):
    +
    +    @staticmethod
    +    def createStream(ssc, zkQuorum, groupId, topics,
    +                     storageLevel=StorageLevel.MEMORY_AND_DISK_SER_2,
    +                     keyDecoder=utf8_decoder, valueDecoder=utf8_decoder):
    +        """
    +        Create an input stream that pulls messages from a Kafka Broker.
    +
    +        :param ssc:  StreamingContext object
    +        :param zkQuorum:  Zookeeper quorum (hostname:port,hostname:port,..).
    +        :param groupId:  The group id for this consumer.
    +        :param topics:  Dict of (topic_name -> numPartitions) to consume.
    +                        Each partition is consumed in its own thread.
    +        :param storageLevel:  RDD storage level.
    +        :param keyDecoder:  A function used to decode key
    +        :param valueDecoder:  A function used to decode value
    +        :return: A DStream object
    +        """
    +        java_import(ssc._jvm, "org.apache.spark.streaming.kafka.KafkaUtils")
    +
    +        param = {
    +            "zookeeper.connect": zkQuorum,
    +            "group.id": groupId,
    +            "zookeeper.connection.timeout.ms": "10000",
    +        }
    +        if not isinstance(topics, dict):
    +            raise TypeError("topics should be dict")
    +        jtopics = MapConverter().convert(topics, ssc.sparkContext._gateway._gateway_client)
    +        jparam = MapConverter().convert(param, ssc.sparkContext._gateway._gateway_client)
    +        jlevel = ssc._sc._getJavaStorageLevel(storageLevel)
    +
    +        def getClassByName(name):
    +            return ssc._jvm.org.apache.spark.util.Utils.classForName(name)
    +
    +        try:
    +            array = getClassByName("[B")
    +            decoder = getClassByName("kafka.serializer.DefaultDecoder")
    +            jstream = ssc._jvm.KafkaUtils.createStream(ssc._jssc, array, array, decoder, decoder,
    +                                                       jparam, jtopics, jlevel)
    +        except Py4JError, e:
    +            # TODO: use --jar once it also work on driver
    +            if not e.message or 'call a package' in e.message:
    +                print "No kafka package, please put the assembly jar into classpath:"
    +                print " $ bin/submit --driver-class-path external/kafka-assembly/target/" + \
    +                      "scala-*/spark-streaming-kafka-assembly-*.jar"
    +            raise e
    +        ser = PairDeserializer(NoOpSerializer(), NoOpSerializer())
    +        stream = DStream(jstream, ssc, ser)
    +        return stream.map(lambda (k, v): (keyDecoder(k), valueDecoder(v)))
    diff --git a/repl/pom.xml b/repl/pom.xml
    index 9b2290429fee..ae7c31aef4f5 100644
    --- a/repl/pom.xml
    +++ b/repl/pom.xml
    @@ -68,10 +68,6 @@
           ${project.version}
           test
         
    -    
    -      org.eclipse.jetty
    -      jetty-server
    -    
         
           org.scala-lang
           scala-compiler
    @@ -86,11 +82,6 @@
           org.slf4j
           jul-to-slf4j
         
    -    
    -      org.scalatest
    -      scalatest_${scala.binary.version}
    -      test
    -    
         
           org.scalacheck
           scalacheck_${scala.binary.version}
    @@ -101,29 +92,6 @@
         target/scala-${scala.binary.version}/classes
         target/scala-${scala.binary.version}/test-classes
         
    -      
    -        org.apache.maven.plugins
    -        maven-deploy-plugin
    -        
    -          true
    -        
    -      
    -      
    -        org.apache.maven.plugins
    -        maven-install-plugin
    -        
    -          true
    -        
    -      
    -      
    -        org.scalatest
    -        scalatest-maven-plugin
    -        
    -          
    -            ${basedir}/..
    -          
    -        
    -      
           
           
             org.codehaus.mojo
    diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkCommandLine.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkCommandLine.scala
    index 05816941b54b..6480e2d24e04 100644
    --- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkCommandLine.scala
    +++ b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkCommandLine.scala
    @@ -19,14 +19,21 @@ package org.apache.spark.repl
     
     import scala.tools.nsc.{Settings, CompilerCommand}
     import scala.Predef._
    +import org.apache.spark.annotation.DeveloperApi
     
     /**
      * Command class enabling Spark-specific command line options (provided by
      * org.apache.spark.repl.SparkRunnerSettings).
    + *
    + * @example new SparkCommandLine(Nil).settings
    + *
    + * @param args The list of command line arguments
    + * @param settings The underlying settings to associate with this set of
    + *                 command-line options
      */
    +@DeveloperApi
     class SparkCommandLine(args: List[String], override val settings: Settings)
         extends CompilerCommand(args, settings) {
    -
       def this(args: List[String], error: String => Unit) {
         this(args, new SparkRunnerSettings(error))
       }
    diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala
    index f8432c8af6ed..5fb378112ef9 100644
    --- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala
    +++ b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala
    @@ -15,7 +15,7 @@ import scala.tools.nsc.ast.parser.Tokens.EOF
     
     import org.apache.spark.Logging
     
    -trait SparkExprTyper extends Logging {
    +private[repl] trait SparkExprTyper extends Logging {
       val repl: SparkIMain
     
       import repl._
    diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkHelper.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkHelper.scala
    index 5340951d9133..955be17a73b8 100644
    --- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkHelper.scala
    +++ b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkHelper.scala
    @@ -17,6 +17,23 @@
     
     package scala.tools.nsc
     
    +import org.apache.spark.annotation.DeveloperApi
    +
    +// NOTE: Forced to be public (and in scala.tools.nsc package) to access the
    +//       settings "explicitParentLoader" method
    +
    +/**
    + * Provides exposure for the explicitParentLoader method on settings instances.
    + */
    +@DeveloperApi
     object SparkHelper {
    +  /**
    +   * Retrieves the explicit parent loader for the provided settings.
    +   *
    +   * @param settings The settings whose explicit parent loader to retrieve
    +   *
    +   * @return The Optional classloader representing the explicit parent loader
    +   */
    +  @DeveloperApi
       def explicitParentLoader(settings: Settings) = settings.explicitParentLoader
     }
    diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala
    index e56b74edba88..72c1a989999b 100644
    --- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala
    +++ b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala
    @@ -10,6 +10,8 @@ package org.apache.spark.repl
     
     import java.net.URL
     
    +import org.apache.spark.annotation.DeveloperApi
    +
     import scala.reflect.io.AbstractFile
     import scala.tools.nsc._
     import scala.tools.nsc.backend.JavaPlatform
    @@ -57,20 +59,22 @@ import org.apache.spark.util.Utils
      *  @author  Lex Spoon
      *  @version 1.2
      */
    -class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter,
    -               val master: Option[String])
    -                extends AnyRef
    -                   with LoopCommands
    -                   with SparkILoopInit
    -                   with Logging
    -{
    +@DeveloperApi
    +class SparkILoop(
    +    private val in0: Option[BufferedReader],
    +    protected val out: JPrintWriter,
    +    val master: Option[String]
    +) extends AnyRef with LoopCommands with SparkILoopInit with Logging {
       def this(in0: BufferedReader, out: JPrintWriter, master: String) = this(Some(in0), out, Some(master))
       def this(in0: BufferedReader, out: JPrintWriter) = this(Some(in0), out, None)
       def this() = this(None, new JPrintWriter(Console.out, true), None)
     
    -  var in: InteractiveReader = _   // the input stream from which commands come
    -  var settings: Settings = _
    -  var intp: SparkIMain = _
    +  private var in: InteractiveReader = _   // the input stream from which commands come
    +
    +  // NOTE: Exposed in package for testing
    +  private[repl] var settings: Settings = _
    +
    +  private[repl] var intp: SparkIMain = _
     
       @deprecated("Use `intp` instead.", "2.9.0") def interpreter = intp
       @deprecated("Use `intp` instead.", "2.9.0") def interpreter_= (i: SparkIMain): Unit = intp = i
    @@ -123,6 +127,8 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter,
         }
       }
     
    +  // NOTE: Must be public for visibility
    +  @DeveloperApi
       var sparkContext: SparkContext = _
     
       override def echoCommandMessage(msg: String) {
    @@ -130,45 +136,45 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter,
       }
     
       // def isAsync = !settings.Yreplsync.value
    -  def isAsync = false
    +  private[repl] def isAsync = false
       // lazy val power = new Power(intp, new StdReplVals(this))(tagOfStdReplVals, classTag[StdReplVals])
    -  def history = in.history
    +  private def history = in.history
     
       /** The context class loader at the time this object was created */
       protected val originalClassLoader = Utils.getContextOrSparkClassLoader
     
       // classpath entries added via :cp
    -  var addedClasspath: String = ""
    +  private var addedClasspath: String = ""
     
       /** A reverse list of commands to replay if the user requests a :replay */
    -  var replayCommandStack: List[String] = Nil
    +  private var replayCommandStack: List[String] = Nil
     
       /** A list of commands to replay if the user requests a :replay */
    -  def replayCommands = replayCommandStack.reverse
    +  private def replayCommands = replayCommandStack.reverse
     
       /** Record a command for replay should the user request a :replay */
    -  def addReplay(cmd: String) = replayCommandStack ::= cmd
    +  private def addReplay(cmd: String) = replayCommandStack ::= cmd
     
    -  def savingReplayStack[T](body: => T): T = {
    +  private def savingReplayStack[T](body: => T): T = {
         val saved = replayCommandStack
         try body
         finally replayCommandStack = saved
       }
    -  def savingReader[T](body: => T): T = {
    +  private def savingReader[T](body: => T): T = {
         val saved = in
         try body
         finally in = saved
       }
     
     
    -  def sparkCleanUp(){
    +  private def sparkCleanUp(){
         echo("Stopping spark context.")
         intp.beQuietDuring {
           command("sc.stop()")
         }
       }
       /** Close the interpreter and set the var to null. */
    -  def closeInterpreter() {
    +  private def closeInterpreter() {
         if (intp ne null) {
           sparkCleanUp()
           intp.close()
    @@ -179,14 +185,16 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter,
       class SparkILoopInterpreter extends SparkIMain(settings, out) {
         outer =>
     
    -    override lazy val formatting = new Formatting {
    +    override private[repl] lazy val formatting = new Formatting {
           def prompt = SparkILoop.this.prompt
         }
         override protected def parentClassLoader = SparkHelper.explicitParentLoader(settings).getOrElse(classOf[SparkILoop].getClassLoader)
       }
     
    -  /** Create a new interpreter. */
    -  def createInterpreter() {
    +  /**
    +   * Constructs a new interpreter.
    +   */
    +  protected def createInterpreter() {
         require(settings != null)
     
         if (addedClasspath != "") settings.classpath.append(addedClasspath)
    @@ -207,7 +215,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter,
       }
     
       /** print a friendly help message */
    -  def helpCommand(line: String): Result = {
    +  private def helpCommand(line: String): Result = {
         if (line == "") helpSummary()
         else uniqueCommand(line) match {
           case Some(lc) => echo("\n" + lc.longHelp)
    @@ -258,7 +266,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter,
       }
     
       /** Show the history */
    -  lazy val historyCommand = new LoopCommand("history", "show the history (optional num is commands to show)") {
    +  private lazy val historyCommand = new LoopCommand("history", "show the history (optional num is commands to show)") {
         override def usage = "[num]"
         def defaultLines = 20
     
    @@ -279,21 +287,21 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter,
     
       // When you know you are most likely breaking into the middle
       // of a line being typed.  This softens the blow.
    -  protected def echoAndRefresh(msg: String) = {
    +  private[repl] def echoAndRefresh(msg: String) = {
         echo("\n" + msg)
         in.redrawLine()
       }
    -  protected def echo(msg: String) = {
    +  private[repl] def echo(msg: String) = {
         out println msg
         out.flush()
       }
    -  protected def echoNoNL(msg: String) = {
    +  private def echoNoNL(msg: String) = {
         out print msg
         out.flush()
       }
     
       /** Search the history */
    -  def searchHistory(_cmdline: String) {
    +  private def searchHistory(_cmdline: String) {
         val cmdline = _cmdline.toLowerCase
         val offset  = history.index - history.size + 1
     
    @@ -302,14 +310,27 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter,
       }
     
       private var currentPrompt = Properties.shellPromptString
    +
    +  /**
    +   * Sets the prompt string used by the REPL.
    +   *
    +   * @param prompt The new prompt string
    +   */
    +  @DeveloperApi
       def setPrompt(prompt: String) = currentPrompt = prompt
    -  /** Prompt to print when awaiting input */
    +
    +  /**
    +   * Represents the current prompt string used by the REPL.
    +   *
    +   * @return The current prompt string
    +   */
    +  @DeveloperApi
       def prompt = currentPrompt
     
       import LoopCommand.{ cmd, nullary }
     
       /** Standard commands */
    -  lazy val standardCommands = List(
    +  private lazy val standardCommands = List(
         cmd("cp", "", "add a jar or directory to the classpath", addClasspath),
         cmd("help", "[command]", "print this summary or command-specific help", helpCommand),
         historyCommand,
    @@ -333,7 +354,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter,
       )
     
       /** Power user commands */
    -  lazy val powerCommands: List[LoopCommand] = List(
    +  private lazy val powerCommands: List[LoopCommand] = List(
         // cmd("phase", "", "set the implicit phase for power commands", phaseCommand)
       )
     
    @@ -459,7 +480,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter,
         }
       }
     
    -  protected def newJavap() = new JavapClass(addToolsJarToLoader(), new SparkIMain.ReplStrippingWriter(intp)) {
    +  private def newJavap() = new JavapClass(addToolsJarToLoader(), new SparkIMain.ReplStrippingWriter(intp)) {
         override def tryClass(path: String): Array[Byte] = {
           val hd :: rest = path split '.' toList;
           // If there are dots in the name, the first segment is the
    @@ -581,7 +602,12 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter,
       //   }
       // }
     
    -  /** Available commands */
    +  /**
    +   * Provides a list of available commands.
    +   *
    +   * @return The list of commands
    +   */
    +  @DeveloperApi
       def commands: List[LoopCommand] = standardCommands /*++ (
         if (isReplPower) powerCommands else Nil
       )*/
    @@ -613,7 +639,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter,
        *  command() for each line of input, and stops when
        *  command() returns false.
        */
    -  def loop() {
    +  private def loop() {
         def readOneLine() = {
           out.flush()
           in readLine prompt
    @@ -642,7 +668,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter,
       }
     
       /** interpret all lines from a specified file */
    -  def interpretAllFrom(file: File) {
    +  private def interpretAllFrom(file: File) {
         savingReader {
           savingReplayStack {
             file applyReader { reader =>
    @@ -655,7 +681,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter,
       }
     
       /** create a new interpreter and replay the given commands */
    -  def replay() {
    +  private def replay() {
         reset()
         if (replayCommandStack.isEmpty)
           echo("Nothing to replay.")
    @@ -665,7 +691,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter,
           echo("")
         }
       }
    -  def resetCommand() {
    +  private def resetCommand() {
         echo("Resetting repl state.")
         if (replayCommandStack.nonEmpty) {
           echo("Forgetting this session history:\n")
    @@ -681,13 +707,13 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter,
         reset()
       }
     
    -  def reset() {
    +  private def reset() {
         intp.reset()
         // unleashAndSetPhase()
       }
     
       /** fork a shell and run a command */
    -  lazy val shCommand = new LoopCommand("sh", "run a shell command (result is implicitly => List[String])") {
    +  private lazy val shCommand = new LoopCommand("sh", "run a shell command (result is implicitly => List[String])") {
         override def usage = ""
         def apply(line: String): Result = line match {
           case ""   => showUsage()
    @@ -698,14 +724,14 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter,
         }
       }
     
    -  def withFile(filename: String)(action: File => Unit) {
    +  private def withFile(filename: String)(action: File => Unit) {
         val f = File(filename)
     
         if (f.exists) action(f)
         else echo("That file does not exist")
       }
     
    -  def loadCommand(arg: String) = {
    +  private def loadCommand(arg: String) = {
         var shouldReplay: Option[String] = None
         withFile(arg)(f => {
           interpretAllFrom(f)
    @@ -714,7 +740,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter,
         Result(true, shouldReplay)
       }
     
    -  def addAllClasspath(args: Seq[String]): Unit = {
    +  private def addAllClasspath(args: Seq[String]): Unit = {
         var added = false
         var totalClasspath = ""
         for (arg <- args) {
    @@ -729,7 +755,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter,
         }
       }
     
    -  def addClasspath(arg: String): Unit = {
    +  private def addClasspath(arg: String): Unit = {
         val f = File(arg).normalize
         if (f.exists) {
           addedClasspath = ClassPath.join(addedClasspath, f.path)
    @@ -741,12 +767,12 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter,
       }
     
     
    -  def powerCmd(): Result = {
    +  private def powerCmd(): Result = {
         if (isReplPower) "Already in power mode."
         else enablePowerMode(false)
       }
     
    -  def enablePowerMode(isDuringInit: Boolean) = {
    +  private[repl] def enablePowerMode(isDuringInit: Boolean) = {
         // replProps.power setValue true
         // unleashAndSetPhase()
         // asyncEcho(isDuringInit, power.banner)
    @@ -759,12 +785,12 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter,
     //     }
     //   }
     
    -  def asyncEcho(async: Boolean, msg: => String) {
    +  private def asyncEcho(async: Boolean, msg: => String) {
         if (async) asyncMessage(msg)
         else echo(msg)
       }
     
    -  def verbosity() = {
    +  private def verbosity() = {
         // val old = intp.printResults
         // intp.printResults = !old
         // echo("Switched " + (if (old) "off" else "on") + " result printing.")
    @@ -773,7 +799,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter,
       /** Run one command submitted by the user.  Two values are returned:
         * (1) whether to keep running, (2) the line to record for replay,
         * if any. */
    -  def command(line: String): Result = {
    +  private[repl] def command(line: String): Result = {
         if (line startsWith ":") {
           val cmd = line.tail takeWhile (x => !x.isWhitespace)
           uniqueCommand(cmd) match {
    @@ -789,7 +815,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter,
         Iterator continually in.readLine("") takeWhile (x => x != null && cond(x))
       }
     
    -  def pasteCommand(): Result = {
    +  private def pasteCommand(): Result = {
         echo("// Entering paste mode (ctrl-D to finish)\n")
         val code = readWhile(_ => true) mkString "\n"
         echo("\n// Exiting paste mode, now interpreting.\n")
    @@ -820,7 +846,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter,
         * read, go ahead and interpret it.  Return the full string
         * to be recorded for replay, if any.
         */
    -  def interpretStartingWith(code: String): Option[String] = {
    +  private def interpretStartingWith(code: String): Option[String] = {
         // signal completion non-completion input has been received
         in.completion.resetVerbosity()
     
    @@ -874,7 +900,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter,
       }
     
       // runs :load `file` on any files passed via -i
    -  def loadFiles(settings: Settings) = settings match {
    +  private def loadFiles(settings: Settings) = settings match {
         case settings: SparkRunnerSettings =>
           for (filename <- settings.loadfiles.value) {
             val cmd = ":load " + filename
    @@ -889,7 +915,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter,
        *  unless settings or properties are such that it should start
        *  with SimpleReader.
        */
    -  def chooseReader(settings: Settings): InteractiveReader = {
    +  private def chooseReader(settings: Settings): InteractiveReader = {
         if (settings.Xnojline.value || Properties.isEmacsShell)
           SimpleReader()
         else try new SparkJLineReader(
    @@ -903,8 +929,8 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter,
         }
       }
     
    -  val u: scala.reflect.runtime.universe.type = scala.reflect.runtime.universe
    -  val m = u.runtimeMirror(Utils.getSparkClassLoader)
    +  private val u: scala.reflect.runtime.universe.type = scala.reflect.runtime.universe
    +  private val m = u.runtimeMirror(Utils.getSparkClassLoader)
       private def tagOfStaticClass[T: ClassTag]: u.TypeTag[T] =
         u.TypeTag[T](
           m,
    @@ -913,7 +939,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter,
               m.staticClass(classTag[T].runtimeClass.getName).toTypeConstructor.asInstanceOf[U # Type]
           })
     
    -  def process(settings: Settings): Boolean = savingContextLoader {
    +  private def process(settings: Settings): Boolean = savingContextLoader {
         if (getMaster() == "yarn-client") System.setProperty("SPARK_YARN_MODE", "true")
     
         this.settings = settings
    @@ -972,6 +998,8 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter,
         true
       }
     
    +  // NOTE: Must be public for visibility
    +  @DeveloperApi
       def createSparkContext(): SparkContext = {
         val execUri = System.getenv("SPARK_EXECUTOR_URI")
         val jars = SparkILoop.getAddedJars
    @@ -979,7 +1007,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter,
           .setMaster(getMaster())
           .setAppName("Spark shell")
           .setJars(jars)
    -      .set("spark.repl.class.uri", intp.classServer.uri)
    +      .set("spark.repl.class.uri", intp.classServerUri)
         if (execUri != null) {
           conf.set("spark.executor.uri", execUri)
         }
    @@ -1014,7 +1042,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter,
       }
     
       @deprecated("Use `process` instead", "2.9.0")
    -  def main(settings: Settings): Unit = process(settings)
    +  private def main(settings: Settings): Unit = process(settings)
     }
     
     object SparkILoop {
    @@ -1033,7 +1061,7 @@ object SparkILoop {
       // Designed primarily for use by test code: take a String with a
       // bunch of code, and prints out a transcript of what it would look
       // like if you'd just typed it into the repl.
    -  def runForTranscript(code: String, settings: Settings): String = {
    +  private[repl] def runForTranscript(code: String, settings: Settings): String = {
         import java.io.{ BufferedReader, StringReader, OutputStreamWriter }
     
         stringFromStream { ostream =>
    @@ -1071,7 +1099,7 @@ object SparkILoop {
       /** Creates an interpreter loop with default settings and feeds
        *  the given code to it as input.
        */
    -  def run(code: String, sets: Settings = new Settings): String = {
    +  private[repl] def run(code: String, sets: Settings = new Settings): String = {
         import java.io.{ BufferedReader, StringReader, OutputStreamWriter }
     
         stringFromStream { ostream =>
    @@ -1087,5 +1115,5 @@ object SparkILoop {
           }
         }
       }
    -  def run(lines: List[String]): String = run(lines map (_ + "\n") mkString)
    +  private[repl] def run(lines: List[String]): String = run(lines map (_ + "\n") mkString)
     }
    diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala
    index da4286c5e487..99bd777c04fd 100644
    --- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala
    +++ b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala
    @@ -19,7 +19,7 @@ import org.apache.spark.SPARK_VERSION
     /**
      *  Machinery for the asynchronous initialization of the repl.
      */
    -trait SparkILoopInit {
    +private[repl] trait SparkILoopInit {
       self: SparkILoop =>
     
       /** Print a welcome message */
    diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkIMain.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkIMain.scala
    index 646c68e60c2e..35fb62564502 100644
    --- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkIMain.scala
    +++ b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkIMain.scala
    @@ -39,6 +39,7 @@ import scala.util.control.ControlThrowable
     
     import org.apache.spark.{Logging, HttpServer, SecurityManager, SparkConf}
     import org.apache.spark.util.Utils
    +import org.apache.spark.annotation.DeveloperApi
     
     // /** directory to save .class files to */
     // private class ReplVirtualDirectory(out: JPrintWriter) extends VirtualDirectory("((memory))", None) {
    @@ -84,17 +85,18 @@ import org.apache.spark.util.Utils
        *  @author Moez A. Abdel-Gawad
        *  @author Lex Spoon
        */
    +  @DeveloperApi
       class SparkIMain(
           initialSettings: Settings,
           val out: JPrintWriter,
           propagateExceptions: Boolean = false)
         extends SparkImports with Logging { imain =>
     
    -    val conf = new SparkConf()
    +    private val conf = new SparkConf()
     
    -    val SPARK_DEBUG_REPL: Boolean = (System.getenv("SPARK_DEBUG_REPL") == "1")
    +    private val SPARK_DEBUG_REPL: Boolean = (System.getenv("SPARK_DEBUG_REPL") == "1")
         /** Local directory to save .class files too */
    -    lazy val outputDir = {
    +    private lazy val outputDir = {
           val tmp = System.getProperty("java.io.tmpdir")
           val rootDir = conf.get("spark.repl.classdir",  tmp)
           Utils.createTempDir(rootDir)
    @@ -103,13 +105,20 @@ import org.apache.spark.util.Utils
           echo("Output directory: " + outputDir)
         }
     
    -    val virtualDirectory                              = new PlainFile(outputDir) // "directory" for classfiles
    +    /**
    +     * Returns the path to the output directory containing all generated
    +     * class files that will be served by the REPL class server.
    +     */
    +    @DeveloperApi
    +    lazy val getClassOutputDirectory = outputDir
    +
    +    private val virtualDirectory                              = new PlainFile(outputDir) // "directory" for classfiles
         /** Jetty server that will serve our classes to worker nodes */
    -    val classServerPort                               = conf.getInt("spark.replClassServer.port", 0)
    -    val classServer                                   = new HttpServer(outputDir, new SecurityManager(conf), classServerPort, "HTTP class server")
    +    private val classServerPort                               = conf.getInt("spark.replClassServer.port", 0)
    +    private val classServer                                   = new HttpServer(conf, outputDir, new SecurityManager(conf), classServerPort, "HTTP class server")
         private var currentSettings: Settings             = initialSettings
    -    var printResults                                  = true      // whether to print result lines
    -    var totalSilence                                  = false     // whether to print anything
    +    private var printResults                                  = true      // whether to print result lines
    +    private var totalSilence                                  = false     // whether to print anything
         private var _initializeComplete                   = false     // compiler is initialized
         private var _isInitialized: Future[Boolean]       = null      // set up initialization future
         private var bindExceptions                        = true      // whether to bind the lastException variable
    @@ -123,6 +132,14 @@ import org.apache.spark.util.Utils
           echo("Class server started, URI = " + classServer.uri)
         }
     
    +    /**
    +     * URI of the class server used to feed REPL compiled classes.
    +     *
    +     * @return The string representing the class server uri
    +     */
    +    @DeveloperApi
    +    def classServerUri = classServer.uri
    +
         /** We're going to go to some trouble to initialize the compiler asynchronously.
          *  It's critical that nothing call into it until it's been initialized or we will
          *  run into unrecoverable issues, but the perceived repl startup time goes
    @@ -141,17 +158,18 @@ import org.apache.spark.util.Utils
           () => { counter += 1 ; counter }
         }
     
    -    def compilerClasspath: Seq[URL] = (
    +    private def compilerClasspath: Seq[URL] = (
           if (isInitializeComplete) global.classPath.asURLs
           else new PathResolver(settings).result.asURLs  // the compiler's classpath
           )
    -    def settings = currentSettings
    -    def mostRecentLine = prevRequestList match {
    +    // NOTE: Exposed to repl package since accessed indirectly from SparkIMain
    +    private[repl] def settings = currentSettings
    +    private def mostRecentLine = prevRequestList match {
           case Nil      => ""
           case req :: _ => req.originalLine
         }
         // Run the code body with the given boolean settings flipped to true.
    -    def withoutWarnings[T](body: => T): T = beQuietDuring {
    +    private def withoutWarnings[T](body: => T): T = beQuietDuring {
           val saved = settings.nowarn.value
           if (!saved)
             settings.nowarn.value = true
    @@ -164,16 +182,28 @@ import org.apache.spark.util.Utils
         def this(settings: Settings) = this(settings, new NewLinePrintWriter(new ConsoleWriter, true))
         def this() = this(new Settings())
     
    -    lazy val repllog: Logger = new Logger {
    +    private lazy val repllog: Logger = new Logger {
           val out: JPrintWriter = imain.out
           val isInfo: Boolean  = BooleanProp keyExists "scala.repl.info"
           val isDebug: Boolean = BooleanProp keyExists "scala.repl.debug"
           val isTrace: Boolean = BooleanProp keyExists "scala.repl.trace"
         }
    -    lazy val formatting: Formatting = new Formatting {
    +    private[repl] lazy val formatting: Formatting = new Formatting {
           val prompt = Properties.shellPromptString
         }
    -    lazy val reporter: ConsoleReporter = new SparkIMain.ReplReporter(this)
    +
    +    // NOTE: Exposed to repl package since used by SparkExprTyper and SparkILoop
    +    private[repl] lazy val reporter: ConsoleReporter = new SparkIMain.ReplReporter(this)
    +
    +    /**
    +     * Determines if errors were reported (typically during compilation).
    +     *
    +     * @note This is not for runtime errors
    +     *
    +     * @return True if had errors, otherwise false
    +     */
    +    @DeveloperApi
    +    def isReportingErrors = reporter.hasErrors
     
         import formatting._
         import reporter.{ printMessage, withoutTruncating }
    @@ -193,7 +223,8 @@ import org.apache.spark.util.Utils
         private def tquoted(s: String) = "\"\"\"" + s + "\"\"\""
     
         // argument is a thunk to execute after init is done
    -    def initialize(postInitSignal: => Unit) {
    +    // NOTE: Exposed to repl package since used by SparkILoop
    +    private[repl] def initialize(postInitSignal: => Unit) {
           synchronized {
             if (_isInitialized == null) {
               _isInitialized = io.spawn {
    @@ -203,15 +234,27 @@ import org.apache.spark.util.Utils
             }
           }
         }
    +
    +    /**
    +     * Initializes the underlying compiler/interpreter in a blocking fashion.
    +     *
    +     * @note Must be executed before using SparkIMain!
    +     */
    +    @DeveloperApi
         def initializeSynchronous(): Unit = {
           if (!isInitializeComplete) {
             _initialize()
             assert(global != null, global)
           }
         }
    -    def isInitializeComplete = _initializeComplete
    +    private def isInitializeComplete = _initializeComplete
     
         /** the public, go through the future compiler */
    +
    +    /**
    +     * The underlying compiler used to generate ASTs and execute code.
    +     */
    +    @DeveloperApi
         lazy val global: Global = {
           if (isInitializeComplete) _compiler
           else {
    @@ -226,13 +269,13 @@ import org.apache.spark.util.Utils
           }
         }
         @deprecated("Use `global` for access to the compiler instance.", "2.9.0")
    -    lazy val compiler: global.type = global
    +    private lazy val compiler: global.type = global
     
         import global._
         import definitions.{ScalaPackage, JavaLangPackage, termMember, typeMember}
         import rootMirror.{RootClass, getClassIfDefined, getModuleIfDefined, getRequiredModule, getRequiredClass}
     
    -    implicit class ReplTypeOps(tp: Type) {
    +    private implicit class ReplTypeOps(tp: Type) {
           def orElse(other: => Type): Type    = if (tp ne NoType) tp else other
           def andAlso(fn: Type => Type): Type = if (tp eq NoType) tp else fn(tp)
         }
    @@ -240,7 +283,8 @@ import org.apache.spark.util.Utils
         // TODO: If we try to make naming a lazy val, we run into big time
         // scalac unhappiness with what look like cycles.  It has not been easy to
         // reduce, but name resolution clearly takes different paths.
    -    object naming extends {
    +    // NOTE: Exposed to repl package since used by SparkExprTyper
    +    private[repl] object naming extends {
           val global: imain.global.type = imain.global
         } with Naming {
           // make sure we don't overwrite their unwisely named res3 etc.
    @@ -254,22 +298,43 @@ import org.apache.spark.util.Utils
         }
         import naming._
     
    -    object deconstruct extends {
    +    // NOTE: Exposed to repl package since used by SparkILoop
    +    private[repl] object deconstruct extends {
           val global: imain.global.type = imain.global
         } with StructuredTypeStrings
     
    -    lazy val memberHandlers = new {
    +    // NOTE: Exposed to repl package since used by SparkImports
    +    private[repl] lazy val memberHandlers = new {
           val intp: imain.type = imain
         } with SparkMemberHandlers
         import memberHandlers._
     
    -    /** Temporarily be quiet */
    +    /**
    +     * Suppresses overwriting print results during the operation.
    +     *
    +     * @param body The block to execute
    +     * @tparam T The return type of the block
    +     *
    +     * @return The result from executing the block
    +     */
    +    @DeveloperApi
         def beQuietDuring[T](body: => T): T = {
           val saved = printResults
           printResults = false
           try body
           finally printResults = saved
         }
    +
    +    /**
    +     * Completely masks all output during the operation (minus JVM standard
    +     * out and error).
    +     *
    +     * @param operation The block to execute
    +     * @tparam T The return type of the block
    +     *
    +     * @return The result from executing the block
    +     */
    +    @DeveloperApi
         def beSilentDuring[T](operation: => T): T = {
           val saved = totalSilence
           totalSilence = true
    @@ -277,10 +342,10 @@ import org.apache.spark.util.Utils
           finally totalSilence = saved
         }
     
    -    def quietRun[T](code: String) = beQuietDuring(interpret(code))
    +    // NOTE: Exposed to repl package since used by SparkILoop
    +    private[repl] def quietRun[T](code: String) = beQuietDuring(interpret(code))
     
    -
    -     private def logAndDiscard[T](label: String, alt: => T): PartialFunction[Throwable, T] = {
    +    private def logAndDiscard[T](label: String, alt: => T): PartialFunction[Throwable, T] = {
           case t: ControlThrowable => throw t
           case t: Throwable        =>
             logDebug(label + ": " + unwrap(t))
    @@ -298,14 +363,44 @@ import org.apache.spark.util.Utils
           finally bindExceptions = true
         }
     
    +    /**
    +     * Contains the code (in string form) representing a wrapper around all
    +     * code executed by this instance.
    +     *
    +     * @return The wrapper code as a string
    +     */
    +    @DeveloperApi
         def executionWrapper = _executionWrapper
    +
    +    /**
    +     * Sets the code to use as a wrapper around all code executed by this
    +     * instance.
    +     *
    +     * @param code The wrapper code as a string
    +     */
    +    @DeveloperApi
         def setExecutionWrapper(code: String) = _executionWrapper = code
    +
    +    /**
    +     * Clears the code used as a wrapper around all code executed by
    +     * this instance.
    +     */
    +    @DeveloperApi
         def clearExecutionWrapper() = _executionWrapper = ""
     
         /** interpreter settings */
    -    lazy val isettings = new SparkISettings(this)
    +    private lazy val isettings = new SparkISettings(this)
     
    -    /** Instantiate a compiler.  Overridable. */
    +    /**
    +     * Instantiates a new compiler used by SparkIMain. Overridable to provide
    +     * own instance of a compiler.
    +     *
    +     * @param settings The settings to provide the compiler
    +     * @param reporter The reporter to use for compiler output
    +     *
    +     * @return The compiler as a Global
    +     */
    +    @DeveloperApi
         protected def newCompiler(settings: Settings, reporter: Reporter): ReplGlobal = {
           settings.outputDirs setSingleOutput virtualDirectory
           settings.exposeEmptyPackage.value = true
    @@ -320,13 +415,14 @@ import org.apache.spark.util.Utils
          * @note Currently only supports jars, not directories
          * @param urls The list of items to add to the compile and runtime classpaths
          */
    +    @DeveloperApi
         def addUrlsToClassPath(urls: URL*): Unit = {
           new Run // Needed to force initialization of "something" to correctly load Scala classes from jars
           urls.foreach(_runtimeClassLoader.addNewUrl) // Add jars/classes to runtime for execution
           updateCompilerClassPath(urls: _*)           // Add jars/classes to compile time for compiling
         }
     
    -    protected def updateCompilerClassPath(urls: URL*): Unit = {
    +    private def updateCompilerClassPath(urls: URL*): Unit = {
           require(!global.forMSIL) // Only support JavaPlatform
     
           val platform = global.platform.asInstanceOf[JavaPlatform]
    @@ -342,7 +438,7 @@ import org.apache.spark.util.Utils
           global.invalidateClassPathEntries(urls.map(_.getPath): _*)
         }
     
    -    protected def mergeUrlsIntoClassPath(platform: JavaPlatform, urls: URL*): MergedClassPath[AbstractFile] = {
    +    private def mergeUrlsIntoClassPath(platform: JavaPlatform, urls: URL*): MergedClassPath[AbstractFile] = {
           // Collect our new jars/directories and add them to the existing set of classpaths
           val allClassPaths = (
             platform.classPath.asInstanceOf[MergedClassPath[AbstractFile]].entries ++
    @@ -365,7 +461,13 @@ import org.apache.spark.util.Utils
           new MergedClassPath(allClassPaths, platform.classPath.context)
         }
     
    -    /** Parent classloader.  Overridable. */
    +    /**
    +     * Represents the parent classloader used by this instance. Can be
    +     * overridden to provide alternative classloader.
    +     *
    +     * @return The classloader used as the parent loader of this instance
    +     */
    +    @DeveloperApi
         protected def parentClassLoader: ClassLoader =
           SparkHelper.explicitParentLoader(settings).getOrElse( this.getClass.getClassLoader() )
     
    @@ -382,16 +484,18 @@ import org.apache.spark.util.Utils
         shadow the old ones, and old code objects refer to the old
         definitions.
         */
    -    def resetClassLoader() = {
    +    private def resetClassLoader() = {
           logDebug("Setting new classloader: was " + _classLoader)
           _classLoader = null
           ensureClassLoader()
         }
    -    final def ensureClassLoader() {
    +    private final def ensureClassLoader() {
           if (_classLoader == null)
             _classLoader = makeClassLoader()
         }
    -    def classLoader: AbstractFileClassLoader = {
    +
    +    // NOTE: Exposed to repl package since used by SparkILoop
    +    private[repl] def classLoader: AbstractFileClassLoader = {
           ensureClassLoader()
           _classLoader
         }
    @@ -418,27 +522,58 @@ import org.apache.spark.util.Utils
               _runtimeClassLoader
           })
     
    -    def getInterpreterClassLoader() = classLoader
    +    private def getInterpreterClassLoader() = classLoader
     
         // Set the current Java "context" class loader to this interpreter's class loader
    -    def setContextClassLoader() = classLoader.setAsContext()
    +    // NOTE: Exposed to repl package since used by SparkILoopInit
    +    private[repl] def setContextClassLoader() = classLoader.setAsContext()
     
    -    /** Given a simple repl-defined name, returns the real name of
    -     *  the class representing it, e.g. for "Bippy" it may return
    -     *  {{{
    -     *    $line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$Bippy
    -     *  }}}
    +    /**
    +     * Returns the real name of a class based on its repl-defined name.
    +     *
    +     * ==Example==
    +     * Given a simple repl-defined name, returns the real name of
    +     * the class representing it, e.g. for "Bippy" it may return
    +     * {{{
    +     *     $line19.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$Bippy
    +     * }}}
    +     *
    +     * @param simpleName The repl-defined name whose real name to retrieve
    +     *
    +     * @return Some real name if the simple name exists, else None
          */
    +    @DeveloperApi
         def generatedName(simpleName: String): Option[String] = {
           if (simpleName endsWith nme.MODULE_SUFFIX_STRING) optFlatName(simpleName.init) map (_ + nme.MODULE_SUFFIX_STRING)
           else optFlatName(simpleName)
         }
    -    def flatName(id: String)    = optFlatName(id) getOrElse id
    -    def optFlatName(id: String) = requestForIdent(id) map (_ fullFlatName id)
     
    +    // NOTE: Exposed to repl package since used by SparkILoop
    +    private[repl] def flatName(id: String)    = optFlatName(id) getOrElse id
    +    // NOTE: Exposed to repl package since used by SparkILoop
    +    private[repl] def optFlatName(id: String) = requestForIdent(id) map (_ fullFlatName id)
    +
    +    /**
    +     * Retrieves all simple names contained in the current instance.
    +     *
    +     * @return A list of sorted names
    +     */
    +    @DeveloperApi
         def allDefinedNames = definedNameMap.keys.toList.sorted
    -    def pathToType(id: String): String = pathToName(newTypeName(id))
    -    def pathToTerm(id: String): String = pathToName(newTermName(id))
    +
    +    private def pathToType(id: String): String = pathToName(newTypeName(id))
    +    // NOTE: Exposed to repl package since used by SparkILoop
    +    private[repl] def pathToTerm(id: String): String = pathToName(newTermName(id))
    +
    +    /**
    +     * Retrieves the full code path to access the specified simple name
    +     * content.
    +     *
    +     * @param name The simple name of the target whose path to determine
    +     *
    +     * @return The full path used to access the specified target (name)
    +     */
    +    @DeveloperApi
         def pathToName(name: Name): String = {
           if (definedNameMap contains name)
             definedNameMap(name) fullPath name
    @@ -457,13 +592,13 @@ import org.apache.spark.util.Utils
         }
     
         /** Stubs for work in progress. */
    -    def handleTypeRedefinition(name: TypeName, old: Request, req: Request) = {
    +    private def handleTypeRedefinition(name: TypeName, old: Request, req: Request) = {
           for (t1 <- old.simpleNameOfType(name) ; t2 <- req.simpleNameOfType(name)) {
             logDebug("Redefining type '%s'\n  %s -> %s".format(name, t1, t2))
           }
         }
     
    -    def handleTermRedefinition(name: TermName, old: Request, req: Request) = {
    +    private def handleTermRedefinition(name: TermName, old: Request, req: Request) = {
           for (t1 <- old.compilerTypeOf get name ; t2 <- req.compilerTypeOf get name) {
         //    Printing the types here has a tendency to cause assertion errors, like
             //   assertion failed: fatal:  has owner value x, but a class owner is required
    @@ -473,7 +608,7 @@ import org.apache.spark.util.Utils
           }
         }
     
    -    def recordRequest(req: Request) {
    +    private def recordRequest(req: Request) {
           if (req == null || referencedNameMap == null)
             return
     
    @@ -504,12 +639,12 @@ import org.apache.spark.util.Utils
           }
         }
     
    -    def replwarn(msg: => String) {
    +    private def replwarn(msg: => String) {
           if (!settings.nowarnings.value)
             printMessage(msg)
         }
     
    -    def isParseable(line: String): Boolean = {
    +    private def isParseable(line: String): Boolean = {
           beSilentDuring {
             try parse(line) match {
               case Some(xs) => xs.nonEmpty  // parses as-is
    @@ -522,22 +657,32 @@ import org.apache.spark.util.Utils
           }
         }
     
    -    def compileSourcesKeepingRun(sources: SourceFile*) = {
    +    private def compileSourcesKeepingRun(sources: SourceFile*) = {
           val run = new Run()
           reporter.reset()
           run compileSources sources.toList
           (!reporter.hasErrors, run)
         }
     
    -    /** Compile an nsc SourceFile.  Returns true if there are
    -     *  no compilation errors, or false otherwise.
    +    /**
    +     * Compiles specified source files.
    +     *
    +     * @param sources The sequence of source files to compile
    +     *
    +     * @return True if successful, otherwise false
          */
    +    @DeveloperApi
         def compileSources(sources: SourceFile*): Boolean =
           compileSourcesKeepingRun(sources: _*)._1
     
    -    /** Compile a string.  Returns true if there are no
    -     *  compilation errors, or false otherwise.
    +    /**
    +     * Compiles a string of code.
    +     *
    +     * @param code The string of code to compile
    +     *
    +     * @return True if successful, otherwise false
          */
    +    @DeveloperApi
         def compileString(code: String): Boolean =
           compileSources(new BatchSourceFile("