diff --git a/.rat-excludes b/.rat-excludes index 0240e81c45ea2..236c2db05367c 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -91,3 +91,5 @@ help/* html/* INDEX .lintr +gen-java.* +.*avpr diff --git a/R/README.md b/R/README.md index d7d65b4f0eca5..005f56da1670c 100644 --- a/R/README.md +++ b/R/README.md @@ -6,7 +6,7 @@ SparkR is an R package that provides a light-weight frontend to use Spark from R #### Build Spark -Build Spark with [Maven](http://spark.apache.org/docs/latest/building-spark.html#building-with-buildmvn) and include the `-PsparkR` profile to build the R package. For example to use the default Hadoop versions you can run +Build Spark with [Maven](http://spark.apache.org/docs/latest/building-spark.html#building-with-buildmvn) and include the `-Psparkr` profile to build the R package. For example to use the default Hadoop versions you can run ``` build/mvn -DskipTests -Psparkr package ``` diff --git a/R/install-dev.bat b/R/install-dev.bat index 008a5c668bc45..f32670b67de96 100644 --- a/R/install-dev.bat +++ b/R/install-dev.bat @@ -25,3 +25,8 @@ set SPARK_HOME=%~dp0.. MKDIR %SPARK_HOME%\R\lib R.exe CMD INSTALL --library="%SPARK_HOME%\R\lib" %SPARK_HOME%\R\pkg\ + +rem Zip the SparkR package so that it can be distributed to worker nodes on YARN +pushd %SPARK_HOME%\R\lib +%JAVA_HOME%\bin\jar.exe cfM "%SPARK_HOME%\R\lib\sparkr.zip" SparkR +popd diff --git a/R/install-dev.sh b/R/install-dev.sh index 1edd551f8d243..4972bb9217072 100755 --- a/R/install-dev.sh +++ b/R/install-dev.sh @@ -34,7 +34,7 @@ LIB_DIR="$FWDIR/lib" mkdir -p $LIB_DIR -pushd $FWDIR +pushd $FWDIR > /dev/null # Generate Rd files if devtools is installed Rscript -e ' if("devtools" %in% rownames(installed.packages())) { library(devtools); devtools::document(pkg="./pkg", roclets=c("rd")) }' @@ -42,4 +42,8 @@ Rscript -e ' if("devtools" %in% rownames(installed.packages())) { library(devtoo # Install SparkR to $LIB_DIR R CMD INSTALL --library=$LIB_DIR $FWDIR/pkg/ -popd +# Zip the SparkR package so that it can be distributed to worker nodes on YARN +cd $LIB_DIR +jar cfM "$LIB_DIR/sparkr.zip" SparkR + +popd > /dev/null diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index efc85bbc4b316..d028821534b1a 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -32,4 +32,3 @@ Collate: 'serialize.R' 'sparkR.R' 'utils.R' - 'zzz.R' diff --git a/R/pkg/R/RDD.R b/R/pkg/R/RDD.R index 89511141d3ef7..d2d096709245d 100644 --- a/R/pkg/R/RDD.R +++ b/R/pkg/R/RDD.R @@ -165,7 +165,6 @@ setMethod("getJRDD", signature(rdd = "PipelinedRDD"), serializedFuncArr, rdd@env$prev_serializedMode, packageNamesArr, - as.character(.sparkREnv[["libname"]]), broadcastArr, callJMethod(prev_jrdd, "classTag")) } else { @@ -175,7 +174,6 @@ setMethod("getJRDD", signature(rdd = "PipelinedRDD"), rdd@env$prev_serializedMode, serializedMode, packageNamesArr, - as.character(.sparkREnv[["libname"]]), broadcastArr, callJMethod(prev_jrdd, "classTag")) } diff --git a/R/pkg/R/SQLContext.R b/R/pkg/R/SQLContext.R index 9a743a3411533..30978bb50d339 100644 --- a/R/pkg/R/SQLContext.R +++ b/R/pkg/R/SQLContext.R @@ -86,7 +86,9 @@ infer_type <- function(x) { createDataFrame <- function(sqlContext, data, schema = NULL, samplingRatio = 1.0) { if (is.data.frame(data)) { # get the names of columns, they will be put into RDD - schema <- names(data) + if (is.null(schema)) { + schema <- names(data) + } n <- nrow(data) m <- ncol(data) # get rid of factor type diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index 79055b7f18558..fad9d71158c51 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -20,7 +20,8 @@ # @rdname aggregateRDD # @seealso reduce # @export -setGeneric("aggregateRDD", function(x, zeroValue, seqOp, combOp) { standardGeneric("aggregateRDD") }) +setGeneric("aggregateRDD", + function(x, zeroValue, seqOp, combOp) { standardGeneric("aggregateRDD") }) # @rdname cache-methods # @export diff --git a/R/pkg/R/pairRDD.R b/R/pkg/R/pairRDD.R index 7f902ba8e683e..ebc6ff65e9d0f 100644 --- a/R/pkg/R/pairRDD.R +++ b/R/pkg/R/pairRDD.R @@ -215,7 +215,6 @@ setMethod("partitionBy", serializedHashFuncBytes, getSerializedMode(x), packageNamesArr, - as.character(.sparkREnv$libname), broadcastArr, callJMethod(jrdd, "classTag")) @@ -560,8 +559,8 @@ setMethod("join", # Left outer join two RDDs # # @description -# \code{leftouterjoin} This function left-outer-joins two RDDs where every element is of the form list(K, V). -# The key types of the two RDDs should be the same. +# \code{leftouterjoin} This function left-outer-joins two RDDs where every element is of +# the form list(K, V). The key types of the two RDDs should be the same. # # @param x An RDD to be joined. Should be an RDD where each element is # list(K, V). @@ -597,8 +596,8 @@ setMethod("leftOuterJoin", # Right outer join two RDDs # # @description -# \code{rightouterjoin} This function right-outer-joins two RDDs where every element is of the form list(K, V). -# The key types of the two RDDs should be the same. +# \code{rightouterjoin} This function right-outer-joins two RDDs where every element is of +# the form list(K, V). The key types of the two RDDs should be the same. # # @param x An RDD to be joined. Should be an RDD where each element is # list(K, V). @@ -634,8 +633,8 @@ setMethod("rightOuterJoin", # Full outer join two RDDs # # @description -# \code{fullouterjoin} This function full-outer-joins two RDDs where every element is of the form list(K, V). -# The key types of the two RDDs should be the same. +# \code{fullouterjoin} This function full-outer-joins two RDDs where every element is of +# the form list(K, V). The key types of the two RDDs should be the same. # # @param x An RDD to be joined. Should be an RDD where each element is # list(K, V). diff --git a/R/pkg/R/sparkR.R b/R/pkg/R/sparkR.R index 86233e01db365..172335809dec2 100644 --- a/R/pkg/R/sparkR.R +++ b/R/pkg/R/sparkR.R @@ -17,10 +17,6 @@ .sparkREnv <- new.env() -sparkR.onLoad <- function(libname, pkgname) { - .sparkREnv$libname <- libname -} - # Utility function that returns TRUE if we have an active connection to the # backend and FALSE otherwise connExists <- function(env) { @@ -80,7 +76,6 @@ sparkR.stop <- function() { #' @param sparkEnvir Named list of environment variables to set on worker nodes. #' @param sparkExecutorEnv Named list of environment variables to be used when launching executors. #' @param sparkJars Character string vector of jar files to pass to the worker nodes. -#' @param sparkRLibDir The path where R is installed on the worker nodes. #' @param sparkPackages Character string vector of packages from spark-packages.org #' @export #' @examples @@ -101,11 +96,11 @@ sparkR.init <- function( sparkEnvir = list(), sparkExecutorEnv = list(), sparkJars = "", - sparkRLibDir = "", sparkPackages = "") { if (exists(".sparkRjsc", envir = .sparkREnv)) { - cat("Re-using existing Spark Context. Please stop SparkR with sparkR.stop() or restart R to create a new Spark Context\n") + cat(paste("Re-using existing Spark Context.", + "Please stop SparkR with sparkR.stop() or restart R to create a new Spark Context\n")) return(get(".sparkRjsc", envir = .sparkREnv)) } @@ -169,10 +164,6 @@ sparkR.init <- function( sparkHome <- normalizePath(sparkHome) } - if (nchar(sparkRLibDir) != 0) { - .sparkREnv$libname <- sparkRLibDir - } - sparkEnvirMap <- new.env() for (varname in names(sparkEnvir)) { sparkEnvirMap[[varname]] <- sparkEnvir[[varname]] @@ -180,14 +171,16 @@ sparkR.init <- function( sparkExecutorEnvMap <- new.env() if (!any(names(sparkExecutorEnv) == "LD_LIBRARY_PATH")) { - sparkExecutorEnvMap[["LD_LIBRARY_PATH"]] <- paste0("$LD_LIBRARY_PATH:",Sys.getenv("LD_LIBRARY_PATH")) + sparkExecutorEnvMap[["LD_LIBRARY_PATH"]] <- + paste0("$LD_LIBRARY_PATH:",Sys.getenv("LD_LIBRARY_PATH")) } for (varname in names(sparkExecutorEnv)) { sparkExecutorEnvMap[[varname]] <- sparkExecutorEnv[[varname]] } nonEmptyJars <- Filter(function(x) { x != "" }, jars) - localJarPaths <- sapply(nonEmptyJars, function(j) { utils::URLencode(paste("file:", uriSep, j, sep = "")) }) + localJarPaths <- sapply(nonEmptyJars, + function(j) { utils::URLencode(paste("file:", uriSep, j, sep = "")) }) # Set the start time to identify jobjs # Seconds resolution is good enough for this purpose, so use ints diff --git a/R/pkg/R/utils.R b/R/pkg/R/utils.R index 13cec0f712fb4..ea629a64f7158 100644 --- a/R/pkg/R/utils.R +++ b/R/pkg/R/utils.R @@ -334,18 +334,21 @@ getStorageLevel <- function(newLevel = c("DISK_ONLY", "MEMORY_ONLY_SER_2", "OFF_HEAP")) { match.arg(newLevel) + storageLevelClass <- "org.apache.spark.storage.StorageLevel" storageLevel <- switch(newLevel, - "DISK_ONLY" = callJStatic("org.apache.spark.storage.StorageLevel", "DISK_ONLY"), - "DISK_ONLY_2" = callJStatic("org.apache.spark.storage.StorageLevel", "DISK_ONLY_2"), - "MEMORY_AND_DISK" = callJStatic("org.apache.spark.storage.StorageLevel", "MEMORY_AND_DISK"), - "MEMORY_AND_DISK_2" = callJStatic("org.apache.spark.storage.StorageLevel", "MEMORY_AND_DISK_2"), - "MEMORY_AND_DISK_SER" = callJStatic("org.apache.spark.storage.StorageLevel", "MEMORY_AND_DISK_SER"), - "MEMORY_AND_DISK_SER_2" = callJStatic("org.apache.spark.storage.StorageLevel", "MEMORY_AND_DISK_SER_2"), - "MEMORY_ONLY" = callJStatic("org.apache.spark.storage.StorageLevel", "MEMORY_ONLY"), - "MEMORY_ONLY_2" = callJStatic("org.apache.spark.storage.StorageLevel", "MEMORY_ONLY_2"), - "MEMORY_ONLY_SER" = callJStatic("org.apache.spark.storage.StorageLevel", "MEMORY_ONLY_SER"), - "MEMORY_ONLY_SER_2" = callJStatic("org.apache.spark.storage.StorageLevel", "MEMORY_ONLY_SER_2"), - "OFF_HEAP" = callJStatic("org.apache.spark.storage.StorageLevel", "OFF_HEAP")) + "DISK_ONLY" = callJStatic(storageLevelClass, "DISK_ONLY"), + "DISK_ONLY_2" = callJStatic(storageLevelClass, "DISK_ONLY_2"), + "MEMORY_AND_DISK" = callJStatic(storageLevelClass, "MEMORY_AND_DISK"), + "MEMORY_AND_DISK_2" = callJStatic(storageLevelClass, "MEMORY_AND_DISK_2"), + "MEMORY_AND_DISK_SER" = callJStatic(storageLevelClass, + "MEMORY_AND_DISK_SER"), + "MEMORY_AND_DISK_SER_2" = callJStatic(storageLevelClass, + "MEMORY_AND_DISK_SER_2"), + "MEMORY_ONLY" = callJStatic(storageLevelClass, "MEMORY_ONLY"), + "MEMORY_ONLY_2" = callJStatic(storageLevelClass, "MEMORY_ONLY_2"), + "MEMORY_ONLY_SER" = callJStatic(storageLevelClass, "MEMORY_ONLY_SER"), + "MEMORY_ONLY_SER_2" = callJStatic(storageLevelClass, "MEMORY_ONLY_SER_2"), + "OFF_HEAP" = callJStatic(storageLevelClass, "OFF_HEAP")) } # Utility function for functions where an argument needs to be integer but we want to allow @@ -545,9 +548,11 @@ mergePartitions <- function(rdd, zip) { lengthOfKeys <- part[[len - lengthOfValues]] stopifnot(len == lengthOfKeys + lengthOfValues) - # For zip operation, check if corresponding partitions of both RDDs have the same number of elements. + # For zip operation, check if corresponding partitions + # of both RDDs have the same number of elements. if (zip && lengthOfKeys != lengthOfValues) { - stop("Can only zip RDDs with same number of elements in each pair of corresponding partitions.") + stop(paste("Can only zip RDDs with same number of elements", + "in each pair of corresponding partitions.")) } if (lengthOfKeys > 1) { diff --git a/R/pkg/inst/profile/general.R b/R/pkg/inst/profile/general.R index 8fe711b622086..2a8a8213d0849 100644 --- a/R/pkg/inst/profile/general.R +++ b/R/pkg/inst/profile/general.R @@ -16,7 +16,7 @@ # .First <- function() { - home <- Sys.getenv("SPARK_HOME") - .libPaths(c(file.path(home, "R", "lib"), .libPaths())) + packageDir <- Sys.getenv("SPARKR_PACKAGE_DIR") + .libPaths(c(packageDir, .libPaths())) Sys.setenv(NOAWT=1) } diff --git a/R/pkg/inst/tests/test_includeJAR.R b/R/pkg/inst/tests/test_includeJAR.R index 844d86f3cc97f..cc1faeabffe30 100644 --- a/R/pkg/inst/tests/test_includeJAR.R +++ b/R/pkg/inst/tests/test_includeJAR.R @@ -18,8 +18,8 @@ context("include an external JAR in SparkContext") runScript <- function() { sparkHome <- Sys.getenv("SPARK_HOME") - jarPath <- paste("--jars", - shQuote(file.path(sparkHome, "R/lib/SparkR/test_support/sparktestjar_2.10-1.0.jar"))) + sparkTestJarPath <- "R/lib/SparkR/test_support/sparktestjar_2.10-1.0.jar" + jarPath <- paste("--jars", shQuote(file.path(sparkHome, sparkTestJarPath))) scriptPath <- file.path(sparkHome, "R/lib/SparkR/tests/jarTest.R") submitPath <- file.path(sparkHome, "bin/spark-submit") res <- system2(command = submitPath, diff --git a/R/pkg/inst/tests/test_rdd.R b/R/pkg/inst/tests/test_rdd.R index fc3c01d837de4..b79692873cec3 100644 --- a/R/pkg/inst/tests/test_rdd.R +++ b/R/pkg/inst/tests/test_rdd.R @@ -669,13 +669,15 @@ test_that("fullOuterJoin() on pairwise RDDs", { rdd1 <- parallelize(sc, list(list(1,2), list(1,3), list(3,3))) rdd2 <- parallelize(sc, list(list(1,1), list(2,4))) actual <- collect(fullOuterJoin(rdd1, rdd2, 2L)) - expected <- list(list(1, list(2, 1)), list(1, list(3, 1)), list(2, list(NULL, 4)), list(3, list(3, NULL))) + expected <- list(list(1, list(2, 1)), list(1, list(3, 1)), + list(2, list(NULL, 4)), list(3, list(3, NULL))) expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) rdd1 <- parallelize(sc, list(list("a",2), list("a",3), list("c", 1))) rdd2 <- parallelize(sc, list(list("a",1), list("b",4))) actual <- collect(fullOuterJoin(rdd1, rdd2, 2L)) - expected <- list(list("b", list(NULL, 4)), list("a", list(2, 1)), list("a", list(3, 1)), list("c", list(1, NULL))) + expected <- list(list("b", list(NULL, 4)), list("a", list(2, 1)), + list("a", list(3, 1)), list("c", list(1, NULL))) expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) @@ -683,13 +685,15 @@ test_that("fullOuterJoin() on pairwise RDDs", { rdd2 <- parallelize(sc, list(list(3,3), list(4,4))) actual <- collect(fullOuterJoin(rdd1, rdd2, 2L)) expect_equal(sortKeyValueList(actual), - sortKeyValueList(list(list(1, list(1, NULL)), list(2, list(2, NULL)), list(3, list(NULL, 3)), list(4, list(NULL, 4))))) + sortKeyValueList(list(list(1, list(1, NULL)), list(2, list(2, NULL)), + list(3, list(NULL, 3)), list(4, list(NULL, 4))))) rdd1 <- parallelize(sc, list(list("a",1), list("b",2))) rdd2 <- parallelize(sc, list(list("c",3), list("d",4))) actual <- collect(fullOuterJoin(rdd1, rdd2, 2L)) expect_equal(sortKeyValueList(actual), - sortKeyValueList(list(list("a", list(1, NULL)), list("b", list(2, NULL)), list("d", list(NULL, 4)), list("c", list(NULL, 3))))) + sortKeyValueList(list(list("a", list(1, NULL)), list("b", list(2, NULL)), + list("d", list(NULL, 4)), list("c", list(NULL, 3))))) }) test_that("sortByKey() on pairwise RDDs", { diff --git a/R/pkg/inst/tests/test_sparkSQL.R b/R/pkg/inst/tests/test_sparkSQL.R index 0e4235ea8b4b3..b0ea38854304e 100644 --- a/R/pkg/inst/tests/test_sparkSQL.R +++ b/R/pkg/inst/tests/test_sparkSQL.R @@ -391,7 +391,7 @@ test_that("collect() and take() on a DataFrame return the same number of rows an expect_equal(ncol(collect(df)), ncol(take(df, 10))) }) -test_that("multiple pipeline transformations starting with a DataFrame result in an RDD with the correct values", { +test_that("multiple pipeline transformations result in an RDD with the correct values", { df <- jsonFile(sqlContext, jsonPath) first <- lapply(df, function(row) { row$age <- row$age + 5 @@ -756,7 +756,14 @@ test_that("toJSON() returns an RDD of the correct values", { test_that("showDF()", { df <- jsonFile(sqlContext, jsonPath) s <- capture.output(showDF(df)) - expect_output(s , "+----+-------+\n| age| name|\n+----+-------+\n|null|Michael|\n| 30| Andy|\n| 19| Justin|\n+----+-------+\n") + expected <- paste("+----+-------+\n", + "| age| name|\n", + "+----+-------+\n", + "|null|Michael|\n", + "| 30| Andy|\n", + "| 19| Justin|\n", + "+----+-------+\n", sep="") + expect_output(s , expected) }) test_that("isLocal()", { diff --git a/build/mvn b/build/mvn index e8364181e8230..f62f61ee1c416 100755 --- a/build/mvn +++ b/build/mvn @@ -112,10 +112,17 @@ install_scala() { # the environment ZINC_PORT=${ZINC_PORT:-"3030"} +# Check for the `--force` flag dictating that `mvn` should be downloaded +# regardless of whether the system already has a `mvn` install +if [ "$1" == "--force" ]; then + FORCE_MVN=1 + shift +fi + # Install Maven if necessary MVN_BIN="$(command -v mvn)" -if [ ! "$MVN_BIN" ]; then +if [ ! "$MVN_BIN" -o -n "$FORCE_MVN" ]; then install_mvn fi @@ -139,5 +146,7 @@ fi # Set any `mvn` options if not already present export MAVEN_OPTS=${MAVEN_OPTS:-"$_COMPILE_JVM_OPTS"} +echo "Using \`mvn\` from path: $MVN_BIN" + # Last, call the `mvn` command as usual ${MVN_BIN} "$@" diff --git a/core/pom.xml b/core/pom.xml index aee0d92620606..558cc3fb9f2f3 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -343,28 +343,28 @@ test - org.mockito - mockito-core + org.hamcrest + hamcrest-core test - org.scalacheck - scalacheck_${scala.binary.version} + org.hamcrest + hamcrest-library test - junit - junit + org.mockito + mockito-core test - org.hamcrest - hamcrest-core + org.scalacheck + scalacheck_${scala.binary.version} test - org.hamcrest - hamcrest-library + junit + junit test diff --git a/core/src/main/java/org/apache/spark/JavaSparkListener.java b/core/src/main/java/org/apache/spark/JavaSparkListener.java index 646496f313507..fa9acf0a15b88 100644 --- a/core/src/main/java/org/apache/spark/JavaSparkListener.java +++ b/core/src/main/java/org/apache/spark/JavaSparkListener.java @@ -17,23 +17,7 @@ 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; +import org.apache.spark.scheduler.*; /** * Java clients should extend this class instead of implementing @@ -94,4 +78,8 @@ public void onExecutorAdded(SparkListenerExecutorAdded executorAdded) { } @Override public void onExecutorRemoved(SparkListenerExecutorRemoved executorRemoved) { } + + @Override + public void onBlockUpdated(SparkListenerBlockUpdated blockUpdated) { } + } diff --git a/core/src/main/java/org/apache/spark/SparkFirehoseListener.java b/core/src/main/java/org/apache/spark/SparkFirehoseListener.java index fbc5666959055..1214d05ba6063 100644 --- a/core/src/main/java/org/apache/spark/SparkFirehoseListener.java +++ b/core/src/main/java/org/apache/spark/SparkFirehoseListener.java @@ -112,4 +112,10 @@ public final void onExecutorAdded(SparkListenerExecutorAdded executorAdded) { public final void onExecutorRemoved(SparkListenerExecutorRemoved executorRemoved) { onEvent(executorRemoved); } + + @Override + public void onBlockUpdated(SparkListenerBlockUpdated blockUpdated) { + onEvent(blockUpdated); + } + } diff --git a/core/src/main/java/org/apache/spark/shuffle/unsafe/DummySerializerInstance.java b/core/src/main/java/org/apache/spark/serializer/DummySerializerInstance.java similarity index 91% rename from core/src/main/java/org/apache/spark/shuffle/unsafe/DummySerializerInstance.java rename to core/src/main/java/org/apache/spark/serializer/DummySerializerInstance.java index 3f746b886bc9b..0399abc63c235 100644 --- a/core/src/main/java/org/apache/spark/shuffle/unsafe/DummySerializerInstance.java +++ b/core/src/main/java/org/apache/spark/serializer/DummySerializerInstance.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.shuffle.unsafe; +package org.apache.spark.serializer; import java.io.IOException; import java.io.InputStream; @@ -24,9 +24,7 @@ import scala.reflect.ClassTag; -import org.apache.spark.serializer.DeserializationStream; -import org.apache.spark.serializer.SerializationStream; -import org.apache.spark.serializer.SerializerInstance; +import org.apache.spark.annotation.Private; import org.apache.spark.unsafe.PlatformDependent; /** @@ -35,7 +33,8 @@ * `write() OutputStream methods), but DiskBlockObjectWriter still calls some methods on it. To work * around this, we pass a dummy no-op serializer. */ -final class DummySerializerInstance extends SerializerInstance { +@Private +public final class DummySerializerInstance extends SerializerInstance { public static final DummySerializerInstance INSTANCE = new DummySerializerInstance(); diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java index d3d6280284beb..0b8b604e18494 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java @@ -75,7 +75,7 @@ final class BypassMergeSortShuffleWriter implements SortShuffleFileWriter< private final Serializer serializer; /** Array of file writers, one for each partition */ - private BlockObjectWriter[] partitionWriters; + private DiskBlockObjectWriter[] partitionWriters; public BypassMergeSortShuffleWriter( SparkConf conf, @@ -101,7 +101,7 @@ public void insertAll(Iterator> records) throws IOException { } final SerializerInstance serInstance = serializer.newInstance(); final long openStartTime = System.nanoTime(); - partitionWriters = new BlockObjectWriter[numPartitions]; + partitionWriters = new DiskBlockObjectWriter[numPartitions]; for (int i = 0; i < numPartitions; i++) { final Tuple2 tempShuffleBlockIdPlusFile = blockManager.diskBlockManager().createTempShuffleBlock(); @@ -121,7 +121,7 @@ public void insertAll(Iterator> records) throws IOException { partitionWriters[partitioner.getPartition(key)].write(key, record._2()); } - for (BlockObjectWriter writer : partitionWriters) { + for (DiskBlockObjectWriter writer : partitionWriters) { writer.commitAndClose(); } } @@ -169,7 +169,7 @@ public void stop() throws IOException { if (partitionWriters != null) { try { final DiskBlockManager diskBlockManager = blockManager.diskBlockManager(); - for (BlockObjectWriter writer : partitionWriters) { + for (DiskBlockObjectWriter writer : partitionWriters) { // This method explicitly does _not_ throw exceptions: writer.revertPartialWritesAndClose(); if (!diskBlockManager.getFile(writer.blockId()).delete()) { diff --git a/core/src/main/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleExternalSorter.java b/core/src/main/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleExternalSorter.java index 9e9ed94b7890c..1d460432be9ff 100644 --- a/core/src/main/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleExternalSorter.java +++ b/core/src/main/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleExternalSorter.java @@ -30,6 +30,7 @@ import org.apache.spark.SparkConf; import org.apache.spark.TaskContext; import org.apache.spark.executor.ShuffleWriteMetrics; +import org.apache.spark.serializer.DummySerializerInstance; import org.apache.spark.serializer.SerializerInstance; import org.apache.spark.shuffle.ShuffleMemoryManager; import org.apache.spark.storage.*; @@ -156,7 +157,7 @@ private void writeSortedFile(boolean isLastFile) throws IOException { // Currently, we need to open a new DiskBlockObjectWriter for each partition; we can avoid this // after SPARK-5581 is fixed. - BlockObjectWriter writer; + DiskBlockObjectWriter writer; // Small writes to DiskBlockObjectWriter will be fairly inefficient. Since there doesn't seem to // be an API to directly transfer bytes from managed memory to the disk writer, we buffer diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/PrefixComparator.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/PrefixComparator.java new file mode 100644 index 0000000000000..45b78829e4cf7 --- /dev/null +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/PrefixComparator.java @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.collection.unsafe.sort; + +import org.apache.spark.annotation.Private; + +/** + * Compares 8-byte key prefixes in prefix sort. Subclasses may implement type-specific + * comparisons, such as lexicographic comparison for strings. + */ +@Private +public abstract class PrefixComparator { + public abstract int compare(long prefix1, long prefix2); +} diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/PrefixComparators.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/PrefixComparators.java new file mode 100644 index 0000000000000..438742565c51d --- /dev/null +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/PrefixComparators.java @@ -0,0 +1,109 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.collection.unsafe.sort; + +import com.google.common.base.Charsets; +import com.google.common.primitives.Longs; +import com.google.common.primitives.UnsignedBytes; + +import org.apache.spark.annotation.Private; +import org.apache.spark.unsafe.types.UTF8String; + +@Private +public class PrefixComparators { + private PrefixComparators() {} + + public static final StringPrefixComparator STRING = new StringPrefixComparator(); + public static final IntegralPrefixComparator INTEGRAL = new IntegralPrefixComparator(); + public static final FloatPrefixComparator FLOAT = new FloatPrefixComparator(); + public static final DoublePrefixComparator DOUBLE = new DoublePrefixComparator(); + + public static final class StringPrefixComparator extends PrefixComparator { + @Override + public int compare(long aPrefix, long bPrefix) { + // TODO: can done more efficiently + byte[] a = Longs.toByteArray(aPrefix); + byte[] b = Longs.toByteArray(bPrefix); + for (int i = 0; i < 8; i++) { + int c = UnsignedBytes.compare(a[i], b[i]); + if (c != 0) return c; + } + return 0; + } + + public long computePrefix(byte[] bytes) { + if (bytes == null) { + return 0L; + } else { + byte[] padded = new byte[8]; + System.arraycopy(bytes, 0, padded, 0, Math.min(bytes.length, 8)); + return Longs.fromByteArray(padded); + } + } + + public long computePrefix(String value) { + return value == null ? 0L : computePrefix(value.getBytes(Charsets.UTF_8)); + } + + public long computePrefix(UTF8String value) { + return value == null ? 0L : computePrefix(value.getBytes()); + } + } + + /** + * Prefix comparator for all integral types (boolean, byte, short, int, long). + */ + public static final class IntegralPrefixComparator extends PrefixComparator { + @Override + public int compare(long a, long b) { + return (a < b) ? -1 : (a > b) ? 1 : 0; + } + + public final long NULL_PREFIX = Long.MIN_VALUE; + } + + public static final class FloatPrefixComparator extends PrefixComparator { + @Override + public int compare(long aPrefix, long bPrefix) { + float a = Float.intBitsToFloat((int) aPrefix); + float b = Float.intBitsToFloat((int) bPrefix); + return (a < b) ? -1 : (a > b) ? 1 : 0; + } + + public long computePrefix(float value) { + return Float.floatToIntBits(value) & 0xffffffffL; + } + + public final long NULL_PREFIX = computePrefix(Float.NEGATIVE_INFINITY); + } + + public static final class DoublePrefixComparator extends PrefixComparator { + @Override + public int compare(long aPrefix, long bPrefix) { + double a = Double.longBitsToDouble(aPrefix); + double b = Double.longBitsToDouble(bPrefix); + return (a < b) ? -1 : (a > b) ? 1 : 0; + } + + public long computePrefix(double value) { + return Double.doubleToLongBits(value); + } + + public final long NULL_PREFIX = computePrefix(Double.NEGATIVE_INFINITY); + } +} diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/RecordComparator.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/RecordComparator.java new file mode 100644 index 0000000000000..09e4258792204 --- /dev/null +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/RecordComparator.java @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.collection.unsafe.sort; + +/** + * Compares records for ordering. In cases where the entire sorting key can fit in the 8-byte + * prefix, this may simply return 0. + */ +public abstract class RecordComparator { + + /** + * Compare two records for order. + * + * @return a negative integer, zero, or a positive integer as the first record is less than, + * equal to, or greater than the second. + */ + public abstract int compare( + Object leftBaseObject, + long leftBaseOffset, + Object rightBaseObject, + long rightBaseOffset); +} diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/RecordPointerAndKeyPrefix.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/RecordPointerAndKeyPrefix.java new file mode 100644 index 0000000000000..0c4ebde407cfc --- /dev/null +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/RecordPointerAndKeyPrefix.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.collection.unsafe.sort; + +final class RecordPointerAndKeyPrefix { + /** + * A pointer to a record; see {@link org.apache.spark.unsafe.memory.TaskMemoryManager} for a + * description of how these addresses are encoded. + */ + public long recordPointer; + + /** + * A key prefix, for use in comparisons. + */ + public long keyPrefix; +} diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java new file mode 100644 index 0000000000000..4d6731ee60af3 --- /dev/null +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java @@ -0,0 +1,282 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.collection.unsafe.sort; + +import java.io.IOException; +import java.util.LinkedList; + +import com.google.common.annotations.VisibleForTesting; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.spark.SparkConf; +import org.apache.spark.TaskContext; +import org.apache.spark.executor.ShuffleWriteMetrics; +import org.apache.spark.shuffle.ShuffleMemoryManager; +import org.apache.spark.storage.BlockManager; +import org.apache.spark.unsafe.PlatformDependent; +import org.apache.spark.unsafe.memory.MemoryBlock; +import org.apache.spark.unsafe.memory.TaskMemoryManager; +import org.apache.spark.util.Utils; + +/** + * External sorter based on {@link UnsafeInMemorySorter}. + */ +public final class UnsafeExternalSorter { + + private final Logger logger = LoggerFactory.getLogger(UnsafeExternalSorter.class); + + private static final int PAGE_SIZE = 1 << 27; // 128 megabytes + @VisibleForTesting + static final int MAX_RECORD_SIZE = PAGE_SIZE - 4; + + private final PrefixComparator prefixComparator; + private final RecordComparator recordComparator; + private final int initialSize; + private final TaskMemoryManager memoryManager; + private final ShuffleMemoryManager shuffleMemoryManager; + private final BlockManager blockManager; + private final TaskContext taskContext; + private ShuffleWriteMetrics writeMetrics; + + /** The buffer size to use when writing spills using DiskBlockObjectWriter */ + private final int fileBufferSizeBytes; + + /** + * Memory pages that hold the records being sorted. The pages in this list are freed when + * spilling, although in principle we could recycle these pages across spills (on the other hand, + * this might not be necessary if we maintained a pool of re-usable pages in the TaskMemoryManager + * itself). + */ + private final LinkedList allocatedPages = new LinkedList(); + + // These variables are reset after spilling: + private UnsafeInMemorySorter sorter; + private MemoryBlock currentPage = null; + private long currentPagePosition = -1; + private long freeSpaceInCurrentPage = 0; + + private final LinkedList spillWriters = new LinkedList<>(); + + public UnsafeExternalSorter( + TaskMemoryManager memoryManager, + ShuffleMemoryManager shuffleMemoryManager, + BlockManager blockManager, + TaskContext taskContext, + RecordComparator recordComparator, + PrefixComparator prefixComparator, + int initialSize, + SparkConf conf) throws IOException { + this.memoryManager = memoryManager; + this.shuffleMemoryManager = shuffleMemoryManager; + this.blockManager = blockManager; + this.taskContext = taskContext; + this.recordComparator = recordComparator; + this.prefixComparator = prefixComparator; + this.initialSize = initialSize; + // Use getSizeAsKb (not bytes) to maintain backwards compatibility for units + this.fileBufferSizeBytes = (int) conf.getSizeAsKb("spark.shuffle.file.buffer", "32k") * 1024; + initializeForWriting(); + } + + // TODO: metrics tracking + integration with shuffle write metrics + // need to connect the write metrics to task metrics so we count the spill IO somewhere. + + /** + * Allocates new sort data structures. Called when creating the sorter and after each spill. + */ + private void initializeForWriting() throws IOException { + this.writeMetrics = new ShuffleWriteMetrics(); + // TODO: move this sizing calculation logic into a static method of sorter: + final long memoryRequested = initialSize * 8L * 2; + final long memoryAcquired = shuffleMemoryManager.tryToAcquire(memoryRequested); + if (memoryAcquired != memoryRequested) { + shuffleMemoryManager.release(memoryAcquired); + throw new IOException("Could not acquire " + memoryRequested + " bytes of memory"); + } + + this.sorter = + new UnsafeInMemorySorter(memoryManager, recordComparator, prefixComparator, initialSize); + } + + /** + * Sort and spill the current records in response to memory pressure. + */ + @VisibleForTesting + public void spill() throws IOException { + logger.info("Thread {} spilling sort data of {} to disk ({} {} so far)", + Thread.currentThread().getId(), + Utils.bytesToString(getMemoryUsage()), + spillWriters.size(), + spillWriters.size() > 1 ? " times" : " time"); + + final UnsafeSorterSpillWriter spillWriter = + new UnsafeSorterSpillWriter(blockManager, fileBufferSizeBytes, writeMetrics, + sorter.numRecords()); + spillWriters.add(spillWriter); + final UnsafeSorterIterator sortedRecords = sorter.getSortedIterator(); + while (sortedRecords.hasNext()) { + sortedRecords.loadNext(); + final Object baseObject = sortedRecords.getBaseObject(); + final long baseOffset = sortedRecords.getBaseOffset(); + final int recordLength = sortedRecords.getRecordLength(); + spillWriter.write(baseObject, baseOffset, recordLength, sortedRecords.getKeyPrefix()); + } + spillWriter.close(); + final long sorterMemoryUsage = sorter.getMemoryUsage(); + sorter = null; + shuffleMemoryManager.release(sorterMemoryUsage); + final long spillSize = freeMemory(); + taskContext.taskMetrics().incMemoryBytesSpilled(spillSize); + initializeForWriting(); + } + + private long getMemoryUsage() { + return sorter.getMemoryUsage() + (allocatedPages.size() * (long) PAGE_SIZE); + } + + public long freeMemory() { + long memoryFreed = 0; + for (MemoryBlock block : allocatedPages) { + memoryManager.freePage(block); + shuffleMemoryManager.release(block.size()); + memoryFreed += block.size(); + } + allocatedPages.clear(); + currentPage = null; + currentPagePosition = -1; + freeSpaceInCurrentPage = 0; + return memoryFreed; + } + + /** + * Checks whether there is enough space to insert a new record into the sorter. + * + * @param requiredSpace the required space in the data page, in bytes, including space for storing + * the record size. + + * @return true if the record can be inserted without requiring more allocations, false otherwise. + */ + private boolean haveSpaceForRecord(int requiredSpace) { + assert (requiredSpace > 0); + return (sorter.hasSpaceForAnotherRecord() && (requiredSpace <= freeSpaceInCurrentPage)); + } + + /** + * Allocates more memory in order to insert an additional record. This will request additional + * memory from the {@link ShuffleMemoryManager} and spill if the requested memory can not be + * obtained. + * + * @param requiredSpace the required space in the data page, in bytes, including space for storing + * the record size. + */ + private void allocateSpaceForRecord(int requiredSpace) throws IOException { + // TODO: merge these steps to first calculate total memory requirements for this insert, + // then try to acquire; no point in acquiring sort buffer only to spill due to no space in the + // data page. + if (!sorter.hasSpaceForAnotherRecord()) { + logger.debug("Attempting to expand sort pointer array"); + final long oldPointerArrayMemoryUsage = sorter.getMemoryUsage(); + final long memoryToGrowPointerArray = oldPointerArrayMemoryUsage * 2; + final long memoryAcquired = shuffleMemoryManager.tryToAcquire(memoryToGrowPointerArray); + if (memoryAcquired < memoryToGrowPointerArray) { + shuffleMemoryManager.release(memoryAcquired); + spill(); + } else { + sorter.expandPointerArray(); + shuffleMemoryManager.release(oldPointerArrayMemoryUsage); + } + } + + if (requiredSpace > freeSpaceInCurrentPage) { + logger.trace("Required space {} is less than free space in current page ({})", requiredSpace, + freeSpaceInCurrentPage); + // TODO: we should track metrics on the amount of space wasted when we roll over to a new page + // without using the free space at the end of the current page. We should also do this for + // BytesToBytesMap. + if (requiredSpace > PAGE_SIZE) { + throw new IOException("Required space " + requiredSpace + " is greater than page size (" + + PAGE_SIZE + ")"); + } else { + final long memoryAcquired = shuffleMemoryManager.tryToAcquire(PAGE_SIZE); + if (memoryAcquired < PAGE_SIZE) { + shuffleMemoryManager.release(memoryAcquired); + spill(); + final long memoryAcquiredAfterSpilling = shuffleMemoryManager.tryToAcquire(PAGE_SIZE); + if (memoryAcquiredAfterSpilling != PAGE_SIZE) { + shuffleMemoryManager.release(memoryAcquiredAfterSpilling); + throw new IOException("Unable to acquire " + PAGE_SIZE + " bytes of memory"); + } + } + currentPage = memoryManager.allocatePage(PAGE_SIZE); + currentPagePosition = currentPage.getBaseOffset(); + freeSpaceInCurrentPage = PAGE_SIZE; + allocatedPages.add(currentPage); + } + } + } + + /** + * Write a record to the sorter. + */ + public void insertRecord( + Object recordBaseObject, + long recordBaseOffset, + int lengthInBytes, + long prefix) throws IOException { + // Need 4 bytes to store the record length. + final int totalSpaceRequired = lengthInBytes + 4; + if (!haveSpaceForRecord(totalSpaceRequired)) { + allocateSpaceForRecord(totalSpaceRequired); + } + + final long recordAddress = + memoryManager.encodePageNumberAndOffset(currentPage, currentPagePosition); + final Object dataPageBaseObject = currentPage.getBaseObject(); + PlatformDependent.UNSAFE.putInt(dataPageBaseObject, currentPagePosition, lengthInBytes); + currentPagePosition += 4; + PlatformDependent.copyMemory( + recordBaseObject, + recordBaseOffset, + dataPageBaseObject, + currentPagePosition, + lengthInBytes); + currentPagePosition += lengthInBytes; + + sorter.insertRecord(recordAddress, prefix); + } + + public UnsafeSorterIterator getSortedIterator() throws IOException { + final UnsafeSorterIterator inMemoryIterator = sorter.getSortedIterator(); + int numIteratorsToMerge = spillWriters.size() + (inMemoryIterator.hasNext() ? 1 : 0); + if (spillWriters.isEmpty()) { + return inMemoryIterator; + } else { + final UnsafeSorterSpillMerger spillMerger = + new UnsafeSorterSpillMerger(recordComparator, prefixComparator, numIteratorsToMerge); + for (UnsafeSorterSpillWriter spillWriter : spillWriters) { + spillMerger.addSpill(spillWriter.getReader(blockManager)); + } + spillWriters.clear(); + if (inMemoryIterator.hasNext()) { + spillMerger.addSpill(inMemoryIterator); + } + return spillMerger.getSortedIterator(); + } + } +} diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java new file mode 100644 index 0000000000000..fc34ad9cff369 --- /dev/null +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java @@ -0,0 +1,189 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.collection.unsafe.sort; + +import java.util.Comparator; + +import org.apache.spark.unsafe.PlatformDependent; +import org.apache.spark.util.collection.Sorter; +import org.apache.spark.unsafe.memory.TaskMemoryManager; + +/** + * Sorts records using an AlphaSort-style key-prefix sort. This sort stores pointers to records + * alongside a user-defined prefix of the record's sorting key. When the underlying sort algorithm + * compares records, it will first compare the stored key prefixes; if the prefixes are not equal, + * then we do not need to traverse the record pointers to compare the actual records. Avoiding these + * random memory accesses improves cache hit rates. + */ +public final class UnsafeInMemorySorter { + + private static final class SortComparator implements Comparator { + + private final RecordComparator recordComparator; + private final PrefixComparator prefixComparator; + private final TaskMemoryManager memoryManager; + + SortComparator( + RecordComparator recordComparator, + PrefixComparator prefixComparator, + TaskMemoryManager memoryManager) { + this.recordComparator = recordComparator; + this.prefixComparator = prefixComparator; + this.memoryManager = memoryManager; + } + + @Override + public int compare(RecordPointerAndKeyPrefix r1, RecordPointerAndKeyPrefix r2) { + final int prefixComparisonResult = prefixComparator.compare(r1.keyPrefix, r2.keyPrefix); + if (prefixComparisonResult == 0) { + final Object baseObject1 = memoryManager.getPage(r1.recordPointer); + final long baseOffset1 = memoryManager.getOffsetInPage(r1.recordPointer) + 4; // skip length + final Object baseObject2 = memoryManager.getPage(r2.recordPointer); + final long baseOffset2 = memoryManager.getOffsetInPage(r2.recordPointer) + 4; // skip length + return recordComparator.compare(baseObject1, baseOffset1, baseObject2, baseOffset2); + } else { + return prefixComparisonResult; + } + } + } + + private final TaskMemoryManager memoryManager; + private final Sorter sorter; + private final Comparator sortComparator; + + /** + * Within this buffer, position {@code 2 * i} holds a pointer pointer to the record at + * index {@code i}, while position {@code 2 * i + 1} in the array holds an 8-byte key prefix. + */ + private long[] pointerArray; + + /** + * The position in the sort buffer where new records can be inserted. + */ + private int pointerArrayInsertPosition = 0; + + public UnsafeInMemorySorter( + final TaskMemoryManager memoryManager, + final RecordComparator recordComparator, + final PrefixComparator prefixComparator, + int initialSize) { + assert (initialSize > 0); + this.pointerArray = new long[initialSize * 2]; + this.memoryManager = memoryManager; + this.sorter = new Sorter<>(UnsafeSortDataFormat.INSTANCE); + this.sortComparator = new SortComparator(recordComparator, prefixComparator, memoryManager); + } + + /** + * @return the number of records that have been inserted into this sorter. + */ + public int numRecords() { + return pointerArrayInsertPosition / 2; + } + + public long getMemoryUsage() { + return pointerArray.length * 8L; + } + + public boolean hasSpaceForAnotherRecord() { + return pointerArrayInsertPosition + 2 < pointerArray.length; + } + + public void expandPointerArray() { + final long[] oldArray = pointerArray; + // Guard against overflow: + final int newLength = oldArray.length * 2 > 0 ? (oldArray.length * 2) : Integer.MAX_VALUE; + pointerArray = new long[newLength]; + System.arraycopy(oldArray, 0, pointerArray, 0, oldArray.length); + } + + /** + * Inserts a record to be sorted. Assumes that the record pointer points to a record length + * stored as a 4-byte integer, followed by the record's bytes. + * + * @param recordPointer pointer to a record in a data page, encoded by {@link TaskMemoryManager}. + * @param keyPrefix a user-defined key prefix + */ + public void insertRecord(long recordPointer, long keyPrefix) { + if (!hasSpaceForAnotherRecord()) { + expandPointerArray(); + } + pointerArray[pointerArrayInsertPosition] = recordPointer; + pointerArrayInsertPosition++; + pointerArray[pointerArrayInsertPosition] = keyPrefix; + pointerArrayInsertPosition++; + } + + private static final class SortedIterator extends UnsafeSorterIterator { + + private final TaskMemoryManager memoryManager; + private final int sortBufferInsertPosition; + private final long[] sortBuffer; + private int position = 0; + private Object baseObject; + private long baseOffset; + private long keyPrefix; + private int recordLength; + + SortedIterator( + TaskMemoryManager memoryManager, + int sortBufferInsertPosition, + long[] sortBuffer) { + this.memoryManager = memoryManager; + this.sortBufferInsertPosition = sortBufferInsertPosition; + this.sortBuffer = sortBuffer; + } + + @Override + public boolean hasNext() { + return position < sortBufferInsertPosition; + } + + @Override + public void loadNext() { + // This pointer points to a 4-byte record length, followed by the record's bytes + final long recordPointer = sortBuffer[position]; + baseObject = memoryManager.getPage(recordPointer); + baseOffset = memoryManager.getOffsetInPage(recordPointer) + 4; // Skip over record length + recordLength = PlatformDependent.UNSAFE.getInt(baseObject, baseOffset - 4); + keyPrefix = sortBuffer[position + 1]; + position += 2; + } + + @Override + public Object getBaseObject() { return baseObject; } + + @Override + public long getBaseOffset() { return baseOffset; } + + @Override + public int getRecordLength() { return recordLength; } + + @Override + public long getKeyPrefix() { return keyPrefix; } + } + + /** + * Return an iterator over record pointers in sorted order. For efficiency, all calls to + * {@code next()} will return the same mutable object. + */ + public UnsafeSorterIterator getSortedIterator() { + sorter.sort(pointerArray, 0, pointerArrayInsertPosition / 2, sortComparator); + return new SortedIterator(memoryManager, pointerArrayInsertPosition, pointerArray); + } +} diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSortDataFormat.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSortDataFormat.java new file mode 100644 index 0000000000000..d09c728a7a638 --- /dev/null +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSortDataFormat.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.collection.unsafe.sort; + +import org.apache.spark.util.collection.SortDataFormat; + +/** + * Supports sorting an array of (record pointer, key prefix) pairs. + * Used in {@link UnsafeInMemorySorter}. + *

+ * Within each long[] buffer, position {@code 2 * i} holds a pointer pointer to the record at + * index {@code i}, while position {@code 2 * i + 1} in the array holds an 8-byte key prefix. + */ +final class UnsafeSortDataFormat extends SortDataFormat { + + public static final UnsafeSortDataFormat INSTANCE = new UnsafeSortDataFormat(); + + private UnsafeSortDataFormat() { } + + @Override + public RecordPointerAndKeyPrefix getKey(long[] data, int pos) { + // Since we re-use keys, this method shouldn't be called. + throw new UnsupportedOperationException(); + } + + @Override + public RecordPointerAndKeyPrefix newKey() { + return new RecordPointerAndKeyPrefix(); + } + + @Override + public RecordPointerAndKeyPrefix getKey(long[] data, int pos, RecordPointerAndKeyPrefix reuse) { + reuse.recordPointer = data[pos * 2]; + reuse.keyPrefix = data[pos * 2 + 1]; + return reuse; + } + + @Override + public void swap(long[] data, int pos0, int pos1) { + long tempPointer = data[pos0 * 2]; + long tempKeyPrefix = data[pos0 * 2 + 1]; + data[pos0 * 2] = data[pos1 * 2]; + data[pos0 * 2 + 1] = data[pos1 * 2 + 1]; + data[pos1 * 2] = tempPointer; + data[pos1 * 2 + 1] = tempKeyPrefix; + } + + @Override + public void copyElement(long[] src, int srcPos, long[] dst, int dstPos) { + dst[dstPos * 2] = src[srcPos * 2]; + dst[dstPos * 2 + 1] = src[srcPos * 2 + 1]; + } + + @Override + public void copyRange(long[] src, int srcPos, long[] dst, int dstPos, int length) { + System.arraycopy(src, srcPos * 2, dst, dstPos * 2, length * 2); + } + + @Override + public long[] allocate(int length) { + assert (length < Integer.MAX_VALUE / 2) : "Length " + length + " is too large"; + return new long[length * 2]; + } + +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MemoryUtils.scala b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterIterator.java similarity index 65% rename from core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MemoryUtils.scala rename to core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterIterator.java index 8df4f3b554c41..16ac2e8d821ba 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MemoryUtils.scala +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterIterator.java @@ -15,17 +15,21 @@ * limitations under the License. */ -package org.apache.spark.scheduler.cluster.mesos +package org.apache.spark.util.collection.unsafe.sort; -import org.apache.spark.SparkContext +import java.io.IOException; -private[spark] object MemoryUtils { - // These defaults copied from YARN - val OVERHEAD_FRACTION = 0.10 - val OVERHEAD_MINIMUM = 384 +public abstract class UnsafeSorterIterator { - def calculateTotalMemory(sc: SparkContext): Int = { - sc.conf.getInt("spark.mesos.executor.memoryOverhead", - math.max(OVERHEAD_FRACTION * sc.executorMemory, OVERHEAD_MINIMUM).toInt) + sc.executorMemory - } + public abstract boolean hasNext(); + + public abstract void loadNext() throws IOException; + + public abstract Object getBaseObject(); + + public abstract long getBaseOffset(); + + public abstract int getRecordLength(); + + public abstract long getKeyPrefix(); } diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillMerger.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillMerger.java new file mode 100644 index 0000000000000..8272c2a5be0d1 --- /dev/null +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillMerger.java @@ -0,0 +1,91 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.collection.unsafe.sort; + +import java.io.IOException; +import java.util.Comparator; +import java.util.PriorityQueue; + +final class UnsafeSorterSpillMerger { + + private final PriorityQueue priorityQueue; + + public UnsafeSorterSpillMerger( + final RecordComparator recordComparator, + final PrefixComparator prefixComparator, + final int numSpills) { + final Comparator comparator = new Comparator() { + + @Override + public int compare(UnsafeSorterIterator left, UnsafeSorterIterator right) { + final int prefixComparisonResult = + prefixComparator.compare(left.getKeyPrefix(), right.getKeyPrefix()); + if (prefixComparisonResult == 0) { + return recordComparator.compare( + left.getBaseObject(), left.getBaseOffset(), + right.getBaseObject(), right.getBaseOffset()); + } else { + return prefixComparisonResult; + } + } + }; + priorityQueue = new PriorityQueue(numSpills, comparator); + } + + public void addSpill(UnsafeSorterIterator spillReader) throws IOException { + if (spillReader.hasNext()) { + spillReader.loadNext(); + } + priorityQueue.add(spillReader); + } + + public UnsafeSorterIterator getSortedIterator() throws IOException { + return new UnsafeSorterIterator() { + + private UnsafeSorterIterator spillReader; + + @Override + public boolean hasNext() { + return !priorityQueue.isEmpty() || (spillReader != null && spillReader.hasNext()); + } + + @Override + public void loadNext() throws IOException { + if (spillReader != null) { + if (spillReader.hasNext()) { + spillReader.loadNext(); + priorityQueue.add(spillReader); + } + } + spillReader = priorityQueue.remove(); + } + + @Override + public Object getBaseObject() { return spillReader.getBaseObject(); } + + @Override + public long getBaseOffset() { return spillReader.getBaseOffset(); } + + @Override + public int getRecordLength() { return spillReader.getRecordLength(); } + + @Override + public long getKeyPrefix() { return spillReader.getKeyPrefix(); } + }; + } +} diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillReader.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillReader.java new file mode 100644 index 0000000000000..29e9e0f30f934 --- /dev/null +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillReader.java @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.collection.unsafe.sort; + +import java.io.*; + +import com.google.common.io.ByteStreams; + +import org.apache.spark.storage.BlockId; +import org.apache.spark.storage.BlockManager; +import org.apache.spark.unsafe.PlatformDependent; + +/** + * Reads spill files written by {@link UnsafeSorterSpillWriter} (see that class for a description + * of the file format). + */ +final class UnsafeSorterSpillReader extends UnsafeSorterIterator { + + private InputStream in; + private DataInputStream din; + + // Variables that change with every record read: + private int recordLength; + private long keyPrefix; + private int numRecordsRemaining; + + private byte[] arr = new byte[1024 * 1024]; + private Object baseObject = arr; + private final long baseOffset = PlatformDependent.BYTE_ARRAY_OFFSET; + + public UnsafeSorterSpillReader( + BlockManager blockManager, + File file, + BlockId blockId) throws IOException { + assert (file.length() > 0); + final BufferedInputStream bs = new BufferedInputStream(new FileInputStream(file)); + this.in = blockManager.wrapForCompression(blockId, bs); + this.din = new DataInputStream(this.in); + numRecordsRemaining = din.readInt(); + } + + @Override + public boolean hasNext() { + return (numRecordsRemaining > 0); + } + + @Override + public void loadNext() throws IOException { + recordLength = din.readInt(); + keyPrefix = din.readLong(); + if (recordLength > arr.length) { + arr = new byte[recordLength]; + baseObject = arr; + } + ByteStreams.readFully(in, arr, 0, recordLength); + numRecordsRemaining--; + if (numRecordsRemaining == 0) { + in.close(); + in = null; + din = null; + } + } + + @Override + public Object getBaseObject() { + return baseObject; + } + + @Override + public long getBaseOffset() { + return baseOffset; + } + + @Override + public int getRecordLength() { + return recordLength; + } + + @Override + public long getKeyPrefix() { + return keyPrefix; + } +} diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillWriter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillWriter.java new file mode 100644 index 0000000000000..71eed29563d4a --- /dev/null +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillWriter.java @@ -0,0 +1,146 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.collection.unsafe.sort; + +import java.io.File; +import java.io.IOException; + +import scala.Tuple2; + +import org.apache.spark.executor.ShuffleWriteMetrics; +import org.apache.spark.serializer.DummySerializerInstance; +import org.apache.spark.storage.BlockId; +import org.apache.spark.storage.BlockManager; +import org.apache.spark.storage.DiskBlockObjectWriter; +import org.apache.spark.storage.TempLocalBlockId; +import org.apache.spark.unsafe.PlatformDependent; + +/** + * Spills a list of sorted records to disk. Spill files have the following format: + * + * [# of records (int)] [[len (int)][prefix (long)][data (bytes)]...] + */ +final class UnsafeSorterSpillWriter { + + static final int DISK_WRITE_BUFFER_SIZE = 1024 * 1024; + + // Small writes to DiskBlockObjectWriter will be fairly inefficient. Since there doesn't seem to + // be an API to directly transfer bytes from managed memory to the disk writer, we buffer + // data through a byte array. + private byte[] writeBuffer = new byte[DISK_WRITE_BUFFER_SIZE]; + + private final File file; + private final BlockId blockId; + private final int numRecordsToWrite; + private DiskBlockObjectWriter writer; + private int numRecordsSpilled = 0; + + public UnsafeSorterSpillWriter( + BlockManager blockManager, + int fileBufferSize, + ShuffleWriteMetrics writeMetrics, + int numRecordsToWrite) throws IOException { + final Tuple2 spilledFileInfo = + blockManager.diskBlockManager().createTempLocalBlock(); + this.file = spilledFileInfo._2(); + this.blockId = spilledFileInfo._1(); + this.numRecordsToWrite = numRecordsToWrite; + // Unfortunately, we need a serializer instance in order to construct a DiskBlockObjectWriter. + // Our write path doesn't actually use this serializer (since we end up calling the `write()` + // OutputStream methods), but DiskBlockObjectWriter still calls some methods on it. To work + // around this, we pass a dummy no-op serializer. + writer = blockManager.getDiskWriter( + blockId, file, DummySerializerInstance.INSTANCE, fileBufferSize, writeMetrics); + // Write the number of records + writeIntToBuffer(numRecordsToWrite, 0); + writer.write(writeBuffer, 0, 4); + } + + // Based on DataOutputStream.writeLong. + private void writeLongToBuffer(long v, int offset) throws IOException { + writeBuffer[offset + 0] = (byte)(v >>> 56); + writeBuffer[offset + 1] = (byte)(v >>> 48); + writeBuffer[offset + 2] = (byte)(v >>> 40); + writeBuffer[offset + 3] = (byte)(v >>> 32); + writeBuffer[offset + 4] = (byte)(v >>> 24); + writeBuffer[offset + 5] = (byte)(v >>> 16); + writeBuffer[offset + 6] = (byte)(v >>> 8); + writeBuffer[offset + 7] = (byte)(v >>> 0); + } + + // Based on DataOutputStream.writeInt. + private void writeIntToBuffer(int v, int offset) throws IOException { + writeBuffer[offset + 0] = (byte)(v >>> 24); + writeBuffer[offset + 1] = (byte)(v >>> 16); + writeBuffer[offset + 2] = (byte)(v >>> 8); + writeBuffer[offset + 3] = (byte)(v >>> 0); + } + + /** + * Write a record to a spill file. + * + * @param baseObject the base object / memory page containing the record + * @param baseOffset the base offset which points directly to the record data. + * @param recordLength the length of the record. + * @param keyPrefix a sort key prefix + */ + public void write( + Object baseObject, + long baseOffset, + int recordLength, + long keyPrefix) throws IOException { + if (numRecordsSpilled == numRecordsToWrite) { + throw new IllegalStateException( + "Number of records written exceeded numRecordsToWrite = " + numRecordsToWrite); + } else { + numRecordsSpilled++; + } + writeIntToBuffer(recordLength, 0); + writeLongToBuffer(keyPrefix, 4); + int dataRemaining = recordLength; + int freeSpaceInWriteBuffer = DISK_WRITE_BUFFER_SIZE - 4 - 8; // space used by prefix + len + long recordReadPosition = baseOffset; + while (dataRemaining > 0) { + final int toTransfer = Math.min(freeSpaceInWriteBuffer, dataRemaining); + PlatformDependent.copyMemory( + baseObject, + recordReadPosition, + writeBuffer, + PlatformDependent.BYTE_ARRAY_OFFSET + (DISK_WRITE_BUFFER_SIZE - freeSpaceInWriteBuffer), + toTransfer); + writer.write(writeBuffer, 0, (DISK_WRITE_BUFFER_SIZE - freeSpaceInWriteBuffer) + toTransfer); + recordReadPosition += toTransfer; + dataRemaining -= toTransfer; + freeSpaceInWriteBuffer = DISK_WRITE_BUFFER_SIZE; + } + if (freeSpaceInWriteBuffer < DISK_WRITE_BUFFER_SIZE) { + writer.write(writeBuffer, 0, (DISK_WRITE_BUFFER_SIZE - freeSpaceInWriteBuffer)); + } + writer.recordWritten(); + } + + public void close() throws IOException { + writer.commitAndClose(); + writer = null; + writeBuffer = null; + } + + public UnsafeSorterSpillReader getReader(BlockManager blockManager) throws IOException { + return new UnsafeSorterSpillReader(blockManager, file, blockId); + } +} diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index 49329423dca76..0c50b4002cf7b 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -102,7 +102,7 @@ private[spark] class ExecutorAllocationManager( "spark.dynamicAllocation.executorIdleTimeout", "60s") private val cachedExecutorIdleTimeoutS = conf.getTimeAsSeconds( - "spark.dynamicAllocation.cachedExecutorIdleTimeout", s"${2 * executorIdleTimeoutS}s") + "spark.dynamicAllocation.cachedExecutorIdleTimeout", s"${Integer.MAX_VALUE}s") // During testing, the methods to actually kill and add executors are mocked out private val testing = conf.getBoolean("spark.dynamicAllocation.testing", false) diff --git a/core/src/main/scala/org/apache/spark/Logging.scala b/core/src/main/scala/org/apache/spark/Logging.scala index 7fcb7830e7b0b..f0598816d6c07 100644 --- a/core/src/main/scala/org/apache/spark/Logging.scala +++ b/core/src/main/scala/org/apache/spark/Logging.scala @@ -121,6 +121,7 @@ trait Logging { if (usingLog4j12) { val log4j12Initialized = LogManager.getRootLogger.getAllAppenders.hasMoreElements if (!log4j12Initialized) { + // scalastyle:off println if (Utils.isInInterpreter) { val replDefaultLogProps = "org/apache/spark/log4j-defaults-repl.properties" Option(Utils.getSparkClassLoader.getResource(replDefaultLogProps)) match { @@ -141,6 +142,7 @@ trait Logging { System.err.println(s"Spark was unable to load $defaultLogProps") } } + // scalastyle:on println } } Logging.initialized = true @@ -157,7 +159,7 @@ private object Logging { try { // We use reflection here to handle the case where users remove the // slf4j-to-jul bridge order to route their logs to JUL. - val bridgeClass = Class.forName("org.slf4j.bridge.SLF4JBridgeHandler") + val bridgeClass = Utils.classForName("org.slf4j.bridge.SLF4JBridgeHandler") bridgeClass.getMethod("removeHandlersForRootLogger").invoke(null) val installed = bridgeClass.getMethod("isInstalled").invoke(null).asInstanceOf[Boolean] if (!installed) { diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index d2547eeff2b4e..bd1cc332a63e7 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -532,7 +532,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli _executorAllocationManager = if (dynamicAllocationEnabled) { assert(supportDynamicAllocation, - "Dynamic allocation of executors is currently only supported in YARN mode") + "Dynamic allocation of executors is currently only supported in YARN and Mesos mode") Some(new ExecutorAllocationManager(this, listenerBus, _conf)) } else { None @@ -853,7 +853,6 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli minPartitions).setName(path) } - /** * :: Experimental :: * @@ -1364,10 +1363,14 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli /** * Return whether dynamically adjusting the amount of resources allocated to - * this application is supported. This is currently only available for YARN. + * this application is supported. This is currently only available for YARN + * and Mesos coarse-grained mode. */ - private[spark] def supportDynamicAllocation = - master.contains("yarn") || _conf.getBoolean("spark.dynamicAllocation.testing", false) + private[spark] def supportDynamicAllocation: Boolean = { + (master.contains("yarn") + || master.contains("mesos") + || _conf.getBoolean("spark.dynamicAllocation.testing", false)) + } /** * :: DeveloperApi :: @@ -1385,7 +1388,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli */ private[spark] override def requestTotalExecutors(numExecutors: Int): Boolean = { assert(supportDynamicAllocation, - "Requesting executors is currently only supported in YARN mode") + "Requesting executors is currently only supported in YARN and Mesos modes") schedulerBackend match { case b: CoarseGrainedSchedulerBackend => b.requestTotalExecutors(numExecutors) @@ -1403,7 +1406,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli @DeveloperApi override def requestExecutors(numAdditionalExecutors: Int): Boolean = { assert(supportDynamicAllocation, - "Requesting executors is currently only supported in YARN mode") + "Requesting executors is currently only supported in YARN and Mesos modes") schedulerBackend match { case b: CoarseGrainedSchedulerBackend => b.requestExecutors(numAdditionalExecutors) @@ -1421,7 +1424,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli @DeveloperApi override def killExecutors(executorIds: Seq[String]): Boolean = { assert(supportDynamicAllocation, - "Killing executors is currently only supported in YARN mode") + "Killing executors is currently only supported in YARN and Mesos modes") schedulerBackend match { case b: CoarseGrainedSchedulerBackend => b.killExecutors(executorIds) @@ -1965,7 +1968,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli for (className <- listenerClassNames) { // Use reflection to find the right constructor val constructors = { - val listenerClass = Class.forName(className) + val listenerClass = Utils.classForName(className) listenerClass.getConstructors.asInstanceOf[Array[Constructor[_ <: SparkListener]]] } val constructorTakingSparkConf = constructors.find { c => @@ -2500,7 +2503,7 @@ object SparkContext extends Logging { "\"yarn-standalone\" is deprecated as of Spark 1.0. Use \"yarn-cluster\" instead.") } val scheduler = try { - val clazz = Class.forName("org.apache.spark.scheduler.cluster.YarnClusterScheduler") + val clazz = Utils.classForName("org.apache.spark.scheduler.cluster.YarnClusterScheduler") val cons = clazz.getConstructor(classOf[SparkContext]) cons.newInstance(sc).asInstanceOf[TaskSchedulerImpl] } catch { @@ -2512,7 +2515,7 @@ object SparkContext extends Logging { } val backend = try { val clazz = - Class.forName("org.apache.spark.scheduler.cluster.YarnClusterSchedulerBackend") + Utils.classForName("org.apache.spark.scheduler.cluster.YarnClusterSchedulerBackend") val cons = clazz.getConstructor(classOf[TaskSchedulerImpl], classOf[SparkContext]) cons.newInstance(scheduler, sc).asInstanceOf[CoarseGrainedSchedulerBackend] } catch { @@ -2525,8 +2528,7 @@ object SparkContext extends Logging { case "yarn-client" => val scheduler = try { - val clazz = - Class.forName("org.apache.spark.scheduler.cluster.YarnScheduler") + val clazz = Utils.classForName("org.apache.spark.scheduler.cluster.YarnScheduler") val cons = clazz.getConstructor(classOf[SparkContext]) cons.newInstance(sc).asInstanceOf[TaskSchedulerImpl] @@ -2538,7 +2540,7 @@ object SparkContext extends Logging { val backend = try { val clazz = - Class.forName("org.apache.spark.scheduler.cluster.YarnClientSchedulerBackend") + Utils.classForName("org.apache.spark.scheduler.cluster.YarnClientSchedulerBackend") val cons = clazz.getConstructor(classOf[TaskSchedulerImpl], classOf[SparkContext]) cons.newInstance(scheduler, sc).asInstanceOf[CoarseGrainedSchedulerBackend] } catch { diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index d18fc599e9890..adfece4d6e7c0 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -261,7 +261,7 @@ object SparkEnv extends Logging { // Create an instance of the class with the given name, possibly initializing it with our conf def instantiateClass[T](className: String): T = { - val cls = Class.forName(className, true, Utils.getContextOrSparkClassLoader) + val cls = Utils.classForName(className) // Look for a constructor taking a SparkConf and a boolean isDriver, then one taking just // SparkConf, then one taking no arguments try { diff --git a/core/src/main/scala/org/apache/spark/api/r/RBackend.scala b/core/src/main/scala/org/apache/spark/api/r/RBackend.scala index 1a5f2bca26c2b..b7e72d4d0ed0b 100644 --- a/core/src/main/scala/org/apache/spark/api/r/RBackend.scala +++ b/core/src/main/scala/org/apache/spark/api/r/RBackend.scala @@ -95,7 +95,9 @@ private[spark] class RBackend { private[spark] object RBackend extends Logging { def main(args: Array[String]): Unit = { if (args.length < 1) { + // scalastyle:off println System.err.println("Usage: RBackend ") + // scalastyle:on println System.exit(-1) } val sparkRBackend = new RBackend() diff --git a/core/src/main/scala/org/apache/spark/api/r/RBackendHandler.scala b/core/src/main/scala/org/apache/spark/api/r/RBackendHandler.scala index 4b8f7fe9242e0..9658e9a696ffa 100644 --- a/core/src/main/scala/org/apache/spark/api/r/RBackendHandler.scala +++ b/core/src/main/scala/org/apache/spark/api/r/RBackendHandler.scala @@ -26,6 +26,7 @@ import io.netty.channel.{ChannelHandlerContext, SimpleChannelInboundHandler} import org.apache.spark.Logging import org.apache.spark.api.r.SerDe._ +import org.apache.spark.util.Utils /** * Handler for RBackend @@ -88,21 +89,6 @@ private[r] class RBackendHandler(server: RBackend) ctx.close() } - // Looks up a class given a class name. This function first checks the - // current class loader and if a class is not found, it looks up the class - // in the context class loader. Address [SPARK-5185] - def getStaticClass(objId: String): Class[_] = { - try { - val clsCurrent = Class.forName(objId) - clsCurrent - } catch { - // Use contextLoader if we can't find the JAR in the system class loader - case e: ClassNotFoundException => - val clsContext = Class.forName(objId, true, Thread.currentThread().getContextClassLoader) - clsContext - } - } - def handleMethodCall( isStatic: Boolean, objId: String, @@ -113,7 +99,7 @@ private[r] class RBackendHandler(server: RBackend) var obj: Object = null try { val cls = if (isStatic) { - getStaticClass(objId) + Utils.classForName(objId) } else { JVMObjectTracker.get(objId) match { case None => throw new IllegalArgumentException("Object not found " + objId) diff --git a/core/src/main/scala/org/apache/spark/api/r/RRDD.scala b/core/src/main/scala/org/apache/spark/api/r/RRDD.scala index 524676544d6f5..23a470d6afcae 100644 --- a/core/src/main/scala/org/apache/spark/api/r/RRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/r/RRDD.scala @@ -39,7 +39,6 @@ private abstract class BaseRRDD[T: ClassTag, U: ClassTag]( deserializer: String, serializer: String, packageNames: Array[Byte], - rLibDir: String, broadcastVars: Array[Broadcast[Object]]) extends RDD[U](parent) with Logging { protected var dataStream: DataInputStream = _ @@ -60,7 +59,7 @@ private abstract class BaseRRDD[T: ClassTag, U: ClassTag]( // The stdout/stderr is shared by multiple tasks, because we use one daemon // to launch child process as worker. - val errThread = RRDD.createRWorker(rLibDir, listenPort) + val errThread = RRDD.createRWorker(listenPort) // We use two sockets to separate input and output, then it's easy to manage // the lifecycle of them to avoid deadlock. @@ -161,7 +160,9 @@ private abstract class BaseRRDD[T: ClassTag, U: ClassTag]( dataOut.write(elem.asInstanceOf[Array[Byte]]) } else if (deserializer == SerializationFormats.STRING) { // write string(for StringRRDD) + // scalastyle:off println printOut.println(elem) + // scalastyle:on println } } @@ -233,11 +234,10 @@ private class PairwiseRRDD[T: ClassTag]( hashFunc: Array[Byte], deserializer: String, packageNames: Array[Byte], - rLibDir: String, broadcastVars: Array[Object]) extends BaseRRDD[T, (Int, Array[Byte])]( parent, numPartitions, hashFunc, deserializer, - SerializationFormats.BYTE, packageNames, rLibDir, + SerializationFormats.BYTE, packageNames, broadcastVars.map(x => x.asInstanceOf[Broadcast[Object]])) { override protected def readData(length: Int): (Int, Array[Byte]) = { @@ -264,10 +264,9 @@ private class RRDD[T: ClassTag]( deserializer: String, serializer: String, packageNames: Array[Byte], - rLibDir: String, broadcastVars: Array[Object]) extends BaseRRDD[T, Array[Byte]]( - parent, -1, func, deserializer, serializer, packageNames, rLibDir, + parent, -1, func, deserializer, serializer, packageNames, broadcastVars.map(x => x.asInstanceOf[Broadcast[Object]])) { override protected def readData(length: Int): Array[Byte] = { @@ -291,10 +290,9 @@ private class StringRRDD[T: ClassTag]( func: Array[Byte], deserializer: String, packageNames: Array[Byte], - rLibDir: String, broadcastVars: Array[Object]) extends BaseRRDD[T, String]( - parent, -1, func, deserializer, SerializationFormats.STRING, packageNames, rLibDir, + parent, -1, func, deserializer, SerializationFormats.STRING, packageNames, broadcastVars.map(x => x.asInstanceOf[Broadcast[Object]])) { override protected def readData(length: Int): String = { @@ -390,9 +388,10 @@ private[r] object RRDD { thread } - private def createRProcess(rLibDir: String, port: Int, script: String): BufferedStreamThread = { + private def createRProcess(port: Int, script: String): BufferedStreamThread = { val rCommand = SparkEnv.get.conf.get("spark.sparkr.r.command", "Rscript") val rOptions = "--vanilla" + val rLibDir = RUtils.sparkRPackagePath(isDriver = false) val rExecScript = rLibDir + "/SparkR/worker/" + script val pb = new ProcessBuilder(List(rCommand, rOptions, rExecScript)) // Unset the R_TESTS environment variable for workers. @@ -411,7 +410,7 @@ private[r] object RRDD { /** * ProcessBuilder used to launch worker R processes. */ - def createRWorker(rLibDir: String, port: Int): BufferedStreamThread = { + def createRWorker(port: Int): BufferedStreamThread = { val useDaemon = SparkEnv.get.conf.getBoolean("spark.sparkr.use.daemon", true) if (!Utils.isWindows && useDaemon) { synchronized { @@ -419,7 +418,7 @@ private[r] object RRDD { // we expect one connections val serverSocket = new ServerSocket(0, 1, InetAddress.getByName("localhost")) val daemonPort = serverSocket.getLocalPort - errThread = createRProcess(rLibDir, daemonPort, "daemon.R") + errThread = createRProcess(daemonPort, "daemon.R") // the socket used to send out the input of task serverSocket.setSoTimeout(10000) val sock = serverSocket.accept() @@ -441,7 +440,7 @@ private[r] object RRDD { errThread } } else { - createRProcess(rLibDir, port, "worker.R") + createRProcess(port, "worker.R") } } diff --git a/core/src/main/scala/org/apache/spark/api/r/RUtils.scala b/core/src/main/scala/org/apache/spark/api/r/RUtils.scala new file mode 100644 index 0000000000000..d53abd3408c55 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/api/r/RUtils.scala @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.api.r + +import java.io.File + +import org.apache.spark.{SparkEnv, SparkException} + +private[spark] object RUtils { + /** + * Get the SparkR package path in the local spark distribution. + */ + def localSparkRPackagePath: Option[String] = { + val sparkHome = sys.env.get("SPARK_HOME") + sparkHome.map( + Seq(_, "R", "lib").mkString(File.separator) + ) + } + + /** + * Get the SparkR package path in various deployment modes. + * This assumes that Spark properties `spark.master` and `spark.submit.deployMode` + * and environment variable `SPARK_HOME` are set. + */ + def sparkRPackagePath(isDriver: Boolean): String = { + val (master, deployMode) = + if (isDriver) { + (sys.props("spark.master"), sys.props("spark.submit.deployMode")) + } else { + val sparkConf = SparkEnv.get.conf + (sparkConf.get("spark.master"), sparkConf.get("spark.submit.deployMode")) + } + + val isYarnCluster = master.contains("yarn") && deployMode == "cluster" + val isYarnClient = master.contains("yarn") && deployMode == "client" + + // In YARN mode, the SparkR package is distributed as an archive symbolically + // linked to the "sparkr" file in the current directory. Note that this does not apply + // to the driver in client mode because it is run outside of the cluster. + if (isYarnCluster || (isYarnClient && !isDriver)) { + new File("sparkr").getAbsolutePath + } else { + // Otherwise, assume the package is local + // TODO: support this for Mesos + localSparkRPackagePath.getOrElse { + throw new SparkException("SPARK_HOME not set. Can't locate SparkR package.") + } + } + } +} diff --git a/core/src/main/scala/org/apache/spark/broadcast/BroadcastManager.scala b/core/src/main/scala/org/apache/spark/broadcast/BroadcastManager.scala index 685313ac009ba..fac6666bb3410 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/BroadcastManager.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/BroadcastManager.scala @@ -22,6 +22,7 @@ import java.util.concurrent.atomic.AtomicLong import scala.reflect.ClassTag import org.apache.spark._ +import org.apache.spark.util.Utils private[spark] class BroadcastManager( val isDriver: Boolean, @@ -42,7 +43,7 @@ private[spark] class BroadcastManager( conf.get("spark.broadcast.factory", "org.apache.spark.broadcast.TorrentBroadcastFactory") broadcastFactory = - Class.forName(broadcastFactoryClass).newInstance.asInstanceOf[BroadcastFactory] + Utils.classForName(broadcastFactoryClass).newInstance.asInstanceOf[BroadcastFactory] // Initialize appropriate BroadcastFactory and BroadcastObject broadcastFactory.initialize(isDriver, conf, securityManager) 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 71f7e2129116f..f03875a3e8c89 100644 --- a/core/src/main/scala/org/apache/spark/deploy/Client.scala +++ b/core/src/main/scala/org/apache/spark/deploy/Client.scala @@ -118,26 +118,26 @@ private class ClientEndpoint( def pollAndReportStatus(driverId: String) { // Since ClientEndpoint is the only RpcEndpoint in the process, blocking the event loop thread // is fine. - println("... waiting before polling master for driver state") + logInfo("... waiting before polling master for driver state") Thread.sleep(5000) - println("... polling master for driver state") + logInfo("... polling master for driver state") val statusResponse = activeMasterEndpoint.askWithRetry[DriverStatusResponse](RequestDriverStatus(driverId)) statusResponse.found match { case false => - println(s"ERROR: Cluster master did not recognize $driverId") + logError(s"ERROR: Cluster master did not recognize $driverId") System.exit(-1) case true => - println(s"State of $driverId is ${statusResponse.state.get}") + logInfo(s"State of $driverId is ${statusResponse.state.get}") // Worker node, if present (statusResponse.workerId, statusResponse.workerHostPort, statusResponse.state) match { case (Some(id), Some(hostPort), Some(DriverState.RUNNING)) => - println(s"Driver running on $hostPort ($id)") + logInfo(s"Driver running on $hostPort ($id)") case _ => } // Exception, if present statusResponse.exception.map { e => - println(s"Exception from cluster was: $e") + logError(s"Exception from cluster was: $e") e.printStackTrace() System.exit(-1) } @@ -148,7 +148,7 @@ private class ClientEndpoint( override def receive: PartialFunction[Any, Unit] = { case SubmitDriverResponse(master, success, driverId, message) => - println(message) + logInfo(message) if (success) { activeMasterEndpoint = master pollAndReportStatus(driverId.get) @@ -158,7 +158,7 @@ private class ClientEndpoint( case KillDriverResponse(master, driverId, success, message) => - println(message) + logInfo(message) if (success) { activeMasterEndpoint = master pollAndReportStatus(driverId) @@ -169,13 +169,13 @@ private class ClientEndpoint( override def onDisconnected(remoteAddress: RpcAddress): Unit = { if (!lostMasters.contains(remoteAddress)) { - println(s"Error connecting to master $remoteAddress.") + logError(s"Error connecting to master $remoteAddress.") lostMasters += remoteAddress // Note that this heuristic does not account for the fact that a Master can recover within // the lifetime of this client. Thus, once a Master is lost it is lost to us forever. This // is not currently a concern, however, because this client does not retry submissions. if (lostMasters.size >= masterEndpoints.size) { - println("No master is available, exiting.") + logError("No master is available, exiting.") System.exit(-1) } } @@ -183,18 +183,18 @@ private class ClientEndpoint( override def onNetworkError(cause: Throwable, remoteAddress: RpcAddress): Unit = { if (!lostMasters.contains(remoteAddress)) { - println(s"Error connecting to master ($remoteAddress).") - println(s"Cause was: $cause") + logError(s"Error connecting to master ($remoteAddress).") + logError(s"Cause was: $cause") lostMasters += remoteAddress if (lostMasters.size >= masterEndpoints.size) { - println("No master is available, exiting.") + logError("No master is available, exiting.") System.exit(-1) } } } override def onError(cause: Throwable): Unit = { - println(s"Error processing messages, exiting.") + logError(s"Error processing messages, exiting.") cause.printStackTrace() System.exit(-1) } @@ -209,10 +209,12 @@ private class ClientEndpoint( */ object Client { def main(args: Array[String]) { + // scalastyle:off println if (!sys.props.contains("SPARK_SUBMIT")) { println("WARNING: This client is deprecated and will be removed in a future version of Spark") println("Use ./bin/spark-submit with \"--master spark://host:port\"") } + // scalastyle:on println val conf = new SparkConf() val driverArgs = new ClientArguments(args) 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 42d3296062e6d..72cc330a398da 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala @@ -72,9 +72,11 @@ private[deploy] class ClientArguments(args: Array[String]) { cmd = "launch" if (!ClientArguments.isValidJarUrl(_jarUrl)) { + // scalastyle:off println println(s"Jar url '${_jarUrl}' is not in valid format.") println(s"Must be a jar file path in URL format " + "(e.g. hdfs://host:port/XX.jar, file:///XX.jar)") + // scalastyle:on println printUsageAndExit(-1) } @@ -110,7 +112,9 @@ private[deploy] class ClientArguments(args: Array[String]) { | (default: $DEFAULT_SUPERVISE) | -v, --verbose Print more debugging output """.stripMargin + // scalastyle:off println System.err.println(usage) + // scalastyle:on println System.exit(exitCode) } } diff --git a/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala index 2954f932b4f41..ccffb36652988 100644 --- a/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala @@ -76,12 +76,13 @@ private[deploy] object JsonProtocol { } def writeMasterState(obj: MasterStateResponse): JObject = { + val aliveWorkers = obj.workers.filter(_.isAlive()) ("url" -> obj.uri) ~ ("workers" -> obj.workers.toList.map(writeWorkerInfo)) ~ - ("cores" -> obj.workers.map(_.cores).sum) ~ - ("coresused" -> obj.workers.map(_.coresUsed).sum) ~ - ("memory" -> obj.workers.map(_.memory).sum) ~ - ("memoryused" -> obj.workers.map(_.memoryUsed).sum) ~ + ("cores" -> aliveWorkers.map(_.cores).sum) ~ + ("coresused" -> aliveWorkers.map(_.coresUsed).sum) ~ + ("memory" -> aliveWorkers.map(_.memory).sum) ~ + ("memoryused" -> aliveWorkers.map(_.memoryUsed).sum) ~ ("activeapps" -> obj.activeApps.toList.map(writeApplicationInfo)) ~ ("completedapps" -> obj.completedApps.toList.map(writeApplicationInfo)) ~ ("activedrivers" -> obj.activeDrivers.toList.map(writeDriverInfo)) ~ diff --git a/core/src/main/scala/org/apache/spark/deploy/RRunner.scala b/core/src/main/scala/org/apache/spark/deploy/RRunner.scala index e99779f299785..c0cab22fa8252 100644 --- a/core/src/main/scala/org/apache/spark/deploy/RRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/RRunner.scala @@ -24,7 +24,7 @@ import scala.collection.JavaConversions._ import org.apache.hadoop.fs.Path -import org.apache.spark.api.r.RBackend +import org.apache.spark.api.r.{RBackend, RUtils} import org.apache.spark.util.RedirectThread /** @@ -71,9 +71,10 @@ object RRunner { val builder = new ProcessBuilder(Seq(rCommand, rFileNormalized) ++ otherArgs) val env = builder.environment() env.put("EXISTING_SPARKR_BACKEND_PORT", sparkRBackendPort.toString) - val sparkHome = System.getenv("SPARK_HOME") + val rPackageDir = RUtils.sparkRPackagePath(isDriver = true) + env.put("SPARKR_PACKAGE_DIR", rPackageDir) env.put("R_PROFILE_USER", - Seq(sparkHome, "R", "lib", "SparkR", "profile", "general.R").mkString(File.separator)) + Seq(rPackageDir, "SparkR", "profile", "general.R").mkString(File.separator)) builder.redirectErrorStream(true) // Ugly but needed for stdout and stderr to synchronize val process = builder.start() @@ -85,7 +86,9 @@ object RRunner { } System.exit(returnCode) } else { + // scalastyle:off println System.err.println("SparkR backend did not initialize in " + backendTimeout + " seconds") + // scalastyle:on println System.exit(-1) } } diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index 6d14590a1d192..9f94118829ff1 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -178,7 +178,7 @@ class SparkHadoopUtil extends Logging { private def getFileSystemThreadStatisticsMethod(methodName: String): Method = { val statisticsDataClass = - Class.forName("org.apache.hadoop.fs.FileSystem$Statistics$StatisticsData") + Utils.classForName("org.apache.hadoop.fs.FileSystem$Statistics$StatisticsData") statisticsDataClass.getDeclaredMethod(methodName) } @@ -356,7 +356,7 @@ object SparkHadoopUtil { System.getProperty("SPARK_YARN_MODE", System.getenv("SPARK_YARN_MODE"))) if (yarnMode) { try { - Class.forName("org.apache.spark.deploy.yarn.YarnSparkHadoopUtil") + Utils.classForName("org.apache.spark.deploy.yarn.YarnSparkHadoopUtil") .newInstance() .asInstanceOf[SparkHadoopUtil] } catch { 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 b1d6ec209d62b..036cb6e054791 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -37,6 +37,7 @@ import org.apache.ivy.core.settings.IvySettings import org.apache.ivy.plugins.matcher.GlobPatternMatcher import org.apache.ivy.plugins.repository.file.FileRepository import org.apache.ivy.plugins.resolver.{FileSystemResolver, ChainResolver, IBiblioResolver} +import org.apache.spark.api.r.RUtils import org.apache.spark.SPARK_VERSION import org.apache.spark.deploy.rest._ import org.apache.spark.util.{ChildFirstURLClassLoader, MutableURLClassLoader, Utils} @@ -79,9 +80,11 @@ object SparkSubmit { private val SPARK_SHELL = "spark-shell" private val PYSPARK_SHELL = "pyspark-shell" private val SPARKR_SHELL = "sparkr-shell" + private val SPARKR_PACKAGE_ARCHIVE = "sparkr.zip" private val CLASS_NOT_FOUND_EXIT_STATUS = 101 + // scalastyle:off println // Exposed for testing private[spark] var exitFn: Int => Unit = (exitCode: Int) => System.exit(exitCode) private[spark] var printStream: PrintStream = System.err @@ -102,11 +105,14 @@ object SparkSubmit { printStream.println("Type --help for more information.") exitFn(0) } + // scalastyle:on println def main(args: Array[String]): Unit = { val appArgs = new SparkSubmitArguments(args) if (appArgs.verbose) { + // scalastyle:off println printStream.println(appArgs) + // scalastyle:on println } appArgs.action match { case SparkSubmitAction.SUBMIT => submit(appArgs) @@ -160,7 +166,9 @@ object SparkSubmit { // makes the message printed to the output by the JVM not very helpful. Instead, // detect exceptions with empty stack traces here, and treat them differently. if (e.getStackTrace().length == 0) { + // scalastyle:off println printStream.println(s"ERROR: ${e.getClass().getName()}: ${e.getMessage()}") + // scalastyle:on println exitFn(1) } else { throw e @@ -178,7 +186,9 @@ object SparkSubmit { // to use the legacy gateway if the master endpoint turns out to be not a REST server. if (args.isStandaloneCluster && args.useRest) { try { + // scalastyle:off println printStream.println("Running Spark using the REST application submission protocol.") + // scalastyle:on println doRunMain() } catch { // Fail over to use the legacy submission gateway @@ -254,6 +264,12 @@ object SparkSubmit { } } + // Update args.deployMode if it is null. It will be passed down as a Spark property later. + (args.deployMode, deployMode) match { + case (null, CLIENT) => args.deployMode = "client" + case (null, CLUSTER) => args.deployMode = "cluster" + case _ => + } val isYarnCluster = clusterManager == YARN && deployMode == CLUSTER val isMesosCluster = clusterManager == MESOS && deployMode == CLUSTER @@ -339,6 +355,23 @@ object SparkSubmit { } } + // In YARN mode for an R app, add the SparkR package archive to archives + // that can be distributed with the job + if (args.isR && clusterManager == YARN) { + val rPackagePath = RUtils.localSparkRPackagePath + if (rPackagePath.isEmpty) { + printErrorAndExit("SPARK_HOME does not exist for R application in YARN mode.") + } + val rPackageFile = new File(rPackagePath.get, SPARKR_PACKAGE_ARCHIVE) + if (!rPackageFile.exists()) { + printErrorAndExit(s"$SPARKR_PACKAGE_ARCHIVE does not exist for R application in YARN mode.") + } + val localURI = Utils.resolveURI(rPackageFile.getAbsolutePath) + + // Assigns a symbol link name "sparkr" to the shipped package. + args.archives = mergeFileLists(args.archives, localURI.toString + "#sparkr") + } + // If we're running a R app, set the main class to our specific R runner if (args.isR && deployMode == CLIENT) { if (args.primaryResource == SPARKR_SHELL) { @@ -367,6 +400,8 @@ object SparkSubmit { // All cluster managers OptionAssigner(args.master, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES, sysProp = "spark.master"), + OptionAssigner(args.deployMode, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES, + sysProp = "spark.submit.deployMode"), OptionAssigner(args.name, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES, sysProp = "spark.app.name"), OptionAssigner(args.jars, ALL_CLUSTER_MGRS, CLIENT, sysProp = "spark.jars"), OptionAssigner(args.ivyRepoPath, ALL_CLUSTER_MGRS, CLIENT, sysProp = "spark.jars.ivy"), @@ -558,6 +593,7 @@ object SparkSubmit { sysProps: Map[String, String], childMainClass: String, verbose: Boolean): Unit = { + // scalastyle:off println if (verbose) { printStream.println(s"Main class:\n$childMainClass") printStream.println(s"Arguments:\n${childArgs.mkString("\n")}") @@ -565,6 +601,7 @@ object SparkSubmit { printStream.println(s"Classpath elements:\n${childClasspath.mkString("\n")}") printStream.println("\n") } + // scalastyle:on println val loader = if (sysProps.getOrElse("spark.driver.userClassPathFirst", "false").toBoolean) { @@ -587,13 +624,15 @@ object SparkSubmit { var mainClass: Class[_] = null try { - mainClass = Class.forName(childMainClass, true, loader) + mainClass = Utils.classForName(childMainClass) } catch { case e: ClassNotFoundException => e.printStackTrace(printStream) if (childMainClass.contains("thriftserver")) { + // scalastyle:off println printStream.println(s"Failed to load main class $childMainClass.") printStream.println("You need to build Spark with -Phive and -Phive-thriftserver.") + // scalastyle:on println } System.exit(CLASS_NOT_FOUND_EXIT_STATUS) } @@ -766,7 +805,9 @@ private[spark] object SparkSubmitUtils { brr.setRoot(repo) brr.setName(s"repo-${i + 1}") cr.add(brr) + // scalastyle:off println printStream.println(s"$repo added as a remote repository with the name: ${brr.getName}") + // scalastyle:on println } } @@ -829,7 +870,9 @@ private[spark] object SparkSubmitUtils { val ri = ModuleRevisionId.newInstance(mvn.groupId, mvn.artifactId, mvn.version) val dd = new DefaultDependencyDescriptor(ri, false, false) dd.addDependencyConfiguration(ivyConfName, ivyConfName) + // scalastyle:off println printStream.println(s"${dd.getDependencyId} added as a dependency") + // scalastyle:on println md.addDependency(dd) } } @@ -896,9 +939,11 @@ private[spark] object SparkSubmitUtils { ivySettings.setDefaultCache(new File(alternateIvyCache, "cache")) new File(alternateIvyCache, "jars") } + // scalastyle:off println printStream.println( s"Ivy Default Cache set to: ${ivySettings.getDefaultCache.getAbsolutePath}") printStream.println(s"The jars for the packages stored in: $packagesDirectory") + // scalastyle:on println // create a pattern matcher ivySettings.addMatcher(new GlobPatternMatcher) // create the dependency resolvers 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 73ab18332feb4..b3710073e330c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -79,6 +79,7 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S /** Default properties present in the currently defined defaults file. */ lazy val defaultSparkProperties: HashMap[String, String] = { val defaultProperties = new HashMap[String, String]() + // scalastyle:off println if (verbose) SparkSubmit.printStream.println(s"Using properties file: $propertiesFile") Option(propertiesFile).foreach { filename => Utils.getPropertiesFromFile(filename).foreach { case (k, v) => @@ -86,6 +87,7 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S if (verbose) SparkSubmit.printStream.println(s"Adding default property: $k=$v") } } + // scalastyle:on println defaultProperties } @@ -162,6 +164,7 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S .orNull executorCores = Option(executorCores) .orElse(sparkProperties.get("spark.executor.cores")) + .orElse(env.get("SPARK_EXECUTOR_CORES")) .orNull totalExecutorCores = Option(totalExecutorCores) .orElse(sparkProperties.get("spark.cores.max")) @@ -451,6 +454,7 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S } private def printUsageAndExit(exitCode: Int, unknownParam: Any = null): Unit = { + // scalastyle:off println val outStream = SparkSubmit.printStream if (unknownParam != null) { outStream.println("Unknown/unsupported param " + unknownParam) @@ -540,6 +544,7 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S outStream.println("CLI options:") outStream.println(getSqlShellOptions()) } + // scalastyle:on println SparkSubmit.exitFn(exitCode) } @@ -571,7 +576,7 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S System.setSecurityManager(sm) try { - Class.forName(mainClass).getMethod("main", classOf[Array[String]]) + Utils.classForName(mainClass).getMethod("main", classOf[Array[String]]) .invoke(null, Array(HELP)) } catch { case e: InvocationTargetException => diff --git a/core/src/main/scala/org/apache/spark/deploy/client/TestExecutor.scala b/core/src/main/scala/org/apache/spark/deploy/client/TestExecutor.scala index c5ac45c6730d3..a98b1fa8f83a1 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/TestExecutor.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/TestExecutor.scala @@ -19,7 +19,9 @@ package org.apache.spark.deploy.client private[spark] object TestExecutor { def main(args: Array[String]) { + // scalastyle:off println println("Hello world!") + // scalastyle:on println while (true) { Thread.sleep(1000) } diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index 10638afb74900..a076a9c3f984d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -228,7 +228,7 @@ object HistoryServer extends Logging { val providerName = conf.getOption("spark.history.provider") .getOrElse(classOf[FsHistoryProvider].getName()) - val provider = Class.forName(providerName) + val provider = Utils.classForName(providerName) .getConstructor(classOf[SparkConf]) .newInstance(conf) .asInstanceOf[ApplicationHistoryProvider] diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala index 4692d22651c93..18265df9faa2c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala @@ -56,6 +56,7 @@ private[history] class HistoryServerArguments(conf: SparkConf, args: Array[Strin Utils.loadDefaultSparkProperties(conf, propertiesFile) private def printUsageAndExit(exitCode: Int) { + // scalastyle:off println System.err.println( """ |Usage: HistoryServer [options] @@ -84,6 +85,7 @@ private[history] class HistoryServerArguments(conf: SparkConf, args: Array[Strin | spark.history.fs.updateInterval How often to reload log data from storage | (in seconds, default: 10) |""".stripMargin) + // scalastyle:on println System.exit(exitCode) } 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 48070768f6edb..245b047e7dfbd 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 @@ -172,7 +172,7 @@ private[master] class Master( new FileSystemRecoveryModeFactory(conf, SerializationExtension(actorSystem)) (fsFactory.createPersistenceEngine(), fsFactory.createLeaderElectionAgent(this)) case "CUSTOM" => - val clazz = Class.forName(conf.get("spark.deploy.recoveryMode.factory")) + val clazz = Utils.classForName(conf.get("spark.deploy.recoveryMode.factory")) val factory = clazz.getConstructor(classOf[SparkConf], classOf[Serialization]) .newInstance(conf, SerializationExtension(actorSystem)) .asInstanceOf[StandaloneRecoveryModeFactory] diff --git a/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala b/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala index 435b9b12f83b8..44cefbc77f08e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala @@ -85,6 +85,7 @@ private[master] class MasterArguments(args: Array[String], conf: SparkConf) { * Print usage and exit JVM with the given exit code. */ private def printUsageAndExit(exitCode: Int) { + // scalastyle:off println System.err.println( "Usage: Master [options]\n" + "\n" + @@ -95,6 +96,7 @@ private[master] class MasterArguments(args: Array[String], conf: SparkConf) { " --webui-port PORT Port for web UI (default: 8080)\n" + " --properties-file FILE Path to a custom Spark properties file.\n" + " Default is conf/spark-defaults.conf.") + // scalastyle:on println System.exit(exitCode) } } 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 471811037e5e2..f751966605206 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 @@ -105,4 +105,6 @@ private[spark] class WorkerInfo( def setState(state: WorkerState.Value): Unit = { this.state = state } + + def isAlive(): Boolean = this.state == WorkerState.ALIVE } diff --git a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArguments.scala b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArguments.scala index 894cb78d8591a..5accaf78d0a51 100644 --- a/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArguments.scala @@ -54,7 +54,9 @@ private[mesos] class MesosClusterDispatcherArguments(args: Array[String], conf: case ("--master" | "-m") :: value :: tail => if (!value.startsWith("mesos://")) { + // scalastyle:off println System.err.println("Cluster dispatcher only supports mesos (uri begins with mesos://)") + // scalastyle:on println System.exit(1) } masterUrl = value.stripPrefix("mesos://") @@ -73,7 +75,9 @@ private[mesos] class MesosClusterDispatcherArguments(args: Array[String], conf: case Nil => { if (masterUrl == null) { + // scalastyle:off println System.err.println("--master is required") + // scalastyle:on println printUsageAndExit(1) } } @@ -83,6 +87,7 @@ private[mesos] class MesosClusterDispatcherArguments(args: Array[String], conf: } private def printUsageAndExit(exitCode: Int): Unit = { + // scalastyle:off println System.err.println( "Usage: MesosClusterDispatcher [options]\n" + "\n" + @@ -96,6 +101,7 @@ private[mesos] class MesosClusterDispatcherArguments(args: Array[String], conf: " Zookeeper for persistence\n" + " --properties-file FILE Path to a custom Spark properties file.\n" + " Default is conf/spark-defaults.conf.") + // scalastyle:on println System.exit(exitCode) } } diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolMessage.scala b/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolMessage.scala index e6615a3174ce1..ef5a7e35ad562 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolMessage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolMessage.scala @@ -128,7 +128,7 @@ private[spark] object SubmitRestProtocolMessage { */ def fromJson(json: String): SubmitRestProtocolMessage = { val className = parseAction(json) - val clazz = Class.forName(packagePrefix + "." + className) + val clazz = Utils.classForName(packagePrefix + "." + className) .asSubclass[SubmitRestProtocolMessage](classOf[SubmitRestProtocolMessage]) fromJson(json, clazz) } diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala index d1a12b01e78f7..6799f78ec0c19 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala @@ -53,14 +53,16 @@ object DriverWrapper { Thread.currentThread.setContextClassLoader(loader) // Delegate to supplied main class - val clazz = Class.forName(mainClass, true, loader) + val clazz = Utils.classForName(mainClass) val mainMethod = clazz.getMethod("main", classOf[Array[String]]) mainMethod.invoke(null, extraArgs.toArray[String]) rpcEnv.shutdown() case _ => + // scalastyle:off println System.err.println("Usage: DriverWrapper [options]") + // scalastyle:on println System.exit(-1) } } diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala index 1d2ecab517613..5181142c5f80e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala @@ -121,6 +121,7 @@ private[worker] class WorkerArguments(args: Array[String], conf: SparkConf) { * Print usage and exit JVM with the given exit code. */ def printUsageAndExit(exitCode: Int) { + // scalastyle:off println System.err.println( "Usage: Worker [options] \n" + "\n" + @@ -136,6 +137,7 @@ private[worker] class WorkerArguments(args: Array[String], conf: SparkConf) { " --webui-port PORT Port for web UI (default: 8081)\n" + " --properties-file FILE Path to a custom Spark properties file.\n" + " Default is conf/spark-defaults.conf.") + // scalastyle:on println System.exit(exitCode) } @@ -147,6 +149,7 @@ private[worker] class WorkerArguments(args: Array[String], conf: SparkConf) { val ibmVendor = System.getProperty("java.vendor").contains("IBM") var totalMb = 0 try { + // scalastyle:off classforname val bean = ManagementFactory.getOperatingSystemMXBean() if (ibmVendor) { val beanClass = Class.forName("com.ibm.lang.management.OperatingSystemMXBean") @@ -157,10 +160,13 @@ private[worker] class WorkerArguments(args: Array[String], conf: SparkConf) { val method = beanClass.getDeclaredMethod("getTotalPhysicalMemorySize") totalMb = (method.invoke(bean).asInstanceOf[Long] / 1024 / 1024).toInt } + // scalastyle:on classforname } catch { case e: Exception => { totalMb = 2*1024 + // scalastyle:off println System.out.println("Failed to get total physical memory. Using " + totalMb + " MB") + // scalastyle:on println } } // Leave out 1 GB for the operating system, but don't return a negative memory size 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 34d4cfdca7732..fcd76ec52742a 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -235,7 +235,9 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { argv = tail case Nil => case tail => + // scalastyle:off println System.err.println(s"Unrecognized options: ${tail.mkString(" ")}") + // scalastyle:on println printUsageAndExit() } } @@ -249,6 +251,7 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { } private def printUsageAndExit() = { + // scalastyle:off println System.err.println( """ |"Usage: CoarseGrainedExecutorBackend [options] @@ -262,6 +265,7 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { | --worker-url | --user-class-path |""".stripMargin) + // scalastyle:on println System.exit(1) } 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 8f916e0502ecb..1a02051c87f19 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -356,7 +356,7 @@ private[spark] class Executor( logInfo("Using REPL class URI: " + classUri) try { val _userClassPathFirst: java.lang.Boolean = userClassPathFirst - val klass = Class.forName("org.apache.spark.repl.ExecutorClassLoader") + val klass = Utils.classForName("org.apache.spark.repl.ExecutorClassLoader") .asInstanceOf[Class[_ <: ClassLoader]] val constructor = klass.getConstructor(classOf[SparkConf], classOf[String], classOf[ClassLoader], classOf[Boolean]) @@ -443,7 +443,7 @@ private[spark] class Executor( try { val response = heartbeatReceiverRef.askWithRetry[HeartbeatResponse](message) if (response.reregisterBlockManager) { - logWarning("Told to re-register on heartbeat") + logInfo("Told to re-register on heartbeat") env.blockManager.reregister() } } catch { 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 a3b4561b07e7f..e80feeeab4142 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -17,11 +17,15 @@ package org.apache.spark.executor +import java.io.{IOException, ObjectInputStream} +import java.util.concurrent.ConcurrentHashMap + import scala.collection.mutable.ArrayBuffer import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.DataReadMethod.DataReadMethod import org.apache.spark.storage.{BlockId, BlockStatus} +import org.apache.spark.util.Utils /** * :: DeveloperApi :: @@ -210,10 +214,26 @@ class TaskMetrics extends Serializable { private[spark] def updateInputMetrics(): Unit = synchronized { inputMetrics.foreach(_.updateBytesRead()) } + + @throws(classOf[IOException]) + private def readObject(in: ObjectInputStream): Unit = Utils.tryOrIOException { + in.defaultReadObject() + // Get the hostname from cached data, since hostname is the order of number of nodes in + // cluster, so using cached hostname will decrease the object number and alleviate the GC + // overhead. + _hostname = TaskMetrics.getCachedHostName(_hostname) + } } private[spark] object TaskMetrics { + private val hostNameCache = new ConcurrentHashMap[String, String]() + def empty: TaskMetrics = new TaskMetrics + + def getCachedHostName(host: String): String = { + val canonicalHost = hostNameCache.putIfAbsent(host, host) + if (canonicalHost != null) canonicalHost else host + } } /** diff --git a/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryInputFormat.scala b/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryInputFormat.scala index c219d21fbefa9..532850dd57716 100644 --- a/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryInputFormat.scala +++ b/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryInputFormat.scala @@ -21,6 +21,8 @@ import org.apache.hadoop.fs.Path import org.apache.hadoop.io.{BytesWritable, LongWritable} import org.apache.hadoop.mapreduce.lib.input.FileInputFormat import org.apache.hadoop.mapreduce.{InputSplit, JobContext, RecordReader, TaskAttemptContext} + +import org.apache.spark.Logging import org.apache.spark.deploy.SparkHadoopUtil /** @@ -39,7 +41,8 @@ private[spark] object FixedLengthBinaryInputFormat { } private[spark] class FixedLengthBinaryInputFormat - extends FileInputFormat[LongWritable, BytesWritable] { + extends FileInputFormat[LongWritable, BytesWritable] + with Logging { private var recordLength = -1 @@ -51,7 +54,7 @@ private[spark] class FixedLengthBinaryInputFormat recordLength = FixedLengthBinaryInputFormat.getRecordLength(context) } if (recordLength <= 0) { - println("record length is less than 0, file cannot be split") + logDebug("record length is less than 0, file cannot be split") false } else { true diff --git a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala index 0d8ac1f80a9f4..607d5a321efca 100644 --- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala +++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala @@ -63,8 +63,7 @@ private[spark] object CompressionCodec { def createCodec(conf: SparkConf, codecName: String): CompressionCodec = { val codecClass = shortCompressionCodecNames.getOrElse(codecName.toLowerCase, codecName) val codec = try { - val ctor = Class.forName(codecClass, true, Utils.getContextOrSparkClassLoader) - .getConstructor(classOf[SparkConf]) + val ctor = Utils.classForName(codecClass).getConstructor(classOf[SparkConf]) Some(ctor.newInstance(conf).asInstanceOf[CompressionCodec]) } catch { case e: ClassNotFoundException => None diff --git a/core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala b/core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala index 818f7a4c8d422..87df42748be44 100644 --- a/core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala +++ b/core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala @@ -26,6 +26,7 @@ import org.apache.hadoop.mapreduce.{OutputCommitter => MapReduceOutputCommitter} import org.apache.spark.executor.CommitDeniedException import org.apache.spark.{Logging, SparkEnv, TaskContext} +import org.apache.spark.util.{Utils => SparkUtils} private[spark] trait SparkHadoopMapRedUtil { @@ -64,10 +65,10 @@ trait SparkHadoopMapRedUtil { private def firstAvailableClass(first: String, second: String): Class[_] = { try { - Class.forName(first) + SparkUtils.classForName(first) } catch { case e: ClassNotFoundException => - Class.forName(second) + SparkUtils.classForName(second) } } } diff --git a/core/src/main/scala/org/apache/spark/mapreduce/SparkHadoopMapReduceUtil.scala b/core/src/main/scala/org/apache/spark/mapreduce/SparkHadoopMapReduceUtil.scala index 390d148bc97f9..943ebcb7bd0a1 100644 --- a/core/src/main/scala/org/apache/spark/mapreduce/SparkHadoopMapReduceUtil.scala +++ b/core/src/main/scala/org/apache/spark/mapreduce/SparkHadoopMapReduceUtil.scala @@ -21,6 +21,7 @@ import java.lang.{Boolean => JBoolean, Integer => JInteger} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.mapreduce.{JobContext, JobID, TaskAttemptContext, TaskAttemptID} +import org.apache.spark.util.Utils private[spark] trait SparkHadoopMapReduceUtil { @@ -46,7 +47,7 @@ trait SparkHadoopMapReduceUtil { isMap: Boolean, taskId: Int, attemptId: Int): TaskAttemptID = { - val klass = Class.forName("org.apache.hadoop.mapreduce.TaskAttemptID") + val klass = Utils.classForName("org.apache.hadoop.mapreduce.TaskAttemptID") try { // First, attempt to use the old-style constructor that takes a boolean isMap // (not available in YARN) @@ -57,7 +58,7 @@ trait SparkHadoopMapReduceUtil { } catch { case exc: NoSuchMethodException => { // If that failed, look for the new constructor that takes a TaskType (not available in 1.x) - val taskTypeClass = Class.forName("org.apache.hadoop.mapreduce.TaskType") + val taskTypeClass = Utils.classForName("org.apache.hadoop.mapreduce.TaskType") .asInstanceOf[Class[Enum[_]]] val taskType = taskTypeClass.getMethod("valueOf", classOf[String]).invoke( taskTypeClass, if (isMap) "MAP" else "REDUCE") @@ -71,10 +72,10 @@ trait SparkHadoopMapReduceUtil { private def firstAvailableClass(first: String, second: String): Class[_] = { try { - Class.forName(first) + Utils.classForName(first) } catch { case e: ClassNotFoundException => - Class.forName(second) + Utils.classForName(second) } } } diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala index ed5131c79fdc5..67f64d5e278de 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala @@ -20,6 +20,8 @@ package org.apache.spark.metrics import java.util.Properties import java.util.concurrent.TimeUnit +import org.apache.spark.util.Utils + import scala.collection.mutable import com.codahale.metrics.{Metric, MetricFilter, MetricRegistry} @@ -166,7 +168,7 @@ private[spark] class MetricsSystem private ( sourceConfigs.foreach { kv => val classPath = kv._2.getProperty("class") try { - val source = Class.forName(classPath).newInstance() + val source = Utils.classForName(classPath).newInstance() registerSource(source.asInstanceOf[Source]) } catch { case e: Exception => logError("Source class " + classPath + " cannot be instantiated", e) @@ -182,7 +184,7 @@ private[spark] class MetricsSystem private ( val classPath = kv._2.getProperty("class") if (null != classPath) { try { - val sink = Class.forName(classPath) + val sink = Utils.classForName(classPath) .getConstructor(classOf[Properties], classOf[MetricRegistry], classOf[SecurityManager]) .newInstance(kv._2, registry, securityMgr) if (kv._1 == "servlet") { diff --git a/core/src/main/scala/org/apache/spark/network/nio/BlockMessage.scala b/core/src/main/scala/org/apache/spark/network/nio/BlockMessage.scala index 67a376102994c..79cb0640c8672 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/BlockMessage.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/BlockMessage.scala @@ -57,16 +57,6 @@ private[nio] class BlockMessage() { } def set(buffer: ByteBuffer) { - /* - println() - println("BlockMessage: ") - while(buffer.remaining > 0) { - print(buffer.get()) - } - buffer.rewind() - println() - println() - */ typ = buffer.getInt() val idLength = buffer.getInt() val idBuilder = new StringBuilder(idLength) @@ -138,18 +128,6 @@ private[nio] class BlockMessage() { buffers += data } - /* - println() - println("BlockMessage: ") - buffers.foreach(b => { - while(b.remaining > 0) { - print(b.get()) - } - b.rewind() - }) - println() - println() - */ Message.createBufferMessage(buffers) } diff --git a/core/src/main/scala/org/apache/spark/network/nio/BlockMessageArray.scala b/core/src/main/scala/org/apache/spark/network/nio/BlockMessageArray.scala index 7d0806f0c2580..f1c9ea8b64ca3 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/BlockMessageArray.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/BlockMessageArray.scala @@ -43,16 +43,6 @@ class BlockMessageArray(var blockMessages: Seq[BlockMessage]) val newBlockMessages = new ArrayBuffer[BlockMessage]() val buffer = bufferMessage.buffers(0) buffer.clear() - /* - println() - println("BlockMessageArray: ") - while(buffer.remaining > 0) { - print(buffer.get()) - } - buffer.rewind() - println() - println() - */ while (buffer.remaining() > 0) { val size = buffer.getInt() logDebug("Creating block message of size " + size + " bytes") @@ -86,23 +76,11 @@ class BlockMessageArray(var blockMessages: Seq[BlockMessage]) logDebug("Buffer list:") buffers.foreach((x: ByteBuffer) => logDebug("" + x)) - /* - println() - println("BlockMessageArray: ") - buffers.foreach(b => { - while(b.remaining > 0) { - print(b.get()) - } - b.rewind() - }) - println() - println() - */ Message.createBufferMessage(buffers) } } -private[nio] object BlockMessageArray { +private[nio] object BlockMessageArray extends Logging { def fromBufferMessage(bufferMessage: BufferMessage): BlockMessageArray = { val newBlockMessageArray = new BlockMessageArray() @@ -123,10 +101,10 @@ private[nio] object BlockMessageArray { } } val blockMessageArray = new BlockMessageArray(blockMessages) - println("Block message array created") + logDebug("Block message array created") val bufferMessage = blockMessageArray.toBufferMessage - println("Converted to buffer message") + logDebug("Converted to buffer message") val totalSize = bufferMessage.size val newBuffer = ByteBuffer.allocate(totalSize) @@ -138,10 +116,11 @@ private[nio] object BlockMessageArray { }) newBuffer.flip val newBufferMessage = Message.createBufferMessage(newBuffer) - println("Copied to new buffer message, size = " + newBufferMessage.size) + logDebug("Copied to new buffer message, size = " + newBufferMessage.size) val newBlockMessageArray = BlockMessageArray.fromBufferMessage(newBufferMessage) - println("Converted back to block message array") + logDebug("Converted back to block message array") + // scalastyle:off println newBlockMessageArray.foreach(blockMessage => { blockMessage.getType match { case BlockMessage.TYPE_PUT_BLOCK => { @@ -154,6 +133,7 @@ private[nio] object BlockMessageArray { } } }) + // scalastyle:on println } } 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 c0bca2c4bc994..9143918790381 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 @@ -1016,7 +1016,9 @@ private[spark] object ConnectionManager { val conf = new SparkConf val manager = new ConnectionManager(9999, conf, new SecurityManager(conf)) manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { + // scalastyle:off println println("Received [" + msg + "] from [" + id + "]") + // scalastyle:on println None }) @@ -1033,6 +1035,7 @@ private[spark] object ConnectionManager { System.gc() } + // scalastyle:off println def testSequentialSending(manager: ConnectionManager) { println("--------------------------") println("Sequential Sending") @@ -1150,4 +1153,5 @@ private[spark] object ConnectionManager { println() } } + // scalastyle:on println } 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 bee59a437f120..f1c17369cb48c 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -383,11 +383,11 @@ private[spark] object HadoopRDD extends Logging { private[spark] class SplitInfoReflections { val inputSplitWithLocationInfo = - Class.forName("org.apache.hadoop.mapred.InputSplitWithLocationInfo") + Utils.classForName("org.apache.hadoop.mapred.InputSplitWithLocationInfo") val getLocationInfo = inputSplitWithLocationInfo.getMethod("getLocationInfo") - val newInputSplit = Class.forName("org.apache.hadoop.mapreduce.InputSplit") + val newInputSplit = Utils.classForName("org.apache.hadoop.mapreduce.InputSplit") val newGetLocationInfo = newInputSplit.getMethod("getLocationInfo") - val splitLocationInfo = Class.forName("org.apache.hadoop.mapred.SplitLocationInfo") + val splitLocationInfo = Utils.classForName("org.apache.hadoop.mapred.SplitLocationInfo") val isInMemory = splitLocationInfo.getMethod("isInMemory") val getLocation = splitLocationInfo.getMethod("getLocation") } diff --git a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala index dc60d48927624..defdabf95ac4b 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala @@ -123,7 +123,9 @@ private[spark] class PipedRDD[T: ClassTag]( new Thread("stderr reader for " + command) { override def run() { for (line <- Source.fromInputStream(proc.getErrorStream).getLines) { + // scalastyle:off println System.err.println(line) + // scalastyle:on println } } }.start() @@ -133,6 +135,7 @@ private[spark] class PipedRDD[T: ClassTag]( override def run() { val out = new PrintWriter(proc.getOutputStream) + // scalastyle:off println // input the pipe context firstly if (printPipeContext != null) { printPipeContext(out.println(_)) @@ -144,6 +147,7 @@ private[spark] class PipedRDD[T: ClassTag]( out.println(elem) } } + // scalastyle:on println out.close() } }.start() diff --git a/core/src/main/scala/org/apache/spark/rpc/RpcEnv.scala b/core/src/main/scala/org/apache/spark/rpc/RpcEnv.scala index 1709bdf560b6f..c9fcc7a36cc04 100644 --- a/core/src/main/scala/org/apache/spark/rpc/RpcEnv.scala +++ b/core/src/main/scala/org/apache/spark/rpc/RpcEnv.scala @@ -39,8 +39,7 @@ private[spark] object RpcEnv { val rpcEnvNames = Map("akka" -> "org.apache.spark.rpc.akka.AkkaRpcEnvFactory") val rpcEnvName = conf.get("spark.rpc", "akka") val rpcEnvFactoryClassName = rpcEnvNames.getOrElse(rpcEnvName.toLowerCase, rpcEnvName) - Class.forName(rpcEnvFactoryClassName, true, Utils.getContextOrSparkClassLoader). - newInstance().asInstanceOf[RpcEnvFactory] + Utils.classForName(rpcEnvFactoryClassName).newInstance().asInstanceOf[RpcEnvFactory] } def create( 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 6841fa835747f..f3d87ee5c4fd1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -872,7 +872,7 @@ class DAGScheduler( // serializable. If tasks are not serializable, a SparkListenerStageCompleted event // will be posted, which should always come after a corresponding SparkListenerStageSubmitted // event. - stage.latestInfo = StageInfo.fromStage(stage, Some(partitionsToCompute.size)) + stage.makeNewStageAttempt(partitionsToCompute.size) outputCommitCoordinator.stageStart(stage.id) listenerBus.post(SparkListenerStageSubmitted(stage.latestInfo, properties)) @@ -937,8 +937,8 @@ class DAGScheduler( logInfo("Submitting " + tasks.size + " missing tasks from " + stage + " (" + stage.rdd + ")") stage.pendingTasks ++= tasks logDebug("New pending tasks: " + stage.pendingTasks) - taskScheduler.submitTasks( - new TaskSet(tasks.toArray, stage.id, stage.newAttemptId(), stage.firstJobId, properties)) + taskScheduler.submitTasks(new TaskSet( + tasks.toArray, stage.id, stage.latestInfo.attemptId, stage.firstJobId, properties)) stage.latestInfo.submissionTime = Some(clock.getTimeMillis()) } else { // Because we posted SparkListenerStageSubmitted earlier, we should mark 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 529a5b2bf1a0d..5a06ef02f5c57 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -140,7 +140,9 @@ private[spark] class EventLoggingListener( /** Log the event as JSON. */ private def logEvent(event: SparkListenerEvent, flushLogger: Boolean = false) { val eventJson = JsonProtocol.sparkEventToJson(event) + // scalastyle:off println writer.foreach(_.println(compact(render(eventJson)))) + // scalastyle:on println if (flushLogger) { writer.foreach(_.flush()) hadoopDataStream.foreach(hadoopFlushMethod.invoke(_)) @@ -197,6 +199,9 @@ private[spark] class EventLoggingListener( logEvent(event, flushLogger = true) } + // No-op because logging every update would be overkill + override def onBlockUpdated(event: SparkListenerBlockUpdated): Unit = {} + // No-op because logging every update would be overkill override def onExecutorMetricsUpdate(event: SparkListenerExecutorMetricsUpdate): Unit = { } diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala index e55b76c36cc5f..f96eb8ca0ae00 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala @@ -125,7 +125,9 @@ class JobLogger(val user: String, val logDirName: String) extends SparkListener val date = new Date(System.currentTimeMillis()) writeInfo = dateFormat.get.format(date) + ": " + info } + // scalastyle:off println jobIdToPrintWriter.get(jobId).foreach(_.println(writeInfo)) + // scalastyle:on println } /** 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 9620915f495ab..896f1743332f1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -26,7 +26,7 @@ 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.storage.{BlockManagerId, BlockUpdatedInfo} import org.apache.spark.util.{Distribution, Utils} @DeveloperApi @@ -98,6 +98,9 @@ case class SparkListenerExecutorAdded(time: Long, executorId: String, executorIn case class SparkListenerExecutorRemoved(time: Long, executorId: String, reason: String) extends SparkListenerEvent +@DeveloperApi +case class SparkListenerBlockUpdated(blockUpdatedInfo: BlockUpdatedInfo) extends SparkListenerEvent + /** * Periodic updates from executors. * @param execId executor id @@ -215,6 +218,11 @@ trait SparkListener { * Called when the driver removes an executor. */ def onExecutorRemoved(executorRemoved: SparkListenerExecutorRemoved) { } + + /** + * Called when the driver receives a block update info. + */ + def onBlockUpdated(blockUpdated: SparkListenerBlockUpdated) { } } /** 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 61e69ecc08387..04afde33f5aad 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala @@ -58,6 +58,8 @@ private[spark] trait SparkListenerBus extends ListenerBus[SparkListener, SparkLi listener.onExecutorAdded(executorAdded) case executorRemoved: SparkListenerExecutorRemoved => listener.onExecutorRemoved(executorRemoved) + case blockUpdated: SparkListenerBlockUpdated => + listener.onBlockUpdated(blockUpdated) case logStart: SparkListenerLogStart => // ignore event log metadata } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala index c59d6e4f5bc04..b86724de2cb73 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala @@ -62,22 +62,28 @@ private[spark] abstract class Stage( var pendingTasks = new HashSet[Task[_]] + /** The ID to use for the next new attempt for this stage. */ private var nextAttemptId: Int = 0 val name = callSite.shortForm val details = callSite.longForm - /** Pointer to the latest [StageInfo] object, set by DAGScheduler. */ - var latestInfo: StageInfo = StageInfo.fromStage(this) + /** + * Pointer to the [StageInfo] object for the most recent attempt. This needs to be initialized + * here, before any attempts have actually been created, because the DAGScheduler uses this + * StageInfo to tell SparkListeners when a job starts (which happens before any stage attempts + * have been created). + */ + private var _latestInfo: StageInfo = StageInfo.fromStage(this, nextAttemptId) - /** Return a new attempt id, starting with 0. */ - def newAttemptId(): Int = { - val id = nextAttemptId + /** Creates a new attempt for this stage by creating a new StageInfo with a new attempt ID. */ + def makeNewStageAttempt(numPartitionsToCompute: Int): Unit = { + _latestInfo = StageInfo.fromStage(this, nextAttemptId, Some(numPartitionsToCompute)) nextAttemptId += 1 - id } - def attemptId: Int = nextAttemptId + /** Returns the StageInfo for the most recent attempt for this stage. */ + def latestInfo: StageInfo = _latestInfo override final def hashCode(): Int = id override final def equals(other: Any): Boolean = other match { diff --git a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala index e439d2a7e1229..5d2abbc67e9d9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala @@ -70,12 +70,12 @@ private[spark] object StageInfo { * shuffle dependencies. Therefore, all ancestor RDDs related to this Stage's RDD through a * sequence of narrow dependencies should also be associated with this Stage. */ - def fromStage(stage: Stage, numTasks: Option[Int] = None): StageInfo = { + def fromStage(stage: Stage, attemptId: Int, numTasks: Option[Int] = None): StageInfo = { val ancestorRddInfos = stage.rdd.getNarrowAncestors.map(RDDInfo.fromRdd) val rddInfos = Seq(RDDInfo.fromRdd(stage.rdd)) ++ ancestorRddInfos new StageInfo( stage.id, - stage.attemptId, + attemptId, stage.name, numTasks.getOrElse(stage.numTasks), rddInfos, diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index 6b8edca5aa485..cbade131494bc 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -18,18 +18,21 @@ package org.apache.spark.scheduler.cluster.mesos import java.io.File -import java.util.{Collections, List => JList} +import java.util.{List => JList, Collections} +import java.util.concurrent.locks.ReentrantLock import scala.collection.JavaConversions._ import scala.collection.mutable.{HashMap, HashSet} +import com.google.common.collect.HashBiMap import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, _} import org.apache.mesos.{Scheduler => MScheduler, _} +import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, _} +import org.apache.spark.{SparkContext, SparkEnv, SparkException, TaskState} import org.apache.spark.rpc.RpcAddress import org.apache.spark.scheduler.TaskSchedulerImpl import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.util.Utils -import org.apache.spark.{SparkContext, SparkEnv, SparkException, TaskState} /** * A SchedulerBackend that runs tasks on Mesos, but uses "coarse-grained" tasks, where it holds @@ -60,12 +63,34 @@ private[spark] class CoarseMesosSchedulerBackend( val slaveIdsWithExecutors = new HashSet[String] - val taskIdToSlaveId = new HashMap[Int, String] - val failuresBySlaveId = new HashMap[String, Int] // How many times tasks on each slave failed + val taskIdToSlaveId: HashBiMap[Int, String] = HashBiMap.create[Int, String] + // How many times tasks on each slave failed + val failuresBySlaveId: HashMap[String, Int] = new HashMap[String, Int] + + /** + * The total number of executors we aim to have. Undefined when not using dynamic allocation + * and before the ExecutorAllocatorManager calls [[doRequesTotalExecutors]]. + */ + private var executorLimitOption: Option[Int] = None + + /** + * Return the current executor limit, which may be [[Int.MaxValue]] + * before properly initialized. + */ + private[mesos] def executorLimit: Int = executorLimitOption.getOrElse(Int.MaxValue) + + private val pendingRemovedSlaveIds = new HashSet[String] + // private lock object protecting mutable state above. Using the intrinsic lock + // may lead to deadlocks since the superclass might also try to lock + private val stateLock = new ReentrantLock val extraCoresPerSlave = conf.getInt("spark.mesos.extra.cores", 0) + // Offer constraints + private val slaveOfferConstraints = + parseConstraintString(sc.conf.get("spark.mesos.constraints", "")) + var nextMesosTaskId = 0 @volatile var appId: String = _ @@ -82,7 +107,7 @@ private[spark] class CoarseMesosSchedulerBackend( startScheduler(master, CoarseMesosSchedulerBackend.this, fwInfo) } - def createCommand(offer: Offer, numCores: Int): CommandInfo = { + def createCommand(offer: Offer, numCores: Int, taskId: Int): CommandInfo = { val executorSparkHome = conf.getOption("spark.mesos.executor.home") .orElse(sc.getSparkHome()) .getOrElse { @@ -116,10 +141,6 @@ private[spark] class CoarseMesosSchedulerBackend( } val command = CommandInfo.newBuilder() .setEnvironment(environment) - val driverUrl = sc.env.rpcEnv.uriOf( - SparkEnv.driverActorSystemName, - RpcAddress(conf.get("spark.driver.host"), conf.get("spark.driver.port").toInt), - CoarseGrainedSchedulerBackend.ENDPOINT_NAME) val uri = conf.getOption("spark.executor.uri") .orElse(Option(System.getenv("SPARK_EXECUTOR_URI"))) @@ -129,7 +150,7 @@ private[spark] class CoarseMesosSchedulerBackend( command.setValue( "%s \"%s\" org.apache.spark.executor.CoarseGrainedExecutorBackend" .format(prefixEnv, runScript) + - s" --driver-url $driverUrl" + + s" --driver-url $driverURL" + s" --executor-id ${offer.getSlaveId.getValue}" + s" --hostname ${offer.getHostname}" + s" --cores $numCores" + @@ -138,11 +159,12 @@ private[spark] class CoarseMesosSchedulerBackend( // Grab everything to the first '.'. We'll use that and '*' to // glob the directory "correctly". val basename = uri.get.split('/').last.split('.').head + val executorId = sparkExecutorId(offer.getSlaveId.getValue, taskId.toString) command.setValue( s"cd $basename*; $prefixEnv " + "./bin/spark-class org.apache.spark.executor.CoarseGrainedExecutorBackend" + - s" --driver-url $driverUrl" + - s" --executor-id ${offer.getSlaveId.getValue}" + + s" --driver-url $driverURL" + + s" --executor-id $executorId" + s" --hostname ${offer.getHostname}" + s" --cores $numCores" + s" --app-id $appId") @@ -151,6 +173,17 @@ private[spark] class CoarseMesosSchedulerBackend( command.build() } + protected def driverURL: String = { + if (conf.contains("spark.testing")) { + "driverURL" + } else { + sc.env.rpcEnv.uriOf( + SparkEnv.driverActorSystemName, + RpcAddress(conf.get("spark.driver.host"), conf.get("spark.driver.port").toInt), + CoarseGrainedSchedulerBackend.ENDPOINT_NAME) + } + } + override def offerRescinded(d: SchedulerDriver, o: OfferID) {} override def registered(d: SchedulerDriver, frameworkId: FrameworkID, masterInfo: MasterInfo) { @@ -168,15 +201,19 @@ private[spark] class CoarseMesosSchedulerBackend( * unless we've already launched more than we wanted to. */ override def resourceOffers(d: SchedulerDriver, offers: JList[Offer]) { - synchronized { + stateLock.synchronized { val filters = Filters.newBuilder().setRefuseSeconds(5).build() - for (offer <- offers) { - val slaveId = offer.getSlaveId.toString + val offerAttributes = toAttributeMap(offer.getAttributesList) + val meetsConstraints = matchesAttributeRequirements(slaveOfferConstraints, offerAttributes) + val slaveId = offer.getSlaveId.getValue val mem = getResource(offer.getResourcesList, "mem") val cpus = getResource(offer.getResourcesList, "cpus").toInt - if (totalCoresAcquired < maxCores && - mem >= MemoryUtils.calculateTotalMemory(sc) && + val id = offer.getId.getValue + if (taskIdToSlaveId.size < executorLimit && + totalCoresAcquired < maxCores && + meetsConstraints && + mem >= calculateTotalMemory(sc) && cpus >= 1 && failuresBySlaveId.getOrElse(slaveId, 0) < MAX_SLAVE_FAILURES && !slaveIdsWithExecutors.contains(slaveId)) { @@ -190,42 +227,36 @@ private[spark] class CoarseMesosSchedulerBackend( val task = MesosTaskInfo.newBuilder() .setTaskId(TaskID.newBuilder().setValue(taskId.toString).build()) .setSlaveId(offer.getSlaveId) - .setCommand(createCommand(offer, cpusToUse + extraCoresPerSlave)) + .setCommand(createCommand(offer, cpusToUse + extraCoresPerSlave, taskId)) .setName("Task " + taskId) .addResources(createResource("cpus", cpusToUse)) - .addResources(createResource("mem", - MemoryUtils.calculateTotalMemory(sc))) + .addResources(createResource("mem", calculateTotalMemory(sc))) sc.conf.getOption("spark.mesos.executor.docker.image").foreach { image => MesosSchedulerBackendUtil - .setupContainerBuilderDockerInfo(image, sc.conf, task.getContainerBuilder()) + .setupContainerBuilderDockerInfo(image, sc.conf, task.getContainerBuilder) } + // accept the offer and launch the task + logDebug(s"Accepting offer: $id with attributes: $offerAttributes mem: $mem cpu: $cpus") d.launchTasks( - Collections.singleton(offer.getId), Collections.singletonList(task.build()), filters) + Collections.singleton(offer.getId), + Collections.singleton(task.build()), filters) } else { - // Filter it out - d.launchTasks( - Collections.singleton(offer.getId), Collections.emptyList[MesosTaskInfo](), filters) + // Decline the offer + logDebug(s"Declining offer: $id with attributes: $offerAttributes mem: $mem cpu: $cpus") + d.declineOffer(offer.getId) } } } } - /** Build a Mesos resource protobuf object */ - private def createResource(resourceName: String, quantity: Double): Protos.Resource = { - Resource.newBuilder() - .setName(resourceName) - .setType(Value.Type.SCALAR) - .setScalar(Value.Scalar.newBuilder().setValue(quantity).build()) - .build() - } override def statusUpdate(d: SchedulerDriver, status: TaskStatus) { val taskId = status.getTaskId.getValue.toInt val state = status.getState logInfo("Mesos task " + taskId + " is now " + state) - synchronized { + stateLock.synchronized { if (TaskState.isFinished(TaskState.fromMesos(state))) { val slaveId = taskIdToSlaveId(taskId) slaveIdsWithExecutors -= slaveId @@ -243,8 +274,9 @@ private[spark] class CoarseMesosSchedulerBackend( "is Spark installed on it?") } } + executorTerminated(d, slaveId, s"Executor finished with state $state") // In case we'd rejected everything before but have now lost a node - mesosDriver.reviveOffers() + d.reviveOffers() } } } @@ -263,18 +295,39 @@ private[spark] class CoarseMesosSchedulerBackend( override def frameworkMessage(d: SchedulerDriver, e: ExecutorID, s: SlaveID, b: Array[Byte]) {} - override def slaveLost(d: SchedulerDriver, slaveId: SlaveID) { - logInfo("Mesos slave lost: " + slaveId.getValue) - synchronized { - if (slaveIdsWithExecutors.contains(slaveId.getValue)) { - // Note that the slave ID corresponds to the executor ID on that slave - slaveIdsWithExecutors -= slaveId.getValue - removeExecutor(slaveId.getValue, "Mesos slave lost") + /** + * Called when a slave is lost or a Mesos task finished. Update local view on + * what tasks are running and remove the terminated slave from the list of pending + * slave IDs that we might have asked to be killed. It also notifies the driver + * that an executor was removed. + */ + private def executorTerminated(d: SchedulerDriver, slaveId: String, reason: String): Unit = { + stateLock.synchronized { + if (slaveIdsWithExecutors.contains(slaveId)) { + val slaveIdToTaskId = taskIdToSlaveId.inverse() + if (slaveIdToTaskId.contains(slaveId)) { + val taskId: Int = slaveIdToTaskId.get(slaveId) + taskIdToSlaveId.remove(taskId) + removeExecutor(sparkExecutorId(slaveId, taskId.toString), reason) + } + // TODO: This assumes one Spark executor per Mesos slave, + // which may no longer be true after SPARK-5095 + pendingRemovedSlaveIds -= slaveId + slaveIdsWithExecutors -= slaveId } } } - override def executorLost(d: SchedulerDriver, e: ExecutorID, s: SlaveID, status: Int) { + private def sparkExecutorId(slaveId: String, taskId: String): String = { + s"$slaveId/$taskId" + } + + override def slaveLost(d: SchedulerDriver, slaveId: SlaveID): Unit = { + logInfo("Mesos slave lost: " + slaveId.getValue) + executorTerminated(d, slaveId.getValue, "Mesos slave lost: " + slaveId.getValue) + } + + override def executorLost(d: SchedulerDriver, e: ExecutorID, s: SlaveID, status: Int): Unit = { logInfo("Executor lost: %s, marking slave %s as lost".format(e.getValue, s.getValue)) slaveLost(d, s) } @@ -285,4 +338,34 @@ private[spark] class CoarseMesosSchedulerBackend( super.applicationId } + override def doRequestTotalExecutors(requestedTotal: Int): Boolean = { + // We don't truly know if we can fulfill the full amount of executors + // since at coarse grain it depends on the amount of slaves available. + logInfo("Capping the total amount of executors to " + requestedTotal) + executorLimitOption = Some(requestedTotal) + true + } + + override def doKillExecutors(executorIds: Seq[String]): Boolean = { + if (mesosDriver == null) { + logWarning("Asked to kill executors before the Mesos driver was started.") + return false + } + + val slaveIdToTaskId = taskIdToSlaveId.inverse() + for (executorId <- executorIds) { + val slaveId = executorId.split("/")(0) + if (slaveIdToTaskId.contains(slaveId)) { + mesosDriver.killTask( + TaskID.newBuilder().setValue(slaveIdToTaskId.get(slaveId).toString).build()) + pendingRemovedSlaveIds += slaveId + } else { + logWarning("Unable to find executor Id '" + executorId + "' in Mesos scheduler") + } + } + // no need to adjust `executorLimitOption` since the AllocationManager already communicated + // the desired limit through a call to `doRequestTotalExecutors`. + // See [[o.a.s.scheduler.cluster.CoarseGrainedSchedulerBackend.killExecutors]] + true + } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala index 1067a7f1caf4c..d3a20f822176e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala @@ -29,6 +29,7 @@ import org.apache.mesos.Protos.Environment.Variable import org.apache.mesos.Protos.TaskStatus.Reason import org.apache.mesos.Protos.{TaskState => MesosTaskState, _} import org.apache.mesos.{Scheduler, SchedulerDriver} + import org.apache.spark.deploy.mesos.MesosDriverDescription import org.apache.spark.deploy.rest.{CreateSubmissionResponse, KillSubmissionResponse, SubmissionStatusResponse} import org.apache.spark.metrics.MetricsSystem 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 49de85ef48ada..d72e2af456e15 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 @@ -23,14 +23,14 @@ import java.util.{ArrayList => JArrayList, Collections, List => JList} import scala.collection.JavaConversions._ import scala.collection.mutable.{HashMap, HashSet} +import org.apache.mesos.{Scheduler => MScheduler, _} import org.apache.mesos.Protos.{ExecutorInfo => MesosExecutorInfo, TaskInfo => MesosTaskInfo, _} import org.apache.mesos.protobuf.ByteString -import org.apache.mesos.{Scheduler => MScheduler, _} +import org.apache.spark.{SparkContext, SparkException, TaskState} import org.apache.spark.executor.MesosExecutorBackend import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.util.Utils -import org.apache.spark.{SparkContext, SparkException, TaskState} /** * A SchedulerBackend for running fine-grained tasks on Mesos. Each Spark task is mapped to a @@ -59,6 +59,10 @@ private[spark] class MesosSchedulerBackend( private[mesos] val mesosExecutorCores = sc.conf.getDouble("spark.mesos.mesosExecutor.cores", 1) + // Offer constraints + private[this] val slaveOfferConstraints = + parseConstraintString(sc.conf.get("spark.mesos.constraints", "")) + @volatile var appId: String = _ override def start() { @@ -71,8 +75,8 @@ private[spark] class MesosSchedulerBackend( val executorSparkHome = sc.conf.getOption("spark.mesos.executor.home") .orElse(sc.getSparkHome()) // Fall back to driver Spark home for backward compatibility .getOrElse { - throw new SparkException("Executor Spark home `spark.mesos.executor.home` is not set!") - } + throw new SparkException("Executor Spark home `spark.mesos.executor.home` is not set!") + } val environment = Environment.newBuilder() sc.conf.getOption("spark.executor.extraClassPath").foreach { cp => environment.addVariables( @@ -115,14 +119,14 @@ private[spark] class MesosSchedulerBackend( .setName("cpus") .setType(Value.Type.SCALAR) .setScalar(Value.Scalar.newBuilder() - .setValue(mesosExecutorCores).build()) + .setValue(mesosExecutorCores).build()) .build() val memory = Resource.newBuilder() .setName("mem") .setType(Value.Type.SCALAR) .setScalar( Value.Scalar.newBuilder() - .setValue(MemoryUtils.calculateTotalMemory(sc)).build()) + .setValue(calculateTotalMemory(sc)).build()) .build() val executorInfo = MesosExecutorInfo.newBuilder() .setExecutorId(ExecutorID.newBuilder().setValue(execId).build()) @@ -191,13 +195,31 @@ private[spark] class MesosSchedulerBackend( val mem = getResource(o.getResourcesList, "mem") val cpus = getResource(o.getResourcesList, "cpus") val slaveId = o.getSlaveId.getValue - (mem >= MemoryUtils.calculateTotalMemory(sc) && - // need at least 1 for executor, 1 for task - cpus >= (mesosExecutorCores + scheduler.CPUS_PER_TASK)) || - (slaveIdsWithExecutors.contains(slaveId) && - cpus >= scheduler.CPUS_PER_TASK) + val offerAttributes = toAttributeMap(o.getAttributesList) + + // check if all constraints are satisfield + // 1. Attribute constraints + // 2. Memory requirements + // 3. CPU requirements - need at least 1 for executor, 1 for task + val meetsConstraints = matchesAttributeRequirements(slaveOfferConstraints, offerAttributes) + val meetsMemoryRequirements = mem >= calculateTotalMemory(sc) + val meetsCPURequirements = cpus >= (mesosExecutorCores + scheduler.CPUS_PER_TASK) + + val meetsRequirements = + (meetsConstraints && meetsMemoryRequirements && meetsCPURequirements) || + (slaveIdsWithExecutors.contains(slaveId) && cpus >= scheduler.CPUS_PER_TASK) + + // add some debug messaging + val debugstr = if (meetsRequirements) "Accepting" else "Declining" + val id = o.getId.getValue + logDebug(s"$debugstr offer: $id with attributes: $offerAttributes mem: $mem cpu: $cpus") + + meetsRequirements } + // Decline offers we ruled out immediately + unUsableOffers.foreach(o => d.declineOffer(o.getId)) + val workerOffers = usableOffers.map { o => val cpus = if (slaveIdsWithExecutors.contains(o.getSlaveId.getValue)) { getResource(o.getResourcesList, "cpus").toInt @@ -223,15 +245,15 @@ private[spark] class MesosSchedulerBackend( val acceptedOffers = scheduler.resourceOffers(workerOffers).filter(!_.isEmpty) acceptedOffers .foreach { offer => - offer.foreach { taskDesc => - val slaveId = taskDesc.executorId - slaveIdsWithExecutors += slaveId - slavesIdsOfAcceptedOffers += slaveId - taskIdToSlaveId(taskDesc.taskId) = slaveId - mesosTasks.getOrElseUpdate(slaveId, new JArrayList[MesosTaskInfo]) - .add(createMesosTask(taskDesc, slaveId)) - } + offer.foreach { taskDesc => + val slaveId = taskDesc.executorId + slaveIdsWithExecutors += slaveId + slavesIdsOfAcceptedOffers += slaveId + taskIdToSlaveId(taskDesc.taskId) = slaveId + mesosTasks.getOrElseUpdate(slaveId, new JArrayList[MesosTaskInfo]) + .add(createMesosTask(taskDesc, slaveId)) } + } // Reply to the offers val filters = Filters.newBuilder().setRefuseSeconds(1).build() // TODO: lower timeout? @@ -251,8 +273,6 @@ private[spark] class MesosSchedulerBackend( d.declineOffer(o.getId) } - // Decline offers we ruled out immediately - unUsableOffers.foreach(o => d.declineOffer(o.getId)) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala index d11228f3d016a..925702e63afd3 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala @@ -17,14 +17,17 @@ package org.apache.spark.scheduler.cluster.mesos -import java.util.List +import java.util.{List => JList} import java.util.concurrent.CountDownLatch import scala.collection.JavaConversions._ +import scala.util.control.NonFatal -import org.apache.mesos.Protos.{FrameworkInfo, Resource, Status} -import org.apache.mesos.{MesosSchedulerDriver, Scheduler} -import org.apache.spark.Logging +import com.google.common.base.Splitter +import org.apache.mesos.{MesosSchedulerDriver, SchedulerDriver, Scheduler, Protos} +import org.apache.mesos.Protos._ +import org.apache.mesos.protobuf.GeneratedMessage +import org.apache.spark.{Logging, SparkContext} import org.apache.spark.util.Utils /** @@ -36,7 +39,7 @@ private[mesos] trait MesosSchedulerUtils extends Logging { private final val registerLatch = new CountDownLatch(1) // Driver for talking to Mesos - protected var mesosDriver: MesosSchedulerDriver = null + protected var mesosDriver: SchedulerDriver = null /** * Starts the MesosSchedulerDriver with the provided information. This method returns @@ -86,10 +89,150 @@ private[mesos] trait MesosSchedulerUtils extends Logging { /** * Get the amount of resources for the specified type from the resource list */ - protected def getResource(res: List[Resource], name: String): Double = { + protected def getResource(res: JList[Resource], name: String): Double = { for (r <- res if r.getName == name) { return r.getScalar.getValue } 0.0 } + + /** Helper method to get the key,value-set pair for a Mesos Attribute protobuf */ + protected def getAttribute(attr: Attribute): (String, Set[String]) = { + (attr.getName, attr.getText.getValue.split(',').toSet) + } + + + /** Build a Mesos resource protobuf object */ + protected def createResource(resourceName: String, quantity: Double): Protos.Resource = { + Resource.newBuilder() + .setName(resourceName) + .setType(Value.Type.SCALAR) + .setScalar(Value.Scalar.newBuilder().setValue(quantity).build()) + .build() + } + + /** + * Converts the attributes from the resource offer into a Map of name -> Attribute Value + * The attribute values are the mesos attribute types and they are + * @param offerAttributes + * @return + */ + protected def toAttributeMap(offerAttributes: JList[Attribute]): Map[String, GeneratedMessage] = { + offerAttributes.map(attr => { + val attrValue = attr.getType match { + case Value.Type.SCALAR => attr.getScalar + case Value.Type.RANGES => attr.getRanges + case Value.Type.SET => attr.getSet + case Value.Type.TEXT => attr.getText + } + (attr.getName, attrValue) + }).toMap + } + + + /** + * Match the requirements (if any) to the offer attributes. + * if attribute requirements are not specified - return true + * else if attribute is defined and no values are given, simple attribute presence is performed + * else if attribute name and value is specified, subset match is performed on slave attributes + */ + def matchesAttributeRequirements( + slaveOfferConstraints: Map[String, Set[String]], + offerAttributes: Map[String, GeneratedMessage]): Boolean = { + slaveOfferConstraints.forall { + // offer has the required attribute and subsumes the required values for that attribute + case (name, requiredValues) => + offerAttributes.get(name) match { + case None => false + case Some(_) if requiredValues.isEmpty => true // empty value matches presence + case Some(scalarValue: Value.Scalar) => + // check if provided values is less than equal to the offered values + requiredValues.map(_.toDouble).exists(_ <= scalarValue.getValue) + case Some(rangeValue: Value.Range) => + val offerRange = rangeValue.getBegin to rangeValue.getEnd + // Check if there is some required value that is between the ranges specified + // Note: We only support the ability to specify discrete values, in the future + // we may expand it to subsume ranges specified with a XX..YY value or something + // similar to that. + requiredValues.map(_.toLong).exists(offerRange.contains(_)) + case Some(offeredValue: Value.Set) => + // check if the specified required values is a subset of offered set + requiredValues.subsetOf(offeredValue.getItemList.toSet) + case Some(textValue: Value.Text) => + // check if the specified value is equal, if multiple values are specified + // we succeed if any of them match. + requiredValues.contains(textValue.getValue) + } + } + } + + /** + * Parses the attributes constraints provided to spark and build a matching data struct: + * Map[, Set[values-to-match]] + * The constraints are specified as ';' separated key-value pairs where keys and values + * are separated by ':'. The ':' implies equality (for singular values) and "is one of" for + * multiple values (comma separated). For example: + * {{{ + * parseConstraintString("tachyon:true;zone:us-east-1a,us-east-1b") + * // would result in + * + * Map( + * "tachyon" -> Set("true"), + * "zone": -> Set("us-east-1a", "us-east-1b") + * ) + * }}} + * + * Mesos documentation: http://mesos.apache.org/documentation/attributes-resources/ + * https://github.com/apache/mesos/blob/master/src/common/values.cpp + * https://github.com/apache/mesos/blob/master/src/common/attributes.cpp + * + * @param constraintsVal constaints string consisting of ';' separated key-value pairs (separated + * by ':') + * @return Map of constraints to match resources offers. + */ + def parseConstraintString(constraintsVal: String): Map[String, Set[String]] = { + /* + Based on mesos docs: + attributes : attribute ( ";" attribute )* + attribute : labelString ":" ( labelString | "," )+ + labelString : [a-zA-Z0-9_/.-] + */ + val splitter = Splitter.on(';').trimResults().withKeyValueSeparator(':') + // kv splitter + if (constraintsVal.isEmpty) { + Map() + } else { + try { + Map() ++ mapAsScalaMap(splitter.split(constraintsVal)).map { + case (k, v) => + if (v == null || v.isEmpty) { + (k, Set[String]()) + } else { + (k, v.split(',').toSet) + } + } + } catch { + case NonFatal(e) => + throw new IllegalArgumentException(s"Bad constraint string: $constraintsVal", e) + } + } + } + + // These defaults copied from YARN + private val MEMORY_OVERHEAD_FRACTION = 0.10 + private val MEMORY_OVERHEAD_MINIMUM = 384 + + /** + * Return the amount of memory to allocate to each executor, taking into account + * container overheads. + * @param sc SparkContext to use to get `spark.mesos.executor.memoryOverhead` value + * @return memory requirement as (0.1 * ) or MEMORY_OVERHEAD_MINIMUM + * (whichever is larger) + */ + def calculateTotalMemory(sc: SparkContext): Int = { + sc.conf.getInt("spark.mesos.executor.memoryOverhead", + math.max(MEMORY_OVERHEAD_FRACTION * sc.executorMemory, MEMORY_OVERHEAD_MINIMUM).toInt) + + sc.executorMemory + } + } 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 3078a1b10be8b..4d48fcfea44e7 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 @@ -17,13 +17,16 @@ package org.apache.spark.scheduler.local +import java.io.File +import java.net.URL import java.nio.ByteBuffer import org.apache.spark.{Logging, SparkConf, SparkContext, SparkEnv, TaskState} import org.apache.spark.TaskState.TaskState import org.apache.spark.executor.{Executor, ExecutorBackend} import org.apache.spark.rpc.{RpcCallContext, RpcEndpointRef, RpcEnv, ThreadSafeRpcEndpoint} -import org.apache.spark.scheduler.{SchedulerBackend, TaskSchedulerImpl, WorkerOffer} +import org.apache.spark.scheduler._ +import org.apache.spark.scheduler.cluster.ExecutorInfo private case class ReviveOffers() @@ -40,6 +43,7 @@ private case class StopExecutor() */ private[spark] class LocalEndpoint( override val rpcEnv: RpcEnv, + userClassPath: Seq[URL], scheduler: TaskSchedulerImpl, executorBackend: LocalBackend, private val totalCores: Int) @@ -47,11 +51,11 @@ private[spark] class LocalEndpoint( private var freeCores = totalCores - private val localExecutorId = SparkContext.DRIVER_IDENTIFIER - private val localExecutorHostname = "localhost" + val localExecutorId = SparkContext.DRIVER_IDENTIFIER + val localExecutorHostname = "localhost" private val executor = new Executor( - localExecutorId, localExecutorHostname, SparkEnv.get, isLocal = true) + localExecutorId, localExecutorHostname, SparkEnv.get, userClassPath, isLocal = true) override def receive: PartialFunction[Any, Unit] = { case ReviveOffers => @@ -96,11 +100,28 @@ private[spark] class LocalBackend( extends SchedulerBackend with ExecutorBackend with Logging { private val appId = "local-" + System.currentTimeMillis - var localEndpoint: RpcEndpointRef = null + private var localEndpoint: RpcEndpointRef = null + private val userClassPath = getUserClasspath(conf) + private val listenerBus = scheduler.sc.listenerBus + + /** + * Returns a list of URLs representing the user classpath. + * + * @param conf Spark configuration. + */ + def getUserClasspath(conf: SparkConf): Seq[URL] = { + val userClassPathStr = conf.getOption("spark.executor.extraClassPath") + userClassPathStr.map(_.split(File.pathSeparator)).toSeq.flatten.map(new File(_).toURI.toURL) + } override def start() { - localEndpoint = SparkEnv.get.rpcEnv.setupEndpoint( - "LocalBackendEndpoint", new LocalEndpoint(SparkEnv.get.rpcEnv, scheduler, this, totalCores)) + val rpcEnv = SparkEnv.get.rpcEnv + val executorEndpoint = new LocalEndpoint(rpcEnv, userClassPath, scheduler, this, totalCores) + localEndpoint = rpcEnv.setupEndpoint("LocalBackendEndpoint", executorEndpoint) + listenerBus.post(SparkListenerExecutorAdded( + System.currentTimeMillis, + executorEndpoint.localExecutorId, + new ExecutorInfo(executorEndpoint.localExecutorHostname, totalCores, Map.empty))) } override def stop() { 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 698d1384d580d..4a5274b46b7a0 100644 --- a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala @@ -62,8 +62,11 @@ private[spark] class JavaDeserializationStream(in: InputStream, loader: ClassLoa extends DeserializationStream { private val objIn = new ObjectInputStream(in) { - override def resolveClass(desc: ObjectStreamClass): Class[_] = + override def resolveClass(desc: ObjectStreamClass): Class[_] = { + // scalastyle:off classforname Class.forName(desc.getName, false, loader) + // scalastyle:on classforname + } } def readObject[T: ClassTag](): T = objIn.readObject().asInstanceOf[T] 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 ed35cffe968f8..7cb6e080533ad 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -102,6 +102,7 @@ class KryoSerializer(conf: SparkConf) kryo.register(classOf[PythonBroadcast], new KryoJavaSerializer()) try { + // scalastyle:off classforname // Use the default classloader when calling the user registrator. Thread.currentThread.setContextClassLoader(classLoader) // Register classes given through spark.kryo.classesToRegister. @@ -111,6 +112,7 @@ class KryoSerializer(conf: SparkConf) userRegistrator .map(Class.forName(_, true, classLoader).newInstance().asInstanceOf[KryoRegistrator]) .foreach { reg => reg.registerClasses(kryo) } + // scalastyle:on classforname } catch { case e: Exception => throw new SparkException(s"Failed to register classes with Kryo", e) diff --git a/core/src/main/scala/org/apache/spark/serializer/SerializationDebugger.scala b/core/src/main/scala/org/apache/spark/serializer/SerializationDebugger.scala index cc2f0506817d3..a1b1e1631eafb 100644 --- a/core/src/main/scala/org/apache/spark/serializer/SerializationDebugger.scala +++ b/core/src/main/scala/org/apache/spark/serializer/SerializationDebugger.scala @@ -407,7 +407,9 @@ private[spark] object SerializationDebugger extends Logging { /** ObjectStreamClass$ClassDataSlot.desc field */ val DescField: Field = { + // scalastyle:off classforname val f = Class.forName("java.io.ObjectStreamClass$ClassDataSlot").getDeclaredField("desc") + // scalastyle:on classforname f.setAccessible(true) f } diff --git a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala index 6c3b3080d2605..f6a96d81e7aa9 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockResolver.scala @@ -35,7 +35,7 @@ import org.apache.spark.util.collection.{PrimitiveKeyOpenHashMap, PrimitiveVecto /** A group of writers for a ShuffleMapTask, one writer per reducer. */ private[spark] trait ShuffleWriterGroup { - val writers: Array[BlockObjectWriter] + val writers: Array[DiskBlockObjectWriter] /** @param success Indicates all writes were successful. If false, no blocks will be recorded. */ def releaseWriters(success: Boolean) @@ -113,15 +113,15 @@ private[spark] class FileShuffleBlockResolver(conf: SparkConf) val openStartTime = System.nanoTime val serializerInstance = serializer.newInstance() - val writers: Array[BlockObjectWriter] = if (consolidateShuffleFiles) { + val writers: Array[DiskBlockObjectWriter] = if (consolidateShuffleFiles) { fileGroup = getUnusedFileGroup() - Array.tabulate[BlockObjectWriter](numBuckets) { bucketId => + Array.tabulate[DiskBlockObjectWriter](numBuckets) { bucketId => val blockId = ShuffleBlockId(shuffleId, mapId, bucketId) blockManager.getDiskWriter(blockId, fileGroup(bucketId), serializerInstance, bufferSize, writeMetrics) } } else { - Array.tabulate[BlockObjectWriter](numBuckets) { bucketId => + Array.tabulate[DiskBlockObjectWriter](numBuckets) { bucketId => val blockId = ShuffleBlockId(shuffleId, mapId, bucketId) val blockFile = blockManager.diskBlockManager.getFile(blockId) // Because of previous failures, the shuffle file may already exist on this machine. diff --git a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala index d9c63b6e7bbb9..fae69551e7330 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala @@ -114,7 +114,7 @@ private[spark] class IndexShuffleBlockResolver(conf: SparkConf) extends ShuffleB } private[spark] object IndexShuffleBlockResolver { - // No-op reduce ID used in interactions with disk store and BlockObjectWriter. + // No-op reduce ID used in interactions with disk store and DiskBlockObjectWriter. // The disk store currently expects puts to relate to a (map, reduce) pair, but in the sort // shuffle outputs for several reduces are glommed into a single file. // TODO: Avoid this entirely by having the DiskBlockObjectWriter not require a BlockId. diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala index eb87cee15903c..41df70c602c30 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala @@ -22,7 +22,7 @@ import org.apache.spark.executor.ShuffleWriteMetrics import org.apache.spark.scheduler.MapStatus import org.apache.spark.serializer.Serializer import org.apache.spark.shuffle._ -import org.apache.spark.storage.BlockObjectWriter +import org.apache.spark.storage.DiskBlockObjectWriter private[spark] class HashShuffleWriter[K, V]( shuffleBlockResolver: FileShuffleBlockResolver, @@ -102,7 +102,7 @@ private[spark] class HashShuffleWriter[K, V]( private def commitWritesAndBuildStatus(): MapStatus = { // Commit the writes. Get the size of each bucket block (total block size). - val sizes: Array[Long] = shuffle.writers.map { writer: BlockObjectWriter => + val sizes: Array[Long] = shuffle.writers.map { writer: DiskBlockObjectWriter => writer.commitAndClose() writer.fileSegment().length } 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 1beafa1771448..86493673d958d 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -648,7 +648,7 @@ private[spark] class BlockManager( file: File, serializerInstance: SerializerInstance, bufferSize: Int, - writeMetrics: ShuffleWriteMetrics): BlockObjectWriter = { + writeMetrics: ShuffleWriteMetrics): DiskBlockObjectWriter = { val compressStream: OutputStream => OutputStream = wrapForCompression(blockId, _) val syncWrites = conf.getBoolean("spark.shuffle.sync", false) new DiskBlockObjectWriter(blockId, file, serializerInstance, bufferSize, compressStream, diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala index 68ed9096731c5..5dc0c537cbb62 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala @@ -60,10 +60,11 @@ class BlockManagerMasterEndpoint( register(blockManagerId, maxMemSize, slaveEndpoint) context.reply(true) - case UpdateBlockInfo( + case _updateBlockInfo @ UpdateBlockInfo( blockManagerId, blockId, storageLevel, deserializedSize, size, externalBlockStoreSize) => context.reply(updateBlockInfo( blockManagerId, blockId, storageLevel, deserializedSize, size, externalBlockStoreSize)) + listenerBus.post(SparkListenerBlockUpdated(BlockUpdatedInfo(_updateBlockInfo))) case GetLocations(blockId) => context.reply(getLocations(blockId)) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockStatusListener.scala b/core/src/main/scala/org/apache/spark/storage/BlockStatusListener.scala new file mode 100644 index 0000000000000..2789e25b8d3ab --- /dev/null +++ b/core/src/main/scala/org/apache/spark/storage/BlockStatusListener.scala @@ -0,0 +1,105 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.storage + +import scala.collection.mutable + +import org.apache.spark.scheduler._ + +private[spark] case class BlockUIData( + blockId: BlockId, + location: String, + storageLevel: StorageLevel, + memSize: Long, + diskSize: Long, + externalBlockStoreSize: Long) + +/** + * The aggregated status of stream blocks in an executor + */ +private[spark] case class ExecutorStreamBlockStatus( + executorId: String, + location: String, + blocks: Seq[BlockUIData]) { + + def totalMemSize: Long = blocks.map(_.memSize).sum + + def totalDiskSize: Long = blocks.map(_.diskSize).sum + + def totalExternalBlockStoreSize: Long = blocks.map(_.externalBlockStoreSize).sum + + def numStreamBlocks: Int = blocks.size + +} + +private[spark] class BlockStatusListener extends SparkListener { + + private val blockManagers = + new mutable.HashMap[BlockManagerId, mutable.HashMap[BlockId, BlockUIData]] + + override def onBlockUpdated(blockUpdated: SparkListenerBlockUpdated): Unit = { + val blockId = blockUpdated.blockUpdatedInfo.blockId + if (!blockId.isInstanceOf[StreamBlockId]) { + // Now we only monitor StreamBlocks + return + } + val blockManagerId = blockUpdated.blockUpdatedInfo.blockManagerId + val storageLevel = blockUpdated.blockUpdatedInfo.storageLevel + val memSize = blockUpdated.blockUpdatedInfo.memSize + val diskSize = blockUpdated.blockUpdatedInfo.diskSize + val externalBlockStoreSize = blockUpdated.blockUpdatedInfo.externalBlockStoreSize + + synchronized { + // Drop the update info if the block manager is not registered + blockManagers.get(blockManagerId).foreach { blocksInBlockManager => + if (storageLevel.isValid) { + blocksInBlockManager.put(blockId, + BlockUIData( + blockId, + blockManagerId.hostPort, + storageLevel, + memSize, + diskSize, + externalBlockStoreSize) + ) + } else { + // If isValid is not true, it means we should drop the block. + blocksInBlockManager -= blockId + } + } + } + } + + override def onBlockManagerAdded(blockManagerAdded: SparkListenerBlockManagerAdded): Unit = { + synchronized { + blockManagers.put(blockManagerAdded.blockManagerId, mutable.HashMap()) + } + } + + override def onBlockManagerRemoved( + blockManagerRemoved: SparkListenerBlockManagerRemoved): Unit = synchronized { + blockManagers -= blockManagerRemoved.blockManagerId + } + + def allExecutorStreamBlockStatus: Seq[ExecutorStreamBlockStatus] = synchronized { + blockManagers.map { case (blockManagerId, blocks) => + ExecutorStreamBlockStatus( + blockManagerId.executorId, blockManagerId.hostPort, blocks.values.toSeq) + }.toSeq + } +} diff --git a/core/src/main/scala/org/apache/spark/storage/BlockUpdatedInfo.scala b/core/src/main/scala/org/apache/spark/storage/BlockUpdatedInfo.scala new file mode 100644 index 0000000000000..a5790e4454a89 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/storage/BlockUpdatedInfo.scala @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.storage + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.storage.BlockManagerMessages.UpdateBlockInfo + +/** + * :: DeveloperApi :: + * Stores information about a block status in a block manager. + */ +@DeveloperApi +case class BlockUpdatedInfo( + blockManagerId: BlockManagerId, + blockId: BlockId, + storageLevel: StorageLevel, + memSize: Long, + diskSize: Long, + externalBlockStoreSize: Long) + +private[spark] object BlockUpdatedInfo { + + private[spark] def apply(updateBlockInfo: UpdateBlockInfo): BlockUpdatedInfo = { + BlockUpdatedInfo( + updateBlockInfo.blockManagerId, + updateBlockInfo.blockId, + updateBlockInfo.storageLevel, + updateBlockInfo.memSize, + updateBlockInfo.diskSize, + updateBlockInfo.externalBlockStoreSize) + } +} 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 91ef86389a0c3..5f537692a16c5 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -124,10 +124,16 @@ private[spark] class DiskBlockManager(blockManager: BlockManager, conf: SparkCon (blockId, getFile(blockId)) } + /** + * Create local directories for storing block data. These directories are + * located inside configured local directories and won't + * be deleted on JVM exit when using the external shuffle service. + */ private def createLocalDirs(conf: SparkConf): Array[File] = { - Utils.getOrCreateLocalRootDirs(conf).flatMap { rootDir => + Utils.getConfiguredLocalDirs(conf).flatMap { rootDir => try { val localDir = Utils.createDirectory(rootDir, "blockmgr") + Utils.chmod700(localDir) logInfo(s"Created local directory at $localDir") Some(localDir) } catch { diff --git a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala similarity index 83% rename from core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala rename to core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala index 7eeabd1e0489c..49d9154f95a5b 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala @@ -26,66 +26,25 @@ import org.apache.spark.executor.ShuffleWriteMetrics import org.apache.spark.util.Utils /** - * An interface for writing JVM objects to some underlying storage. This interface allows - * appending data to an existing block, and can guarantee atomicity in the case of faults - * as it allows the caller to revert partial writes. + * A class for writing JVM objects directly to a file on disk. This class allows data to be appended + * to an existing block and can guarantee atomicity in the case of faults as it allows the caller to + * revert partial writes. * - * This interface does not support concurrent writes. Also, once the writer has - * been opened, it cannot be reopened again. - */ -private[spark] abstract class BlockObjectWriter(val blockId: BlockId) extends OutputStream { - - def open(): BlockObjectWriter - - def close() - - def isOpen: Boolean - - /** - * Flush the partial writes and commit them as a single atomic block. - */ - def commitAndClose(): Unit - - /** - * Reverts writes that haven't been flushed yet. Callers should invoke this function - * when there are runtime exceptions. This method will not throw, though it may be - * unsuccessful in truncating written data. - */ - def revertPartialWritesAndClose() - - /** - * Writes a key-value pair. - */ - def write(key: Any, value: Any) - - /** - * Notify the writer that a record worth of bytes has been written with OutputStream#write. - */ - def recordWritten() - - /** - * Returns the file segment of committed data that this Writer has written. - * This is only valid after commitAndClose() has been called. - */ - def fileSegment(): FileSegment -} - -/** - * BlockObjectWriter which writes directly to a file on disk. Appends to the given file. + * This class does not support concurrent writes. Also, once the writer has been opened it cannot be + * reopened again. */ private[spark] class DiskBlockObjectWriter( - blockId: BlockId, + val blockId: BlockId, file: File, serializerInstance: SerializerInstance, bufferSize: Int, compressStream: OutputStream => OutputStream, syncWrites: Boolean, - // These write metrics concurrently shared with other active BlockObjectWriter's who + // These write metrics concurrently shared with other active DiskBlockObjectWriters who // are themselves performing writes. All updates must be relative. writeMetrics: ShuffleWriteMetrics) - extends BlockObjectWriter(blockId) - with Logging -{ + extends OutputStream + with Logging { /** The file channel, used for repositioning / truncating the file. */ private var channel: FileChannel = null @@ -122,7 +81,7 @@ private[spark] class DiskBlockObjectWriter( */ private var numRecordsWritten = 0 - override def open(): BlockObjectWriter = { + def open(): DiskBlockObjectWriter = { if (hasBeenClosed) { throw new IllegalStateException("Writer already closed. Cannot be reopened.") } @@ -159,9 +118,12 @@ private[spark] class DiskBlockObjectWriter( } } - override def isOpen: Boolean = objOut != null + def isOpen: Boolean = objOut != null - override def commitAndClose(): Unit = { + /** + * Flush the partial writes and commit them as a single atomic block. + */ + def commitAndClose(): Unit = { if (initialized) { // NOTE: Because Kryo doesn't flush the underlying stream we explicitly flush both the // serializer stream and the lower level stream. @@ -177,9 +139,15 @@ private[spark] class DiskBlockObjectWriter( commitAndCloseHasBeenCalled = true } - // Discard current writes. We do this by flushing the outstanding writes and then - // truncating the file to its initial position. - override def revertPartialWritesAndClose() { + + /** + * Reverts writes that haven't been flushed yet. Callers should invoke this function + * when there are runtime exceptions. This method will not throw, though it may be + * unsuccessful in truncating written data. + */ + def revertPartialWritesAndClose() { + // Discard current writes. We do this by flushing the outstanding writes and then + // truncating the file to its initial position. try { if (initialized) { writeMetrics.decShuffleBytesWritten(reportedPosition - initialPosition) @@ -201,7 +169,10 @@ private[spark] class DiskBlockObjectWriter( } } - override def write(key: Any, value: Any) { + /** + * Writes a key-value pair. + */ + def write(key: Any, value: Any) { if (!initialized) { open() } @@ -221,7 +192,10 @@ private[spark] class DiskBlockObjectWriter( bs.write(kvBytes, offs, len) } - override def recordWritten(): Unit = { + /** + * Notify the writer that a record worth of bytes has been written with OutputStream#write. + */ + def recordWritten(): Unit = { numRecordsWritten += 1 writeMetrics.incShuffleRecordsWritten(1) @@ -230,7 +204,11 @@ private[spark] class DiskBlockObjectWriter( } } - override def fileSegment(): FileSegment = { + /** + * Returns the file segment of committed data that this Writer has written. + * This is only valid after commitAndClose() has been called. + */ + def fileSegment(): FileSegment = { if (!commitAndCloseHasBeenCalled) { throw new IllegalStateException( "fileSegment() is only valid after commitAndClose() has been called") diff --git a/core/src/main/scala/org/apache/spark/storage/ExternalBlockStore.scala b/core/src/main/scala/org/apache/spark/storage/ExternalBlockStore.scala index 291394ed34816..db965d54bafd6 100644 --- a/core/src/main/scala/org/apache/spark/storage/ExternalBlockStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/ExternalBlockStore.scala @@ -192,7 +192,7 @@ private[spark] class ExternalBlockStore(blockManager: BlockManager, executorId: .getOrElse(ExternalBlockStore.DEFAULT_BLOCK_MANAGER_NAME) try { - val instance = Class.forName(clsName) + val instance = Utils.classForName(clsName) .newInstance() .asInstanceOf[ExternalBlockManager] instance.init(blockManager, executorId) 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 06e616220c706..c8356467fab87 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -68,7 +68,9 @@ private[spark] object JettyUtils extends Logging { response.setStatus(HttpServletResponse.SC_OK) val result = servletParams.responder(request) response.setHeader("Cache-Control", "no-cache, no-store, must-revalidate") + // scalastyle:off println response.getWriter.println(servletParams.extractFn(result)) + // scalastyle:on println } else { response.setStatus(HttpServletResponse.SC_UNAUTHORIZED) response.setHeader("Cache-Control", "no-cache, no-store, must-revalidate") @@ -210,10 +212,16 @@ private[spark] object JettyUtils extends Logging { conf: SparkConf, serverName: String = ""): ServerInfo = { - val collection = new ContextHandlerCollection - collection.setHandlers(handlers.toArray) addFilters(handlers, conf) + val collection = new ContextHandlerCollection + val gzipHandlers = handlers.map { h => + val gzipHandler = new GzipHandler + gzipHandler.setHandler(h) + gzipHandler + } + collection.setHandlers(gzipHandlers.toArray) + // Bind to the given port, or throw a java.net.BindException if the port is occupied def connect(currentPort: Int): (Server, Int) = { val server = new Server(new InetSocketAddress(hostName, currentPort)) 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 7898039519201..718aea7e1dc22 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -27,7 +27,7 @@ import org.apache.spark.ui.scope.RDDOperationGraph /** Utility functions for generating XML pages with spark content. */ private[spark] object UIUtils extends Logging { - val TABLE_CLASS_NOT_STRIPED = "table table-bordered table-condensed sortable" + val TABLE_CLASS_NOT_STRIPED = "table table-bordered table-condensed" val TABLE_CLASS_STRIPED = TABLE_CLASS_NOT_STRIPED + " table-striped" // SimpleDateFormat is not thread-safe. Don't expose it to avoid improper use. @@ -267,9 +267,17 @@ private[spark] object UIUtils extends Logging { fixedWidth: Boolean = false, id: Option[String] = None, headerClasses: Seq[String] = Seq.empty, - stripeRowsWithCss: Boolean = true): Seq[Node] = { + stripeRowsWithCss: Boolean = true, + sortable: Boolean = true): Seq[Node] = { - val listingTableClass = if (stripeRowsWithCss) TABLE_CLASS_STRIPED else TABLE_CLASS_NOT_STRIPED + val listingTableClass = { + val _tableClass = if (stripeRowsWithCss) TABLE_CLASS_STRIPED else TABLE_CLASS_NOT_STRIPED + if (sortable) { + _tableClass + " sortable" + } else { + _tableClass + } + } val colWidth = 100.toDouble / headers.size val colWidthAttr = if (fixedWidth) colWidth + "%" else "" 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 ba03acdb38cc5..5a8c2914314c2 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala @@ -38,9 +38,11 @@ private[spark] object UIWorkloadGenerator { def main(args: Array[String]) { if (args.length < 3) { + // scalastyle:off println println( - "usage: ./bin/spark-class org.apache.spark.ui.UIWorkloadGenerator " + + "Usage: ./bin/spark-class org.apache.spark.ui.UIWorkloadGenerator " + "[master] [FIFO|FAIR] [#job set (4 jobs per set)]") + // scalastyle:on println System.exit(1) } @@ -96,6 +98,7 @@ private[spark] object UIWorkloadGenerator { for ((desc, job) <- jobs) { new Thread { override def run() { + // scalastyle:off println try { setProperties(desc) job() @@ -106,6 +109,7 @@ private[spark] object UIWorkloadGenerator { } finally { barrier.release() } + // scalastyle:on println } }.start Thread.sleep(INTER_JOB_WAIT_MS) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 60e3c6343122c..ff0a339a39c65 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -332,7 +332,7 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { +: getFormattedTimeQuantiles(serializationTimes) val gettingResultTimes = validTasks.map { case TaskUIData(info, _, _) => - getGettingResultTime(info).toDouble + getGettingResultTime(info, currentTime).toDouble } val gettingResultQuantiles = @@ -346,7 +346,7 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { // machine and to send back the result (but not the time to fetch the task result, // if it needed to be fetched from the block manager on the worker). val schedulerDelays = validTasks.map { case TaskUIData(info, metrics, _) => - getSchedulerDelay(info, metrics.get).toDouble + getSchedulerDelay(info, metrics.get, currentTime).toDouble } val schedulerDelayTitle = Scheduler Delay @@ -544,7 +544,7 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { val serializationTimeProportion = toProportion(serializationTime) val deserializationTime = metricsOpt.map(_.executorDeserializeTime).getOrElse(0L) val deserializationTimeProportion = toProportion(deserializationTime) - val gettingResultTime = getGettingResultTime(taskUIData.taskInfo) + val gettingResultTime = getGettingResultTime(taskUIData.taskInfo, currentTime) val gettingResultTimeProportion = toProportion(gettingResultTime) val schedulerDelay = totalExecutionTime - (executorComputingTime + shuffleReadTime + shuffleWriteTime + @@ -685,11 +685,11 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { else metrics.map(_.executorRunTime).getOrElse(1L) val formatDuration = if (info.status == "RUNNING") UIUtils.formatDuration(duration) else metrics.map(m => UIUtils.formatDuration(m.executorRunTime)).getOrElse("") - val schedulerDelay = metrics.map(getSchedulerDelay(info, _)).getOrElse(0L) + val schedulerDelay = metrics.map(getSchedulerDelay(info, _, currentTime)).getOrElse(0L) val gcTime = metrics.map(_.jvmGCTime).getOrElse(0L) val taskDeserializationTime = metrics.map(_.executorDeserializeTime).getOrElse(0L) val serializationTime = metrics.map(_.resultSerializationTime).getOrElse(0L) - val gettingResultTime = getGettingResultTime(info) + val gettingResultTime = getGettingResultTime(info, currentTime) val maybeAccumulators = info.accumulables val accumulatorsReadable = maybeAccumulators.map{acc => s"${acc.name}: ${acc.update.get}"} @@ -852,32 +852,31 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { {errorSummary}{details} } - private def getGettingResultTime(info: TaskInfo): Long = { - if (info.gettingResultTime > 0) { - if (info.finishTime > 0) { + private def getGettingResultTime(info: TaskInfo, currentTime: Long): Long = { + if (info.gettingResult) { + if (info.finished) { info.finishTime - info.gettingResultTime } else { // The task is still fetching the result. - System.currentTimeMillis - info.gettingResultTime + currentTime - info.gettingResultTime } } else { 0L } } - private def getSchedulerDelay(info: TaskInfo, metrics: TaskMetrics): Long = { - val totalExecutionTime = - if (info.gettingResult) { - info.gettingResultTime - info.launchTime - } else if (info.finished) { - info.finishTime - info.launchTime - } else { - 0 - } - val executorOverhead = (metrics.executorDeserializeTime + - metrics.resultSerializationTime) - math.max( - 0, - totalExecutionTime - metrics.executorRunTime - executorOverhead - getGettingResultTime(info)) + private def getSchedulerDelay(info: TaskInfo, metrics: TaskMetrics, currentTime: Long): Long = { + if (info.finished) { + val totalExecutionTime = info.finishTime - info.launchTime + val executorOverhead = (metrics.executorDeserializeTime + + metrics.resultSerializationTime) + math.max( + 0, + totalExecutionTime - metrics.executorRunTime - executorOverhead - + getGettingResultTime(info, currentTime)) + } else { + // The task is still running and the metrics like executorRunTime are not available. + 0L + } } } diff --git a/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala b/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala index 07db783c572cf..04f584621e71e 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala @@ -21,7 +21,7 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node -import org.apache.spark.storage.RDDInfo +import org.apache.spark.storage._ import org.apache.spark.ui.{UIUtils, WebUIPage} import org.apache.spark.util.Utils @@ -30,13 +30,25 @@ private[ui] class StoragePage(parent: StorageTab) extends WebUIPage("") { private val listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { - val rdds = listener.rddInfoList - val content = UIUtils.listingTable(rddHeader, rddRow, rdds, id = Some("storage-by-rdd-table")) + val content = rddTable(listener.rddInfoList) ++ + receiverBlockTables(listener.allExecutorStreamBlockStatus.sortBy(_.executorId)) UIUtils.headerSparkPage("Storage", content, parent) } + private[storage] def rddTable(rdds: Seq[RDDInfo]): Seq[Node] = { + if (rdds.isEmpty) { + // Don't show the rdd table if there is no RDD persisted. + Nil + } else { +

+

RDDs

+ {UIUtils.listingTable(rddHeader, rddRow, rdds, id = Some("storage-by-rdd-table"))} +
+ } + } + /** Header fields for the RDD table */ - private def rddHeader = Seq( + private val rddHeader = Seq( "RDD Name", "Storage Level", "Cached Partitions", @@ -56,7 +68,7 @@ private[ui] class StoragePage(parent: StorageTab) extends WebUIPage("") { {rdd.storageLevel.description} - {rdd.numCachedPartitions} + {rdd.numCachedPartitions.toString} {"%.0f%%".format(rdd.numCachedPartitions * 100.0 / rdd.numPartitions)} {Utils.bytesToString(rdd.memSize)} {Utils.bytesToString(rdd.externalBlockStoreSize)} @@ -64,4 +76,130 @@ private[ui] class StoragePage(parent: StorageTab) extends WebUIPage("") { // scalastyle:on } + + private[storage] def receiverBlockTables(statuses: Seq[ExecutorStreamBlockStatus]): Seq[Node] = { + if (statuses.map(_.numStreamBlocks).sum == 0) { + // Don't show the tables if there is no stream block + Nil + } else { + val blocks = statuses.flatMap(_.blocks).groupBy(_.blockId).toSeq.sortBy(_._1.toString) + +
+

Receiver Blocks

+ {executorMetricsTable(statuses)} + {streamBlockTable(blocks)} +
+ } + } + + private def executorMetricsTable(statuses: Seq[ExecutorStreamBlockStatus]): Seq[Node] = { +
+
Aggregated Block Metrics by Executor
+ {UIUtils.listingTable(executorMetricsTableHeader, executorMetricsTableRow, statuses, + id = Some("storage-by-executor-stream-blocks"))} +
+ } + + private val executorMetricsTableHeader = Seq( + "Executor ID", + "Address", + "Total Size in Memory", + "Total Size in ExternalBlockStore", + "Total Size on Disk", + "Stream Blocks") + + private def executorMetricsTableRow(status: ExecutorStreamBlockStatus): Seq[Node] = { + + + {status.executorId} + + + {status.location} + + + {Utils.bytesToString(status.totalMemSize)} + + + {Utils.bytesToString(status.totalExternalBlockStoreSize)} + + + {Utils.bytesToString(status.totalDiskSize)} + + + {status.numStreamBlocks.toString} + + + } + + private def streamBlockTable(blocks: Seq[(BlockId, Seq[BlockUIData])]): Seq[Node] = { + if (blocks.isEmpty) { + Nil + } else { +
+
Blocks
+ {UIUtils.listingTable( + streamBlockTableHeader, + streamBlockTableRow, + blocks, + id = Some("storage-by-block-table"), + sortable = false)} +
+ } + } + + private val streamBlockTableHeader = Seq( + "Block ID", + "Replication Level", + "Location", + "Storage Level", + "Size") + + /** Render a stream block */ + private def streamBlockTableRow(block: (BlockId, Seq[BlockUIData])): Seq[Node] = { + val replications = block._2 + assert(replications.size > 0) // This must be true because it's the result of "groupBy" + if (replications.size == 1) { + streamBlockTableSubrow(block._1, replications.head, replications.size, true) + } else { + streamBlockTableSubrow(block._1, replications.head, replications.size, true) ++ + replications.tail.map(streamBlockTableSubrow(block._1, _, replications.size, false)).flatten + } + } + + private def streamBlockTableSubrow( + blockId: BlockId, block: BlockUIData, replication: Int, firstSubrow: Boolean): Seq[Node] = { + val (storageLevel, size) = streamBlockStorageLevelDescriptionAndSize(block) + + + { + if (firstSubrow) { + + {block.blockId.toString} + + + {replication.toString} + + } + } + {block.location} + {storageLevel} + {Utils.bytesToString(size)} + + } + + private[storage] def streamBlockStorageLevelDescriptionAndSize( + block: BlockUIData): (String, Long) = { + if (block.storageLevel.useDisk) { + ("Disk", block.diskSize) + } else if (block.storageLevel.useMemory && block.storageLevel.deserialized) { + ("Memory", block.memSize) + } else if (block.storageLevel.useMemory && !block.storageLevel.deserialized) { + ("Memory Serialized", block.memSize) + } else if (block.storageLevel.useOffHeap) { + ("External", block.externalBlockStoreSize) + } else { + throw new IllegalStateException(s"Invalid Storage Level: ${block.storageLevel}") + } + } + } diff --git a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala index 0351749700962..22e2993b3b5bd 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala @@ -39,7 +39,8 @@ private[ui] class StorageTab(parent: SparkUI) extends SparkUITab(parent, "storag * This class is thread-safe (unlike JobProgressListener) */ @DeveloperApi -class StorageListener(storageStatusListener: StorageStatusListener) extends SparkListener { +class StorageListener(storageStatusListener: StorageStatusListener) extends BlockStatusListener { + private[ui] val _rddInfoMap = mutable.Map[Int, RDDInfo]() // exposed for testing def storageStatusList: Seq[StorageStatus] = storageStatusListener.storageStatusList diff --git a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala index 305de4c75539d..43626b4ef4880 100644 --- a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala +++ b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala @@ -448,10 +448,12 @@ private class InnerClosureFinder(output: Set[Class[_]]) extends ClassVisitor(ASM if (op == INVOKESPECIAL && name == "" && argTypes.length > 0 && argTypes(0).toString.startsWith("L") // is it an object? && argTypes(0).getInternalName == myName) { + // scalastyle:off classforname output += Class.forName( owner.replace('/', '.'), false, Thread.currentThread.getContextClassLoader) + // scalastyle:on classforname } } } diff --git a/core/src/main/scala/org/apache/spark/util/Distribution.scala b/core/src/main/scala/org/apache/spark/util/Distribution.scala index 1bab707235b89..950b69f7db641 100644 --- a/core/src/main/scala/org/apache/spark/util/Distribution.scala +++ b/core/src/main/scala/org/apache/spark/util/Distribution.scala @@ -52,9 +52,11 @@ private[spark] class Distribution(val data: Array[Double], val startIdx: Int, va } def showQuantiles(out: PrintStream = System.out): Unit = { + // scalastyle:off println out.println("min\t25%\t50%\t75%\tmax") getQuantiles(defaultProbabilities).foreach{q => out.print(q + "\t")} out.println + // scalastyle:on println } def statCounter: StatCounter = StatCounter(data.slice(startIdx, endIdx)) @@ -64,8 +66,10 @@ private[spark] class Distribution(val data: Array[Double], val startIdx: Int, va * @param out */ def summary(out: PrintStream = System.out) { + // scalastyle:off println out.println(statCounter) showQuantiles(out) + // scalastyle:on println } } @@ -80,8 +84,10 @@ private[spark] object Distribution { } def showQuantiles(out: PrintStream = System.out, quantiles: Traversable[Double]) { + // scalastyle:off println out.println("min\t25%\t50%\t75%\tmax") quantiles.foreach{q => out.print(q + "\t")} out.println + // scalastyle:on println } } diff --git a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala index 0180399c9dad5..7d84468f62ab1 100644 --- a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala +++ b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala @@ -124,9 +124,11 @@ object SizeEstimator extends Logging { val server = ManagementFactory.getPlatformMBeanServer() // NOTE: This should throw an exception in non-Sun JVMs + // scalastyle:off classforname val hotSpotMBeanClass = Class.forName("com.sun.management.HotSpotDiagnosticMXBean") val getVMMethod = hotSpotMBeanClass.getDeclaredMethod("getVMOption", Class.forName("java.lang.String")) + // scalastyle:on classforname val bean = ManagementFactory.newPlatformMXBeanProxy(server, hotSpotMBeanName, hotSpotMBeanClass) 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 944560a91354a..e6374f17d858f 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -113,8 +113,11 @@ private[spark] object Utils extends Logging { def deserialize[T](bytes: Array[Byte], loader: ClassLoader): T = { val bis = new ByteArrayInputStream(bytes) val ois = new ObjectInputStream(bis) { - override def resolveClass(desc: ObjectStreamClass): Class[_] = + override def resolveClass(desc: ObjectStreamClass): Class[_] = { + // scalastyle:off classforname Class.forName(desc.getName, false, loader) + // scalastyle:on classforname + } } ois.readObject.asInstanceOf[T] } @@ -177,12 +180,16 @@ private[spark] object Utils extends Logging { /** Determines whether the provided class is loadable in the current thread. */ def classIsLoadable(clazz: String): Boolean = { + // scalastyle:off classforname Try { Class.forName(clazz, false, getContextOrSparkClassLoader) }.isSuccess + // scalastyle:on classforname } + // scalastyle:off classforname /** Preferred alternative to Class.forName(className) */ def classForName(className: String): Class[_] = { Class.forName(className, true, getContextOrSparkClassLoader) + // scalastyle:on classforname } /** @@ -733,7 +740,12 @@ private[spark] object Utils extends Logging { localRootDirs } - private def getOrCreateLocalRootDirsImpl(conf: SparkConf): Array[String] = { + /** + * Return the configured local directories where Spark can write files. This + * method does not create any directories on its own, it only encapsulates the + * logic of locating the local directories according to deployment mode. + */ + def getConfiguredLocalDirs(conf: SparkConf): Array[String] = { if (isRunningInYarnContainer(conf)) { // If we are in yarn mode, systems can have different disk layouts so we must set it // to what Yarn on this system said was available. Note this assumes that Yarn has @@ -749,27 +761,29 @@ private[spark] object Utils extends Logging { Option(conf.getenv("SPARK_LOCAL_DIRS")) .getOrElse(conf.get("spark.local.dir", System.getProperty("java.io.tmpdir"))) .split(",") - .flatMap { root => - try { - val rootDir = new File(root) - if (rootDir.exists || rootDir.mkdirs()) { - val dir = createTempDir(root) - chmod700(dir) - Some(dir.getAbsolutePath) - } else { - logError(s"Failed to create dir in $root. Ignoring this directory.") - None - } - } catch { - case e: IOException => - logError(s"Failed to create local root dir in $root. Ignoring this directory.") - None - } - } - .toArray } } + private def getOrCreateLocalRootDirsImpl(conf: SparkConf): Array[String] = { + getConfiguredLocalDirs(conf).flatMap { root => + try { + val rootDir = new File(root) + if (rootDir.exists || rootDir.mkdirs()) { + val dir = createTempDir(root) + chmod700(dir) + Some(dir.getAbsolutePath) + } else { + logError(s"Failed to create dir in $root. Ignoring this directory.") + None + } + } catch { + case e: IOException => + logError(s"Failed to create local root dir in $root. Ignoring this directory.") + None + } + }.toArray + } + /** Get the Yarn approved local directories. */ private def getYarnLocalDirs(conf: SparkConf): String = { // Hadoop 0.23 and 2.x have different Environment variable names for the @@ -2259,7 +2273,7 @@ private [util] class SparkShutdownHookManager { val hookTask = new Runnable() { override def run(): Unit = runAll() } - Try(Class.forName("org.apache.hadoop.util.ShutdownHookManager")) match { + Try(Utils.classForName("org.apache.hadoop.util.ShutdownHookManager")) match { case Success(shmClass) => val fsPriority = classOf[FileSystem].getField("SHUTDOWN_HOOK_PRIORITY").get() .asInstanceOf[Int] diff --git a/core/src/main/scala/org/apache/spark/util/collection/ChainedBuffer.scala b/core/src/main/scala/org/apache/spark/util/collection/ChainedBuffer.scala index 516aaa44d03fc..ae60f3b0cb555 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ChainedBuffer.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ChainedBuffer.scala @@ -37,7 +37,7 @@ private[spark] class ChainedBuffer(chunkSize: Int) { private var _size: Long = 0 /** - * Feed bytes from this buffer into a BlockObjectWriter. + * Feed bytes from this buffer into a DiskBlockObjectWriter. * * @param pos Offset in the buffer to read from. * @param os OutputStream to read into. 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 757dec66c203b..ba7ec834d622d 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 @@ -30,7 +30,7 @@ import org.apache.spark._ import org.apache.spark.serializer._ import org.apache.spark.executor.ShuffleWriteMetrics import org.apache.spark.shuffle.sort.{SortShuffleFileWriter, SortShuffleWriter} -import org.apache.spark.storage.{BlockId, BlockObjectWriter} +import org.apache.spark.storage.{BlockId, DiskBlockObjectWriter} /** * Sorts and potentially merges a number of key-value pairs of type (K, V) to produce key-combiner @@ -250,7 +250,7 @@ private[spark] class ExternalSorter[K, V, C]( // These variables are reset after each flush var objectsWritten: Long = 0 var spillMetrics: ShuffleWriteMetrics = null - var writer: BlockObjectWriter = null + var writer: DiskBlockObjectWriter = null def openWriter(): Unit = { assert (writer == null && spillMetrics == null) spillMetrics = new ShuffleWriteMetrics diff --git a/core/src/main/scala/org/apache/spark/util/collection/PartitionedPairBuffer.scala b/core/src/main/scala/org/apache/spark/util/collection/PartitionedPairBuffer.scala index 04bb7fc78c13b..f5844d5353be7 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/PartitionedPairBuffer.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/PartitionedPairBuffer.scala @@ -19,7 +19,6 @@ package org.apache.spark.util.collection import java.util.Comparator -import org.apache.spark.storage.BlockObjectWriter import org.apache.spark.util.collection.WritablePartitionedPairCollection._ /** diff --git a/core/src/main/scala/org/apache/spark/util/collection/PartitionedSerializedPairBuffer.scala b/core/src/main/scala/org/apache/spark/util/collection/PartitionedSerializedPairBuffer.scala index ae9a48729e201..87a786b02d651 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/PartitionedSerializedPairBuffer.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/PartitionedSerializedPairBuffer.scala @@ -21,9 +21,8 @@ import java.io.InputStream import java.nio.IntBuffer import java.util.Comparator -import org.apache.spark.SparkEnv import org.apache.spark.serializer.{JavaSerializerInstance, SerializerInstance} -import org.apache.spark.storage.BlockObjectWriter +import org.apache.spark.storage.DiskBlockObjectWriter import org.apache.spark.util.collection.PartitionedSerializedPairBuffer._ /** @@ -136,7 +135,7 @@ private[spark] class PartitionedSerializedPairBuffer[K, V]( // current position in the meta buffer in ints var pos = 0 - def writeNext(writer: BlockObjectWriter): Unit = { + def writeNext(writer: DiskBlockObjectWriter): Unit = { val keyStart = getKeyStartPos(metaBuffer, pos) val keyValLen = metaBuffer.get(pos + KEY_VAL_LEN) pos += RECORD_SIZE diff --git a/core/src/main/scala/org/apache/spark/util/collection/WritablePartitionedPairCollection.scala b/core/src/main/scala/org/apache/spark/util/collection/WritablePartitionedPairCollection.scala index 7bc59898658e4..38848e9018c6c 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/WritablePartitionedPairCollection.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/WritablePartitionedPairCollection.scala @@ -19,7 +19,7 @@ package org.apache.spark.util.collection import java.util.Comparator -import org.apache.spark.storage.BlockObjectWriter +import org.apache.spark.storage.DiskBlockObjectWriter /** * A common interface for size-tracking collections of key-value pairs that @@ -51,7 +51,7 @@ private[spark] trait WritablePartitionedPairCollection[K, V] { new WritablePartitionedIterator { private[this] var cur = if (it.hasNext) it.next() else null - def writeNext(writer: BlockObjectWriter): Unit = { + def writeNext(writer: DiskBlockObjectWriter): Unit = { writer.write(cur._1._2, cur._2) cur = if (it.hasNext) it.next() else null } @@ -91,11 +91,11 @@ private[spark] object WritablePartitionedPairCollection { } /** - * Iterator that writes elements to a BlockObjectWriter instead of returning them. Each element + * Iterator that writes elements to a DiskBlockObjectWriter instead of returning them. Each element * has an associated partition. */ private[spark] trait WritablePartitionedIterator { - def writeNext(writer: BlockObjectWriter): Unit + def writeNext(writer: DiskBlockObjectWriter): Unit def hasNext(): Boolean diff --git a/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala b/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala index c4a7b4441c85c..85fb923cd9bc7 100644 --- a/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala +++ b/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala @@ -70,12 +70,14 @@ private[spark] object XORShiftRandom { * @param args takes one argument - the number of random numbers to generate */ def main(args: Array[String]): Unit = { + // scalastyle:off println if (args.length != 1) { println("Benchmark of XORShiftRandom vis-a-vis java.util.Random") println("Usage: XORShiftRandom number_of_random_numbers_to_generate") System.exit(1) } println(benchmark(args(0).toInt)) + // scalastyle:on println } /** diff --git a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java new file mode 100644 index 0000000000000..ea8755e21eb68 --- /dev/null +++ b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java @@ -0,0 +1,202 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.collection.unsafe.sort; + +import java.io.File; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.UUID; + +import scala.Tuple2; +import scala.Tuple2$; +import scala.runtime.AbstractFunction1; + +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; +import static org.junit.Assert.*; +import static org.mockito.AdditionalAnswers.returnsFirstArg; +import static org.mockito.AdditionalAnswers.returnsSecondArg; +import static org.mockito.Answers.RETURNS_SMART_NULLS; +import static org.mockito.Mockito.*; + +import org.apache.spark.SparkConf; +import org.apache.spark.TaskContext; +import org.apache.spark.executor.ShuffleWriteMetrics; +import org.apache.spark.executor.TaskMetrics; +import org.apache.spark.serializer.SerializerInstance; +import org.apache.spark.shuffle.ShuffleMemoryManager; +import org.apache.spark.storage.*; +import org.apache.spark.unsafe.PlatformDependent; +import org.apache.spark.unsafe.memory.ExecutorMemoryManager; +import org.apache.spark.unsafe.memory.MemoryAllocator; +import org.apache.spark.unsafe.memory.TaskMemoryManager; +import org.apache.spark.util.Utils; + +public class UnsafeExternalSorterSuite { + + final TaskMemoryManager memoryManager = + new TaskMemoryManager(new ExecutorMemoryManager(MemoryAllocator.HEAP)); + // Use integer comparison for comparing prefixes (which are partition ids, in this case) + final PrefixComparator prefixComparator = new PrefixComparator() { + @Override + public int compare(long prefix1, long prefix2) { + return (int) prefix1 - (int) prefix2; + } + }; + // Since the key fits within the 8-byte prefix, we don't need to do any record comparison, so + // use a dummy comparator + final RecordComparator recordComparator = new RecordComparator() { + @Override + public int compare( + Object leftBaseObject, + long leftBaseOffset, + Object rightBaseObject, + long rightBaseOffset) { + return 0; + } + }; + + @Mock(answer = RETURNS_SMART_NULLS) ShuffleMemoryManager shuffleMemoryManager; + @Mock(answer = RETURNS_SMART_NULLS) BlockManager blockManager; + @Mock(answer = RETURNS_SMART_NULLS) DiskBlockManager diskBlockManager; + @Mock(answer = RETURNS_SMART_NULLS) TaskContext taskContext; + + File tempDir; + + private static final class CompressStream extends AbstractFunction1 { + @Override + public OutputStream apply(OutputStream stream) { + return stream; + } + } + + @Before + public void setUp() { + MockitoAnnotations.initMocks(this); + tempDir = new File(Utils.createTempDir$default$1()); + taskContext = mock(TaskContext.class); + when(taskContext.taskMetrics()).thenReturn(new TaskMetrics()); + when(shuffleMemoryManager.tryToAcquire(anyLong())).then(returnsFirstArg()); + when(blockManager.diskBlockManager()).thenReturn(diskBlockManager); + when(diskBlockManager.createTempLocalBlock()).thenAnswer(new Answer>() { + @Override + public Tuple2 answer(InvocationOnMock invocationOnMock) throws Throwable { + TempLocalBlockId blockId = new TempLocalBlockId(UUID.randomUUID()); + File file = File.createTempFile("spillFile", ".spill", tempDir); + return Tuple2$.MODULE$.apply(blockId, file); + } + }); + when(blockManager.getDiskWriter( + any(BlockId.class), + any(File.class), + any(SerializerInstance.class), + anyInt(), + any(ShuffleWriteMetrics.class))).thenAnswer(new Answer() { + @Override + public DiskBlockObjectWriter answer(InvocationOnMock invocationOnMock) throws Throwable { + Object[] args = invocationOnMock.getArguments(); + + return new DiskBlockObjectWriter( + (BlockId) args[0], + (File) args[1], + (SerializerInstance) args[2], + (Integer) args[3], + new CompressStream(), + false, + (ShuffleWriteMetrics) args[4] + ); + } + }); + when(blockManager.wrapForCompression(any(BlockId.class), any(InputStream.class))) + .then(returnsSecondArg()); + } + + private static void insertNumber(UnsafeExternalSorter sorter, int value) throws Exception { + final int[] arr = new int[] { value }; + sorter.insertRecord(arr, PlatformDependent.INT_ARRAY_OFFSET, 4, value); + } + + @Test + public void testSortingOnlyByPrefix() throws Exception { + + final UnsafeExternalSorter sorter = new UnsafeExternalSorter( + memoryManager, + shuffleMemoryManager, + blockManager, + taskContext, + recordComparator, + prefixComparator, + 1024, + new SparkConf()); + + insertNumber(sorter, 5); + insertNumber(sorter, 1); + insertNumber(sorter, 3); + sorter.spill(); + insertNumber(sorter, 4); + sorter.spill(); + insertNumber(sorter, 2); + + UnsafeSorterIterator iter = sorter.getSortedIterator(); + + for (int i = 1; i <= 5; i++) { + iter.loadNext(); + assertEquals(i, iter.getKeyPrefix()); + assertEquals(4, iter.getRecordLength()); + // TODO: read rest of value. + } + + // TODO: test for cleanup: + // assert(tempDir.isEmpty) + } + + @Test + public void testSortingEmptyArrays() throws Exception { + + final UnsafeExternalSorter sorter = new UnsafeExternalSorter( + memoryManager, + shuffleMemoryManager, + blockManager, + taskContext, + recordComparator, + prefixComparator, + 1024, + new SparkConf()); + + sorter.insertRecord(null, 0, 0, 0); + sorter.insertRecord(null, 0, 0, 0); + sorter.spill(); + sorter.insertRecord(null, 0, 0, 0); + sorter.spill(); + sorter.insertRecord(null, 0, 0, 0); + sorter.insertRecord(null, 0, 0, 0); + + UnsafeSorterIterator iter = sorter.getSortedIterator(); + + for (int i = 1; i <= 5; i++) { + iter.loadNext(); + assertEquals(0, iter.getKeyPrefix()); + assertEquals(0, iter.getRecordLength()); + } + } + +} diff --git a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java new file mode 100644 index 0000000000000..909500930539c --- /dev/null +++ b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java @@ -0,0 +1,139 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.collection.unsafe.sort; + +import java.util.Arrays; + +import org.junit.Test; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.*; +import static org.junit.Assert.*; +import static org.mockito.Mockito.mock; + +import org.apache.spark.HashPartitioner; +import org.apache.spark.unsafe.PlatformDependent; +import org.apache.spark.unsafe.memory.ExecutorMemoryManager; +import org.apache.spark.unsafe.memory.MemoryAllocator; +import org.apache.spark.unsafe.memory.MemoryBlock; +import org.apache.spark.unsafe.memory.TaskMemoryManager; + +public class UnsafeInMemorySorterSuite { + + private static String getStringFromDataPage(Object baseObject, long baseOffset, int length) { + final byte[] strBytes = new byte[length]; + PlatformDependent.copyMemory( + baseObject, + baseOffset, + strBytes, + PlatformDependent.BYTE_ARRAY_OFFSET, length); + return new String(strBytes); + } + + @Test + public void testSortingEmptyInput() { + final UnsafeInMemorySorter sorter = new UnsafeInMemorySorter( + new TaskMemoryManager(new ExecutorMemoryManager(MemoryAllocator.HEAP)), + mock(RecordComparator.class), + mock(PrefixComparator.class), + 100); + final UnsafeSorterIterator iter = sorter.getSortedIterator(); + assert(!iter.hasNext()); + } + + @Test + public void testSortingOnlyByIntegerPrefix() throws Exception { + final String[] dataToSort = new String[] { + "Boba", + "Pearls", + "Tapioca", + "Taho", + "Condensed Milk", + "Jasmine", + "Milk Tea", + "Lychee", + "Mango" + }; + final TaskMemoryManager memoryManager = + new TaskMemoryManager(new ExecutorMemoryManager(MemoryAllocator.HEAP)); + final MemoryBlock dataPage = memoryManager.allocatePage(2048); + final Object baseObject = dataPage.getBaseObject(); + // Write the records into the data page: + long position = dataPage.getBaseOffset(); + for (String str : dataToSort) { + final byte[] strBytes = str.getBytes("utf-8"); + PlatformDependent.UNSAFE.putInt(baseObject, position, strBytes.length); + position += 4; + PlatformDependent.copyMemory( + strBytes, + PlatformDependent.BYTE_ARRAY_OFFSET, + baseObject, + position, + strBytes.length); + position += strBytes.length; + } + // Since the key fits within the 8-byte prefix, we don't need to do any record comparison, so + // use a dummy comparator + final RecordComparator recordComparator = new RecordComparator() { + @Override + public int compare( + Object leftBaseObject, + long leftBaseOffset, + Object rightBaseObject, + long rightBaseOffset) { + return 0; + } + }; + // Compute key prefixes based on the records' partition ids + final HashPartitioner hashPartitioner = new HashPartitioner(4); + // Use integer comparison for comparing prefixes (which are partition ids, in this case) + final PrefixComparator prefixComparator = new PrefixComparator() { + @Override + public int compare(long prefix1, long prefix2) { + return (int) prefix1 - (int) prefix2; + } + }; + UnsafeInMemorySorter sorter = new UnsafeInMemorySorter(memoryManager, recordComparator, + prefixComparator, dataToSort.length); + // Given a page of records, insert those records into the sorter one-by-one: + position = dataPage.getBaseOffset(); + for (int i = 0; i < dataToSort.length; i++) { + // position now points to the start of a record (which holds its length). + final int recordLength = PlatformDependent.UNSAFE.getInt(baseObject, position); + final long address = memoryManager.encodePageNumberAndOffset(dataPage, position); + final String str = getStringFromDataPage(baseObject, position + 4, recordLength); + final int partitionId = hashPartitioner.getPartition(str); + sorter.insertRecord(address, partitionId); + position += 4 + recordLength; + } + final UnsafeSorterIterator iter = sorter.getSortedIterator(); + int iterLength = 0; + long prevPrefix = -1; + Arrays.sort(dataToSort); + while (iter.hasNext()) { + iter.loadNext(); + final String str = + getStringFromDataPage(iter.getBaseObject(), iter.getBaseOffset(), iter.getRecordLength()); + final long keyPrefix = iter.getKeyPrefix(); + assertThat(str, isIn(Arrays.asList(dataToSort))); + assertThat(keyPrefix, greaterThanOrEqualTo(prevPrefix)); + prevPrefix = keyPrefix; + iterLength++; + } + assertEquals(dataToSort.length, iterLength); + } +} diff --git a/core/src/test/scala/org/apache/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala index 9c191ed52206d..2300bcff4f118 100644 --- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala @@ -107,7 +107,9 @@ class DistributedSuite extends SparkFunSuite with Matchers with LocalSparkContex sc = new SparkContext(clusterUrl, "test") val accum = sc.accumulator(0) val thrown = intercept[SparkException] { + // scalastyle:off println sc.parallelize(1 to 10, 10).foreach(x => println(x / 0)) + // scalastyle:on println } assert(thrown.getClass === classOf[SparkException]) assert(thrown.getMessage.contains("failed 4 times")) diff --git a/core/src/test/scala/org/apache/spark/FailureSuite.scala b/core/src/test/scala/org/apache/spark/FailureSuite.scala index a8c8c6f73fb5a..b099cd3fb7965 100644 --- a/core/src/test/scala/org/apache/spark/FailureSuite.scala +++ b/core/src/test/scala/org/apache/spark/FailureSuite.scala @@ -130,7 +130,9 @@ class FailureSuite extends SparkFunSuite with LocalSparkContext { // Non-serializable closure in foreach function val thrown2 = intercept[SparkException] { + // scalastyle:off println sc.parallelize(1 to 10, 2).foreach(x => println(a)) + // scalastyle:on println } assert(thrown2.getClass === classOf[SparkException]) assert(thrown2.getMessage.contains("NotSerializableException") || diff --git a/core/src/test/scala/org/apache/spark/FileServerSuite.scala b/core/src/test/scala/org/apache/spark/FileServerSuite.scala index 6e65b0a8f6c76..876418aa13029 100644 --- a/core/src/test/scala/org/apache/spark/FileServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileServerSuite.scala @@ -51,7 +51,9 @@ class FileServerSuite extends SparkFunSuite with LocalSparkContext { val textFile = new File(testTempDir, "FileServerSuite.txt") val pw = new PrintWriter(textFile) + // scalastyle:off println pw.println("100") + // scalastyle:on println pw.close() val jarFile = new File(testTempDir, "test.jar") diff --git a/core/src/test/scala/org/apache/spark/FileSuite.scala b/core/src/test/scala/org/apache/spark/FileSuite.scala index 1d8fade90f398..418763f4e5ffa 100644 --- a/core/src/test/scala/org/apache/spark/FileSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileSuite.scala @@ -179,6 +179,7 @@ class FileSuite extends SparkFunSuite with LocalSparkContext { } test("object files of classes from a JAR") { + // scalastyle:off classforname val original = Thread.currentThread().getContextClassLoader val className = "FileSuiteObjectFileTest" val jar = TestUtils.createJarWithClasses(Seq(className)) @@ -201,6 +202,7 @@ class FileSuite extends SparkFunSuite with LocalSparkContext { finally { Thread.currentThread().setContextClassLoader(original) } + // scalastyle:on classforname } test("write SequenceFile using new Hadoop API") { diff --git a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala index f89e3d0a49920..dba46f101c580 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark import org.scalatest.PrivateMethodTester +import org.apache.spark.util.Utils import org.apache.spark.scheduler.{SchedulerBackend, TaskScheduler, TaskSchedulerImpl} import org.apache.spark.scheduler.cluster.{SimrSchedulerBackend, SparkDeploySchedulerBackend} import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} @@ -131,7 +132,7 @@ class SparkContextSchedulerCreationSuite def testYarn(master: String, expectedClassName: String) { try { val sched = createTaskScheduler(master) - assert(sched.getClass === Class.forName(expectedClassName)) + assert(sched.getClass === Utils.classForName(expectedClassName)) } catch { case e: SparkException => assert(e.getMessage.contains("YARN mode not available")) diff --git a/core/src/test/scala/org/apache/spark/ThreadingSuite.scala b/core/src/test/scala/org/apache/spark/ThreadingSuite.scala index 6580139df6c60..48509f0759a3b 100644 --- a/core/src/test/scala/org/apache/spark/ThreadingSuite.scala +++ b/core/src/test/scala/org/apache/spark/ThreadingSuite.scala @@ -36,7 +36,7 @@ object ThreadingSuiteState { } } -class ThreadingSuite extends SparkFunSuite with LocalSparkContext { +class ThreadingSuite extends SparkFunSuite with LocalSparkContext with Logging { test("accessing SparkContext form a different thread") { sc = new SparkContext("local", "test") @@ -130,8 +130,6 @@ class ThreadingSuite extends SparkFunSuite with LocalSparkContext { Thread.sleep(100) } if (running.get() != 4) { - println("Waited 1 second without seeing runningThreads = 4 (it was " + - running.get() + "); failing test") ThreadingSuiteState.failed.set(true) } number @@ -143,6 +141,8 @@ class ThreadingSuite extends SparkFunSuite with LocalSparkContext { } sem.acquire(2) if (ThreadingSuiteState.failed.get()) { + logError("Waited 1 second without seeing runningThreads = 4 (it was " + + ThreadingSuiteState.runningThreads.get() + "); failing test") fail("One or more threads didn't see runningThreads = 4") } } 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 2e05dec99b6bf..343d28eef8359 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -51,9 +51,11 @@ class SparkSubmitSuite /** Simple PrintStream that reads data into a buffer */ private class BufferPrintStream extends PrintStream(noOpOutputStream) { var lineBuffer = ArrayBuffer[String]() + // scalastyle:off println override def println(line: String) { lineBuffer += line } + // scalastyle:on println } /** Returns true if the script exits and the given search string is printed. */ @@ -81,6 +83,7 @@ class SparkSubmitSuite } } + // scalastyle:off println test("prints usage on empty input") { testPrematureExit(Array[String](), "Usage: spark-submit") } @@ -243,7 +246,7 @@ class SparkSubmitSuite mainClass should be ("org.apache.spark.deploy.Client") } classpath should have size 0 - sysProps should have size 8 + sysProps should have size 9 sysProps.keys should contain ("SPARK_SUBMIT") sysProps.keys should contain ("spark.master") sysProps.keys should contain ("spark.app.name") @@ -252,6 +255,7 @@ class SparkSubmitSuite sysProps.keys should contain ("spark.driver.cores") sysProps.keys should contain ("spark.driver.supervise") sysProps.keys should contain ("spark.shuffle.spill") + sysProps.keys should contain ("spark.submit.deployMode") sysProps("spark.shuffle.spill") should be ("false") } @@ -491,6 +495,7 @@ class SparkSubmitSuite appArgs.executorMemory should be ("2.3g") } } + // scalastyle:on println // NOTE: This is an expensive operation in terms of time (10 seconds+). Use sparingly. private def runSparkSubmit(args: Seq[String]): Unit = { @@ -536,8 +541,8 @@ object JarCreationTest extends Logging { val result = sc.makeRDD(1 to 100, 10).mapPartitions { x => var exception: String = null try { - Class.forName(args(0), true, Thread.currentThread().getContextClassLoader) - Class.forName(args(1), true, Thread.currentThread().getContextClassLoader) + Utils.classForName(args(0)) + Utils.classForName(args(1)) } catch { case t: Throwable => exception = t + "\n" + t.getStackTraceString diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala index c9b435a9228d3..01ece1a10f46d 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala @@ -41,9 +41,11 @@ class SparkSubmitUtilsSuite extends SparkFunSuite with BeforeAndAfterAll { /** Simple PrintStream that reads data into a buffer */ private class BufferPrintStream extends PrintStream(noOpOutputStream) { var lineBuffer = ArrayBuffer[String]() + // scalastyle:off println override def println(line: String) { lineBuffer += line } + // scalastyle:on println } override def beforeAll() { diff --git a/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala b/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala index 63947df3d43a2..8a199459c1ddf 100644 --- a/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala +++ b/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala @@ -27,7 +27,7 @@ import org.scalatest.BeforeAndAfterAll import org.apache.hadoop.io.Text -import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark.{Logging, SparkConf, SparkContext, SparkFunSuite} import org.apache.spark.util.Utils import org.apache.hadoop.io.compress.{DefaultCodec, CompressionCodecFactory, GzipCodec} @@ -36,7 +36,7 @@ import org.apache.hadoop.io.compress.{DefaultCodec, CompressionCodecFactory, Gzi * [[org.apache.spark.input.WholeTextFileRecordReader WholeTextFileRecordReader]]. A temporary * directory is created as fake input. Temporal storage would be deleted in the end. */ -class WholeTextFileRecordReaderSuite extends SparkFunSuite with BeforeAndAfterAll { +class WholeTextFileRecordReaderSuite extends SparkFunSuite with BeforeAndAfterAll with Logging { private var sc: SparkContext = _ private var factory: CompressionCodecFactory = _ @@ -85,7 +85,7 @@ class WholeTextFileRecordReaderSuite extends SparkFunSuite with BeforeAndAfterAl */ test("Correctness of WholeTextFileRecordReader.") { val dir = Utils.createTempDir() - println(s"Local disk address is ${dir.toString}.") + logInfo(s"Local disk address is ${dir.toString}.") WholeTextFileRecordReaderSuite.files.foreach { case (filename, contents) => createNativeFile(dir, filename, contents, false) @@ -109,7 +109,7 @@ class WholeTextFileRecordReaderSuite extends SparkFunSuite with BeforeAndAfterAl test("Correctness of WholeTextFileRecordReader with GzipCodec.") { val dir = Utils.createTempDir() - println(s"Local disk address is ${dir.toString}.") + logInfo(s"Local disk address is ${dir.toString}.") WholeTextFileRecordReaderSuite.files.foreach { case (filename, contents) => createNativeFile(dir, filename, contents, true) 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 9e4d34fb7d382..d3218a548efc7 100644 --- a/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala +++ b/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala @@ -60,7 +60,9 @@ class InputOutputMetricsSuite extends SparkFunSuite with SharedSparkContext tmpFile = new File(testTempDir, getClass.getSimpleName + ".txt") val pw = new PrintWriter(new FileWriter(tmpFile)) for (x <- 1 to numRecords) { + // scalastyle:off println pw.println(RandomUtils.nextInt(0, numBuckets)) + // scalastyle:on println } pw.close() diff --git a/core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala index 08215a2bafc09..05013fbc49b8e 100644 --- a/core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala @@ -22,11 +22,12 @@ import java.sql._ import org.scalatest.BeforeAndAfter import org.apache.spark.{LocalSparkContext, SparkContext, SparkFunSuite} +import org.apache.spark.util.Utils class JdbcRDDSuite extends SparkFunSuite with BeforeAndAfter with LocalSparkContext { before { - Class.forName("org.apache.derby.jdbc.EmbeddedDriver") + Utils.classForName("org.apache.derby.jdbc.EmbeddedDriver") val conn = DriverManager.getConnection("jdbc:derby:target/JdbcRDDSuiteDb;create=true") try { diff --git a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala index ff3fa95ec32ae..4e3defb43a021 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala @@ -52,8 +52,10 @@ class ReplayListenerSuite extends SparkFunSuite with BeforeAndAfter { val applicationStart = SparkListenerApplicationStart("Greatest App (N)ever", None, 125L, "Mickey", None) val applicationEnd = SparkListenerApplicationEnd(1000L) + // scalastyle:off println writer.println(compact(render(JsonProtocol.sparkEventToJson(applicationStart)))) writer.println(compact(render(JsonProtocol.sparkEventToJson(applicationEnd)))) + // scalastyle:on println writer.close() val conf = EventLoggingListenerSuite.getLoggingConf(logFilePath) diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackendSuite.scala new file mode 100644 index 0000000000000..3f1692917a357 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackendSuite.scala @@ -0,0 +1,175 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.util +import java.util.Collections + +import org.apache.mesos.Protos.Value.Scalar +import org.apache.mesos.Protos._ +import org.apache.mesos.SchedulerDriver +import org.mockito.Matchers._ +import org.mockito.Mockito._ +import org.mockito.Matchers +import org.scalatest.mock.MockitoSugar +import org.scalatest.BeforeAndAfter + +import org.apache.spark.scheduler.TaskSchedulerImpl +import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkFunSuite} + +class CoarseMesosSchedulerBackendSuite extends SparkFunSuite + with LocalSparkContext + with MockitoSugar + with BeforeAndAfter { + + private def createOffer(offerId: String, slaveId: String, mem: Int, cpu: Int): Offer = { + val builder = Offer.newBuilder() + builder.addResourcesBuilder() + .setName("mem") + .setType(Value.Type.SCALAR) + .setScalar(Scalar.newBuilder().setValue(mem)) + builder.addResourcesBuilder() + .setName("cpus") + .setType(Value.Type.SCALAR) + .setScalar(Scalar.newBuilder().setValue(cpu)) + builder.setId(OfferID.newBuilder() + .setValue(offerId).build()) + .setFrameworkId(FrameworkID.newBuilder() + .setValue("f1")) + .setSlaveId(SlaveID.newBuilder().setValue(slaveId)) + .setHostname(s"host${slaveId}") + .build() + } + + private def createSchedulerBackend( + taskScheduler: TaskSchedulerImpl, + driver: SchedulerDriver): CoarseMesosSchedulerBackend = { + val backend = new CoarseMesosSchedulerBackend(taskScheduler, sc, "master") { + mesosDriver = driver + markRegistered() + } + backend.start() + backend + } + + var sparkConf: SparkConf = _ + + before { + sparkConf = (new SparkConf) + .setMaster("local[*]") + .setAppName("test-mesos-dynamic-alloc") + .setSparkHome("/path") + + sc = new SparkContext(sparkConf) + } + + test("mesos supports killing and limiting executors") { + val driver = mock[SchedulerDriver] + val taskScheduler = mock[TaskSchedulerImpl] + when(taskScheduler.sc).thenReturn(sc) + + sparkConf.set("spark.driver.host", "driverHost") + sparkConf.set("spark.driver.port", "1234") + + val backend = createSchedulerBackend(taskScheduler, driver) + val minMem = backend.calculateTotalMemory(sc).toInt + val minCpu = 4 + + val mesosOffers = new java.util.ArrayList[Offer] + mesosOffers.add(createOffer("o1", "s1", minMem, minCpu)) + + val taskID0 = TaskID.newBuilder().setValue("0").build() + + backend.resourceOffers(driver, mesosOffers) + verify(driver, times(1)).launchTasks( + Matchers.eq(Collections.singleton(mesosOffers.get(0).getId)), + any[util.Collection[TaskInfo]], + any[Filters]) + + // simulate the allocation manager down-scaling executors + backend.doRequestTotalExecutors(0) + assert(backend.doKillExecutors(Seq("s1/0"))) + verify(driver, times(1)).killTask(taskID0) + + val mesosOffers2 = new java.util.ArrayList[Offer] + mesosOffers2.add(createOffer("o2", "s2", minMem, minCpu)) + backend.resourceOffers(driver, mesosOffers2) + + verify(driver, times(1)) + .declineOffer(OfferID.newBuilder().setValue("o2").build()) + + // Verify we didn't launch any new executor + assert(backend.slaveIdsWithExecutors.size === 1) + + backend.doRequestTotalExecutors(2) + backend.resourceOffers(driver, mesosOffers2) + verify(driver, times(1)).launchTasks( + Matchers.eq(Collections.singleton(mesosOffers2.get(0).getId)), + any[util.Collection[TaskInfo]], + any[Filters]) + + assert(backend.slaveIdsWithExecutors.size === 2) + backend.slaveLost(driver, SlaveID.newBuilder().setValue("s1").build()) + assert(backend.slaveIdsWithExecutors.size === 1) + } + + test("mesos supports killing and relaunching tasks with executors") { + val driver = mock[SchedulerDriver] + val taskScheduler = mock[TaskSchedulerImpl] + when(taskScheduler.sc).thenReturn(sc) + + val backend = createSchedulerBackend(taskScheduler, driver) + val minMem = backend.calculateTotalMemory(sc).toInt + 1024 + val minCpu = 4 + + val mesosOffers = new java.util.ArrayList[Offer] + val offer1 = createOffer("o1", "s1", minMem, minCpu) + mesosOffers.add(offer1) + + val offer2 = createOffer("o2", "s1", minMem, 1); + + backend.resourceOffers(driver, mesosOffers) + + verify(driver, times(1)).launchTasks( + Matchers.eq(Collections.singleton(offer1.getId)), + anyObject(), + anyObject[Filters]) + + // Simulate task killed, executor no longer running + val status = TaskStatus.newBuilder() + .setTaskId(TaskID.newBuilder().setValue("0").build()) + .setSlaveId(SlaveID.newBuilder().setValue("s1").build()) + .setState(TaskState.TASK_KILLED) + .build + + backend.statusUpdate(driver, status) + assert(!backend.slaveIdsWithExecutors.contains("s1")) + + mesosOffers.clear() + mesosOffers.add(offer2) + backend.resourceOffers(driver, mesosOffers) + assert(backend.slaveIdsWithExecutors.contains("s1")) + + verify(driver, times(1)).launchTasks( + Matchers.eq(Collections.singleton(offer2.getId)), + anyObject(), + anyObject[Filters]) + + verify(driver, times(1)).reviveOffers() + } +} diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MemoryUtilsSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MemoryUtilsSuite.scala deleted file mode 100644 index e72285d03d3ee..0000000000000 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MemoryUtilsSuite.scala +++ /dev/null @@ -1,46 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.scheduler.cluster.mesos - -import org.mockito.Mockito._ -import org.scalatest.mock.MockitoSugar - -import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} - -class MemoryUtilsSuite extends SparkFunSuite with MockitoSugar { - test("MesosMemoryUtils should always override memoryOverhead when it's set") { - val sparkConf = new SparkConf - - val sc = mock[SparkContext] - when(sc.conf).thenReturn(sparkConf) - - // 384 > sc.executorMemory * 0.1 => 512 + 384 = 896 - when(sc.executorMemory).thenReturn(512) - assert(MemoryUtils.calculateTotalMemory(sc) === 896) - - // 384 < sc.executorMemory * 0.1 => 4096 + (4096 * 0.1) = 4505.6 - when(sc.executorMemory).thenReturn(4096) - assert(MemoryUtils.calculateTotalMemory(sc) === 4505) - - // set memoryOverhead - sparkConf.set("spark.mesos.executor.memoryOverhead", "100") - assert(MemoryUtils.calculateTotalMemory(sc) === 4196) - sparkConf.set("spark.mesos.executor.memoryOverhead", "400") - assert(MemoryUtils.calculateTotalMemory(sc) === 4496) - } -} diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala index 68df46a41ddc8..d01837fe78957 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala @@ -149,7 +149,9 @@ class MesosSchedulerBackendSuite extends SparkFunSuite with LocalSparkContext wi when(sc.conf).thenReturn(new SparkConf) when(sc.listenerBus).thenReturn(listenerBus) - val minMem = MemoryUtils.calculateTotalMemory(sc).toInt + val backend = new MesosSchedulerBackend(taskScheduler, sc, "master") + + val minMem = backend.calculateTotalMemory(sc) val minCpu = 4 val mesosOffers = new java.util.ArrayList[Offer] @@ -157,8 +159,6 @@ class MesosSchedulerBackendSuite extends SparkFunSuite with LocalSparkContext wi mesosOffers.add(createOffer(2, minMem - 1, minCpu)) mesosOffers.add(createOffer(3, minMem, minCpu)) - val backend = new MesosSchedulerBackend(taskScheduler, sc, "master") - val expectedWorkerOffers = new ArrayBuffer[WorkerOffer](2) expectedWorkerOffers.append(new WorkerOffer( mesosOffers.get(0).getSlaveId.getValue, diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtilsSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtilsSuite.scala new file mode 100644 index 0000000000000..b354914b6ffd0 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtilsSuite.scala @@ -0,0 +1,140 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.scheduler.cluster.mesos + +import org.apache.mesos.Protos.Value +import org.mockito.Mockito._ +import org.scalatest._ +import org.scalatest.mock.MockitoSugar +import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} + +class MesosSchedulerUtilsSuite extends SparkFunSuite with Matchers with MockitoSugar { + + // scalastyle:off structural.type + // this is the documented way of generating fixtures in scalatest + def fixture: Object {val sc: SparkContext; val sparkConf: SparkConf} = new { + val sparkConf = new SparkConf + val sc = mock[SparkContext] + when(sc.conf).thenReturn(sparkConf) + } + val utils = new MesosSchedulerUtils { } + // scalastyle:on structural.type + + test("use at-least minimum overhead") { + val f = fixture + when(f.sc.executorMemory).thenReturn(512) + utils.calculateTotalMemory(f.sc) shouldBe 896 + } + + test("use overhead if it is greater than minimum value") { + val f = fixture + when(f.sc.executorMemory).thenReturn(4096) + utils.calculateTotalMemory(f.sc) shouldBe 4505 + } + + test("use spark.mesos.executor.memoryOverhead (if set)") { + val f = fixture + when(f.sc.executorMemory).thenReturn(1024) + f.sparkConf.set("spark.mesos.executor.memoryOverhead", "512") + utils.calculateTotalMemory(f.sc) shouldBe 1536 + } + + test("parse a non-empty constraint string correctly") { + val expectedMap = Map( + "tachyon" -> Set("true"), + "zone" -> Set("us-east-1a", "us-east-1b") + ) + utils.parseConstraintString("tachyon:true;zone:us-east-1a,us-east-1b") should be (expectedMap) + } + + test("parse an empty constraint string correctly") { + utils.parseConstraintString("") shouldBe Map() + } + + test("throw an exception when the input is malformed") { + an[IllegalArgumentException] should be thrownBy + utils.parseConstraintString("tachyon;zone:us-east") + } + + test("empty values for attributes' constraints matches all values") { + val constraintsStr = "tachyon:" + val parsedConstraints = utils.parseConstraintString(constraintsStr) + + parsedConstraints shouldBe Map("tachyon" -> Set()) + + val zoneSet = Value.Set.newBuilder().addItem("us-east-1a").addItem("us-east-1b").build() + val noTachyonOffer = Map("zone" -> zoneSet) + val tachyonTrueOffer = Map("tachyon" -> Value.Text.newBuilder().setValue("true").build()) + val tachyonFalseOffer = Map("tachyon" -> Value.Text.newBuilder().setValue("false").build()) + + utils.matchesAttributeRequirements(parsedConstraints, noTachyonOffer) shouldBe false + utils.matchesAttributeRequirements(parsedConstraints, tachyonTrueOffer) shouldBe true + utils.matchesAttributeRequirements(parsedConstraints, tachyonFalseOffer) shouldBe true + } + + test("subset match is performed for set attributes") { + val supersetConstraint = Map( + "tachyon" -> Value.Text.newBuilder().setValue("true").build(), + "zone" -> Value.Set.newBuilder() + .addItem("us-east-1a") + .addItem("us-east-1b") + .addItem("us-east-1c") + .build()) + + val zoneConstraintStr = "tachyon:;zone:us-east-1a,us-east-1c" + val parsedConstraints = utils.parseConstraintString(zoneConstraintStr) + + utils.matchesAttributeRequirements(parsedConstraints, supersetConstraint) shouldBe true + } + + test("less than equal match is performed on scalar attributes") { + val offerAttribs = Map("gpus" -> Value.Scalar.newBuilder().setValue(3).build()) + + val ltConstraint = utils.parseConstraintString("gpus:2") + val eqConstraint = utils.parseConstraintString("gpus:3") + val gtConstraint = utils.parseConstraintString("gpus:4") + + utils.matchesAttributeRequirements(ltConstraint, offerAttribs) shouldBe true + utils.matchesAttributeRequirements(eqConstraint, offerAttribs) shouldBe true + utils.matchesAttributeRequirements(gtConstraint, offerAttribs) shouldBe false + } + + test("contains match is performed for range attributes") { + val offerAttribs = Map("ports" -> Value.Range.newBuilder().setBegin(7000).setEnd(8000).build()) + val ltConstraint = utils.parseConstraintString("ports:6000") + val eqConstraint = utils.parseConstraintString("ports:7500") + val gtConstraint = utils.parseConstraintString("ports:8002") + val multiConstraint = utils.parseConstraintString("ports:5000,7500,8300") + + utils.matchesAttributeRequirements(ltConstraint, offerAttribs) shouldBe false + utils.matchesAttributeRequirements(eqConstraint, offerAttribs) shouldBe true + utils.matchesAttributeRequirements(gtConstraint, offerAttribs) shouldBe false + utils.matchesAttributeRequirements(multiConstraint, offerAttribs) shouldBe true + } + + test("equality match is performed for text attributes") { + val offerAttribs = Map("tachyon" -> Value.Text.newBuilder().setValue("true").build()) + + val trueConstraint = utils.parseConstraintString("tachyon:true") + val falseConstraint = utils.parseConstraintString("tachyon:false") + + utils.matchesAttributeRequirements(trueConstraint, offerAttribs) shouldBe true + utils.matchesAttributeRequirements(falseConstraint, offerAttribs) shouldBe false + } + +} diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala index 63a8480c9b57b..353b97469cd11 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala @@ -59,7 +59,9 @@ object KryoDistributedTest { class AppJarRegistrator extends KryoRegistrator { override def registerClasses(k: Kryo) { val classLoader = Thread.currentThread.getContextClassLoader + // scalastyle:off classforname k.register(Class.forName(AppJarRegistrator.customClassName, true, classLoader)) + // scalastyle:on classforname } } diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala index 542f8f45125a4..cc7342f1ecd78 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala @@ -68,8 +68,8 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte any[SerializerInstance], anyInt(), any[ShuffleWriteMetrics] - )).thenAnswer(new Answer[BlockObjectWriter] { - override def answer(invocation: InvocationOnMock): BlockObjectWriter = { + )).thenAnswer(new Answer[DiskBlockObjectWriter] { + override def answer(invocation: InvocationOnMock): DiskBlockObjectWriter = { val args = invocation.getArguments new DiskBlockObjectWriter( args(0).asInstanceOf[BlockId], diff --git a/core/src/test/scala/org/apache/spark/storage/BlockStatusListenerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockStatusListenerSuite.scala new file mode 100644 index 0000000000000..d7ffde1e7864e --- /dev/null +++ b/core/src/test/scala/org/apache/spark/storage/BlockStatusListenerSuite.scala @@ -0,0 +1,119 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.storage + +import org.apache.spark.SparkFunSuite +import org.apache.spark.scheduler._ + +class BlockStatusListenerSuite extends SparkFunSuite { + + test("basic functions") { + val blockManagerId = BlockManagerId("0", "localhost", 10000) + val listener = new BlockStatusListener() + + // Add a block manager and a new block status + listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(0, blockManagerId, 0)) + listener.onBlockUpdated(SparkListenerBlockUpdated( + BlockUpdatedInfo( + blockManagerId, + StreamBlockId(0, 100), + StorageLevel.MEMORY_AND_DISK, + memSize = 100, + diskSize = 100, + externalBlockStoreSize = 0))) + // The new block status should be added to the listener + val expectedBlock = BlockUIData( + StreamBlockId(0, 100), + "localhost:10000", + StorageLevel.MEMORY_AND_DISK, + memSize = 100, + diskSize = 100, + externalBlockStoreSize = 0 + ) + val expectedExecutorStreamBlockStatus = Seq( + ExecutorStreamBlockStatus("0", "localhost:10000", Seq(expectedBlock)) + ) + assert(listener.allExecutorStreamBlockStatus === expectedExecutorStreamBlockStatus) + + // Add the second block manager + val blockManagerId2 = BlockManagerId("1", "localhost", 10001) + listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(0, blockManagerId2, 0)) + // Add a new replication of the same block id from the second manager + listener.onBlockUpdated(SparkListenerBlockUpdated( + BlockUpdatedInfo( + blockManagerId2, + StreamBlockId(0, 100), + StorageLevel.MEMORY_AND_DISK, + memSize = 100, + diskSize = 100, + externalBlockStoreSize = 0))) + val expectedBlock2 = BlockUIData( + StreamBlockId(0, 100), + "localhost:10001", + StorageLevel.MEMORY_AND_DISK, + memSize = 100, + diskSize = 100, + externalBlockStoreSize = 0 + ) + // Each block manager should contain one block + val expectedExecutorStreamBlockStatus2 = Set( + ExecutorStreamBlockStatus("0", "localhost:10000", Seq(expectedBlock)), + ExecutorStreamBlockStatus("1", "localhost:10001", Seq(expectedBlock2)) + ) + assert(listener.allExecutorStreamBlockStatus.toSet === expectedExecutorStreamBlockStatus2) + + // Remove a replication of the same block + listener.onBlockUpdated(SparkListenerBlockUpdated( + BlockUpdatedInfo( + blockManagerId2, + StreamBlockId(0, 100), + StorageLevel.NONE, // StorageLevel.NONE means removing it + memSize = 0, + diskSize = 0, + externalBlockStoreSize = 0))) + // Only the first block manager contains a block + val expectedExecutorStreamBlockStatus3 = Set( + ExecutorStreamBlockStatus("0", "localhost:10000", Seq(expectedBlock)), + ExecutorStreamBlockStatus("1", "localhost:10001", Seq.empty) + ) + assert(listener.allExecutorStreamBlockStatus.toSet === expectedExecutorStreamBlockStatus3) + + // Remove the second block manager at first but add a new block status + // from this removed block manager + listener.onBlockManagerRemoved(SparkListenerBlockManagerRemoved(0, blockManagerId2)) + listener.onBlockUpdated(SparkListenerBlockUpdated( + BlockUpdatedInfo( + blockManagerId2, + StreamBlockId(0, 100), + StorageLevel.MEMORY_AND_DISK, + memSize = 100, + diskSize = 100, + externalBlockStoreSize = 0))) + // The second block manager is removed so we should not see the new block + val expectedExecutorStreamBlockStatus4 = Seq( + ExecutorStreamBlockStatus("0", "localhost:10000", Seq(expectedBlock)) + ) + assert(listener.allExecutorStreamBlockStatus === expectedExecutorStreamBlockStatus4) + + // Remove the last block manager + listener.onBlockManagerRemoved(SparkListenerBlockManagerRemoved(0, blockManagerId)) + // No block manager now so we should dop all block managers + assert(listener.allExecutorStreamBlockStatus.isEmpty) + } + +} diff --git a/core/src/test/scala/org/apache/spark/storage/BlockObjectWriterSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskBlockObjectWriterSuite.scala similarity index 98% rename from core/src/test/scala/org/apache/spark/storage/BlockObjectWriterSuite.scala rename to core/src/test/scala/org/apache/spark/storage/DiskBlockObjectWriterSuite.scala index 7bdea724fea58..66af6e1a79740 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockObjectWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/DiskBlockObjectWriterSuite.scala @@ -26,7 +26,7 @@ import org.apache.spark.executor.ShuffleWriteMetrics import org.apache.spark.serializer.JavaSerializer import org.apache.spark.util.Utils -class BlockObjectWriterSuite extends SparkFunSuite with BeforeAndAfterEach { +class DiskBlockObjectWriterSuite extends SparkFunSuite with BeforeAndAfterEach { var tempDir: File = _ diff --git a/core/src/test/scala/org/apache/spark/ui/storage/StoragePageSuite.scala b/core/src/test/scala/org/apache/spark/ui/storage/StoragePageSuite.scala new file mode 100644 index 0000000000000..3dab15a9d4691 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/ui/storage/StoragePageSuite.scala @@ -0,0 +1,230 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ui.storage + +import scala.xml.Utility + +import org.mockito.Mockito._ + +import org.apache.spark.SparkFunSuite +import org.apache.spark.storage._ + +class StoragePageSuite extends SparkFunSuite { + + val storageTab = mock(classOf[StorageTab]) + when(storageTab.basePath).thenReturn("http://localhost:4040") + val storagePage = new StoragePage(storageTab) + + test("rddTable") { + val rdd1 = new RDDInfo(1, + "rdd1", + 10, + StorageLevel.MEMORY_ONLY, + Seq.empty) + rdd1.memSize = 100 + rdd1.numCachedPartitions = 10 + + val rdd2 = new RDDInfo(2, + "rdd2", + 10, + StorageLevel.DISK_ONLY, + Seq.empty) + rdd2.diskSize = 200 + rdd2.numCachedPartitions = 5 + + val rdd3 = new RDDInfo(3, + "rdd3", + 10, + StorageLevel.MEMORY_AND_DISK_SER, + Seq.empty) + rdd3.memSize = 400 + rdd3.diskSize = 500 + rdd3.numCachedPartitions = 10 + + val xmlNodes = storagePage.rddTable(Seq(rdd1, rdd2, rdd3)) + + val headers = Seq( + "RDD Name", + "Storage Level", + "Cached Partitions", + "Fraction Cached", + "Size in Memory", + "Size in ExternalBlockStore", + "Size on Disk") + assert((xmlNodes \\ "th").map(_.text) === headers) + + assert((xmlNodes \\ "tr").size === 3) + assert(((xmlNodes \\ "tr")(0) \\ "td").map(_.text.trim) === + Seq("rdd1", "Memory Deserialized 1x Replicated", "10", "100%", "100.0 B", "0.0 B", "0.0 B")) + // Check the url + assert(((xmlNodes \\ "tr")(0) \\ "td" \ "a")(0).attribute("href").map(_.text) === + Some("http://localhost:4040/storage/rdd?id=1")) + + assert(((xmlNodes \\ "tr")(1) \\ "td").map(_.text.trim) === + Seq("rdd2", "Disk Serialized 1x Replicated", "5", "50%", "0.0 B", "0.0 B", "200.0 B")) + // Check the url + assert(((xmlNodes \\ "tr")(1) \\ "td" \ "a")(0).attribute("href").map(_.text) === + Some("http://localhost:4040/storage/rdd?id=2")) + + assert(((xmlNodes \\ "tr")(2) \\ "td").map(_.text.trim) === + Seq("rdd3", "Disk Memory Serialized 1x Replicated", "10", "100%", "400.0 B", "0.0 B", + "500.0 B")) + // Check the url + assert(((xmlNodes \\ "tr")(2) \\ "td" \ "a")(0).attribute("href").map(_.text) === + Some("http://localhost:4040/storage/rdd?id=3")) + } + + test("empty rddTable") { + assert(storagePage.rddTable(Seq.empty).isEmpty) + } + + test("streamBlockStorageLevelDescriptionAndSize") { + val memoryBlock = BlockUIData(StreamBlockId(0, 0), + "localhost:1111", + StorageLevel.MEMORY_ONLY, + memSize = 100, + diskSize = 0, + externalBlockStoreSize = 0) + assert(("Memory", 100) === storagePage.streamBlockStorageLevelDescriptionAndSize(memoryBlock)) + + val memorySerializedBlock = BlockUIData(StreamBlockId(0, 0), + "localhost:1111", + StorageLevel.MEMORY_ONLY_SER, + memSize = 100, + diskSize = 0, + externalBlockStoreSize = 0) + assert(("Memory Serialized", 100) === + storagePage.streamBlockStorageLevelDescriptionAndSize(memorySerializedBlock)) + + val diskBlock = BlockUIData(StreamBlockId(0, 0), + "localhost:1111", + StorageLevel.DISK_ONLY, + memSize = 0, + diskSize = 100, + externalBlockStoreSize = 0) + assert(("Disk", 100) === storagePage.streamBlockStorageLevelDescriptionAndSize(diskBlock)) + + val externalBlock = BlockUIData(StreamBlockId(0, 0), + "localhost:1111", + StorageLevel.OFF_HEAP, + memSize = 0, + diskSize = 0, + externalBlockStoreSize = 100) + assert(("External", 100) === + storagePage.streamBlockStorageLevelDescriptionAndSize(externalBlock)) + } + + test("receiverBlockTables") { + val blocksForExecutor0 = Seq( + BlockUIData(StreamBlockId(0, 0), + "localhost:10000", + StorageLevel.MEMORY_ONLY, + memSize = 100, + diskSize = 0, + externalBlockStoreSize = 0), + BlockUIData(StreamBlockId(1, 1), + "localhost:10000", + StorageLevel.DISK_ONLY, + memSize = 0, + diskSize = 100, + externalBlockStoreSize = 0) + ) + val executor0 = ExecutorStreamBlockStatus("0", "localhost:10000", blocksForExecutor0) + + val blocksForExecutor1 = Seq( + BlockUIData(StreamBlockId(0, 0), + "localhost:10001", + StorageLevel.MEMORY_ONLY, + memSize = 100, + diskSize = 0, + externalBlockStoreSize = 0), + BlockUIData(StreamBlockId(2, 2), + "localhost:10001", + StorageLevel.OFF_HEAP, + memSize = 0, + diskSize = 0, + externalBlockStoreSize = 200), + BlockUIData(StreamBlockId(1, 1), + "localhost:10001", + StorageLevel.MEMORY_ONLY_SER, + memSize = 100, + diskSize = 0, + externalBlockStoreSize = 0) + ) + val executor1 = ExecutorStreamBlockStatus("1", "localhost:10001", blocksForExecutor1) + val xmlNodes = storagePage.receiverBlockTables(Seq(executor0, executor1)) + + val executorTable = (xmlNodes \\ "table")(0) + val executorHeaders = Seq( + "Executor ID", + "Address", + "Total Size in Memory", + "Total Size in ExternalBlockStore", + "Total Size on Disk", + "Stream Blocks") + assert((executorTable \\ "th").map(_.text) === executorHeaders) + + assert((executorTable \\ "tr").size === 2) + assert(((executorTable \\ "tr")(0) \\ "td").map(_.text.trim) === + Seq("0", "localhost:10000", "100.0 B", "0.0 B", "100.0 B", "2")) + assert(((executorTable \\ "tr")(1) \\ "td").map(_.text.trim) === + Seq("1", "localhost:10001", "200.0 B", "200.0 B", "0.0 B", "3")) + + val blockTable = (xmlNodes \\ "table")(1) + val blockHeaders = Seq( + "Block ID", + "Replication Level", + "Location", + "Storage Level", + "Size") + assert((blockTable \\ "th").map(_.text) === blockHeaders) + + assert((blockTable \\ "tr").size === 5) + assert(((blockTable \\ "tr")(0) \\ "td").map(_.text.trim) === + Seq("input-0-0", "2", "localhost:10000", "Memory", "100.0 B")) + // Check "rowspan=2" for the first 2 columns + assert(((blockTable \\ "tr")(0) \\ "td")(0).attribute("rowspan").map(_.text) === Some("2")) + assert(((blockTable \\ "tr")(0) \\ "td")(1).attribute("rowspan").map(_.text) === Some("2")) + + assert(((blockTable \\ "tr")(1) \\ "td").map(_.text.trim) === + Seq("localhost:10001", "Memory", "100.0 B")) + + assert(((blockTable \\ "tr")(2) \\ "td").map(_.text.trim) === + Seq("input-1-1", "2", "localhost:10000", "Disk", "100.0 B")) + // Check "rowspan=2" for the first 2 columns + assert(((blockTable \\ "tr")(2) \\ "td")(0).attribute("rowspan").map(_.text) === Some("2")) + assert(((blockTable \\ "tr")(2) \\ "td")(1).attribute("rowspan").map(_.text) === Some("2")) + + assert(((blockTable \\ "tr")(3) \\ "td").map(_.text.trim) === + Seq("localhost:10001", "Memory Serialized", "100.0 B")) + + assert(((blockTable \\ "tr")(4) \\ "td").map(_.text.trim) === + Seq("input-2-2", "1", "localhost:10001", "External", "200.0 B")) + // Check "rowspan=1" for the first 2 columns + assert(((blockTable \\ "tr")(4) \\ "td")(0).attribute("rowspan").map(_.text) === Some("1")) + assert(((blockTable \\ "tr")(4) \\ "td")(1).attribute("rowspan").map(_.text) === Some("1")) + } + + test("empty receiverBlockTables") { + assert(storagePage.receiverBlockTables(Seq.empty).isEmpty) + + val executor0 = ExecutorStreamBlockStatus("0", "localhost:10000", Seq.empty) + val executor1 = ExecutorStreamBlockStatus("1", "localhost:10001", Seq.empty) + assert(storagePage.receiverBlockTables(Seq(executor0, executor1)).isEmpty) + } +} diff --git a/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala b/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala index 1053c6caf7718..480722a5ac182 100644 --- a/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala @@ -375,6 +375,7 @@ class TestCreateNullValue { // parameters of the closure constructor. This allows us to test whether // null values are created correctly for each type. val nestedClosure = () => { + // scalastyle:off println if (s.toString == "123") { // Don't really output them to avoid noisy println(bo) println(c) @@ -389,6 +390,7 @@ class TestCreateNullValue { val closure = () => { println(getX) } + // scalastyle:on println ClosureCleaner.clean(closure) } nestedClosure() diff --git a/core/src/test/scala/org/apache/spark/util/MutableURLClassLoaderSuite.scala b/core/src/test/scala/org/apache/spark/util/MutableURLClassLoaderSuite.scala index 42125547436cb..d3d464e84ffd7 100644 --- a/core/src/test/scala/org/apache/spark/util/MutableURLClassLoaderSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/MutableURLClassLoaderSuite.scala @@ -84,7 +84,9 @@ class MutableURLClassLoaderSuite extends SparkFunSuite { try { sc.makeRDD(1 to 5, 2).mapPartitions { x => val loader = Thread.currentThread().getContextClassLoader + // scalastyle:off classforname Class.forName(className, true, loader).newInstance() + // scalastyle:on classforname Seq().iterator }.count() } 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 251a797dc28a2..c7638507c88c6 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -684,7 +684,9 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging { val buffer = new CircularBuffer(25) val stream = new java.io.PrintStream(buffer, true, "UTF-8") + // scalastyle:off println stream.println("test circular test circular test circular test circular test circular") + // scalastyle:on println assert(buffer.toString === "t circular test circular\n") } } diff --git a/core/src/test/scala/org/apache/spark/util/collection/PartitionedSerializedPairBufferSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/PartitionedSerializedPairBufferSuite.scala index 6d2459d48d326..3b67f6206495a 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/PartitionedSerializedPairBufferSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/PartitionedSerializedPairBufferSuite.scala @@ -17,15 +17,20 @@ package org.apache.spark.util.collection -import java.io.{ByteArrayInputStream, ByteArrayOutputStream, InputStream} +import java.io.{ByteArrayInputStream, ByteArrayOutputStream} import com.google.common.io.ByteStreams +import org.mockito.Matchers.any +import org.mockito.Mockito._ +import org.mockito.Mockito.RETURNS_SMART_NULLS +import org.mockito.invocation.InvocationOnMock +import org.mockito.stubbing.Answer import org.scalatest.Matchers._ import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.serializer.KryoSerializer -import org.apache.spark.storage.{FileSegment, BlockObjectWriter} +import org.apache.spark.storage.DiskBlockObjectWriter class PartitionedSerializedPairBufferSuite extends SparkFunSuite { test("OrderedInputStream single record") { @@ -79,13 +84,13 @@ class PartitionedSerializedPairBufferSuite extends SparkFunSuite { val struct = SomeStruct("something", 5) buffer.insert(4, 10, struct) val it = buffer.destructiveSortedWritablePartitionedIterator(None) - val writer = new SimpleBlockObjectWriter + val (writer, baos) = createMockWriter() assert(it.hasNext) it.nextPartition should be (4) it.writeNext(writer) assert(!it.hasNext) - val stream = serializerInstance.deserializeStream(writer.getInputStream) + val stream = serializerInstance.deserializeStream(new ByteArrayInputStream(baos.toByteArray)) stream.readObject[AnyRef]() should be (10) stream.readObject[AnyRef]() should be (struct) } @@ -101,7 +106,7 @@ class PartitionedSerializedPairBufferSuite extends SparkFunSuite { buffer.insert(5, 3, struct3) val it = buffer.destructiveSortedWritablePartitionedIterator(None) - val writer = new SimpleBlockObjectWriter + val (writer, baos) = createMockWriter() assert(it.hasNext) it.nextPartition should be (4) it.writeNext(writer) @@ -113,7 +118,7 @@ class PartitionedSerializedPairBufferSuite extends SparkFunSuite { it.writeNext(writer) assert(!it.hasNext) - val stream = serializerInstance.deserializeStream(writer.getInputStream) + val stream = serializerInstance.deserializeStream(new ByteArrayInputStream(baos.toByteArray)) val iter = stream.asIterator iter.next() should be (2) iter.next() should be (struct2) @@ -123,26 +128,21 @@ class PartitionedSerializedPairBufferSuite extends SparkFunSuite { iter.next() should be (struct1) assert(!iter.hasNext) } -} - -case class SomeStruct(val str: String, val num: Int) - -class SimpleBlockObjectWriter extends BlockObjectWriter(null) { - val baos = new ByteArrayOutputStream() - override def write(bytes: Array[Byte], offs: Int, len: Int): Unit = { - baos.write(bytes, offs, len) + def createMockWriter(): (DiskBlockObjectWriter, ByteArrayOutputStream) = { + val writer = mock(classOf[DiskBlockObjectWriter], RETURNS_SMART_NULLS) + val baos = new ByteArrayOutputStream() + when(writer.write(any(), any(), any())).thenAnswer(new Answer[Unit] { + override def answer(invocationOnMock: InvocationOnMock): Unit = { + val args = invocationOnMock.getArguments + val bytes = args(0).asInstanceOf[Array[Byte]] + val offset = args(1).asInstanceOf[Int] + val length = args(2).asInstanceOf[Int] + baos.write(bytes, offset, length) + } + }) + (writer, baos) } - - def getInputStream(): InputStream = new ByteArrayInputStream(baos.toByteArray) - - override def open(): BlockObjectWriter = this - override def close(): Unit = { } - override def isOpen: Boolean = true - override def commitAndClose(): Unit = { } - override def revertPartialWritesAndClose(): Unit = { } - override def fileSegment(): FileSegment = null - override def write(key: Any, value: Any): Unit = { } - override def recordWritten(): Unit = { } - override def write(b: Int): Unit = { } } + +case class SomeStruct(str: String, num: Int) diff --git a/core/src/test/scala/org/apache/spark/util/collection/SizeTrackerSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/SizeTrackerSuite.scala index 5a5919fca2469..4f382414a8dd7 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/SizeTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/SizeTrackerSuite.scala @@ -103,7 +103,9 @@ private object SizeTrackerSuite { */ def main(args: Array[String]): Unit = { if (args.size < 1) { + // scalastyle:off println println("Usage: SizeTrackerSuite [num elements]") + // scalastyle:on println System.exit(1) } val numElements = args(0).toInt @@ -180,11 +182,13 @@ private object SizeTrackerSuite { baseTimes: Seq[Long], sampledTimes: Seq[Long], unsampledTimes: Seq[Long]): Unit = { + // scalastyle:off println println(s"Average times for $testName (ms):") println(" Base - " + averageTime(baseTimes)) println(" SizeTracker (sampled) - " + averageTime(sampledTimes)) println(" SizeEstimator (unsampled) - " + averageTime(unsampledTimes)) println() + // scalastyle:on println } def time(f: => Unit): Long = { diff --git a/core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala index b2f5d9009ee5d..fefa5165db197 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala @@ -20,10 +20,10 @@ package org.apache.spark.util.collection import java.lang.{Float => JFloat, Integer => JInteger} import java.util.{Arrays, Comparator} -import org.apache.spark.SparkFunSuite +import org.apache.spark.{Logging, SparkFunSuite} import org.apache.spark.util.random.XORShiftRandom -class SorterSuite extends SparkFunSuite { +class SorterSuite extends SparkFunSuite with Logging { test("equivalent to Arrays.sort") { val rand = new XORShiftRandom(123) @@ -74,7 +74,7 @@ class SorterSuite extends SparkFunSuite { /** Runs an experiment several times. */ def runExperiment(name: String, skip: Boolean = false)(f: => Unit, prepare: () => Unit): Unit = { if (skip) { - println(s"Skipped experiment $name.") + logInfo(s"Skipped experiment $name.") return } @@ -86,11 +86,11 @@ class SorterSuite extends SparkFunSuite { while (i < 10) { val time = org.apache.spark.util.Utils.timeIt(1)(f, Some(prepare)) next10 += time - println(s"$name: Took $time ms") + logInfo(s"$name: Took $time ms") i += 1 } - println(s"$name: ($firstTry ms first try, ${next10 / 10} ms average)") + logInfo(s"$name: ($firstTry ms first try, ${next10 / 10} ms average)") } /** diff --git a/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/PrefixComparatorsSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/PrefixComparatorsSuite.scala new file mode 100644 index 0000000000000..dd505dfa7d758 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/PrefixComparatorsSuite.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.util.collection.unsafe.sort + +import org.scalatest.prop.PropertyChecks + +import org.apache.spark.SparkFunSuite + +class PrefixComparatorsSuite extends SparkFunSuite with PropertyChecks { + + test("String prefix comparator") { + + def testPrefixComparison(s1: String, s2: String): Unit = { + val s1Prefix = PrefixComparators.STRING.computePrefix(s1) + val s2Prefix = PrefixComparators.STRING.computePrefix(s2) + val prefixComparisonResult = PrefixComparators.STRING.compare(s1Prefix, s2Prefix) + assert( + (prefixComparisonResult == 0) || + (prefixComparisonResult < 0 && s1 < s2) || + (prefixComparisonResult > 0 && s1 > s2)) + } + + // scalastyle:off + val regressionTests = Table( + ("s1", "s2"), + ("abc", "世界"), + ("你好", "世界"), + ("你好123", "你好122") + ) + // scalastyle:on + + forAll (regressionTests) { (s1: String, s2: String) => testPrefixComparison(s1, s2) } + forAll { (s1: String, s2: String) => testPrefixComparison(s1, s2) } + } +} diff --git a/dev/audit-release/sbt_app_core/src/main/scala/SparkApp.scala b/dev/audit-release/sbt_app_core/src/main/scala/SparkApp.scala index fc03fec9866a6..61d91c70e9709 100644 --- a/dev/audit-release/sbt_app_core/src/main/scala/SparkApp.scala +++ b/dev/audit-release/sbt_app_core/src/main/scala/SparkApp.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package main.scala import scala.util.Try @@ -59,3 +60,4 @@ object SimpleApp { } } } +// scalastyle:on println diff --git a/dev/audit-release/sbt_app_ganglia/src/main/scala/SparkApp.scala b/dev/audit-release/sbt_app_ganglia/src/main/scala/SparkApp.scala index 0be8e64fbfabd..9f7ae75d0b477 100644 --- a/dev/audit-release/sbt_app_ganglia/src/main/scala/SparkApp.scala +++ b/dev/audit-release/sbt_app_ganglia/src/main/scala/SparkApp.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package main.scala import scala.util.Try @@ -37,3 +38,4 @@ object SimpleApp { } } } +// scalastyle:on println diff --git a/dev/audit-release/sbt_app_graphx/src/main/scala/GraphxApp.scala b/dev/audit-release/sbt_app_graphx/src/main/scala/GraphxApp.scala index 24c7f8d667296..2f0b6ef9a5672 100644 --- a/dev/audit-release/sbt_app_graphx/src/main/scala/GraphxApp.scala +++ b/dev/audit-release/sbt_app_graphx/src/main/scala/GraphxApp.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package main.scala import org.apache.spark.{SparkContext, SparkConf} @@ -51,3 +52,4 @@ object GraphXApp { println("Test succeeded") } } +// scalastyle:on println diff --git a/dev/audit-release/sbt_app_hive/src/main/scala/HiveApp.scala b/dev/audit-release/sbt_app_hive/src/main/scala/HiveApp.scala index 5111bc0adb772..4a980ec071ae4 100644 --- a/dev/audit-release/sbt_app_hive/src/main/scala/HiveApp.scala +++ b/dev/audit-release/sbt_app_hive/src/main/scala/HiveApp.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package main.scala import scala.collection.mutable.{ListBuffer, Queue} @@ -55,3 +56,4 @@ object SparkSqlExample { sc.stop() } } +// scalastyle:on println diff --git a/dev/audit-release/sbt_app_kinesis/src/main/scala/SparkApp.scala b/dev/audit-release/sbt_app_kinesis/src/main/scala/SparkApp.scala index 9f85066501472..adc25b57d6aa5 100644 --- a/dev/audit-release/sbt_app_kinesis/src/main/scala/SparkApp.scala +++ b/dev/audit-release/sbt_app_kinesis/src/main/scala/SparkApp.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package main.scala import scala.util.Try @@ -31,3 +32,4 @@ object SimpleApp { } } } +// scalastyle:on println diff --git a/dev/audit-release/sbt_app_sql/src/main/scala/SqlApp.scala b/dev/audit-release/sbt_app_sql/src/main/scala/SqlApp.scala index cc86ef45858c9..69c1154dc0955 100644 --- a/dev/audit-release/sbt_app_sql/src/main/scala/SqlApp.scala +++ b/dev/audit-release/sbt_app_sql/src/main/scala/SqlApp.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package main.scala import scala.collection.mutable.{ListBuffer, Queue} @@ -57,3 +58,4 @@ object SparkSqlExample { sc.stop() } } +// scalastyle:on println diff --git a/dev/audit-release/sbt_app_streaming/src/main/scala/StreamingApp.scala b/dev/audit-release/sbt_app_streaming/src/main/scala/StreamingApp.scala index 58a662bd9b2e8..d6a074687f4a1 100644 --- a/dev/audit-release/sbt_app_streaming/src/main/scala/StreamingApp.scala +++ b/dev/audit-release/sbt_app_streaming/src/main/scala/StreamingApp.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package main.scala import scala.collection.mutable.{ListBuffer, Queue} @@ -61,3 +62,4 @@ object SparkStreamingExample { ssc.stop() } } +// scalastyle:on println diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index 54274a83f6d66..30190dcd41ec5 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -118,13 +118,13 @@ if [[ ! "$@" =~ --skip-publish ]]; then rm -rf $SPARK_REPO - build/mvn -DskipTests -Pyarn -Phive \ + build/mvn -DskipTests -Pyarn -Phive -Prelease\ -Phive-thriftserver -Phadoop-2.2 -Pspark-ganglia-lgpl -Pkinesis-asl \ clean install ./dev/change-version-to-2.11.sh - build/mvn -DskipTests -Pyarn -Phive \ + build/mvn -DskipTests -Pyarn -Phive -Prelease\ -Dscala-2.11 -Phadoop-2.2 -Pspark-ganglia-lgpl -Pkinesis-asl \ clean install diff --git a/docker/spark-test/base/Dockerfile b/docker/spark-test/base/Dockerfile index 5956d59130fbf..5dbdb8b22a44f 100644 --- a/docker/spark-test/base/Dockerfile +++ b/docker/spark-test/base/Dockerfile @@ -17,13 +17,13 @@ FROM ubuntu:precise -RUN echo "deb http://archive.ubuntu.com/ubuntu precise main universe" > /etc/apt/sources.list - # Upgrade package index -RUN apt-get update - # install a few other useful packages plus Open Jdk 7 -RUN apt-get install -y less openjdk-7-jre-headless net-tools vim-tiny sudo openssh-server +# Remove unneeded /var/lib/apt/lists/* after install to reduce the +# docker image size (by ~30MB) +RUN apt-get update && \ + apt-get install -y less openjdk-7-jre-headless net-tools vim-tiny sudo openssh-server && \ + rm -rf /var/lib/apt/lists/* ENV SCALA_VERSION 2.10.4 ENV CDH_VERSION cdh4 diff --git a/docs/configuration.md b/docs/configuration.md index bebaf6f62e90a..8a186ee51c1ca 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -665,7 +665,7 @@ Apart from these, the following properties are also available, and may be useful Initial size of Kryo's serialization buffer. Note that there will be one buffer per core on each worker. This buffer will grow up to - spark.kryoserializer.buffer.max.mb if needed. + spark.kryoserializer.buffer.max if needed. @@ -1007,9 +1007,9 @@ Apart from these, the following properties are also available, and may be useful spark.rpc.numRetries 3 + Number of times to retry before an RPC task gives up. An RPC task will run at most times of this number. - @@ -1029,8 +1029,8 @@ Apart from these, the following properties are also available, and may be useful spark.rpc.lookupTimeout 120s - Duration for an RPC remote endpoint lookup operation to wait before timing out. + Duration for an RPC remote endpoint lookup operation to wait before timing out. @@ -1206,7 +1206,7 @@ Apart from these, the following properties are also available, and may be useful spark.dynamicAllocation.cachedExecutorIdleTimeout - 2 * executorIdleTimeout + infinity If dynamic allocation is enabled and an executor which has cached data blocks has been idle for more than this duration, the executor will be removed. For more details, see this @@ -1222,7 +1222,7 @@ Apart from these, the following properties are also available, and may be useful spark.dynamicAllocation.maxExecutors - Integer.MAX_VALUE + infinity Upper bound for the number of executors if dynamic allocation is enabled. diff --git a/docs/ml-features.md b/docs/ml-features.md index f88c0248c1a8a..54068debe2159 100644 --- a/docs/ml-features.md +++ b/docs/ml-features.md @@ -288,6 +288,94 @@ for words_label in wordsDataFrame.select("words", "label").take(3): +## $n$-gram + +An [n-gram](https://en.wikipedia.org/wiki/N-gram) is a sequence of $n$ tokens (typically words) for some integer $n$. The `NGram` class can be used to transform input features into $n$-grams. + +`NGram` takes as input a sequence of strings (e.g. the output of a [Tokenizer](ml-features.html#tokenizer). The parameter `n` is used to determine the number of terms in each $n$-gram. The output will consist of a sequence of $n$-grams where each $n$-gram is represented by a space-delimited string of $n$ consecutive words. If the input sequence contains fewer than `n` strings, no output is produced. + +
+
+
+ +
+ +[`NGram`](api/scala/index.html#org.apache.spark.ml.feature.NGram) takes an input column name, an output column name, and an optional length parameter n (n=2 by default). + +{% highlight scala %} +import org.apache.spark.ml.feature.NGram + +val wordDataFrame = sqlContext.createDataFrame(Seq( + (0, Array("Hi", "I", "heard", "about", "Spark")), + (1, Array("I", "wish", "Java", "could", "use", "case", "classes")), + (2, Array("Logistic", "regression", "models", "are", "neat")) +)).toDF("label", "words") + +val ngram = new NGram().setInputCol("words").setOutputCol("ngrams") +val ngramDataFrame = ngram.transform(wordDataFrame) +ngramDataFrame.take(3).map(_.getAs[Stream[String]]("ngrams").toList).foreach(println) +{% endhighlight %} +
+ +
+ +[`NGram`](api/java/org/apache/spark/ml/feature/NGram.html) takes an input column name, an output column name, and an optional length parameter n (n=2 by default). + +{% highlight java %} +import com.google.common.collect.Lists; + +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.ml.feature.NGram; +import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.sql.DataFrame; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; + +JavaRDD jrdd = jsc.parallelize(Lists.newArrayList( + RowFactory.create(0D, Lists.newArrayList("Hi", "I", "heard", "about", "Spark")), + RowFactory.create(1D, Lists.newArrayList("I", "wish", "Java", "could", "use", "case", "classes")), + RowFactory.create(2D, Lists.newArrayList("Logistic", "regression", "models", "are", "neat")) +)); +StructType schema = new StructType(new StructField[]{ + new StructField("label", DataTypes.DoubleType, false, Metadata.empty()), + new StructField("words", DataTypes.createArrayType(DataTypes.StringType), false, Metadata.empty()) +}); +DataFrame wordDataFrame = sqlContext.createDataFrame(jrdd, schema); +NGram ngramTransformer = new NGram().setInputCol("words").setOutputCol("ngrams"); +DataFrame ngramDataFrame = ngramTransformer.transform(wordDataFrame); +for (Row r : ngramDataFrame.select("ngrams", "label").take(3)) { + java.util.List ngrams = r.getList(0); + for (String ngram : ngrams) System.out.print(ngram + " --- "); + System.out.println(); +} +{% endhighlight %} +
+ +
+ +[`NGram`](api/python/pyspark.ml.html#pyspark.ml.feature.NGram) takes an input column name, an output column name, and an optional length parameter n (n=2 by default). + +{% highlight python %} +from pyspark.ml.feature import NGram + +wordDataFrame = sqlContext.createDataFrame([ + (0, ["Hi", "I", "heard", "about", "Spark"]), + (1, ["I", "wish", "Java", "could", "use", "case", "classes"]), + (2, ["Logistic", "regression", "models", "are", "neat"]) +], ["label", "words"]) +ngram = NGram(inputCol="words", outputCol="ngrams") +ngramDataFrame = ngram.transform(wordDataFrame) +for ngrams_label in ngramDataFrame.select("ngrams", "label").take(3): + print(ngrams_label) +{% endhighlight %} +
+
+ + ## Binarizer Binarization is the process of thresholding numerical features to binary features. As some probabilistic estimators make assumption that the input data is distributed according to [Bernoulli distribution](http://en.wikipedia.org/wiki/Bernoulli_distribution), a binarizer is useful for pre-processing the input data with continuous numerical features. diff --git a/docs/mllib-clustering.md b/docs/mllib-clustering.md index 3aad4149f99db..d72dc20a5ad6e 100644 --- a/docs/mllib-clustering.md +++ b/docs/mllib-clustering.md @@ -447,7 +447,7 @@ It supports different inference algorithms via `setOptimizer` function. EMLDAOpt on the likelihood function and yields comprehensive results, while OnlineLDAOptimizer uses iterative mini-batch sampling for [online variational inference](https://www.cs.princeton.edu/~blei/papers/HoffmanBleiBach2010b.pdf) and is generally memory friendly. After fitting on the documents, LDA provides: * Topics: Inferred topics, each of which is a probability distribution over terms (words). -* Topic distributions for documents: For each document in the training set, LDA gives a probability distribution over topics. (EM only) +* Topic distributions for documents: For each non empty document in the training set, LDA gives a probability distribution over topics. (EM only). Note that for empty documents, we don't create the topic distributions. (EM only) LDA takes the following parameters: diff --git a/docs/mllib-data-types.md b/docs/mllib-data-types.md index d824dab1d7f7b..3aa040046fca5 100644 --- a/docs/mllib-data-types.md +++ b/docs/mllib-data-types.md @@ -226,7 +226,8 @@ examples = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") A local matrix has integer-typed row and column indices and double-typed values, stored on a single machine. MLlib supports dense matrices, whose entry values are stored in a single double array in -column major. For example, the following matrix `\[ \begin{pmatrix} +column-major order, and sparse matrices, whose non-zero entry values are stored in the Compressed Sparse +Column (CSC) format in column-major order. For example, the following dense matrix `\[ \begin{pmatrix} 1.0 & 2.0 \\ 3.0 & 4.0 \\ 5.0 & 6.0 @@ -238,28 +239,33 @@ is stored in a one-dimensional array `[1.0, 3.0, 5.0, 2.0, 4.0, 6.0]` with the m
The base class of local matrices is -[`Matrix`](api/scala/index.html#org.apache.spark.mllib.linalg.Matrix), and we provide one -implementation: [`DenseMatrix`](api/scala/index.html#org.apache.spark.mllib.linalg.DenseMatrix). +[`Matrix`](api/scala/index.html#org.apache.spark.mllib.linalg.Matrix), and we provide two +implementations: [`DenseMatrix`](api/scala/index.html#org.apache.spark.mllib.linalg.DenseMatrix), +and [`SparseMatrix`](api/scala/index.html#org.apache.spark.mllib.linalg.SparseMatrix). We recommend using the factory methods implemented in [`Matrices`](api/scala/index.html#org.apache.spark.mllib.linalg.Matrices$) to create local -matrices. +matrices. Remember, local matrices in MLlib are stored in column-major order. {% highlight scala %} import org.apache.spark.mllib.linalg.{Matrix, Matrices} // Create a dense matrix ((1.0, 2.0), (3.0, 4.0), (5.0, 6.0)) val dm: Matrix = Matrices.dense(3, 2, Array(1.0, 3.0, 5.0, 2.0, 4.0, 6.0)) + +// Create a sparse matrix ((9.0, 0.0), (0.0, 8.0), (0.0, 6.0)) +val sm: Matrix = Matrices.sparse(3, 2, Array(0, 1, 3), Array(0, 2, 1), Array(9, 6, 8)) {% endhighlight %}
The base class of local matrices is -[`Matrix`](api/java/org/apache/spark/mllib/linalg/Matrix.html), and we provide one -implementation: [`DenseMatrix`](api/java/org/apache/spark/mllib/linalg/DenseMatrix.html). +[`Matrix`](api/java/org/apache/spark/mllib/linalg/Matrix.html), and we provide two +implementations: [`DenseMatrix`](api/java/org/apache/spark/mllib/linalg/DenseMatrix.html), +and [`SparseMatrix`](api/java/org/apache/spark/mllib/linalg/SparseMatrix.html). We recommend using the factory methods implemented in [`Matrices`](api/java/org/apache/spark/mllib/linalg/Matrices.html) to create local -matrices. +matrices. Remember, local matrices in MLlib are stored in column-major order. {% highlight java %} import org.apache.spark.mllib.linalg.Matrix; @@ -267,6 +273,30 @@ import org.apache.spark.mllib.linalg.Matrices; // Create a dense matrix ((1.0, 2.0), (3.0, 4.0), (5.0, 6.0)) Matrix dm = Matrices.dense(3, 2, new double[] {1.0, 3.0, 5.0, 2.0, 4.0, 6.0}); + +// Create a sparse matrix ((9.0, 0.0), (0.0, 8.0), (0.0, 6.0)) +Matrix sm = Matrices.sparse(3, 2, new int[] {0, 1, 3}, new int[] {0, 2, 1}, new double[] {9, 6, 8}); +{% endhighlight %} +
+ +
+ +The base class of local matrices is +[`Matrix`](api/python/pyspark.mllib.html#pyspark.mllib.linalg.Matrix), and we provide two +implementations: [`DenseMatrix`](api/python/pyspark.mllib.html#pyspark.mllib.linalg.DenseMatrix), +and [`SparseMatrix`](api/python/pyspark.mllib.html#pyspark.mllib.linalg.SparseMatrix). +We recommend using the factory methods implemented +in [`Matrices`](api/python/pyspark.mllib.html#pyspark.mllib.linalg.Matrices) to create local +matrices. Remember, local matrices in MLlib are stored in column-major order. + +{% highlight python %} +import org.apache.spark.mllib.linalg.{Matrix, Matrices} + +// Create a dense matrix ((1.0, 2.0), (3.0, 4.0), (5.0, 6.0)) +dm2 = Matrices.dense(3, 2, [1, 2, 3, 4, 5, 6]) + +// Create a sparse matrix ((9.0, 0.0), (0.0, 8.0), (0.0, 6.0)) +sm = Matrices.sparse(3, 2, [0, 1, 3], [0, 2, 1], [9, 6, 8]) {% endhighlight %}
diff --git a/docs/mllib-statistics.md b/docs/mllib-statistics.md index 887eae7f4f07b..de5d6485f9b5f 100644 --- a/docs/mllib-statistics.md +++ b/docs/mllib-statistics.md @@ -283,7 +283,7 @@ approxSample = data.sampleByKey(False, fractions); Hypothesis testing is a powerful tool in statistics to determine whether a result is statistically significant, whether this result occurred by chance or not. MLlib currently supports Pearson's -chi-squared ( $\chi^2$) tests for goodness of fit and independence. The input data types determine +chi-squared ( $\chi^2$) tests for goodness of fit and independence. The input data types determine whether the goodness of fit or the independence test is conducted. The goodness of fit test requires an input type of `Vector`, whereas the independence test requires a `Matrix` as input. @@ -422,6 +422,41 @@ for i, result in enumerate(featureTestResults):
+Additionally, MLlib provides a 1-sample, 2-sided implementation of the Kolmogorov-Smirnov (KS) test +for equality of probability distributions. By providing the name of a theoretical distribution +(currently solely supported for the normal distribution) and its parameters, or a function to +calculate the cumulative distribution according to a given theoretical distribution, the user can +test the null hypothesis that their sample is drawn from that distribution. In the case that the +user tests against the normal distribution (`distName="norm"`), but does not provide distribution +parameters, the test initializes to the standard normal distribution and logs an appropriate +message. + +
+
+[`Statistics`](api/scala/index.html#org.apache.spark.mllib.stat.Statistics$) provides methods to +run a 1-sample, 2-sided Kolmogorov-Smirnov test. The following example demonstrates how to run +and interpret the hypothesis tests. + +{% highlight scala %} +import org.apache.spark.SparkContext +import org.apache.spark.mllib.stat.Statistics._ + +val data: RDD[Double] = ... // an RDD of sample data + +// run a KS test for the sample versus a standard normal distribution +val testResult = Statistics.kolmogorovSmirnovTest(data, "norm", 0, 1) +println(testResult) // summary of the test including the p-value, test statistic, + // and null hypothesis + // if our p-value indicates significance, we can reject the null hypothesis + +// perform a KS test using a cumulative distribution function of our making +val myCDF: Double => Double = ... +val testResult2 = Statistics.kolmogorovSmirnovTest(data, myCDF) +{% endhighlight %} +
+
+ + ## Random data generation Random data generation is useful for randomized algorithms, prototyping, and performance testing. diff --git a/docs/running-on-mesos.md b/docs/running-on-mesos.md index 5f1d6daeb27f0..1f915d8ea1d73 100644 --- a/docs/running-on-mesos.md +++ b/docs/running-on-mesos.md @@ -184,6 +184,14 @@ acquire. By default, it will acquire *all* cores in the cluster (that get offere only makes sense if you run just one application at a time. You can cap the maximum number of cores using `conf.set("spark.cores.max", "10")` (for example). +You may also make use of `spark.mesos.constraints` to set attribute based constraints on mesos resource offers. By default, all resource offers will be accepted. + +{% highlight scala %} +conf.set("spark.mesos.constraints", "tachyon=true;us-east-1=false") +{% endhighlight %} + +For example, Let's say `spark.mesos.constraints` is set to `tachyon=true;us-east-1=false`, then the resource offers will be checked to see if they meet both these constraints and only then will be accepted to start new executors. + # Mesos Docker Support Spark can make use of a Mesos Docker containerizer by setting the property `spark.mesos.executor.docker.image` @@ -298,6 +306,20 @@ See the [configuration page](configuration.html) for information on Spark config the final overhead will be this value. + + spark.mesos.constraints + Attribute based constraints to be matched against when accepting resource offers. + + Attribute based constraints on mesos resource offers. By default, all resource offers will be accepted. Refer to Mesos Attributes & Resources for more information on attributes. +
    +
  • Scalar constraints are matched with "less than equal" semantics i.e. value in the constraint must be less than or equal to the value in the resource offer.
  • +
  • Range constraints are matched with "contains" semantics i.e. value in the constraint must be within the resource offer's value.
  • +
  • Set constraints are matched with "subset of" semantics i.e. value in the constraint must be a subset of the resource offer's value.
  • +
  • Text constraints are metched with "equality" semantics i.e. value in the constraint must be exactly equal to the resource offer's value.
  • +
  • In case there is no value present as a part of the constraint any offer with the corresponding attribute will be accepted (without value check).
  • +
+ + # Troubleshooting and Debugging diff --git a/docs/sparkr.md b/docs/sparkr.md index 095ea4308cfeb..4385a4eeacd5c 100644 --- a/docs/sparkr.md +++ b/docs/sparkr.md @@ -68,7 +68,7 @@ you can specify the packages with the `packages` argument.
{% highlight r %} -sc <- sparkR.init(packages="com.databricks:spark-csv_2.11:1.0.3") +sc <- sparkR.init(sparkPackages="com.databricks:spark-csv_2.11:1.0.3") sqlContext <- sparkRSQL.init(sc) {% endhighlight %}
@@ -116,7 +116,7 @@ sql(hiveContext, "CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") sql(hiveContext, "LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src") # Queries can be expressed in HiveQL. -results <- hiveContext.sql("FROM src SELECT key, value") +results <- sql(hiveContext, "FROM src SELECT key, value") # results is now a DataFrame head(results) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 88c96a9a095b3..5838bc172fe86 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -828,7 +828,7 @@ using this syntax. {% highlight scala %} val df = sqlContext.read.format("json").load("examples/src/main/resources/people.json") -df.select("name", "age").write.format("json").save("namesAndAges.json") +df.select("name", "age").write.format("parquet").save("namesAndAges.parquet") {% endhighlight %}
@@ -1637,7 +1637,7 @@ sql(sqlContext, "CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") sql(sqlContext, "LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src") # Queries can be expressed in HiveQL. -results = sqlContext.sql("FROM src SELECT key, value").collect() +results <- collect(sql(sqlContext, "FROM src SELECT key, value")) {% endhighlight %} @@ -1798,7 +1798,7 @@ DataFrame jdbcDF = sqlContext.read().format("jdbc"). options(options).load(); {% highlight python %} -df = sqlContext.read.format('jdbc').options(url = 'jdbc:postgresql:dbserver', dbtable='schema.tablename').load() +df = sqlContext.read.format('jdbc').options(url='jdbc:postgresql:dbserver', dbtable='schema.tablename').load() {% endhighlight %} diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index e72d5580dae55..2f3013b533eb0 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -854,6 +854,8 @@ it with new information. To use this, you will have to do two steps. 1. Define the state update function - Specify with a function how to update the state using the previous state and the new values from an input stream. +In every batch, Spark will apply the state update function for all existing keys, regardless of whether they have new data in a batch or not. If the update function returns `None` then the key-value pair will be eliminated. + Let's illustrate this with an example. Say you want to maintain a running count of each word seen in a text data stream. Here, the running count is the state and it is an integer. We define the update function as: diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 18ccbc0a3edd0..7c83d68e7993e 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -125,7 +125,7 @@ def setup_external_libs(libs): ) with open(tgz_file_path, "wb") as tgz_file: tgz_file.write(download_stream.read()) - with open(tgz_file_path) as tar: + with open(tgz_file_path, "rb") as tar: if hashlib.md5(tar.read()).hexdigest() != lib["md5"]: print("ERROR: Got wrong md5sum for {lib}.".format(lib=lib["name"]), file=stderr) sys.exit(1) @@ -325,14 +325,16 @@ def parse_args(): home_dir = os.getenv('HOME') if home_dir is None or not os.path.isfile(home_dir + '/.boto'): if not os.path.isfile('/etc/boto.cfg'): - if os.getenv('AWS_ACCESS_KEY_ID') is None: - print("ERROR: The environment variable AWS_ACCESS_KEY_ID must be set", - file=stderr) - sys.exit(1) - if os.getenv('AWS_SECRET_ACCESS_KEY') is None: - print("ERROR: The environment variable AWS_SECRET_ACCESS_KEY must be set", - file=stderr) - sys.exit(1) + # If there is no boto config, check aws credentials + if not os.path.isfile(home_dir + '/.aws/credentials'): + if os.getenv('AWS_ACCESS_KEY_ID') is None: + print("ERROR: The environment variable AWS_ACCESS_KEY_ID must be set", + file=stderr) + sys.exit(1) + if os.getenv('AWS_SECRET_ACCESS_KEY') is None: + print("ERROR: The environment variable AWS_SECRET_ACCESS_KEY must be set", + file=stderr) + sys.exit(1) return (opts, action, cluster_name) @@ -791,7 +793,7 @@ def setup_cluster(conn, master_nodes, slave_nodes, opts, deploy_ssh_key): ssh_write(slave_address, opts, ['tar', 'x'], dot_ssh_tar) modules = ['spark', 'ephemeral-hdfs', 'persistent-hdfs', - 'mapreduce', 'spark-standalone', 'tachyon'] + 'mapreduce', 'spark-standalone', 'tachyon', 'rstudio'] if opts.hadoop_major_version == "1": modules = list(filter(lambda x: x != "mapreduce", modules)) @@ -1153,8 +1155,8 @@ def ssh(host, opts, command): # If this was an ssh failure, provide the user with hints. if e.returncode == 255: raise UsageError( - "Failed to SSH to remote host {0}.\n" + - "Please check that you have provided the correct --identity-file and " + + "Failed to SSH to remote host {0}.\n" + "Please check that you have provided the correct --identity-file and " "--key-pair parameters and try again.".format(host)) else: raise e diff --git a/examples/src/main/r/data-manipulation.R b/examples/src/main/r/data-manipulation.R new file mode 100644 index 0000000000000..aa2336e300a91 --- /dev/null +++ b/examples/src/main/r/data-manipulation.R @@ -0,0 +1,107 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# For this example, we shall use the "flights" dataset +# The dataset consists of every flight departing Houston in 2011. +# The data set is made up of 227,496 rows x 14 columns. + +# To run this example use +# ./bin/sparkR --packages com.databricks:spark-csv_2.10:1.0.3 +# examples/src/main/r/data-manipulation.R + +# Load SparkR library into your R session +library(SparkR) + +args <- commandArgs(trailing = TRUE) + +if (length(args) != 1) { + print("Usage: data-manipulation.R % + summarize(avg(flightsDF$dep_delay), avg(flightsDF$arr_delay)) -> dailyDelayDF + + # Print the computed data frame + head(dailyDelayDF) +} + +# Stop the SparkContext now +sparkR.stop() 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 4c129dbe2d12d..d812262fd87dc 100644 --- a/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples import org.apache.spark.{SparkConf, SparkContext} @@ -52,3 +53,4 @@ object BroadcastTest { sc.stop() } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/CassandraCQLTest.scala b/examples/src/main/scala/org/apache/spark/examples/CassandraCQLTest.scala index 023bb3ee2d108..36832f51d2ad4 100644 --- a/examples/src/main/scala/org/apache/spark/examples/CassandraCQLTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/CassandraCQLTest.scala @@ -15,6 +15,7 @@ * limitations under the License. */ + // scalastyle:off println package org.apache.spark.examples import java.nio.ByteBuffer @@ -140,3 +141,4 @@ object CassandraCQLTest { sc.stop() } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/CassandraTest.scala b/examples/src/main/scala/org/apache/spark/examples/CassandraTest.scala index ec689474aecb0..96ef3e198e380 100644 --- a/examples/src/main/scala/org/apache/spark/examples/CassandraTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/CassandraTest.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples import java.nio.ByteBuffer @@ -130,6 +131,7 @@ object CassandraTest { sc.stop() } } +// scalastyle:on println /* create keyspace casDemo; diff --git a/examples/src/main/scala/org/apache/spark/examples/DFSReadWriteTest.scala b/examples/src/main/scala/org/apache/spark/examples/DFSReadWriteTest.scala index 1f12034ce0f57..d651fe4d6ee75 100644 --- a/examples/src/main/scala/org/apache/spark/examples/DFSReadWriteTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/DFSReadWriteTest.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples import java.io.File @@ -136,3 +137,4 @@ object DFSReadWriteTest { } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/DriverSubmissionTest.scala b/examples/src/main/scala/org/apache/spark/examples/DriverSubmissionTest.scala index e757283823fc3..c42df2b8845d2 100644 --- a/examples/src/main/scala/org/apache/spark/examples/DriverSubmissionTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/DriverSubmissionTest.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples import scala.collection.JavaConversions._ @@ -46,3 +47,4 @@ object DriverSubmissionTest { } } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala index 15f6678648b29..fa4a3afeecd19 100644 --- a/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples import java.util.Random @@ -53,3 +54,4 @@ object GroupByTest { sc.stop() } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala b/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala index 95c96111c9b1f..244742327a907 100644 --- a/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples import org.apache.hadoop.hbase.client.HBaseAdmin @@ -62,3 +63,4 @@ object HBaseTest { admin.close() } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala b/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala index ed2b38e2ca6f8..124dc9af6390f 100644 --- a/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples import org.apache.spark._ @@ -41,3 +42,4 @@ object HdfsTest { sc.stop() } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala b/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala index 3d5259463003d..af5f216f28ba4 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples import org.apache.commons.math3.linear._ @@ -142,3 +143,4 @@ object LocalALS { new Array2DRowRealMatrix(Array.fill(rows, cols)(math.random)) } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalFileLR.scala b/examples/src/main/scala/org/apache/spark/examples/LocalFileLR.scala index ac2ea35bbd0e0..9c8aae53cf48d 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LocalFileLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LocalFileLR.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples import java.util.Random @@ -73,3 +74,4 @@ object LocalFileLR { println("Final w: " + w) } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalKMeans.scala b/examples/src/main/scala/org/apache/spark/examples/LocalKMeans.scala index 04fc0a033014a..e7b28d38bdfc6 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LocalKMeans.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LocalKMeans.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples import java.util.Random @@ -119,3 +120,4 @@ object LocalKMeans { println("Final centers: " + kPoints) } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalLR.scala b/examples/src/main/scala/org/apache/spark/examples/LocalLR.scala index c3fc74a116c0a..4f6b092a59ca5 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LocalLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LocalLR.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples import java.util.Random @@ -77,3 +78,4 @@ object LocalLR { println("Final w: " + w) } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalPi.scala b/examples/src/main/scala/org/apache/spark/examples/LocalPi.scala index ee6b3ee34aeb2..3d923625f11b6 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LocalPi.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LocalPi.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples import scala.math.random @@ -33,3 +34,4 @@ object LocalPi { println("Pi is roughly " + 4 * count / 100000.0) } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala b/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala index 75c82117cbad2..a80de10f4610a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples import org.apache.spark.{SparkConf, SparkContext} @@ -83,3 +84,4 @@ object LogQuery { sc.stop() } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala b/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala index 2a5c0c0defe13..61ce9db914f9f 100644 --- a/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples import org.apache.spark.rdd.RDD @@ -53,3 +54,4 @@ object MultiBroadcastTest { sc.stop() } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala index 5291ab81f459e..3b0b00fe4dd0a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples import java.util.Random @@ -67,3 +68,4 @@ object SimpleSkewedGroupByTest { sc.stop() } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala index 017d4e1e5ce13..719e2176fed3f 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples import java.util.Random @@ -57,3 +58,4 @@ object SkewedGroupByTest { sc.stop() } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala index 30c4261551837..69799b7c2bb30 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples import org.apache.commons.math3.linear._ @@ -144,3 +145,4 @@ object SparkALS { new Array2DRowRealMatrix(Array.fill(rows, cols)(math.random)) } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala index 9099c2fcc90b3..505ea5a4c7a85 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples import java.util.Random @@ -97,3 +98,4 @@ object SparkHdfsLR { sc.stop() } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala b/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala index b514d9123f5e7..c56e1124ad415 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples import breeze.linalg.{Vector, DenseVector, squaredDistance} @@ -100,3 +101,4 @@ object SparkKMeans { sc.stop() } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala index 1e6b4fb0c7514..d265c227f4ed2 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples import java.util.Random @@ -86,3 +87,4 @@ object SparkLR { sc.stop() } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala b/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala index bd7894f184c4c..0fd79660dd196 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples import org.apache.spark.SparkContext._ @@ -74,3 +75,4 @@ object SparkPageRank { ctx.stop() } } +// scalastyle:on println 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 35b8dd6c29b66..818d4f2b81f82 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples import scala.math.random @@ -37,3 +38,4 @@ object SparkPi { spark.stop() } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala index 772cd897f5140..95072071ccddb 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples import scala.util.Random @@ -70,3 +71,4 @@ object SparkTC { spark.stop() } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala index 4393b99e636b6..cfbdae02212a5 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples import java.util.Random @@ -94,3 +95,4 @@ object SparkTachyonHdfsLR { sc.stop() } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala index 7743f7968b100..e46ac655beb58 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples import scala.math.random @@ -46,3 +47,4 @@ object SparkTachyonPi { spark.stop() } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala b/examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala index 409721b01c8fd..8dd6c9706e7df 100644 --- a/examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala +++ b/examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples.graphx import scala.collection.mutable @@ -151,3 +152,4 @@ object Analytics extends Logging { } } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/graphx/LiveJournalPageRank.scala b/examples/src/main/scala/org/apache/spark/examples/graphx/LiveJournalPageRank.scala index f6f8d9f90c275..da3ffca1a6f2a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/graphx/LiveJournalPageRank.scala +++ b/examples/src/main/scala/org/apache/spark/examples/graphx/LiveJournalPageRank.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples.graphx /** @@ -42,3 +43,4 @@ object LiveJournalPageRank { Analytics.main(args.patch(0, List("pagerank"), 0)) } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/graphx/SynthBenchmark.scala b/examples/src/main/scala/org/apache/spark/examples/graphx/SynthBenchmark.scala index 3ec20d594b784..46e52aacd90bb 100644 --- a/examples/src/main/scala/org/apache/spark/examples/graphx/SynthBenchmark.scala +++ b/examples/src/main/scala/org/apache/spark/examples/graphx/SynthBenchmark.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples.graphx import org.apache.spark.SparkContext._ @@ -128,3 +129,4 @@ object SynthBenchmark { sc.stop() } } +// scalastyle:on println 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 6c0af20461d3b..14b358d46f6ab 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 @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples.ml import org.apache.spark.{SparkConf, SparkContext} @@ -110,3 +111,4 @@ object CrossValidatorExample { sc.stop() } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala index 54e4073941056..f28671f7869fc 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DecisionTreeExample.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples.ml import scala.collection.mutable @@ -355,3 +356,4 @@ object DecisionTreeExample { println(s" Root mean squared error (RMSE): $RMSE") } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala index 7b8cc21ed8982..78f31b4ffe56a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples.ml import org.apache.spark.{SparkConf, SparkContext} @@ -181,3 +182,4 @@ private class MyLogisticRegressionModel( copyValues(new MyLogisticRegressionModel(uid, weights), extra) } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/GBTExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/GBTExample.scala index 33905277c7341..f4a15f806ea81 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/GBTExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/GBTExample.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples.ml import scala.collection.mutable @@ -236,3 +237,4 @@ object GBTExample { sc.stop() } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/LinearRegressionExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/LinearRegressionExample.scala index b54466fd48bc5..b73299fb12d3f 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/LinearRegressionExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/LinearRegressionExample.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples.ml import scala.collection.mutable @@ -140,3 +141,4 @@ object LinearRegressionExample { sc.stop() } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionExample.scala index 3cf193f353fbc..7682557127b51 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/LogisticRegressionExample.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples.ml import scala.collection.mutable @@ -157,3 +158,4 @@ object LogisticRegressionExample { sc.stop() } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/MovieLensALS.scala b/examples/src/main/scala/org/apache/spark/examples/ml/MovieLensALS.scala index 25f21113bf622..cd411397a4b9d 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/MovieLensALS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/MovieLensALS.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples.ml import scopt.OptionParser @@ -178,3 +179,4 @@ object MovieLensALS { sc.stop() } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/OneVsRestExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/OneVsRestExample.scala index 6927eb8f275cf..bab31f585b0ef 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/OneVsRestExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/OneVsRestExample.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples.ml import java.util.concurrent.TimeUnit.{NANOSECONDS => NANO} @@ -183,3 +184,4 @@ object OneVsRestExample { (NANO.toSeconds(t1 - t0), result) } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestExample.scala index 9f7cad68a4594..109178f4137b2 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/RandomForestExample.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples.ml import scala.collection.mutable @@ -244,3 +245,4 @@ object RandomForestExample { sc.stop() } } +// scalastyle:on println 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 a0561e2573fc9..58d7b67674ff7 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 @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples.ml import org.apache.spark.{SparkConf, SparkContext} @@ -100,3 +101,4 @@ object SimpleParamsExample { sc.stop() } } +// scalastyle:on println 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 1324b066c30c3..960280137cbf9 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 @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples.ml import scala.beans.BeanInfo @@ -89,3 +90,4 @@ object SimpleTextClassificationPipeline { sc.stop() } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassification.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassification.scala index a113653810b93..1a4016f76c2ad 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassification.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassification.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples.mllib import org.apache.log4j.{Level, Logger} @@ -153,3 +154,4 @@ object BinaryClassification { sc.stop() } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/Correlations.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/Correlations.scala index e49129c4e7844..026d4ecc6d10a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/Correlations.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/Correlations.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples.mllib import scopt.OptionParser @@ -91,3 +92,4 @@ object Correlations { sc.stop() } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/CosineSimilarity.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/CosineSimilarity.scala index cb1abbd18fd4d..69988cc1b9334 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/CosineSimilarity.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/CosineSimilarity.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples.mllib import scopt.OptionParser @@ -106,3 +107,4 @@ object CosineSimilarity { sc.stop() } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/DatasetExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/DatasetExample.scala index 520893b26d595..dc13f82488af7 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/DatasetExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/DatasetExample.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples.mllib import java.io.File @@ -119,3 +120,4 @@ object DatasetExample { } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala index 3381941673db8..57ffe3dd2524f 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples.mllib import scala.language.reflectiveCalls @@ -368,3 +369,4 @@ object DecisionTreeRunner { } // scalastyle:on structural.type } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/DenseGaussianMixture.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/DenseGaussianMixture.scala index f8c71ccabc43b..1fce4ba7efd60 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/DenseGaussianMixture.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/DenseGaussianMixture.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples.mllib import org.apache.spark.{SparkConf, SparkContext} @@ -65,3 +66,4 @@ object DenseGaussianMixture { println() } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/DenseKMeans.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/DenseKMeans.scala index 14cc5cbb679c5..380d85d60e7b4 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/DenseKMeans.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/DenseKMeans.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples.mllib import org.apache.log4j.{Level, Logger} @@ -107,3 +108,4 @@ object DenseKMeans { sc.stop() } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/FPGrowthExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/FPGrowthExample.scala index 13f24a1e59610..14b930550d554 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/FPGrowthExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/FPGrowthExample.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples.mllib import scopt.OptionParser @@ -80,3 +81,4 @@ object FPGrowthExample { sc.stop() } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/GradientBoostedTreesRunner.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/GradientBoostedTreesRunner.scala index 7416fb5a40848..e16a6bf033574 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/GradientBoostedTreesRunner.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/GradientBoostedTreesRunner.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples.mllib import scopt.OptionParser @@ -145,3 +146,4 @@ object GradientBoostedTreesRunner { sc.stop() } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala index 31d629f853161..75b0f69cf91aa 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/LDAExample.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples.mllib import java.text.BreakIterator @@ -302,3 +303,4 @@ private class SimpleTokenizer(sc: SparkContext, stopwordFile: String) extends Se } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala index 6a456ba7ec07b..8878061a0970b 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples.mllib import org.apache.log4j.{Level, Logger} @@ -134,3 +135,4 @@ object LinearRegression { sc.stop() } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala index 99588b0984ab2..e43a6f2864c73 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples.mllib import scala.collection.mutable @@ -189,3 +190,4 @@ object MovieLensALS { math.sqrt(predictionsAndRatings.map(x => (x._1 - x._2) * (x._1 - x._2)).mean()) } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/MultivariateSummarizer.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/MultivariateSummarizer.scala index 6e4e2d07f284b..5f839c75dd581 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/MultivariateSummarizer.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/MultivariateSummarizer.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples.mllib import scopt.OptionParser @@ -97,3 +98,4 @@ object MultivariateSummarizer { sc.stop() } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/PowerIterationClusteringExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/PowerIterationClusteringExample.scala index 6d8b806569dfd..0723223954610 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/PowerIterationClusteringExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/PowerIterationClusteringExample.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples.mllib import org.apache.log4j.{Level, Logger} @@ -154,4 +155,4 @@ object PowerIterationClusteringExample { coeff * math.exp(expCoeff * ssquares) } } - +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/RandomRDDGeneration.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/RandomRDDGeneration.scala index 924b586e3af99..bee85ba0f9969 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/RandomRDDGeneration.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/RandomRDDGeneration.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples.mllib import org.apache.spark.mllib.random.RandomRDDs @@ -58,3 +59,4 @@ object RandomRDDGeneration { } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/SampledRDDs.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/SampledRDDs.scala index 663c12734af68..6963f43e082c4 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/SampledRDDs.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/SampledRDDs.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples.mllib import org.apache.spark.mllib.util.MLUtils @@ -125,3 +126,4 @@ object SampledRDDs { sc.stop() } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/SparseNaiveBayes.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/SparseNaiveBayes.scala index f1ff4e6911f5e..f81fc292a3bd1 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/SparseNaiveBayes.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/SparseNaiveBayes.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples.mllib import org.apache.log4j.{Level, Logger} @@ -100,3 +101,4 @@ object SparseNaiveBayes { sc.stop() } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingKMeansExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingKMeansExample.scala index 8bb12d2ee9ed2..af03724a8ac62 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingKMeansExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingKMeansExample.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples.mllib import org.apache.spark.SparkConf @@ -75,3 +76,4 @@ object StreamingKMeansExample { ssc.awaitTermination() } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLinearRegression.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLinearRegression.scala index 1a95048bbfe2d..b4a5dca031abd 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLinearRegression.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLinearRegression.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples.mllib import org.apache.spark.mllib.linalg.Vectors @@ -69,3 +70,4 @@ object StreamingLinearRegression { } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLogisticRegression.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLogisticRegression.scala index e1998099c2d78..b42f4cb5f9338 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLogisticRegression.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLogisticRegression.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples.mllib import org.apache.spark.mllib.linalg.Vectors @@ -71,3 +72,4 @@ object StreamingLogisticRegression { } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala index 3cd9cb743e309..464fbd385ab5d 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples.mllib import org.apache.spark.{SparkConf, SparkContext} @@ -58,3 +59,4 @@ object TallSkinnyPCA { sc.stop() } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala index 4d6690318615a..65b4bc46f0266 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples.mllib import org.apache.spark.{SparkConf, SparkContext} @@ -58,3 +59,4 @@ object TallSkinnySVD { sc.stop() } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala b/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala index b11e32047dc34..2cc56f04e5c1f 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples.sql import org.apache.spark.{SparkConf, SparkContext} @@ -73,3 +74,4 @@ object RDDRelation { sc.stop() } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala index b7ba60ec28155..bf40bd1ef13df 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples.sql.hive import com.google.common.io.{ByteStreams, Files} @@ -77,3 +78,4 @@ object HiveFromSpark { sc.stop() } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/ActorWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/ActorWordCount.scala index 016de4c63d1d2..e9c9907198769 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/ActorWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/ActorWordCount.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples.streaming import scala.collection.mutable.LinkedList @@ -170,3 +171,4 @@ object ActorWordCount { ssc.awaitTermination() } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala index 30269a7ccae97..28e9bf520e568 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples.streaming import java.io.{InputStreamReader, BufferedReader, InputStream} @@ -100,3 +101,4 @@ class CustomReceiver(host: String, port: Int) } } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/DirectKafkaWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/DirectKafkaWordCount.scala index fbe394de4a179..bd78526f8c299 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/DirectKafkaWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/DirectKafkaWordCount.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples.streaming import kafka.serializer.StringDecoder @@ -70,3 +71,4 @@ object DirectKafkaWordCount { ssc.awaitTermination() } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/FlumeEventCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/FlumeEventCount.scala index 20e7df7c45b1b..91e52e4eff5a7 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/FlumeEventCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/FlumeEventCount.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples.streaming import org.apache.spark.SparkConf @@ -66,3 +67,4 @@ object FlumeEventCount { ssc.awaitTermination() } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/FlumePollingEventCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/FlumePollingEventCount.scala index 1cc8c8d5c23b6..2bdbc37e2a289 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/FlumePollingEventCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/FlumePollingEventCount.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples.streaming import org.apache.spark.SparkConf @@ -65,3 +66,4 @@ object FlumePollingEventCount { ssc.awaitTermination() } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/HdfsWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/HdfsWordCount.scala index 4b4667fec44e6..1f282d437dc38 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/HdfsWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/HdfsWordCount.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples.streaming import org.apache.spark.SparkConf @@ -53,3 +54,4 @@ object HdfsWordCount { ssc.awaitTermination() } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala index 60416ee343544..b40d17e9c2fa3 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples.streaming import java.util.HashMap @@ -101,3 +102,4 @@ object KafkaWordCountProducer { } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/MQTTWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/MQTTWordCount.scala index 813c8554f5193..d772ae309f40d 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/MQTTWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/MQTTWordCount.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples.streaming import org.eclipse.paho.client.mqttv3._ @@ -96,8 +97,10 @@ object MQTTWordCount { def main(args: Array[String]) { if (args.length < 2) { + // scalastyle:off println System.err.println( "Usage: MQTTWordCount ") + // scalastyle:on println System.exit(1) } @@ -113,3 +116,4 @@ object MQTTWordCount { ssc.awaitTermination() } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/NetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/NetworkWordCount.scala index 2cd8073dada14..9a57fe286d1ae 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/NetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/NetworkWordCount.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples.streaming import org.apache.spark.SparkConf @@ -57,3 +58,4 @@ object NetworkWordCount { ssc.awaitTermination() } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/RawNetworkGrep.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/RawNetworkGrep.scala index a9aaa445bccb6..5322929d177b4 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/RawNetworkGrep.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/RawNetworkGrep.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples.streaming import org.apache.spark.SparkConf @@ -58,3 +59,4 @@ object RawNetworkGrep { ssc.awaitTermination() } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala index 751b30ea15782..9916882e4f94a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples.streaming import java.io.File @@ -108,3 +109,4 @@ object RecoverableNetworkWordCount { ssc.awaitTermination() } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala index 5a6b9216a3fbc..ed617754cbf1c 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/SqlNetworkWordCount.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples.streaming import org.apache.spark.SparkConf @@ -99,3 +100,4 @@ object SQLContextSingleton { instance } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala index 345d0bc441351..02ba1c2eed0f7 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples.streaming import org.apache.spark.SparkConf @@ -78,3 +79,4 @@ object StatefulNetworkWordCount { ssc.awaitTermination() } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala index c10de84a80ffe..825c671a929b1 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples.streaming import com.twitter.algebird._ @@ -113,3 +114,4 @@ object TwitterAlgebirdCMS { ssc.awaitTermination() } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdHLL.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdHLL.scala index 62db5e663b8af..49826ede70418 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdHLL.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdHLL.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples.streaming import com.twitter.algebird.HyperLogLogMonoid @@ -90,3 +91,4 @@ object TwitterAlgebirdHLL { ssc.awaitTermination() } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterPopularTags.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterPopularTags.scala index f253d75b279f7..49cee1b43c2dc 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterPopularTags.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterPopularTags.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples.streaming import org.apache.spark.streaming.{Seconds, StreamingContext} @@ -82,3 +83,4 @@ object TwitterPopularTags { ssc.awaitTermination() } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/ZeroMQWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/ZeroMQWordCount.scala index e99d1baa72b9f..6ac9a72c37941 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/ZeroMQWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/ZeroMQWordCount.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples.streaming import akka.actor.ActorSystem @@ -97,3 +98,4 @@ object ZeroMQWordCount { ssc.awaitTermination() } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewGenerator.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewGenerator.scala index 889f052c70263..bea7a47cb2855 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewGenerator.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewGenerator.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples.streaming.clickstream import java.net.ServerSocket @@ -108,3 +109,4 @@ object PageViewGenerator { } } } +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewStream.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewStream.scala index fbacaee98690f..ec7d39da8b2e9 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewStream.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewStream.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples.streaming.clickstream import org.apache.spark.SparkContext._ @@ -107,3 +108,4 @@ object PageViewStream { ssc.start() } } +// scalastyle:on println diff --git a/external/flume-assembly/pom.xml b/external/flume-assembly/pom.xml index 8565cd83edfa2..13189595d1d6c 100644 --- a/external/flume-assembly/pom.xml +++ b/external/flume-assembly/pom.xml @@ -32,6 +32,7 @@ http://spark.apache.org/ + provided streaming-flume-assembly @@ -40,6 +41,16 @@ org.apache.spark spark-streaming-flume_${scala.binary.version} ${project.version} + + + org.mortbay.jetty + jetty + + + org.mortbay.jetty + jetty-util + +
org.apache.spark @@ -47,89 +58,101 @@ ${project.version} provided + + + commons-codec + commons-codec + provided + + + commons-net + commons-net + provided + + + com.google.protobuf + protobuf-java + provided + org.apache.avro avro - ${avro.version} + provided org.apache.avro avro-ipc - ${avro.version} - - - io.netty - netty - - - org.mortbay.jetty - jetty - - - org.mortbay.jetty - jetty-util - - - org.mortbay.jetty - servlet-api - - - org.apache.velocity - velocity - - + provided + + + org.scala-lang + scala-library + provided - target/scala-${scala.binary.version}/classes - target/scala-${scala.binary.version}/test-classes - - - org.apache.maven.plugins - maven-shade-plugin - - false - ${project.build.directory}/scala-${scala.binary.version}/spark-streaming-flume-assembly-${project.version}.jar - - - *:* - - - - - *:* - - META-INF/*.SF - META-INF/*.DSA - META-INF/*.RSA - - - - - - - package - - shade - - - - - - reference.conf - - - log4j.properties - - - - - - - - - - + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + org.apache.maven.plugins + maven-shade-plugin + + false + ${project.build.directory}/scala-${scala.binary.version}/spark-streaming-flume-assembly-${project.version}.jar + + + *:* + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + package + + shade + + + + + + reference.conf + + + log4j.properties + + + + + + + + + + + + + + flume-provided + + provided + + + diff --git a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/Logging.scala b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/Logging.scala index 17cbc6707b5ea..d87b86932dd41 100644 --- a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/Logging.scala +++ b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/Logging.scala @@ -113,7 +113,9 @@ private[sink] object Logging { try { // We use reflection here to handle the case where users remove the // slf4j-to-jul bridge order to route their logs to JUL. + // scalastyle:off classforname val bridgeClass = Class.forName("org.slf4j.bridge.SLF4JBridgeHandler") + // scalastyle:on classforname bridgeClass.getMethod("removeHandlersForRootLogger").invoke(null) val installed = bridgeClass.getMethod("isInstalled").invoke(null).asInstanceOf[Boolean] if (!installed) { diff --git a/external/kafka-assembly/pom.xml b/external/kafka-assembly/pom.xml index 8059c443827ef..977514fa5a1ec 100644 --- a/external/kafka-assembly/pom.xml +++ b/external/kafka-assembly/pom.xml @@ -58,6 +58,7 @@ maven-shade-plugin false + ${project.build.directory}/scala-${scala.binary.version}/spark-streaming-kafka-assembly-${project.version}.jar *:* diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala index 876456c964770..48a1933d92f85 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala @@ -19,7 +19,7 @@ package org.apache.spark.streaming.kafka import scala.annotation.tailrec import scala.collection.mutable -import scala.reflect.{classTag, ClassTag} +import scala.reflect.ClassTag import kafka.common.TopicAndPartition import kafka.message.MessageAndMetadata @@ -29,7 +29,7 @@ import org.apache.spark.{Logging, SparkException} import org.apache.spark.streaming.{StreamingContext, Time} import org.apache.spark.streaming.dstream._ import org.apache.spark.streaming.kafka.KafkaCluster.LeaderOffset -import org.apache.spark.streaming.scheduler.InputInfo +import org.apache.spark.streaming.scheduler.StreamInputInfo /** * A stream of {@link org.apache.spark.streaming.kafka.KafkaRDD} where @@ -119,8 +119,23 @@ class DirectKafkaInputDStream[ val rdd = KafkaRDD[K, V, U, T, R]( context.sparkContext, kafkaParams, currentOffsets, untilOffsets, messageHandler) - // Report the record number of this batch interval to InputInfoTracker. - val inputInfo = InputInfo(id, rdd.count) + // Report the record number and metadata of this batch interval to InputInfoTracker. + val offsetRanges = currentOffsets.map { case (tp, fo) => + val uo = untilOffsets(tp) + OffsetRange(tp.topic, tp.partition, fo, uo.offset) + } + val description = offsetRanges.filter { offsetRange => + // Don't display empty ranges. + offsetRange.fromOffset != offsetRange.untilOffset + }.map { offsetRange => + s"topic: ${offsetRange.topic}\tpartition: ${offsetRange.partition}\t" + + s"offsets: ${offsetRange.fromOffset} to ${offsetRange.untilOffset}" + }.mkString("\n") + // Copy offsetRanges to immutable.List to prevent from being modified by the user + val metadata = Map( + "offsets" -> offsetRanges.toList, + StreamInputInfo.METADATA_KEY_DESCRIPTION -> description) + val inputInfo = StreamInputInfo(id, rdd.count, metadata) ssc.scheduler.inputInfoTracker.reportInfo(validTime, inputInfo) currentOffsets = untilOffsets.map(kv => kv._1 -> kv._2.offset) diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala index 3e6b937af57b0..8465432c5850f 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala @@ -410,7 +410,7 @@ object KafkaCluster { } Seq("zookeeper.connect", "group.id").foreach { s => - if (!props.contains(s)) { + if (!props.containsKey(s)) { props.setProperty(s, "") } } diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala index 0e33362d34acd..f3b01bd60b178 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala @@ -670,4 +670,17 @@ private class KafkaUtilsPythonHelper { TopicAndPartition(topic, partition) def createBroker(host: String, port: JInt): Broker = Broker(host, port) + + def offsetRangesOfKafkaRDD(rdd: RDD[_]): JList[OffsetRange] = { + val parentRDDs = rdd.getNarrowAncestors + val kafkaRDDs = parentRDDs.filter(rdd => rdd.isInstanceOf[KafkaRDD[_, _, _, _, _]]) + + require( + kafkaRDDs.length == 1, + "Cannot get offset ranges, as there may be multiple Kafka RDDs or no Kafka RDD associated" + + "with this RDD, please call this method only on a Kafka RDD.") + + val kafkaRDD = kafkaRDDs.head.asInstanceOf[KafkaRDD[_, _, _, _, _]] + kafkaRDD.offsetRanges.toSeq + } } diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala index 2675042666304..f326e7f1f6f8d 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala @@ -75,7 +75,7 @@ final class OffsetRange private( } override def toString(): String = { - s"OffsetRange(topic: '$topic', partition: $partition, range: [$fromOffset -> $untilOffset]" + s"OffsetRange(topic: '$topic', partition: $partition, range: [$fromOffset -> $untilOffset])" } /** this is to avoid ClassNotFoundException during checkpoint restore */ diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala index 8e1715f6dbb95..5b3c79444aa68 100644 --- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala +++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala @@ -111,7 +111,7 @@ class DirectKafkaStreamSuite rdd }.foreachRDD { rdd => for (o <- offsetRanges) { - println(s"${o.topic} ${o.partition} ${o.fromOffset} ${o.untilOffset}") + logInfo(s"${o.topic} ${o.partition} ${o.fromOffset} ${o.untilOffset}") } val collected = rdd.mapPartitionsWithIndex { (i, iter) => // For each partition, get size of the range in the partition, diff --git a/extras/kinesis-asl/pom.xml b/extras/kinesis-asl/pom.xml index 5289073eb457a..c242e7a57b9ab 100644 --- a/extras/kinesis-asl/pom.xml +++ b/extras/kinesis-asl/pom.xml @@ -40,6 +40,13 @@ spark-streaming_${scala.binary.version} ${project.version} + + org.apache.spark + spark-core_${scala.binary.version} + ${project.version} + test-jar + test + org.apache.spark spark-streaming_${scala.binary.version} diff --git a/extras/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala b/extras/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala index be8b62d3cc6ba..de749626ec09c 100644 --- a/extras/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala +++ b/extras/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala @@ -15,6 +15,7 @@ * limitations under the License. */ +// scalastyle:off println package org.apache.spark.examples.streaming import java.nio.ByteBuffer @@ -272,3 +273,4 @@ private[streaming] object StreamingExamples extends Logging { } } } +// scalastyle:on println diff --git a/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisReceiverSuite.scala b/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisReceiverSuite.scala index 6c262624833cd..2103dca6b766f 100644 --- a/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisReceiverSuite.scala +++ b/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisReceiverSuite.scala @@ -26,23 +26,18 @@ import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionIn import com.amazonaws.services.kinesis.clientlibrary.types.ShutdownReason import com.amazonaws.services.kinesis.model.Record import org.mockito.Mockito._ -// scalastyle:off -// To avoid introducing a dependency on Spark core tests, simply use scalatest's FunSuite -// here instead of our own SparkFunSuite. Introducing the dependency has caused problems -// in the past (SPARK-8781) that are complicated by bugs in the maven shade plugin (MSHADE-148). -import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} +import org.scalatest.{BeforeAndAfter, Matchers} import org.scalatest.mock.MockitoSugar import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.{Milliseconds, Seconds, StreamingContext} +import org.apache.spark.streaming.{Milliseconds, Seconds, StreamingContext, TestSuiteBase} import org.apache.spark.util.{Clock, ManualClock, Utils} /** * Suite of Kinesis streaming receiver tests focusing mostly on the KinesisRecordProcessor */ -class KinesisReceiverSuite extends FunSuite with Matchers with BeforeAndAfter - with MockitoSugar { -// scalastyle:on +class KinesisReceiverSuite extends TestSuiteBase with Matchers with BeforeAndAfter + with MockitoSugar { val app = "TestKinesisReceiver" val stream = "mySparkStream" @@ -62,7 +57,7 @@ class KinesisReceiverSuite extends FunSuite with Matchers with BeforeAndAfter var checkpointStateMock: KinesisCheckpointState = _ var currentClockMock: Clock = _ - before { + override def beforeFunction(): Unit = { receiverMock = mock[KinesisReceiver] checkpointerMock = mock[IRecordProcessorCheckpointer] checkpointClockMock = mock[ManualClock] @@ -70,7 +65,8 @@ class KinesisReceiverSuite extends FunSuite with Matchers with BeforeAndAfter currentClockMock = mock[Clock] } - after { + override def afterFunction(): Unit = { + super.afterFunction() // Since this suite was originally written using EasyMock, add this to preserve the old // mocking semantics (see SPARK-5735 for more details) verifyNoMoreInteractions(receiverMock, checkpointerMock, checkpointClockMock, @@ -78,7 +74,7 @@ class KinesisReceiverSuite extends FunSuite with Matchers with BeforeAndAfter } test("KinesisUtils API") { - val ssc = new StreamingContext("local[2]", getClass.getSimpleName, Seconds(1)) + val ssc = new StreamingContext(master, framework, batchDuration) // Tests the API, does not actually test data receiving val kinesisStream1 = KinesisUtils.createStream(ssc, "mySparkStream", "https://kinesis.us-west-2.amazonaws.com", Seconds(2), 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 7372dfbd9fe98..70a7592da8ae3 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala @@ -32,7 +32,7 @@ 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) - 1` bound on vertex replication. + * guaranteeing a `2 * sqrt(numParts)` bound on vertex replication. * * 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,26 +61,36 @@ 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) - 1` machines. + * replicated to at most `2 * sqrt(numParts)` 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 * vertex locations. * - * One of the limitations of this approach is that the number of machines must either be a - * perfect square. We partially address this limitation by computing the machine assignment to - * the next - * largest perfect square and then mapping back down to the actual number of machines. - * Unfortunately, this can also lead to work imbalance and so it is suggested that a perfect - * square is used. + * When the number of partitions requested is not a perfect square we use a slightly different + * method where the last column can have a different number of rows than the others while still + * maintaining the same size per block. */ case object EdgePartition2D extends PartitionStrategy { override def getPartition(src: VertexId, dst: VertexId, numParts: PartitionID): PartitionID = { val ceilSqrtNumParts: PartitionID = math.ceil(math.sqrt(numParts)).toInt val mixingPrime: VertexId = 1125899906842597L - val col: PartitionID = (math.abs(src * mixingPrime) % ceilSqrtNumParts).toInt - val row: PartitionID = (math.abs(dst * mixingPrime) % ceilSqrtNumParts).toInt - (col * ceilSqrtNumParts + row) % numParts + if (numParts == ceilSqrtNumParts * ceilSqrtNumParts) { + // Use old method for perfect squared to ensure we get same results + val col: PartitionID = (math.abs(src * mixingPrime) % ceilSqrtNumParts).toInt + val row: PartitionID = (math.abs(dst * mixingPrime) % ceilSqrtNumParts).toInt + (col * ceilSqrtNumParts + row) % numParts + + } else { + // Otherwise use new method + val cols = ceilSqrtNumParts + val rows = (numParts + cols - 1) / cols + val lastColRows = numParts - rows * (cols - 1) + val col = (math.abs(src * mixingPrime) % numParts / rows).toInt + val row = (math.abs(dst * mixingPrime) % (if (col < cols - 1) rows else lastColRows)).toInt + col * rows + row + + } } } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala index be6b9047d932d..74a7de18d4161 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala @@ -66,7 +66,6 @@ private[graphx] object BytecodeUtils { val finder = new MethodInvocationFinder(c.getName, m) getClassReader(c).accept(finder, 0) for (classMethod <- finder.methodsInvoked) { - // println(classMethod) if (classMethod._1 == targetClass && classMethod._2 == targetMethod) { return true } else if (!seen.contains(classMethod)) { @@ -122,7 +121,7 @@ private[graphx] object BytecodeUtils { override def visitMethodInsn(op: Int, owner: String, name: String, desc: String) { if (op == INVOKEVIRTUAL || op == INVOKESPECIAL || op == INVOKESTATIC) { if (!skipClass(owner)) { - methodsInvoked.add((Class.forName(owner.replace("/", ".")), name)) + methodsInvoked.add((Utils.classForName(owner.replace("/", ".")), name)) } } } 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 9591c4e9b8f4e..989e226305265 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 @@ -33,7 +33,7 @@ import org.apache.spark.graphx.Edge import org.apache.spark.graphx.impl.GraphImpl /** A collection of graph generating functions. */ -object GraphGenerators { +object GraphGenerators extends Logging { val RMATa = 0.45 val RMATb = 0.15 @@ -142,7 +142,7 @@ object GraphGenerators { var edges: Set[Edge[Int]] = Set() while (edges.size < numEdges) { if (edges.size % 100 == 0) { - println(edges.size + " edges") + logDebug(edges.size + " edges") } edges += addEdge(numVertices) } diff --git a/graphx/src/test/scala/org/apache/spark/graphx/util/BytecodeUtilsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/util/BytecodeUtilsSuite.scala index 186d0cc2a977b..61e44dcab578c 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/util/BytecodeUtilsSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/util/BytecodeUtilsSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.graphx.util import org.apache.spark.SparkFunSuite +// scalastyle:off println class BytecodeUtilsSuite extends SparkFunSuite { import BytecodeUtilsSuite.TestClass @@ -102,6 +103,7 @@ class BytecodeUtilsSuite extends SparkFunSuite { private val c = {e: TestClass => println(e.baz)} } +// scalastyle:on println object BytecodeUtilsSuite { class TestClass(val foo: Int, val bar: Long) { diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java index d4cfeacb6ef18..c0f89c9230692 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java @@ -25,11 +25,12 @@ import static org.apache.spark.launcher.CommandBuilderUtils.*; -/** +/** * Launcher for Spark applications. - *

+ *

* Use this class to start Spark applications programmatically. The class uses a builder pattern * to allow clients to configure the Spark application and launch it as a child process. + *

*/ public class SparkLauncher { diff --git a/launcher/src/main/java/org/apache/spark/launcher/package-info.java b/launcher/src/main/java/org/apache/spark/launcher/package-info.java index 7ed756f4b8591..7c97dba511b28 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/package-info.java +++ b/launcher/src/main/java/org/apache/spark/launcher/package-info.java @@ -17,13 +17,17 @@ /** * Library for launching Spark applications. - *

+ * + *

* This library allows applications to launch Spark programmatically. There's only one entry * point to the library - the {@link org.apache.spark.launcher.SparkLauncher} class. - *

+ *

+ * + *

* To launch a Spark application, just instantiate a {@link org.apache.spark.launcher.SparkLauncher} * and configure the application to run. For example: - * + *

+ * *
  * {@code
  *   import org.apache.spark.launcher.SparkLauncher;
diff --git a/make-distribution.sh b/make-distribution.sh
index 9f063da3a16c0..cac7032bb2e87 100755
--- a/make-distribution.sh
+++ b/make-distribution.sh
@@ -219,6 +219,7 @@ cp -r "$SPARK_HOME/ec2" "$DISTDIR"
 if [ -d "$SPARK_HOME"/R/lib/SparkR ]; then
   mkdir -p "$DISTDIR"/R/lib
   cp -r "$SPARK_HOME/R/lib/SparkR" "$DISTDIR"/R/lib
+  cp "$SPARK_HOME/R/lib/sparkr.zip" "$DISTDIR"/R/lib
 fi
 
 # Download and copy in tachyon, if requested
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 a1f3851d804ff..aef2c019d2871 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala
@@ -95,6 +95,8 @@ class Pipeline(override val uid: String) extends Estimator[PipelineModel] {
   /** @group setParam */
   def setStages(value: Array[PipelineStage]): this.type = { set(stages, value); this }
 
+  // Below, we clone stages so that modifications to the list of stages will not change
+  // the Param value in the Pipeline.
   /** @group getParam */
   def getStages: Array[PipelineStage] = $(stages).clone()
 
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 2e6eedd45ab07..8fc9199fb4602 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
@@ -19,7 +19,7 @@ package org.apache.spark.ml.classification
 
 import scala.collection.mutable
 
-import breeze.linalg.{DenseVector => BDV, norm => brzNorm}
+import breeze.linalg.{DenseVector => BDV}
 import breeze.optimize.{CachedDiffFunction, DiffFunction, LBFGS => BreezeLBFGS, OWLQN => BreezeOWLQN}
 
 import org.apache.spark.{Logging, SparkException}
@@ -41,7 +41,7 @@ import org.apache.spark.storage.StorageLevel
  */
 private[classification] trait LogisticRegressionParams extends ProbabilisticClassifierParams
   with HasRegParam with HasElasticNetParam with HasMaxIter with HasFitIntercept with HasTol
-  with HasThreshold
+  with HasThreshold with HasStandardization
 
 /**
  * :: Experimental ::
@@ -98,6 +98,18 @@ class LogisticRegression(override val uid: String)
   def setFitIntercept(value: Boolean): this.type = set(fitIntercept, value)
   setDefault(fitIntercept -> true)
 
+  /**
+   * Whether to standardize the training features before fitting the model.
+   * The coefficients of models will be always returned on the original scale,
+   * so it will be transparent for users. Note that when no regularization,
+   * with or without standardization, the models should be always converged to
+   * the same solution.
+   * Default is true.
+   * @group setParam
+   * */
+  def setStandardization(value: Boolean): this.type = set(standardization, value)
+  setDefault(standardization -> true)
+
   /** @group setParam */
   def setThreshold(value: Double): this.type = set(threshold, value)
   setDefault(threshold -> 0.5)
@@ -116,7 +128,7 @@ class LogisticRegression(override val uid: String)
           case ((summarizer: MultivariateOnlineSummarizer, labelSummarizer: MultiClassSummarizer),
           (label: Double, features: Vector)) =>
             (summarizer.add(features), labelSummarizer.add(label))
-      },
+        },
         combOp = (c1, c2) => (c1, c2) match {
           case ((summarizer1: MultivariateOnlineSummarizer,
           classSummarizer1: MultiClassSummarizer), (summarizer2: MultivariateOnlineSummarizer,
@@ -149,15 +161,28 @@ class LogisticRegression(override val uid: String)
     val regParamL1 = $(elasticNetParam) * $(regParam)
     val regParamL2 = (1.0 - $(elasticNetParam)) * $(regParam)
 
-    val costFun = new LogisticCostFun(instances, numClasses, $(fitIntercept),
+    val costFun = new LogisticCostFun(instances, numClasses, $(fitIntercept), $(standardization),
       featuresStd, featuresMean, regParamL2)
 
     val optimizer = if ($(elasticNetParam) == 0.0 || $(regParam) == 0.0) {
       new BreezeLBFGS[BDV[Double]]($(maxIter), 10, $(tol))
     } else {
-      // Remove the L1 penalization on the intercept
       def regParamL1Fun = (index: Int) => {
-        if (index == numFeatures) 0.0 else regParamL1
+        // Remove the L1 penalization on the intercept
+        if (index == numFeatures) {
+          0.0
+        } else {
+          if ($(standardization)) {
+            regParamL1
+          } else {
+            // If `standardization` is false, we still standardize the data
+            // to improve the rate of convergence; as a result, we have to
+            // perform this reverse standardization by penalizing each component
+            // differently to get effectively the same objective function when
+            // the training dataset is not standardized.
+            if (featuresStd(index) != 0.0) regParamL1 / featuresStd(index) else 0.0
+          }
+        }
       }
       new BreezeOWLQN[Int, BDV[Double]]($(maxIter), 10, regParamL1Fun, $(tol))
     }
@@ -166,18 +191,18 @@ class LogisticRegression(override val uid: String)
       Vectors.zeros(if ($(fitIntercept)) numFeatures + 1 else numFeatures)
 
     if ($(fitIntercept)) {
-      /**
-       * For binary logistic regression, when we initialize the weights as zeros,
-       * it will converge faster if we initialize the intercept such that
-       * it follows the distribution of the labels.
-       *
-       * {{{
-       * P(0) = 1 / (1 + \exp(b)), and
-       * P(1) = \exp(b) / (1 + \exp(b))
-       * }}}, hence
-       * {{{
-       * b = \log{P(1) / P(0)} = \log{count_1 / count_0}
-       * }}}
+      /*
+         For binary logistic regression, when we initialize the weights as zeros,
+         it will converge faster if we initialize the intercept such that
+         it follows the distribution of the labels.
+
+         {{{
+         P(0) = 1 / (1 + \exp(b)), and
+         P(1) = \exp(b) / (1 + \exp(b))
+         }}}, hence
+         {{{
+         b = \log{P(1) / P(0)} = \log{count_1 / count_0}
+         }}}
        */
       initialWeightsWithIntercept.toArray(numFeatures)
         = math.log(histogram(1).toDouble / histogram(0).toDouble)
@@ -186,39 +211,48 @@ class LogisticRegression(override val uid: String)
     val states = optimizer.iterations(new CachedDiffFunction(costFun),
       initialWeightsWithIntercept.toBreeze.toDenseVector)
 
-    var state = states.next()
-    val lossHistory = mutable.ArrayBuilder.make[Double]
+    val (weights, intercept, objectiveHistory) = {
+      /*
+         Note that in Logistic Regression, the objective history (loss + regularization)
+         is log-likelihood which is invariance under feature standardization. As a result,
+         the objective history from optimizer is the same as the one in the original space.
+       */
+      val arrayBuilder = mutable.ArrayBuilder.make[Double]
+      var state: optimizer.State = null
+      while (states.hasNext) {
+        state = states.next()
+        arrayBuilder += state.adjustedValue
+      }
 
-    while (states.hasNext) {
-      lossHistory += state.value
-      state = states.next()
-    }
-    lossHistory += state.value
+      if (state == null) {
+        val msg = s"${optimizer.getClass.getName} failed."
+        logError(msg)
+        throw new SparkException(msg)
+      }
 
-    // The weights are trained in the scaled space; we're converting them back to
-    // the original space.
-    val weightsWithIntercept = {
+      /*
+         The weights are trained in the scaled space; we're converting them back to
+         the original space.
+         Note that the intercept in scaled space and original space is the same;
+         as a result, no scaling is needed.
+       */
       val rawWeights = state.x.toArray.clone()
       var i = 0
-      // Note that the intercept in scaled space and original space is the same;
-      // as a result, no scaling is needed.
       while (i < numFeatures) {
         rawWeights(i) *= { if (featuresStd(i) != 0.0) 1.0 / featuresStd(i) else 0.0 }
         i += 1
       }
-      Vectors.dense(rawWeights)
+
+      if ($(fitIntercept)) {
+        (Vectors.dense(rawWeights.dropRight(1)).compressed, rawWeights.last, arrayBuilder.result())
+      } else {
+        (Vectors.dense(rawWeights).compressed, 0.0, arrayBuilder.result())
+      }
     }
 
     if (handlePersistence) instances.unpersist()
 
-    val (weights, intercept) = if ($(fitIntercept)) {
-      (Vectors.dense(weightsWithIntercept.toArray.slice(0, weightsWithIntercept.size - 1)),
-        weightsWithIntercept(weightsWithIntercept.size - 1))
-    } else {
-      (weightsWithIntercept, 0.0)
-    }
-
-    new LogisticRegressionModel(uid, weights.compressed, intercept)
+    copyValues(new LogisticRegressionModel(uid, weights, intercept))
   }
 
   override def copy(extra: ParamMap): LogisticRegression = defaultCopy(extra)
@@ -423,16 +457,12 @@ private class LogisticAggregator(
     require(dim == data.size, s"Dimensions mismatch when adding new sample." +
       s" Expecting $dim but got ${data.size}.")
 
-    val dataSize = data.size
-
     val localWeightsArray = weightsArray
     val localGradientSumArray = gradientSumArray
 
     numClasses match {
       case 2 =>
-        /**
-         * For Binary Logistic Regression.
-         */
+        // For Binary Logistic Regression.
         val margin = - {
           var sum = 0.0
           data.foreachActive { (index, value) =>
@@ -518,11 +548,13 @@ private class LogisticCostFun(
     data: RDD[(Double, Vector)],
     numClasses: Int,
     fitIntercept: Boolean,
+    standardization: Boolean,
     featuresStd: Array[Double],
     featuresMean: Array[Double],
     regParamL2: Double) extends DiffFunction[BDV[Double]] {
 
   override def calculate(weights: BDV[Double]): (Double, BDV[Double]) = {
+    val numFeatures = featuresStd.length
     val w = Vectors.fromBreeze(weights)
 
     val logisticAggregator = data.treeAggregate(new LogisticAggregator(w, numClasses, fitIntercept,
@@ -534,27 +566,43 @@ private class LogisticCostFun(
           case (aggregator1, aggregator2) => aggregator1.merge(aggregator2)
         })
 
-    // regVal is the sum of weight squares for L2 regularization
-    val norm = if (regParamL2 == 0.0) {
-      0.0
-    } else if (fitIntercept) {
-      brzNorm(Vectors.dense(weights.toArray.slice(0, weights.size -1)).toBreeze, 2.0)
-    } else {
-      brzNorm(weights, 2.0)
-    }
-    val regVal = 0.5 * regParamL2 * norm * norm
-
-    val loss = logisticAggregator.loss + regVal
-    val gradient = logisticAggregator.gradient
+    val totalGradientArray = logisticAggregator.gradient.toArray
 
-    if (fitIntercept) {
-      val wArray = w.toArray.clone()
-      wArray(wArray.length - 1) = 0.0
-      axpy(regParamL2, Vectors.dense(wArray), gradient)
+    // regVal is the sum of weight squares excluding intercept for L2 regularization.
+    val regVal = if (regParamL2 == 0.0) {
+      0.0
     } else {
-      axpy(regParamL2, w, gradient)
+      var sum = 0.0
+      w.foreachActive { (index, value) =>
+        // If `fitIntercept` is true, the last term which is intercept doesn't
+        // contribute to the regularization.
+        if (index != numFeatures) {
+          // The following code will compute the loss of the regularization; also
+          // the gradient of the regularization, and add back to totalGradientArray.
+          sum += {
+            if (standardization) {
+              totalGradientArray(index) += regParamL2 * value
+              value * value
+            } else {
+              if (featuresStd(index) != 0.0) {
+                // If `standardization` is false, we still standardize the data
+                // to improve the rate of convergence; as a result, we have to
+                // perform this reverse standardization by penalizing each component
+                // differently to get effectively the same objective function when
+                // the training dataset is not standardized.
+                val temp = value / (featuresStd(index) * featuresStd(index))
+                totalGradientArray(index) += regParamL2 * temp
+                value * temp
+              } else {
+                0.0
+              }
+            }
+          }
+        }
+      }
+      0.5 * regParamL2 * sum
     }
 
-    (loss, gradient.toBreeze.asInstanceOf[BDV[Double]])
+    (logisticAggregator.loss + regVal, new BDV(totalGradientArray))
   }
 }
diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/CountVectorizerModel.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/CountVectorizerModel.scala
new file mode 100644
index 0000000000000..6b77de89a0330
--- /dev/null
+++ b/mllib/src/main/scala/org/apache/spark/ml/feature/CountVectorizerModel.scala
@@ -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.
+ */
+package org.apache.spark.ml.feature
+
+import scala.collection.mutable
+
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.ml.UnaryTransformer
+import org.apache.spark.ml.param.{ParamMap, ParamValidators, IntParam}
+import org.apache.spark.ml.util.Identifiable
+import org.apache.spark.mllib.linalg.{Vectors, VectorUDT, Vector}
+import org.apache.spark.sql.types.{StringType, ArrayType, DataType}
+
+/**
+ * :: Experimental ::
+ * Converts a text document to a sparse vector of token counts.
+ * @param vocabulary An Array over terms. Only the terms in the vocabulary will be counted.
+ */
+@Experimental
+class CountVectorizerModel (override val uid: String, val vocabulary: Array[String])
+  extends UnaryTransformer[Seq[String], Vector, CountVectorizerModel] {
+
+  def this(vocabulary: Array[String]) =
+    this(Identifiable.randomUID("cntVec"), vocabulary)
+
+  /**
+   * Corpus-specific filter to ignore scarce words in a document. For each document, terms with
+   * frequency (count) less than the given threshold are ignored.
+   * Default: 1
+   * @group param
+   */
+  val minTermFreq: IntParam = new IntParam(this, "minTermFreq",
+    "minimum frequency (count) filter used to neglect scarce words (>= 1). For each document, " +
+      "terms with frequency less than the given threshold are ignored.", ParamValidators.gtEq(1))
+
+  /** @group setParam */
+  def setMinTermFreq(value: Int): this.type = set(minTermFreq, value)
+
+  /** @group getParam */
+  def getMinTermFreq: Int = $(minTermFreq)
+
+  setDefault(minTermFreq -> 1)
+
+  override protected def createTransformFunc: Seq[String] => Vector = {
+    val dict = vocabulary.zipWithIndex.toMap
+    document =>
+      val termCounts = mutable.HashMap.empty[Int, Double]
+      document.foreach { term =>
+        dict.get(term) match {
+          case Some(index) => termCounts.put(index, termCounts.getOrElse(index, 0.0) + 1.0)
+          case None => // ignore terms not in the vocabulary
+        }
+      }
+      Vectors.sparse(dict.size, termCounts.filter(_._2 >= $(minTermFreq)).toSeq)
+  }
+
+  override protected def validateInputType(inputType: DataType): Unit = {
+    require(inputType.sameType(ArrayType(StringType)),
+      s"Input type must be ArrayType(StringType) but got $inputType.")
+  }
+
+  override protected def outputDataType: DataType = new VectorUDT()
+
+  override def copy(extra: ParamMap): CountVectorizerModel = {
+    val copied = new CountVectorizerModel(uid, vocabulary)
+    copyValues(copied, extra)
+  }
+}
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 ca3c1cfb56b7f..72b545e5db3e4 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
@@ -106,6 +106,12 @@ class StandardScalerModel private[ml] (
     scaler: feature.StandardScalerModel)
   extends Model[StandardScalerModel] with StandardScalerParams {
 
+  /** Standard deviation of the StandardScalerModel */
+  val std: Vector = scaler.std
+
+  /** Mean of the StandardScalerModel */
+  val mean: Vector = scaler.mean
+
   /** @group setParam */
   def setInputCol(value: String): this.type = set(inputCol, value)
 
diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/params.scala b/mllib/src/main/scala/org/apache/spark/ml/param/params.scala
index 50c0d855066f8..d034d7ec6b60e 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/param/params.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/param/params.scala
@@ -341,9 +341,7 @@ trait Params extends Identifiable with Serializable {
    * those are checked during schema validation.
    */
   def validateParams(): Unit = {
-    params.filter(isDefined).foreach { param =>
-      param.asInstanceOf[Param[Any]].validate($(param))
-    }
+    // Do nothing by default.  Override to handle Param interactions.
   }
 
   /**
diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala b/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala
index b0a6af171c01f..f7ae1de522e01 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala
@@ -54,8 +54,7 @@ private[shared] object SharedParamsCodeGen {
         isValid = "ParamValidators.gtEq(1)"),
       ParamDesc[Boolean]("fitIntercept", "whether to fit an intercept term", Some("true")),
       ParamDesc[Boolean]("standardization", "whether to standardize the training features" +
-        " prior to fitting the model sequence. Note that the coefficients of models are" +
-        " always returned on the original scale.", Some("true")),
+        " before fitting the model.", Some("true")),
       ParamDesc[Long]("seed", "random seed", Some("this.getClass.getName.hashCode.toLong")),
       ParamDesc[Double]("elasticNetParam", "the ElasticNet mixing parameter, in range [0, 1]." +
         " For alpha = 0, the penalty is an L2 penalty. For alpha = 1, it is an L1 penalty.",
@@ -135,7 +134,7 @@ private[shared] object SharedParamsCodeGen {
 
     s"""
       |/**
-      | * (private[ml]) Trait for shared param $name$defaultValueDoc.
+      | * Trait for shared param $name$defaultValueDoc.
       | */
       |private[ml] trait Has$Name extends Params {
       |
@@ -174,7 +173,6 @@ private[shared] object SharedParamsCodeGen {
         |package org.apache.spark.ml.param.shared
         |
         |import org.apache.spark.ml.param._
-        |import org.apache.spark.util.Utils
         |
         |// DO NOT MODIFY THIS FILE! It was generated by SharedParamsCodeGen.
         |
diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala b/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala
index bbe08939b6d75..65e48e4ee5083 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala
@@ -18,14 +18,13 @@
 package org.apache.spark.ml.param.shared
 
 import org.apache.spark.ml.param._
-import org.apache.spark.util.Utils
 
 // DO NOT MODIFY THIS FILE! It was generated by SharedParamsCodeGen.
 
 // scalastyle:off
 
 /**
- * (private[ml]) Trait for shared param regParam.
+ * Trait for shared param regParam.
  */
 private[ml] trait HasRegParam extends Params {
 
@@ -40,7 +39,7 @@ private[ml] trait HasRegParam extends Params {
 }
 
 /**
- * (private[ml]) Trait for shared param maxIter.
+ * Trait for shared param maxIter.
  */
 private[ml] trait HasMaxIter extends Params {
 
@@ -55,7 +54,7 @@ private[ml] trait HasMaxIter extends Params {
 }
 
 /**
- * (private[ml]) Trait for shared param featuresCol (default: "features").
+ * Trait for shared param featuresCol (default: "features").
  */
 private[ml] trait HasFeaturesCol extends Params {
 
@@ -72,7 +71,7 @@ private[ml] trait HasFeaturesCol extends Params {
 }
 
 /**
- * (private[ml]) Trait for shared param labelCol (default: "label").
+ * Trait for shared param labelCol (default: "label").
  */
 private[ml] trait HasLabelCol extends Params {
 
@@ -89,7 +88,7 @@ private[ml] trait HasLabelCol extends Params {
 }
 
 /**
- * (private[ml]) Trait for shared param predictionCol (default: "prediction").
+ * Trait for shared param predictionCol (default: "prediction").
  */
 private[ml] trait HasPredictionCol extends Params {
 
@@ -106,7 +105,7 @@ private[ml] trait HasPredictionCol extends Params {
 }
 
 /**
- * (private[ml]) Trait for shared param rawPredictionCol (default: "rawPrediction").
+ * Trait for shared param rawPredictionCol (default: "rawPrediction").
  */
 private[ml] trait HasRawPredictionCol extends Params {
 
@@ -123,7 +122,7 @@ private[ml] trait HasRawPredictionCol extends Params {
 }
 
 /**
- * (private[ml]) Trait for shared param probabilityCol (default: "probability").
+ * Trait for shared param probabilityCol (default: "probability").
  */
 private[ml] trait HasProbabilityCol extends Params {
 
@@ -140,7 +139,7 @@ private[ml] trait HasProbabilityCol extends Params {
 }
 
 /**
- * (private[ml]) Trait for shared param threshold.
+ * Trait for shared param threshold.
  */
 private[ml] trait HasThreshold extends Params {
 
@@ -155,7 +154,7 @@ private[ml] trait HasThreshold extends Params {
 }
 
 /**
- * (private[ml]) Trait for shared param inputCol.
+ * Trait for shared param inputCol.
  */
 private[ml] trait HasInputCol extends Params {
 
@@ -170,7 +169,7 @@ private[ml] trait HasInputCol extends Params {
 }
 
 /**
- * (private[ml]) Trait for shared param inputCols.
+ * Trait for shared param inputCols.
  */
 private[ml] trait HasInputCols extends Params {
 
@@ -185,7 +184,7 @@ private[ml] trait HasInputCols extends Params {
 }
 
 /**
- * (private[ml]) Trait for shared param outputCol (default: uid + "__output").
+ * Trait for shared param outputCol (default: uid + "__output").
  */
 private[ml] trait HasOutputCol extends Params {
 
@@ -202,7 +201,7 @@ private[ml] trait HasOutputCol extends Params {
 }
 
 /**
- * (private[ml]) Trait for shared param checkpointInterval.
+ * Trait for shared param checkpointInterval.
  */
 private[ml] trait HasCheckpointInterval extends Params {
 
@@ -217,7 +216,7 @@ private[ml] trait HasCheckpointInterval extends Params {
 }
 
 /**
- * (private[ml]) Trait for shared param fitIntercept (default: true).
+ * Trait for shared param fitIntercept (default: true).
  */
 private[ml] trait HasFitIntercept extends Params {
 
@@ -234,15 +233,15 @@ private[ml] trait HasFitIntercept extends Params {
 }
 
 /**
- * (private[ml]) Trait for shared param standardization (default: true).
+ * Trait for shared param standardization (default: true).
  */
 private[ml] trait HasStandardization extends Params {
 
   /**
-   * Param for whether to standardize the training features prior to fitting the model sequence. Note that the coefficients of models are always returned on the original scale..
+   * Param for whether to standardize the training features before fitting the model..
    * @group param
    */
-  final val standardization: BooleanParam = new BooleanParam(this, "standardization", "whether to standardize the training features prior to fitting the model sequence. Note that the coefficients of models are always returned on the original scale.")
+  final val standardization: BooleanParam = new BooleanParam(this, "standardization", "whether to standardize the training features before fitting the model.")
 
   setDefault(standardization, true)
 
@@ -251,7 +250,7 @@ private[ml] trait HasStandardization extends Params {
 }
 
 /**
- * (private[ml]) Trait for shared param seed (default: this.getClass.getName.hashCode.toLong).
+ * Trait for shared param seed (default: this.getClass.getName.hashCode.toLong).
  */
 private[ml] trait HasSeed extends Params {
 
@@ -268,7 +267,7 @@ private[ml] trait HasSeed extends Params {
 }
 
 /**
- * (private[ml]) Trait for shared param elasticNetParam.
+ * Trait for shared param elasticNetParam.
  */
 private[ml] trait HasElasticNetParam extends Params {
 
@@ -283,7 +282,7 @@ private[ml] trait HasElasticNetParam extends Params {
 }
 
 /**
- * (private[ml]) Trait for shared param tol.
+ * Trait for shared param tol.
  */
 private[ml] trait HasTol extends Params {
 
@@ -298,7 +297,7 @@ private[ml] trait HasTol extends Params {
 }
 
 /**
- * (private[ml]) Trait for shared param stepSize.
+ * Trait for shared param stepSize.
  */
 private[ml] trait HasStepSize extends Params {
 
diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala
index 1b1d7299fb496..8fc986056657d 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala
@@ -22,18 +22,20 @@ import scala.collection.mutable
 import breeze.linalg.{DenseVector => BDV, norm => brzNorm}
 import breeze.optimize.{CachedDiffFunction, DiffFunction, LBFGS => BreezeLBFGS, OWLQN => BreezeOWLQN}
 
-import org.apache.spark.Logging
+import org.apache.spark.{Logging, SparkException}
 import org.apache.spark.annotation.Experimental
 import org.apache.spark.ml.PredictorParams
 import org.apache.spark.ml.param.ParamMap
 import org.apache.spark.ml.param.shared._
 import org.apache.spark.ml.util.Identifiable
+import org.apache.spark.mllib.evaluation.RegressionMetrics
 import org.apache.spark.mllib.linalg.{Vector, Vectors}
 import org.apache.spark.mllib.linalg.BLAS._
 import org.apache.spark.mllib.regression.LabeledPoint
 import org.apache.spark.mllib.stat.MultivariateOnlineSummarizer
 import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.DataFrame
+import org.apache.spark.sql.{DataFrame, Row}
+import org.apache.spark.sql.functions.{col, udf}
 import org.apache.spark.storage.StorageLevel
 import org.apache.spark.util.StatCounter
 
@@ -132,7 +134,6 @@ class LinearRegression(override val uid: String)
     val numFeatures = summarizer.mean.size
     val yMean = statCounter.mean
     val yStd = math.sqrt(statCounter.variance)
-    // look at glmnet5.m L761 maaaybe that has info
 
     // If the yStd is zero, then the intercept is yMean with zero weights;
     // as a result, training is not needed.
@@ -140,7 +141,16 @@ class LinearRegression(override val uid: String)
       logWarning(s"The standard deviation of the label is zero, so the weights will be zeros " +
         s"and the intercept will be the mean of the label; as a result, training is not needed.")
       if (handlePersistence) instances.unpersist()
-      return new LinearRegressionModel(uid, Vectors.sparse(numFeatures, Seq()), yMean)
+      val weights = Vectors.sparse(numFeatures, Seq())
+      val intercept = yMean
+
+      val model = new LinearRegressionModel(uid, weights, intercept)
+      val trainingSummary = new LinearRegressionTrainingSummary(
+        model.transform(dataset).select($(predictionCol), $(labelCol)),
+        $(predictionCol),
+        $(labelCol),
+        Array(0D))
+      return copyValues(model.setSummary(trainingSummary))
     }
 
     val featuresMean = summarizer.mean.toArray
@@ -162,21 +172,33 @@ class LinearRegression(override val uid: String)
     }
 
     val initialWeights = Vectors.zeros(numFeatures)
-    val states =
-      optimizer.iterations(new CachedDiffFunction(costFun), initialWeights.toBreeze.toDenseVector)
-
-    var state = states.next()
-    val lossHistory = mutable.ArrayBuilder.make[Double]
-
-    while (states.hasNext) {
-      lossHistory += state.value
-      state = states.next()
-    }
-    lossHistory += state.value
+    val states = optimizer.iterations(new CachedDiffFunction(costFun),
+      initialWeights.toBreeze.toDenseVector)
+
+    val (weights, objectiveHistory) = {
+      /*
+         Note that in Linear Regression, the objective history (loss + regularization) returned
+         from optimizer is computed in the scaled space given by the following formula.
+         {{{
+         L = 1/2n||\sum_i w_i(x_i - \bar{x_i}) / \hat{x_i} - (y - \bar{y}) / \hat{y}||^2 + regTerms
+         }}}
+       */
+      val arrayBuilder = mutable.ArrayBuilder.make[Double]
+      var state: optimizer.State = null
+      while (states.hasNext) {
+        state = states.next()
+        arrayBuilder += state.adjustedValue
+      }
+      if (state == null) {
+        val msg = s"${optimizer.getClass.getName} failed."
+        logError(msg)
+        throw new SparkException(msg)
+      }
 
-    // The weights are trained in the scaled space; we're converting them back to
-    // the original space.
-    val weights = {
+      /*
+         The weights are trained in the scaled space; we're converting them back to
+         the original space.
+       */
       val rawWeights = state.x.toArray.clone()
       var i = 0
       val len = rawWeights.length
@@ -184,17 +206,26 @@ class LinearRegression(override val uid: String)
         rawWeights(i) *= { if (featuresStd(i) != 0.0) yStd / featuresStd(i) else 0.0 }
         i += 1
       }
-      Vectors.dense(rawWeights)
+
+      (Vectors.dense(rawWeights).compressed, arrayBuilder.result())
     }
 
-    // The intercept in R's GLMNET is computed using closed form after the coefficients are
-    // converged. See the following discussion for detail.
-    // http://stats.stackexchange.com/questions/13617/how-is-the-intercept-computed-in-glmnet
+    /*
+       The intercept in R's GLMNET is computed using closed form after the coefficients are
+       converged. See the following discussion for detail.
+       http://stats.stackexchange.com/questions/13617/how-is-the-intercept-computed-in-glmnet
+     */
     val intercept = if ($(fitIntercept)) yMean - dot(weights, Vectors.dense(featuresMean)) else 0.0
+
     if (handlePersistence) instances.unpersist()
 
-    // TODO: Converts to sparse format based on the storage, but may base on the scoring speed.
-    copyValues(new LinearRegressionModel(uid, weights.compressed, intercept))
+    val model = copyValues(new LinearRegressionModel(uid, weights, intercept))
+    val trainingSummary = new LinearRegressionTrainingSummary(
+      model.transform(dataset).select($(predictionCol), $(labelCol)),
+      $(predictionCol),
+      $(labelCol),
+      objectiveHistory)
+    model.setSummary(trainingSummary)
   }
 
   override def copy(extra: ParamMap): LinearRegression = defaultCopy(extra)
@@ -212,13 +243,124 @@ class LinearRegressionModel private[ml] (
   extends RegressionModel[Vector, LinearRegressionModel]
   with LinearRegressionParams {
 
+  private var trainingSummary: Option[LinearRegressionTrainingSummary] = None
+
+  /**
+   * Gets summary (e.g. residuals, mse, r-squared ) of model on training set. An exception is
+   * thrown if `trainingSummary == None`.
+   */
+  def summary: LinearRegressionTrainingSummary = trainingSummary match {
+    case Some(summ) => summ
+    case None =>
+      throw new SparkException(
+        "No training summary available for this LinearRegressionModel",
+        new NullPointerException())
+  }
+
+  private[regression] def setSummary(summary: LinearRegressionTrainingSummary): this.type = {
+    this.trainingSummary = Some(summary)
+    this
+  }
+
+  /** Indicates whether a training summary exists for this model instance. */
+  def hasSummary: Boolean = trainingSummary.isDefined
+
+  /**
+   * Evaluates the model on a testset.
+   * @param dataset Test dataset to evaluate model on.
+   */
+  // TODO: decide on a good name before exposing to public API
+  private[regression] def evaluate(dataset: DataFrame): LinearRegressionSummary = {
+    val t = udf { features: Vector => predict(features) }
+    val predictionAndObservations = dataset
+      .select(col($(labelCol)), t(col($(featuresCol))).as($(predictionCol)))
+
+    new LinearRegressionSummary(predictionAndObservations, $(predictionCol), $(labelCol))
+  }
+
   override protected def predict(features: Vector): Double = {
     dot(features, weights) + intercept
   }
 
   override def copy(extra: ParamMap): LinearRegressionModel = {
-    copyValues(new LinearRegressionModel(uid, weights, intercept), extra)
+    val newModel = copyValues(new LinearRegressionModel(uid, weights, intercept))
+    if (trainingSummary.isDefined) newModel.setSummary(trainingSummary.get)
+    newModel
+  }
+}
+
+/**
+ * :: Experimental ::
+ * Linear regression training results.
+ * @param predictions predictions outputted by the model's `transform` method.
+ * @param objectiveHistory objective function (scaled loss + regularization) at each iteration.
+ */
+@Experimental
+class LinearRegressionTrainingSummary private[regression] (
+    predictions: DataFrame,
+    predictionCol: String,
+    labelCol: String,
+    val objectiveHistory: Array[Double])
+  extends LinearRegressionSummary(predictions, predictionCol, labelCol) {
+
+  /** Number of training iterations until termination */
+  val totalIterations = objectiveHistory.length
+
+}
+
+/**
+ * :: Experimental ::
+ * Linear regression results evaluated on a dataset.
+ * @param predictions predictions outputted by the model's `transform` method.
+ */
+@Experimental
+class LinearRegressionSummary private[regression] (
+    @transient val predictions: DataFrame,
+    val predictionCol: String,
+    val labelCol: String) extends Serializable {
+
+  @transient private val metrics = new RegressionMetrics(
+    predictions
+      .select(predictionCol, labelCol)
+      .map { case Row(pred: Double, label: Double) => (pred, label) } )
+
+  /**
+   * Returns the explained variance regression score.
+   * explainedVariance = 1 - variance(y - \hat{y}) / variance(y)
+   * Reference: [[http://en.wikipedia.org/wiki/Explained_variation]]
+   */
+  val explainedVariance: Double = metrics.explainedVariance
+
+  /**
+   * Returns the mean absolute error, which is a risk function corresponding to the
+   * expected value of the absolute error loss or l1-norm loss.
+   */
+  val meanAbsoluteError: Double = metrics.meanAbsoluteError
+
+  /**
+   * Returns the mean squared error, which is a risk function corresponding to the
+   * expected value of the squared error loss or quadratic loss.
+   */
+  val meanSquaredError: Double = metrics.meanSquaredError
+
+  /**
+   * Returns the root mean squared error, which is defined as the square root of
+   * the mean squared error.
+   */
+  val rootMeanSquaredError: Double = metrics.rootMeanSquaredError
+
+  /**
+   * Returns R^2^, the coefficient of determination.
+   * Reference: [[http://en.wikipedia.org/wiki/Coefficient_of_determination]]
+   */
+  val r2: Double = metrics.r2
+
+  /** Residuals (predicted value - label value) */
+  @transient lazy val residuals: DataFrame = {
+    val t = udf { (pred: Double, label: Double) => pred - label}
+    predictions.select(t(col(predictionCol), col(labelCol)).as("residuals"))
   }
+
 }
 
 /**
diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/treeModels.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/treeModels.scala
index 1929f9d02156e..22873909c33fa 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/tree/treeModels.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/tree/treeModels.scala
@@ -17,6 +17,7 @@
 
 package org.apache.spark.ml.tree
 
+import org.apache.spark.mllib.linalg.{Vectors, Vector}
 
 /**
  * Abstraction for Decision Tree models.
@@ -70,6 +71,10 @@ private[ml] trait TreeEnsembleModel {
   /** Weights for each tree, zippable with [[trees]] */
   def treeWeights: Array[Double]
 
+  /** Weights used by the python wrappers. */
+  // Note: An array cannot be returned directly due to serialization problems.
+  private[spark] def javaTreeWeights: Vector = Vectors.dense(treeWeights)
+
   /** Summary of the model */
   override def toString: String = {
     // Implementing classes should generally override this method to be more descriptive.
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 f51ee36d0dfcb..9e379d7d74b2f 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,26 +93,70 @@ class NaiveBayesModel private[mllib] (
   override def predict(testData: Vector): Double = {
     modelType match {
       case Multinomial =>
-        val prob = thetaMatrix.multiply(testData)
-        BLAS.axpy(1.0, piVector, prob)
-        labels(prob.argmax)
+        labels(multinomialCalculation(testData).argmax)
       case Bernoulli =>
-        testData.foreachActive { (index, value) =>
-          if (value != 0.0 && value != 1.0) {
-            throw new SparkException(
-              s"Bernoulli naive Bayes requires 0 or 1 feature values but found $testData.")
-          }
-        }
-        val prob = thetaMinusNegTheta.get.multiply(testData)
-        BLAS.axpy(1.0, piVector, prob)
-        BLAS.axpy(1.0, negThetaSum.get, prob)
-        labels(prob.argmax)
-      case _ =>
-        // This should never happen.
-        throw new UnknownError(s"Invalid modelType: $modelType.")
+        labels(bernoulliCalculation(testData).argmax)
+    }
+  }
+
+  /**
+   * Predict values for the given data set using the model trained.
+   *
+   * @param testData RDD representing data points to be predicted
+   * @return an RDD[Vector] where each entry contains the predicted posterior class probabilities,
+   *         in the same order as class labels
+   */
+  def predictProbabilities(testData: RDD[Vector]): RDD[Vector] = {
+    val bcModel = testData.context.broadcast(this)
+    testData.mapPartitions { iter =>
+      val model = bcModel.value
+      iter.map(model.predictProbabilities)
     }
   }
 
+  /**
+   * Predict posterior class probabilities for a single data point using the model trained.
+   *
+   * @param testData array representing a single data point
+   * @return predicted posterior class probabilities from the trained model,
+   *         in the same order as class labels
+   */
+  def predictProbabilities(testData: Vector): Vector = {
+    modelType match {
+      case Multinomial =>
+        posteriorProbabilities(multinomialCalculation(testData))
+      case Bernoulli =>
+        posteriorProbabilities(bernoulliCalculation(testData))
+    }
+  }
+
+  private def multinomialCalculation(testData: Vector) = {
+    val prob = thetaMatrix.multiply(testData)
+    BLAS.axpy(1.0, piVector, prob)
+    prob
+  }
+
+  private def bernoulliCalculation(testData: Vector) = {
+    testData.foreachActive((_, value) =>
+      if (value != 0.0 && value != 1.0) {
+        throw new SparkException(
+          s"Bernoulli naive Bayes requires 0 or 1 feature values but found $testData.")
+      }
+    )
+    val prob = thetaMinusNegTheta.get.multiply(testData)
+    BLAS.axpy(1.0, piVector, prob)
+    BLAS.axpy(1.0, negThetaSum.get, prob)
+    prob
+  }
+
+  private def posteriorProbabilities(logProb: DenseVector) = {
+    val logProbArray = logProb.toArray
+    val maxLog = logProbArray.max
+    val scaledProbs = logProbArray.map(lp => math.exp(lp - maxLog))
+    val probSum = scaledProbs.sum
+    new DenseVector(scaledProbs.map(_ / probSum))
+  }
+
   override def save(sc: SparkContext, path: String): Unit = {
     val data = NaiveBayesModel.SaveLoadV2_0.Data(labels, pi, theta, modelType)
     NaiveBayesModel.SaveLoadV2_0.save(sc, path, data)
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixture.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixture.scala
index fc509d2ba1470..e459367333d26 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixture.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixture.scala
@@ -140,6 +140,10 @@ class GaussianMixture private (
     // Get length of the input vectors
     val d = breezeData.first().length
 
+    // Heuristic to distribute the computation of the [[MultivariateGaussian]]s, approximately when
+    // d > 25 except for when k is very small
+    val distributeGaussians = ((k - 1.0) / k) * d > 25
+
     // 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
@@ -171,14 +175,25 @@ class GaussianMixture private (
       // 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),
-          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
+
+      if (distributeGaussians) {
+        val numPartitions = math.min(k, 1024)
+        val tuples =
+          Seq.tabulate(k)(i => (sums.means(i), sums.sigmas(i), sums.weights(i)))
+        val (ws, gs) = sc.parallelize(tuples, numPartitions).map { case (mean, sigma, weight) =>
+          updateWeightsAndGaussians(mean, sigma, weight, sumWeights)
+        }.collect.unzip
+        Array.copy(ws, 0, weights, 0, ws.length)
+        Array.copy(gs, 0, gaussians, 0, gs.length)
+      } else {
+        var i = 0
+        while (i < k) {
+          val (weight, gaussian) =
+            updateWeightsAndGaussians(sums.means(i), sums.sigmas(i), sums.weights(i), sumWeights)
+          weights(i) = weight
+          gaussians(i) = gaussian
+          i = i + 1
+        }
       }
 
       llhp = llh // current becomes previous
@@ -192,6 +207,19 @@ class GaussianMixture private (
   /** Java-friendly version of [[run()]] */
   def run(data: JavaRDD[Vector]): GaussianMixtureModel = run(data.rdd)
 
+  private def updateWeightsAndGaussians(
+      mean: BDV[Double],
+      sigma: BreezeMatrix[Double],
+      weight: Double,
+      sumWeights: Double): (Double, MultivariateGaussian) = {
+    val mu = (mean /= weight)
+    BLAS.syr(-weight, Vectors.fromBreeze(mu),
+      Matrices.fromBreeze(sigma).asInstanceOf[DenseMatrix])
+    val newWeight = weight / sumWeights
+    val newGaussian = new MultivariateGaussian(mu, sigma / weight)
+    (newWeight, newGaussian)
+  }
+
   /** Average of dense breeze vectors */
   private def vectorMean(x: IndexedSeq[BV[Double]]): BDV[Double] = {
     val v = BDV.zeros[Double](x(0).length)
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala
new file mode 100644
index 0000000000000..72d0ea0c12e1e
--- /dev/null
+++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.mllib.fpm
+
+import scala.reflect.ClassTag
+
+import org.apache.spark.Logging
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.api.java.JavaRDD
+import org.apache.spark.api.java.JavaSparkContext.fakeClassTag
+import org.apache.spark.mllib.fpm.AssociationRules.Rule
+import org.apache.spark.mllib.fpm.FPGrowth.FreqItemset
+import org.apache.spark.rdd.RDD
+
+/**
+ * :: Experimental ::
+ *
+ * Generates association rules from a [[RDD[FreqItemset[Item]]]. This method only generates
+ * association rules which have a single item as the consequent.
+ *
+ * @since 1.5.0
+ */
+@Experimental
+class AssociationRules private[fpm] (
+    private var minConfidence: Double) extends Logging with Serializable {
+
+  /**
+   * Constructs a default instance with default parameters {minConfidence = 0.8}.
+   *
+   * @since 1.5.0
+   */
+  def this() = this(0.8)
+
+  /**
+   * Sets the minimal confidence (default: `0.8`).
+   *
+   * @since 1.5.0
+   */
+  def setMinConfidence(minConfidence: Double): this.type = {
+    require(minConfidence >= 0.0 && minConfidence <= 1.0)
+    this.minConfidence = minConfidence
+    this
+  }
+
+  /**
+   * Computes the association rules with confidence above [[minConfidence]].
+   * @param freqItemsets frequent itemset model obtained from [[FPGrowth]]
+   * @return a [[Set[Rule[Item]]] containing the assocation rules.
+   *
+   * @since 1.5.0
+   */
+  def run[Item: ClassTag](freqItemsets: RDD[FreqItemset[Item]]): RDD[Rule[Item]] = {
+    // For candidate rule X => Y, generate (X, (Y, freq(X union Y)))
+    val candidates = freqItemsets.flatMap { itemset =>
+      val items = itemset.items
+      items.flatMap { item =>
+        items.partition(_ == item) match {
+          case (consequent, antecedent) if !antecedent.isEmpty =>
+            Some((antecedent.toSeq, (consequent.toSeq, itemset.freq)))
+          case _ => None
+        }
+      }
+    }
+
+    // Join to get (X, ((Y, freq(X union Y)), freq(X))), generate rules, and filter by confidence
+    candidates.join(freqItemsets.map(x => (x.items.toSeq, x.freq)))
+      .map { case (antecendent, ((consequent, freqUnion), freqAntecedent)) =>
+      new Rule(antecendent.toArray, consequent.toArray, freqUnion, freqAntecedent)
+    }.filter(_.confidence >= minConfidence)
+  }
+
+  def run[Item](freqItemsets: JavaRDD[FreqItemset[Item]]): JavaRDD[Rule[Item]] = {
+    val tag = fakeClassTag[Item]
+    run(freqItemsets.rdd)(tag)
+  }
+}
+
+object AssociationRules {
+
+  /**
+   * :: Experimental ::
+   *
+   * An association rule between sets of items.
+   * @param antecedent hypotheses of the rule
+   * @param consequent conclusion of the rule
+   * @tparam Item item type
+   *
+   * @since 1.5.0
+   */
+  @Experimental
+  class Rule[Item] private[fpm] (
+      val antecedent: Array[Item],
+      val consequent: Array[Item],
+      freqUnion: Double,
+      freqAntecedent: Double) extends Serializable {
+
+    def confidence: Double = freqUnion.toDouble / freqAntecedent
+
+    require(antecedent.toSet.intersect(consequent.toSet).isEmpty, {
+      val sharedItems = antecedent.toSet.intersect(consequent.toSet)
+      s"A valid association rule must have disjoint antecedent and " +
+        s"consequent but ${sharedItems} is present in both."
+    })
+  }
+}
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala
index abac08022ea47..e2370a52f4930 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala
@@ -28,7 +28,7 @@ import org.apache.spark.{HashPartitioner, Logging, Partitioner, SparkException}
 import org.apache.spark.annotation.Experimental
 import org.apache.spark.api.java.JavaRDD
 import org.apache.spark.api.java.JavaSparkContext.fakeClassTag
-import org.apache.spark.mllib.fpm.FPGrowth.FreqItemset
+import org.apache.spark.mllib.fpm.FPGrowth._
 import org.apache.spark.rdd.RDD
 import org.apache.spark.storage.StorageLevel
 
@@ -36,11 +36,23 @@ import org.apache.spark.storage.StorageLevel
  * :: Experimental ::
  *
  * Model trained by [[FPGrowth]], which holds frequent itemsets.
- * @param freqItemsets frequent itemsets, which is an RDD of [[FreqItemset]]
+ * @param freqItemsets frequent itemset, which is an RDD of [[FreqItemset]]
  * @tparam Item item type
+ *
+ * @since 1.3.0
  */
 @Experimental
-class FPGrowthModel[Item: ClassTag](val freqItemsets: RDD[FreqItemset[Item]]) extends Serializable
+class FPGrowthModel[Item: ClassTag](val freqItemsets: RDD[FreqItemset[Item]]) extends Serializable {
+  /**
+   * Generates association rules for the [[Item]]s in [[freqItemsets]].
+   * @param confidence minimal confidence of the rules produced
+   * @since 1.5.0
+   */
+  def generateAssociationRules(confidence: Double): RDD[AssociationRules.Rule[Item]] = {
+    val associationRules = new AssociationRules(confidence)
+    associationRules.run(freqItemsets)
+  }
+}
 
 /**
  * :: Experimental ::
@@ -58,21 +70,26 @@ class FPGrowthModel[Item: ClassTag](val freqItemsets: RDD[FreqItemset[Item]]) ex
  *
  * @see [[http://en.wikipedia.org/wiki/Association_rule_learning Association rule learning
  *       (Wikipedia)]]
+ *
+ * @since 1.3.0
  */
 @Experimental
 class FPGrowth private (
     private var minSupport: Double,
-    private var numPartitions: Int,
-    private var ordered: Boolean) extends Logging with Serializable {
+    private var numPartitions: Int) extends Logging with Serializable {
 
   /**
    * Constructs a default instance with default parameters {minSupport: `0.3`, numPartitions: same
-   * as the input data, ordered: `false`}.
+   * as the input data}.
+   *
+   * @since 1.3.0
    */
-  def this() = this(0.3, -1, false)
+  def this() = this(0.3, -1)
 
   /**
    * Sets the minimal support level (default: `0.3`).
+   *
+   * @since 1.3.0
    */
   def setMinSupport(minSupport: Double): this.type = {
     this.minSupport = minSupport
@@ -81,25 +98,20 @@ class FPGrowth private (
 
   /**
    * Sets the number of partitions used by parallel FP-growth (default: same as input data).
+   *
+   * @since 1.3.0
    */
   def setNumPartitions(numPartitions: Int): this.type = {
     this.numPartitions = numPartitions
     this
   }
 
-  /**
-   * Indicates whether to mine itemsets (unordered) or sequences (ordered) (default: false, mine
-   * itemsets).
-   */
-  def setOrdered(ordered: Boolean): this.type = {
-    this.ordered = ordered
-    this
-  }
-
   /**
    * Computes an FP-Growth model that contains frequent itemsets.
    * @param data input data set, each element contains a transaction
    * @return an [[FPGrowthModel]]
+   *
+   * @since 1.3.0
    */
   def run[Item: ClassTag](data: RDD[Array[Item]]): FPGrowthModel[Item] = {
     if (data.getStorageLevel == StorageLevel.NONE) {
@@ -165,7 +177,7 @@ class FPGrowth private (
     .flatMap { case (part, tree) =>
       tree.extract(minCount, x => partitioner.getPartition(x) == part)
     }.map { case (ranks, count) =>
-      new FreqItemset(ranks.map(i => freqItems(i)).reverse.toArray, count, ordered)
+      new FreqItemset(ranks.map(i => freqItems(i)).toArray, count)
     }
   }
 
@@ -181,12 +193,9 @@ class FPGrowth private (
       itemToRank: Map[Item, Int],
       partitioner: Partitioner): mutable.Map[Int, Array[Int]] = {
     val output = mutable.Map.empty[Int, Array[Int]]
-    // Filter the basket by frequent items pattern
+    // Filter the basket by frequent items pattern and sort their ranks.
     val filtered = transaction.flatMap(itemToRank.get)
-    if (!this.ordered) {
-      ju.Arrays.sort(filtered)
-    }
-    // Generate conditional transactions
+    ju.Arrays.sort(filtered)
     val n = filtered.length
     var i = n - 1
     while (i >= 0) {
@@ -203,6 +212,8 @@ class FPGrowth private (
 
 /**
  * :: Experimental ::
+ *
+ * @since 1.3.0
  */
 @Experimental
 object FPGrowth {
@@ -211,21 +222,16 @@ object FPGrowth {
    * Frequent itemset.
    * @param items items in this itemset. Java users should call [[FreqItemset#javaItems]] instead.
    * @param freq frequency
-   * @param ordered indicates if items represents an itemset (false) or sequence (true)
    * @tparam Item item type
+   *
+   * @since 1.3.0
    */
-  class FreqItemset[Item](val items: Array[Item], val freq: Long, val ordered: Boolean)
-    extends Serializable {
-
-    /**
-     * Auxillary constructor, assumes unordered by default.
-     */
-    def this(items: Array[Item], freq: Long) {
-      this(items, freq, false)
-    }
+  class FreqItemset[Item](val items: Array[Item], val freq: Long) extends Serializable {
 
     /**
      * Returns items in a Java List.
+     *
+     * @since 1.3.0
      */
     def javaItems: java.util.List[Item] = {
       items.toList.asJava
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 75e7004464af9..0df07663405a3 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
@@ -24,9 +24,9 @@ import scala.collection.mutable.{ArrayBuilder => MArrayBuilder, HashSet => MHash
 import breeze.linalg.{CSCMatrix => BSM, DenseMatrix => BDM, Matrix => BM}
 
 import org.apache.spark.annotation.DeveloperApi
-import org.apache.spark.sql.Row
-import org.apache.spark.sql.types._
 import org.apache.spark.sql.catalyst.expressions.GenericMutableRow
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.types._
 
 /**
  * Trait for a local matrix.
@@ -147,7 +147,7 @@ private[spark] class MatrixUDT extends UserDefinedType[Matrix] {
       ))
   }
 
-  override def serialize(obj: Any): Row = {
+  override def serialize(obj: Any): InternalRow = {
     val row = new GenericMutableRow(7)
     obj match {
       case sm: SparseMatrix =>
@@ -173,9 +173,7 @@ private[spark] class MatrixUDT extends UserDefinedType[Matrix] {
 
   override def deserialize(datum: Any): Matrix = {
     datum match {
-      // TODO: something wrong with UDT serialization, should never happen.
-      case m: Matrix => m
-      case row: Row =>
+      case row: InternalRow =>
         require(row.length == 7,
           s"MatrixUDT.deserialize given row with length ${row.length} but requires length == 7")
         val tpe = row.getByte(0)
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 c9c27425d2877..e048b01d92462 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
@@ -28,7 +28,7 @@ import breeze.linalg.{DenseVector => BDV, SparseVector => BSV, Vector => BV}
 import org.apache.spark.SparkException
 import org.apache.spark.annotation.DeveloperApi
 import org.apache.spark.mllib.util.NumericParser
-import org.apache.spark.sql.Row
+import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.expressions.GenericMutableRow
 import org.apache.spark.sql.types._
 
@@ -175,7 +175,7 @@ private[spark] class VectorUDT extends UserDefinedType[Vector] {
       StructField("values", ArrayType(DoubleType, containsNull = false), nullable = true)))
   }
 
-  override def serialize(obj: Any): Row = {
+  override def serialize(obj: Any): InternalRow = {
     obj match {
       case SparseVector(size, indices, values) =>
         val row = new GenericMutableRow(4)
@@ -191,17 +191,12 @@ private[spark] class VectorUDT extends UserDefinedType[Vector] {
         row.setNullAt(2)
         row.update(3, values.toSeq)
         row
-      // TODO: There are bugs in UDT serialization because we don't have a clear separation between
-      // TODO: internal SQL types and language specific types (including UDT). UDT serialize and
-      // TODO: deserialize may get called twice. See SPARK-7186.
-      case row: Row =>
-        row
     }
   }
 
   override def deserialize(datum: Any): Vector = {
     datum match {
-      case row: Row =>
+      case row: InternalRow =>
         require(row.length == 4,
           s"VectorUDT.deserialize given row with length ${row.length} but requires length == 4")
         val tpe = row.getByte(0)
@@ -215,11 +210,6 @@ private[spark] class VectorUDT extends UserDefinedType[Vector] {
             val values = row.getAs[Iterable[Double]](3).toArray
             new DenseVector(values)
         }
-      // TODO: There are bugs in UDT serialization because we don't have a clear separation between
-      // TODO: internal SQL types and language specific types (including UDT). UDT serialize and
-      // TODO: deserialize may get called twice. See SPARK-7186.
-      case v: Vector =>
-        v
     }
   }
 
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/Statistics.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/Statistics.scala
index 900007ec6bc74..90332028cfb3a 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/stat/Statistics.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/Statistics.scala
@@ -17,13 +17,16 @@
 
 package org.apache.spark.mllib.stat
 
+import scala.annotation.varargs
+
 import org.apache.spark.annotation.Experimental
 import org.apache.spark.api.java.JavaRDD
 import org.apache.spark.mllib.linalg.distributed.RowMatrix
 import org.apache.spark.mllib.linalg.{Matrix, Vector}
 import org.apache.spark.mllib.regression.LabeledPoint
 import org.apache.spark.mllib.stat.correlation.Correlations
-import org.apache.spark.mllib.stat.test.{ChiSqTest, ChiSqTestResult}
+import org.apache.spark.mllib.stat.test.{ChiSqTest, ChiSqTestResult, KolmogorovSmirnovTest,
+  KolmogorovSmirnovTestResult}
 import org.apache.spark.rdd.RDD
 
 /**
@@ -158,4 +161,39 @@ object Statistics {
   def chiSqTest(data: RDD[LabeledPoint]): Array[ChiSqTestResult] = {
     ChiSqTest.chiSquaredFeatures(data)
   }
+
+  /**
+   * Conduct the two-sided Kolmogorov-Smirnov (KS) test for data sampled from a
+   * continuous distribution. By comparing the largest difference between the empirical cumulative
+   * distribution of the sample data and the theoretical distribution we can provide a test for the
+   * the null hypothesis that the sample data comes from that theoretical distribution.
+   * For more information on KS Test:
+   * @see [[https://en.wikipedia.org/wiki/Kolmogorov%E2%80%93Smirnov_test]]
+   *
+   * @param data an `RDD[Double]` containing the sample of data to test
+   * @param cdf a `Double => Double` function to calculate the theoretical CDF at a given value
+   * @return [[org.apache.spark.mllib.stat.test.KolmogorovSmirnovTestResult]] object containing test
+   *        statistic, p-value, and null hypothesis.
+   */
+  def kolmogorovSmirnovTest(data: RDD[Double], cdf: Double => Double)
+    : KolmogorovSmirnovTestResult = {
+    KolmogorovSmirnovTest.testOneSample(data, cdf)
+  }
+
+  /**
+   * Convenience function to conduct a one-sample, two-sided Kolmogorov-Smirnov test for probability
+   * distribution equality. Currently supports the normal distribution, taking as parameters
+   * the mean and standard deviation.
+   * (distName = "norm")
+   * @param data an `RDD[Double]` containing the sample of data to test
+   * @param distName a `String` name for a theoretical distribution
+   * @param params `Double*` specifying the parameters to be used for the theoretical distribution
+   * @return [[org.apache.spark.mllib.stat.test.KolmogorovSmirnovTestResult]] object containing test
+   *        statistic, p-value, and null hypothesis.
+   */
+  @varargs
+  def kolmogorovSmirnovTest(data: RDD[Double], distName: String, params: Double*)
+    : KolmogorovSmirnovTestResult = {
+    KolmogorovSmirnovTest.testOneSample(data, distName, params: _*)
+  }
 }
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/KolmogorovSmirnovTest.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/KolmogorovSmirnovTest.scala
new file mode 100644
index 0000000000000..d89b0059d83f3
--- /dev/null
+++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/KolmogorovSmirnovTest.scala
@@ -0,0 +1,194 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.mllib.stat.test
+
+import scala.annotation.varargs
+
+import org.apache.commons.math3.distribution.{NormalDistribution, RealDistribution}
+import org.apache.commons.math3.stat.inference.KolmogorovSmirnovTest
+
+import org.apache.spark.Logging
+import org.apache.spark.rdd.RDD
+
+/**
+ * Conduct the two-sided Kolmogorov Smirnov (KS) test for data sampled from a
+ * continuous distribution. By comparing the largest difference between the empirical cumulative
+ * distribution of the sample data and the theoretical distribution we can provide a test for the
+ * the null hypothesis that the sample data comes from that theoretical distribution.
+ * For more information on KS Test:
+ * @see [[https://en.wikipedia.org/wiki/Kolmogorov%E2%80%93Smirnov_test]]
+ *
+ * Implementation note: We seek to implement the KS test with a minimal number of distributed
+ * passes. We sort the RDD, and then perform the following operations on a per-partition basis:
+ * calculate an empirical cumulative distribution value for each observation, and a theoretical
+ * cumulative distribution value. We know the latter to be correct, while the former will be off by
+ * a constant (how large the constant is depends on how many values precede it in other partitions).
+ * However, given that this constant simply shifts the empirical CDF upwards, but doesn't
+ * change its shape, and furthermore, that constant is the same within a given partition, we can
+ * pick 2 values in each partition that can potentially resolve to the largest global distance.
+ * Namely, we pick the minimum distance and the maximum distance. Additionally, we keep track of how
+ * many elements are in each partition. Once these three values have been returned for every
+ * partition, we can collect and operate locally. Locally, we can now adjust each distance by the
+ * appropriate constant (the cumulative sum of number of elements in the prior partitions divided by
+ * thedata set size). Finally, we take the maximum absolute value, and this is the statistic.
+ */
+private[stat] object KolmogorovSmirnovTest extends Logging {
+
+  // Null hypothesis for the type of KS test to be included in the result.
+  object NullHypothesis extends Enumeration {
+    type NullHypothesis = Value
+    val OneSampleTwoSided = Value("Sample follows theoretical distribution")
+  }
+
+  /**
+   * Runs a KS test for 1 set of sample data, comparing it to a theoretical distribution
+   * @param data `RDD[Double]` data on which to run test
+   * @param cdf `Double => Double` function to calculate the theoretical CDF
+   * @return [[org.apache.spark.mllib.stat.test.KolmogorovSmirnovTestResult]] summarizing the test
+   *        results (p-value, statistic, and null hypothesis)
+   */
+  def testOneSample(data: RDD[Double], cdf: Double => Double): KolmogorovSmirnovTestResult = {
+    val n = data.count().toDouble
+    val localData = data.sortBy(x => x).mapPartitions { part =>
+      val partDiffs = oneSampleDifferences(part, n, cdf) // local distances
+      searchOneSampleCandidates(partDiffs) // candidates: local extrema
+    }.collect()
+    val ksStat = searchOneSampleStatistic(localData, n) // result: global extreme
+    evalOneSampleP(ksStat, n.toLong)
+  }
+
+  /**
+   * Runs a KS test for 1 set of sample data, comparing it to a theoretical distribution
+   * @param data `RDD[Double]` data on which to run test
+   * @param distObj `RealDistribution` a theoretical distribution
+   * @return [[org.apache.spark.mllib.stat.test.KolmogorovSmirnovTestResult]] summarizing the test
+   *        results (p-value, statistic, and null hypothesis)
+   */
+  def testOneSample(data: RDD[Double], distObj: RealDistribution): KolmogorovSmirnovTestResult = {
+    val cdf = (x: Double) => distObj.cumulativeProbability(x)
+    testOneSample(data, cdf)
+  }
+
+  /**
+   * Calculate unadjusted distances between the empirical CDF and the theoretical CDF in a
+   * partition
+   * @param partData `Iterator[Double]` 1 partition of a sorted RDD
+   * @param n `Double` the total size of the RDD
+   * @param cdf `Double => Double` a function the calculates the theoretical CDF of a value
+   * @return `Iterator[(Double, Double)] `Unadjusted (ie. off by a constant) potential extrema
+   *        in a partition. The first element corresponds to the (empirical CDF - 1/N) - CDF,
+   *        the second element corresponds to empirical CDF - CDF.  We can then search the resulting
+   *        iterator for the minimum of the first and the maximum of the second element, and provide
+   *        this as a partition's candidate extrema
+   */
+  private def oneSampleDifferences(partData: Iterator[Double], n: Double, cdf: Double => Double)
+    : Iterator[(Double, Double)] = {
+    // zip data with index (within that partition)
+    // calculate local (unadjusted) empirical CDF and subtract CDF
+    partData.zipWithIndex.map { case (v, ix) =>
+      // dp and dl are later adjusted by constant, when global info is available
+      val dp = (ix + 1) / n
+      val dl = ix / n
+      val cdfVal = cdf(v)
+      (dl - cdfVal, dp - cdfVal)
+    }
+  }
+
+  /**
+   * Search the unadjusted differences in a partition and return the
+   * two extrema (furthest below and furthest above CDF), along with a count of elements in that
+   * partition
+   * @param partDiffs `Iterator[(Double, Double)]` the unadjusted differences between empirical CDF
+   *                 and CDFin a partition, which come as a tuple of
+   *                 (empirical CDF - 1/N - CDF, empirical CDF - CDF)
+   * @return `Iterator[(Double, Double, Double)]` the local extrema and a count of elements
+   */
+  private def searchOneSampleCandidates(partDiffs: Iterator[(Double, Double)])
+    : Iterator[(Double, Double, Double)] = {
+    val initAcc = (Double.MaxValue, Double.MinValue, 0.0)
+    val pResults = partDiffs.foldLeft(initAcc) { case ((pMin, pMax, pCt), (dl, dp)) =>
+      (math.min(pMin, dl), math.max(pMax, dp), pCt + 1)
+    }
+    val results = if (pResults == initAcc) Array[(Double, Double, Double)]() else Array(pResults)
+    results.iterator
+  }
+
+  /**
+   * Find the global maximum distance between empirical CDF and CDF (i.e. the KS statistic) after
+   * adjusting local extrema estimates from individual partitions with the amount of elements in
+   * preceding partitions
+   * @param localData `Array[(Double, Double, Double)]` A local array containing the collected
+   *                 results of `searchOneSampleCandidates` across all partitions
+   * @param n `Double`The size of the RDD
+   * @return The one-sample Kolmogorov Smirnov Statistic
+   */
+  private def searchOneSampleStatistic(localData: Array[(Double, Double, Double)], n: Double)
+    : Double = {
+    val initAcc = (Double.MinValue, 0.0)
+    // adjust differences based on the number of elements preceding it, which should provide
+    // the correct distance between empirical CDF and CDF
+    val results = localData.foldLeft(initAcc) { case ((prevMax, prevCt), (minCand, maxCand, ct)) =>
+      val adjConst = prevCt / n
+      val dist1 = math.abs(minCand + adjConst)
+      val dist2 = math.abs(maxCand + adjConst)
+      val maxVal = Array(prevMax, dist1, dist2).max
+      (maxVal, prevCt + ct)
+    }
+    results._1
+  }
+
+  /**
+   * A convenience function that allows running the KS test for 1 set of sample data against
+   * a named distribution
+   * @param data the sample data that we wish to evaluate
+   * @param distName the name of the theoretical distribution
+   * @param params Variable length parameter for distribution's parameters
+   * @return [[org.apache.spark.mllib.stat.test.KolmogorovSmirnovTestResult]] summarizing the
+   *        test results (p-value, statistic, and null hypothesis)
+   */
+  @varargs
+  def testOneSample(data: RDD[Double], distName: String, params: Double*)
+    : KolmogorovSmirnovTestResult = {
+    val distObj =
+      distName match {
+        case "norm" => {
+          if (params.nonEmpty) {
+            // parameters are passed, then can only be 2
+            require(params.length == 2, "Normal distribution requires mean and standard " +
+              "deviation as parameters")
+            new NormalDistribution(params(0), params(1))
+          } else {
+            // if no parameters passed in initializes to standard normal
+            logInfo("No parameters specified for normal distribution," +
+              "initialized to standard normal (i.e. N(0, 1))")
+            new NormalDistribution(0, 1)
+          }
+        }
+        case  _ => throw new UnsupportedOperationException(s"$distName not yet supported through" +
+          s" convenience method. Current options are:['norm'].")
+      }
+
+    testOneSample(data, distObj)
+  }
+
+  private def evalOneSampleP(ksStat: Double, n: Long): KolmogorovSmirnovTestResult = {
+    val pval = 1 - new KolmogorovSmirnovTest().cdf(ksStat, n.toInt)
+    new KolmogorovSmirnovTestResult(pval, ksStat, NullHypothesis.OneSampleTwoSided.toString)
+  }
+}
+
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/TestResult.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/TestResult.scala
index 4784f9e947908..f44be13706695 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/TestResult.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/TestResult.scala
@@ -90,3 +90,20 @@ class ChiSqTestResult private[stat] (override val pValue: Double,
       super.toString
   }
 }
+
+/**
+ * :: Experimental ::
+ * Object containing the test results for the Kolmogorov-Smirnov test.
+ */
+@Experimental
+class KolmogorovSmirnovTestResult private[stat] (
+    override val pValue: Double,
+    override val statistic: Double,
+    override val nullHypothesis: String) extends TestResult[Int] {
+
+  override val degreesOfFreedom = 0
+
+  override def toString: String = {
+    "Kolmogorov-Smirnov test summary:\n" + super.toString
+  }
+}
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala
index 6eaebaf7dba9f..e6bcff48b022c 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala
@@ -64,8 +64,10 @@ object KMeansDataGenerator {
 
   def main(args: Array[String]) {
     if (args.length < 6) {
+      // scalastyle:off println
       println("Usage: KMeansGenerator " +
         "      []")
+      // scalastyle:on println
       System.exit(1)
     }
 
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala
index b4e33c98ba7e5..87eeb5db05d26 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala
@@ -153,8 +153,10 @@ object LinearDataGenerator {
 
   def main(args: Array[String]) {
     if (args.length < 2) {
+      // scalastyle:off println
       println("Usage: LinearDataGenerator " +
         "  [num_examples] [num_features] [num_partitions]")
+      // scalastyle:on println
       System.exit(1)
     }
 
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala
index 9d802678c4a77..c09cbe69bb971 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala
@@ -64,8 +64,10 @@ object LogisticRegressionDataGenerator {
 
   def main(args: Array[String]) {
     if (args.length != 5) {
+      // scalastyle:off println
       println("Usage: LogisticRegressionGenerator " +
         "    ")
+      // scalastyle:on println
       System.exit(1)
     }
 
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala
index bd73a866c8a82..16f430599a515 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala
@@ -55,8 +55,10 @@ import org.apache.spark.rdd.RDD
 object MFDataGenerator {
   def main(args: Array[String]) {
     if (args.length < 2) {
+      // scalastyle:off println
       println("Usage: MFDataGenerator " +
         "  [m] [n] [rank] [trainSampFact] [noise] [sigma] [test] [testSampFact]")
+      // scalastyle:on println
       System.exit(1)
     }
 
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala
index a8e30cc9d730c..ad20b7694a779 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala
@@ -37,8 +37,10 @@ object SVMDataGenerator {
 
   def main(args: Array[String]) {
     if (args.length < 2) {
+      // scalastyle:off println
       println("Usage: SVMGenerator " +
         "  [num_examples] [num_features] [num_partitions]")
+      // scalastyle:on println
       System.exit(1)
     }
 
diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaPCASuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaPCASuite.java
new file mode 100644
index 0000000000000..5cf43fec6f29e
--- /dev/null
+++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaPCASuite.java
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.ml.feature;
+
+import java.io.Serializable;
+import java.util.List;
+
+import scala.Tuple2;
+
+import com.google.common.collect.Lists;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import org.apache.spark.api.java.function.Function;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.mllib.linalg.distributed.RowMatrix;
+import org.apache.spark.mllib.linalg.Matrix;
+import org.apache.spark.mllib.linalg.Vector;
+import org.apache.spark.mllib.linalg.Vectors;
+import org.apache.spark.sql.DataFrame;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SQLContext;
+
+public class JavaPCASuite implements Serializable {
+  private transient JavaSparkContext jsc;
+  private transient SQLContext sqlContext;
+
+  @Before
+  public void setUp() {
+    jsc = new JavaSparkContext("local", "JavaPCASuite");
+    sqlContext = new SQLContext(jsc);
+  }
+
+  @After
+  public void tearDown() {
+    jsc.stop();
+    jsc = null;
+  }
+
+  public static class VectorPair implements Serializable {
+    private Vector features = Vectors.dense(0.0);
+    private Vector expected = Vectors.dense(0.0);
+
+    public void setFeatures(Vector features) {
+      this.features = features;
+    }
+
+    public Vector getFeatures() {
+      return this.features;
+    }
+
+    public void setExpected(Vector expected) {
+      this.expected = expected;
+    }
+
+    public Vector getExpected() {
+      return this.expected;
+    }
+  }
+
+  @Test
+  public void testPCA() {
+    List points = Lists.newArrayList(
+      Vectors.sparse(5, new int[]{1, 3}, new double[]{1.0, 7.0}),
+      Vectors.dense(2.0, 0.0, 3.0, 4.0, 5.0),
+      Vectors.dense(4.0, 0.0, 0.0, 6.0, 7.0)
+    );
+    JavaRDD dataRDD = jsc.parallelize(points, 2);
+
+    RowMatrix mat = new RowMatrix(dataRDD.rdd());
+    Matrix pc = mat.computePrincipalComponents(3);
+    JavaRDD expected = mat.multiply(pc).rows().toJavaRDD();
+
+    JavaRDD featuresExpected = dataRDD.zip(expected).map(
+      new Function, VectorPair>() {
+        public VectorPair call(Tuple2 pair) {
+          VectorPair featuresExpected = new VectorPair();
+          featuresExpected.setFeatures(pair._1());
+          featuresExpected.setExpected(pair._2());
+          return featuresExpected;
+        }
+      }
+    );
+
+    DataFrame df = sqlContext.createDataFrame(featuresExpected, VectorPair.class);
+    PCAModel pca = new PCA()
+      .setInputCol("features")
+      .setOutputCol("pca_features")
+      .setK(3)
+      .fit(df);
+    List result = pca.transform(df).select("pca_features", "expected").toJavaRDD().collect();
+    for (Row r : result) {
+      Assert.assertEquals(r.get(1), r.get(0));
+    }
+  }
+}
diff --git a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaLDASuite.java b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaLDASuite.java
index 581c033f08ebe..b48f190f599a2 100644
--- a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaLDASuite.java
+++ b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaLDASuite.java
@@ -28,12 +28,13 @@
 import org.junit.Before;
 import org.junit.Test;
 
+import org.apache.spark.api.java.function.Function;
 import org.apache.spark.api.java.JavaPairRDD;
 import org.apache.spark.api.java.JavaRDD;
 import org.apache.spark.api.java.JavaSparkContext;
 import org.apache.spark.mllib.linalg.Matrix;
 import org.apache.spark.mllib.linalg.Vector;
-
+import org.apache.spark.mllib.linalg.Vectors;
 
 public class JavaLDASuite implements Serializable {
   private transient JavaSparkContext sc;
@@ -110,7 +111,15 @@ public void distributedLDAModel() {
 
     // Check: topic distributions
     JavaPairRDD topicDistributions = model.javaTopicDistributions();
-    assertEquals(topicDistributions.count(), corpus.count());
+    // SPARK-5562. since the topicDistribution returns the distribution of the non empty docs
+    // over topics. Compare it against nonEmptyCorpus instead of corpus
+    JavaPairRDD nonEmptyCorpus = corpus.filter(
+      new Function, Boolean>() {
+        public Boolean call(Tuple2 tuple2) {
+          return Vectors.norm(tuple2._2(), 1.0) != 0.0;
+        }
+    });
+    assertEquals(topicDistributions.count(), nonEmptyCorpus.count());
   }
 
   @Test
diff --git a/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaAssociationRulesSuite.java b/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaAssociationRulesSuite.java
new file mode 100644
index 0000000000000..b3815ae6039c0
--- /dev/null
+++ b/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaAssociationRulesSuite.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.mllib.fpm;
+
+import java.io.Serializable;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import com.google.common.collect.Lists;
+
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.mllib.fpm.FPGrowth.FreqItemset;
+
+
+public class JavaAssociationRulesSuite implements Serializable {
+  private transient JavaSparkContext sc;
+
+  @Before
+  public void setUp() {
+    sc = new JavaSparkContext("local", "JavaFPGrowth");
+  }
+
+  @After
+  public void tearDown() {
+    sc.stop();
+    sc = null;
+  }
+
+  @Test
+  public void runAssociationRules() {
+
+    @SuppressWarnings("unchecked")
+    JavaRDD> freqItemsets = sc.parallelize(Lists.newArrayList(
+      new FreqItemset(new String[] {"a"}, 15L),
+      new FreqItemset(new String[] {"b"}, 35L),
+      new FreqItemset(new String[] {"a", "b"}, 18L)
+    ));
+
+    JavaRDD> results = (new AssociationRules()).run(freqItemsets);
+  }
+}
+
diff --git a/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaFPGrowthSuite.java b/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaFPGrowthSuite.java
index bd0edf2b9ea62..9ce2c52dca8b6 100644
--- a/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaFPGrowthSuite.java
+++ b/mllib/src/test/java/org/apache/spark/mllib/fpm/JavaFPGrowthSuite.java
@@ -29,7 +29,6 @@
 
 import org.apache.spark.api.java.JavaRDD;
 import org.apache.spark.api.java.JavaSparkContext;
-import org.apache.spark.mllib.fpm.FPGrowth.FreqItemset;
 
 public class JavaFPGrowthSuite implements Serializable {
   private transient JavaSparkContext sc;
@@ -62,10 +61,10 @@ public void runFPGrowth() {
       .setNumPartitions(2)
       .run(rdd);
 
-    List> freqItemsets = model.freqItemsets().toJavaRDD().collect();
+    List> freqItemsets = model.freqItemsets().toJavaRDD().collect();
     assertEquals(18, freqItemsets.size());
 
-    for (FreqItemset itemset: freqItemsets) {
+    for (FPGrowth.FreqItemset itemset: freqItemsets) {
       // Test return types.
       List items = itemset.javaItems();
       long freq = itemset.freq();
diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala
index ba8fbee84197c..b7dd44753896a 100644
--- a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala
@@ -77,6 +77,7 @@ class LogisticRegressionSuite extends SparkFunSuite with MLlibTestSparkContext {
     assert(lr.getRawPredictionCol === "rawPrediction")
     assert(lr.getProbabilityCol === "probability")
     assert(lr.getFitIntercept)
+    assert(lr.getStandardization)
     val model = lr.fit(dataset)
     model.transform(dataset)
       .select("label", "probability", "prediction", "rawPrediction")
@@ -208,8 +209,11 @@ class LogisticRegressionSuite extends SparkFunSuite with MLlibTestSparkContext {
   }
 
   test("binary logistic regression with intercept without regularization") {
-    val trainer = (new LogisticRegression).setFitIntercept(true)
-    val model = trainer.fit(binaryDataset)
+    val trainer1 = (new LogisticRegression).setFitIntercept(true).setStandardization(true)
+    val trainer2 = (new LogisticRegression).setFitIntercept(true).setStandardization(false)
+
+    val model1 = trainer1.fit(binaryDataset)
+    val model2 = trainer2.fit(binaryDataset)
 
     /*
        Using the following R code to load the data and train the model using glmnet package.
@@ -230,18 +234,22 @@ class LogisticRegressionSuite extends SparkFunSuite with MLlibTestSparkContext {
        data.V5     -0.7996864
      */
     val interceptR = 2.8366423
-    val weightsR = Array(-0.5895848, 0.8931147, -0.3925051, -0.7996864)
+    val weightsR = Vectors.dense(-0.5895848, 0.8931147, -0.3925051, -0.7996864)
+
+    assert(model1.intercept ~== interceptR relTol 1E-3)
+    assert(model1.weights ~= weightsR relTol 1E-3)
 
-    assert(model.intercept ~== interceptR relTol 1E-3)
-    assert(model.weights(0) ~== weightsR(0) relTol 1E-3)
-    assert(model.weights(1) ~== weightsR(1) relTol 1E-3)
-    assert(model.weights(2) ~== weightsR(2) relTol 1E-3)
-    assert(model.weights(3) ~== weightsR(3) relTol 1E-3)
+    // Without regularization, with or without standardization will converge to the same solution.
+    assert(model2.intercept ~== interceptR relTol 1E-3)
+    assert(model2.weights ~= weightsR relTol 1E-3)
   }
 
   test("binary logistic regression without intercept without regularization") {
-    val trainer = (new LogisticRegression).setFitIntercept(false)
-    val model = trainer.fit(binaryDataset)
+    val trainer1 = (new LogisticRegression).setFitIntercept(false).setStandardization(true)
+    val trainer2 = (new LogisticRegression).setFitIntercept(false).setStandardization(false)
+
+    val model1 = trainer1.fit(binaryDataset)
+    val model2 = trainer2.fit(binaryDataset)
 
     /*
        Using the following R code to load the data and train the model using glmnet package.
@@ -263,19 +271,24 @@ class LogisticRegressionSuite extends SparkFunSuite with MLlibTestSparkContext {
        data.V5     -0.7407946
      */
     val interceptR = 0.0
-    val weightsR = Array(-0.3534996, 1.2964482, -0.3571741, -0.7407946)
+    val weightsR = Vectors.dense(-0.3534996, 1.2964482, -0.3571741, -0.7407946)
+
+    assert(model1.intercept ~== interceptR relTol 1E-3)
+    assert(model1.weights ~= weightsR relTol 1E-2)
 
-    assert(model.intercept ~== interceptR relTol 1E-3)
-    assert(model.weights(0) ~== weightsR(0) relTol 1E-2)
-    assert(model.weights(1) ~== weightsR(1) relTol 1E-2)
-    assert(model.weights(2) ~== weightsR(2) relTol 1E-3)
-    assert(model.weights(3) ~== weightsR(3) relTol 1E-3)
+    // Without regularization, with or without standardization should converge to the same solution.
+    assert(model2.intercept ~== interceptR relTol 1E-3)
+    assert(model2.weights ~= weightsR relTol 1E-2)
   }
 
   test("binary logistic regression with intercept with L1 regularization") {
-    val trainer = (new LogisticRegression).setFitIntercept(true)
-      .setElasticNetParam(1.0).setRegParam(0.12)
-    val model = trainer.fit(binaryDataset)
+    val trainer1 = (new LogisticRegression).setFitIntercept(true)
+      .setElasticNetParam(1.0).setRegParam(0.12).setStandardization(true)
+    val trainer2 = (new LogisticRegression).setFitIntercept(true)
+      .setElasticNetParam(1.0).setRegParam(0.12).setStandardization(false)
+
+    val model1 = trainer1.fit(binaryDataset)
+    val model2 = trainer2.fit(binaryDataset)
 
     /*
        Using the following R code to load the data and train the model using glmnet package.
@@ -295,20 +308,46 @@ class LogisticRegressionSuite extends SparkFunSuite with MLlibTestSparkContext {
        data.V4     -0.04325749
        data.V5     -0.02481551
      */
-    val interceptR = -0.05627428
-    val weightsR = Array(0.0, 0.0, -0.04325749, -0.02481551)
-
-    assert(model.intercept ~== interceptR relTol 1E-2)
-    assert(model.weights(0) ~== weightsR(0) relTol 1E-3)
-    assert(model.weights(1) ~== weightsR(1) relTol 1E-3)
-    assert(model.weights(2) ~== weightsR(2) relTol 1E-2)
-    assert(model.weights(3) ~== weightsR(3) relTol 2E-2)
+    val interceptR1 = -0.05627428
+    val weightsR1 = Vectors.dense(0.0, 0.0, -0.04325749, -0.02481551)
+
+    assert(model1.intercept ~== interceptR1 relTol 1E-2)
+    assert(model1.weights ~= weightsR1 absTol 2E-2)
+
+    /*
+       Using the following R code to load the data and train the model using glmnet package.
+
+       library("glmnet")
+       data <- read.csv("path", header=FALSE)
+       label = factor(data$V1)
+       features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5))
+       weights = coef(glmnet(features,label, family="binomial", alpha = 1, lambda = 0.12,
+           standardize=FALSE))
+       weights
+
+       5 x 1 sparse Matrix of class "dgCMatrix"
+                           s0
+       (Intercept)  0.3722152
+       data.V2       .
+       data.V3       .
+       data.V4     -0.1665453
+       data.V5       .
+     */
+    val interceptR2 = 0.3722152
+    val weightsR2 = Vectors.dense(0.0, 0.0, -0.1665453, 0.0)
+
+    assert(model2.intercept ~== interceptR2 relTol 1E-2)
+    assert(model2.weights ~= weightsR2 absTol 1E-3)
   }
 
   test("binary logistic regression without intercept with L1 regularization") {
-    val trainer = (new LogisticRegression).setFitIntercept(false)
-      .setElasticNetParam(1.0).setRegParam(0.12)
-    val model = trainer.fit(binaryDataset)
+    val trainer1 = (new LogisticRegression).setFitIntercept(false)
+      .setElasticNetParam(1.0).setRegParam(0.12).setStandardization(true)
+    val trainer2 = (new LogisticRegression).setFitIntercept(false)
+      .setElasticNetParam(1.0).setRegParam(0.12).setStandardization(false)
+
+    val model1 = trainer1.fit(binaryDataset)
+    val model2 = trainer2.fit(binaryDataset)
 
     /*
        Using the following R code to load the data and train the model using glmnet package.
@@ -329,20 +368,46 @@ class LogisticRegressionSuite extends SparkFunSuite with MLlibTestSparkContext {
        data.V4     -0.05189203
        data.V5     -0.03891782
      */
-    val interceptR = 0.0
-    val weightsR = Array(0.0, 0.0, -0.05189203, -0.03891782)
+    val interceptR1 = 0.0
+    val weightsR1 = Vectors.dense(0.0, 0.0, -0.05189203, -0.03891782)
+
+    assert(model1.intercept ~== interceptR1 relTol 1E-3)
+    assert(model1.weights ~= weightsR1 absTol 1E-3)
+
+    /*
+       Using the following R code to load the data and train the model using glmnet package.
 
-    assert(model.intercept ~== interceptR relTol 1E-3)
-    assert(model.weights(0) ~== weightsR(0) relTol 1E-3)
-    assert(model.weights(1) ~== weightsR(1) relTol 1E-3)
-    assert(model.weights(2) ~== weightsR(2) relTol 1E-2)
-    assert(model.weights(3) ~== weightsR(3) relTol 1E-2)
+       library("glmnet")
+       data <- read.csv("path", header=FALSE)
+       label = factor(data$V1)
+       features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5))
+       weights = coef(glmnet(features,label, family="binomial", alpha = 1, lambda = 0.12,
+           intercept=FALSE, standardize=FALSE))
+       weights
+
+       5 x 1 sparse Matrix of class "dgCMatrix"
+                            s0
+       (Intercept)   .
+       data.V2       .
+       data.V3       .
+       data.V4     -0.08420782
+       data.V5       .
+     */
+    val interceptR2 = 0.0
+    val weightsR2 = Vectors.dense(0.0, 0.0, -0.08420782, 0.0)
+
+    assert(model2.intercept ~== interceptR2 absTol 1E-3)
+    assert(model2.weights ~= weightsR2 absTol 1E-3)
   }
 
   test("binary logistic regression with intercept with L2 regularization") {
-    val trainer = (new LogisticRegression).setFitIntercept(true)
-      .setElasticNetParam(0.0).setRegParam(1.37)
-    val model = trainer.fit(binaryDataset)
+    val trainer1 = (new LogisticRegression).setFitIntercept(true)
+      .setElasticNetParam(0.0).setRegParam(1.37).setStandardization(true)
+    val trainer2 = (new LogisticRegression).setFitIntercept(true)
+      .setElasticNetParam(0.0).setRegParam(1.37).setStandardization(false)
+
+    val model1 = trainer1.fit(binaryDataset)
+    val model2 = trainer2.fit(binaryDataset)
 
     /*
        Using the following R code to load the data and train the model using glmnet package.
@@ -362,20 +427,46 @@ class LogisticRegressionSuite extends SparkFunSuite with MLlibTestSparkContext {
        data.V4     -0.04865309
        data.V5     -0.10062872
      */
-    val interceptR = 0.15021751
-    val weightsR = Array(-0.07251837, 0.10724191, -0.04865309, -0.10062872)
-
-    assert(model.intercept ~== interceptR relTol 1E-3)
-    assert(model.weights(0) ~== weightsR(0) relTol 1E-3)
-    assert(model.weights(1) ~== weightsR(1) relTol 1E-3)
-    assert(model.weights(2) ~== weightsR(2) relTol 1E-3)
-    assert(model.weights(3) ~== weightsR(3) relTol 1E-3)
+    val interceptR1 = 0.15021751
+    val weightsR1 = Vectors.dense(-0.07251837, 0.10724191, -0.04865309, -0.10062872)
+
+    assert(model1.intercept ~== interceptR1 relTol 1E-3)
+    assert(model1.weights ~= weightsR1 relTol 1E-3)
+
+    /*
+       Using the following R code to load the data and train the model using glmnet package.
+
+       library("glmnet")
+       data <- read.csv("path", header=FALSE)
+       label = factor(data$V1)
+       features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5))
+       weights = coef(glmnet(features,label, family="binomial", alpha = 0, lambda = 1.37,
+           standardize=FALSE))
+       weights
+
+       5 x 1 sparse Matrix of class "dgCMatrix"
+                            s0
+       (Intercept)  0.48657516
+       data.V2     -0.05155371
+       data.V3      0.02301057
+       data.V4     -0.11482896
+       data.V5     -0.06266838
+     */
+    val interceptR2 = 0.48657516
+    val weightsR2 = Vectors.dense(-0.05155371, 0.02301057, -0.11482896, -0.06266838)
+
+    assert(model2.intercept ~== interceptR2 relTol 1E-3)
+    assert(model2.weights ~= weightsR2 relTol 1E-3)
   }
 
   test("binary logistic regression without intercept with L2 regularization") {
-    val trainer = (new LogisticRegression).setFitIntercept(false)
-      .setElasticNetParam(0.0).setRegParam(1.37)
-    val model = trainer.fit(binaryDataset)
+    val trainer1 = (new LogisticRegression).setFitIntercept(false)
+      .setElasticNetParam(0.0).setRegParam(1.37).setStandardization(true)
+    val trainer2 = (new LogisticRegression).setFitIntercept(false)
+      .setElasticNetParam(0.0).setRegParam(1.37).setStandardization(false)
+
+    val model1 = trainer1.fit(binaryDataset)
+    val model2 = trainer2.fit(binaryDataset)
 
     /*
        Using the following R code to load the data and train the model using glmnet package.
@@ -396,20 +487,46 @@ class LogisticRegressionSuite extends SparkFunSuite with MLlibTestSparkContext {
        data.V4     -0.04708770
        data.V5     -0.09799775
      */
-    val interceptR = 0.0
-    val weightsR = Array(-0.06099165, 0.12857058, -0.04708770, -0.09799775)
+    val interceptR1 = 0.0
+    val weightsR1 = Vectors.dense(-0.06099165, 0.12857058, -0.04708770, -0.09799775)
+
+    assert(model1.intercept ~== interceptR1 absTol 1E-3)
+    assert(model1.weights ~= weightsR1 relTol 1E-2)
+
+    /*
+       Using the following R code to load the data and train the model using glmnet package.
+
+       library("glmnet")
+       data <- read.csv("path", header=FALSE)
+       label = factor(data$V1)
+       features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5))
+       weights = coef(glmnet(features,label, family="binomial", alpha = 0, lambda = 1.37,
+           intercept=FALSE, standardize=FALSE))
+       weights
 
-    assert(model.intercept ~== interceptR relTol 1E-3)
-    assert(model.weights(0) ~== weightsR(0) relTol 1E-2)
-    assert(model.weights(1) ~== weightsR(1) relTol 1E-2)
-    assert(model.weights(2) ~== weightsR(2) relTol 1E-3)
-    assert(model.weights(3) ~== weightsR(3) relTol 1E-3)
+       5 x 1 sparse Matrix of class "dgCMatrix"
+                             s0
+       (Intercept)   .
+       data.V2     -0.005679651
+       data.V3      0.048967094
+       data.V4     -0.093714016
+       data.V5     -0.053314311
+     */
+    val interceptR2 = 0.0
+    val weightsR2 = Vectors.dense(-0.005679651, 0.048967094, -0.093714016, -0.053314311)
+
+    assert(model2.intercept ~== interceptR2 absTol 1E-3)
+    assert(model2.weights ~= weightsR2 relTol 1E-2)
   }
 
   test("binary logistic regression with intercept with ElasticNet regularization") {
-    val trainer = (new LogisticRegression).setFitIntercept(true)
-      .setElasticNetParam(0.38).setRegParam(0.21)
-    val model = trainer.fit(binaryDataset)
+    val trainer1 = (new LogisticRegression).setFitIntercept(true)
+      .setElasticNetParam(0.38).setRegParam(0.21).setStandardization(true)
+    val trainer2 = (new LogisticRegression).setFitIntercept(true)
+      .setElasticNetParam(0.38).setRegParam(0.21).setStandardization(false)
+
+    val model1 = trainer1.fit(binaryDataset)
+    val model2 = trainer2.fit(binaryDataset)
 
     /*
        Using the following R code to load the data and train the model using glmnet package.
@@ -429,20 +546,46 @@ class LogisticRegressionSuite extends SparkFunSuite with MLlibTestSparkContext {
        data.V4     -0.08849250
        data.V5     -0.15458796
      */
-    val interceptR = 0.57734851
-    val weightsR = Array(-0.05310287, 0.0, -0.08849250, -0.15458796)
-
-    assert(model.intercept ~== interceptR relTol 6E-3)
-    assert(model.weights(0) ~== weightsR(0) relTol 5E-3)
-    assert(model.weights(1) ~== weightsR(1) relTol 1E-3)
-    assert(model.weights(2) ~== weightsR(2) relTol 5E-3)
-    assert(model.weights(3) ~== weightsR(3) relTol 1E-3)
+    val interceptR1 = 0.57734851
+    val weightsR1 = Vectors.dense(-0.05310287, 0.0, -0.08849250, -0.15458796)
+
+    assert(model1.intercept ~== interceptR1 relTol 6E-3)
+    assert(model1.weights ~== weightsR1 absTol 5E-3)
+
+    /*
+       Using the following R code to load the data and train the model using glmnet package.
+
+       library("glmnet")
+       data <- read.csv("path", header=FALSE)
+       label = factor(data$V1)
+       features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5))
+       weights = coef(glmnet(features,label, family="binomial", alpha = 0.38, lambda = 0.21,
+           standardize=FALSE))
+       weights
+
+       5 x 1 sparse Matrix of class "dgCMatrix"
+                            s0
+       (Intercept)  0.51555993
+       data.V2       .
+       data.V3       .
+       data.V4     -0.18807395
+       data.V5     -0.05350074
+     */
+    val interceptR2 = 0.51555993
+    val weightsR2 = Vectors.dense(0.0, 0.0, -0.18807395, -0.05350074)
+
+    assert(model2.intercept ~== interceptR2 relTol 6E-3)
+    assert(model2.weights ~= weightsR2 absTol 1E-3)
   }
 
   test("binary logistic regression without intercept with ElasticNet regularization") {
-    val trainer = (new LogisticRegression).setFitIntercept(false)
-      .setElasticNetParam(0.38).setRegParam(0.21)
-    val model = trainer.fit(binaryDataset)
+    val trainer1 = (new LogisticRegression).setFitIntercept(false)
+      .setElasticNetParam(0.38).setRegParam(0.21).setStandardization(true)
+    val trainer2 = (new LogisticRegression).setFitIntercept(false)
+      .setElasticNetParam(0.38).setRegParam(0.21).setStandardization(false)
+
+    val model1 = trainer1.fit(binaryDataset)
+    val model2 = trainer2.fit(binaryDataset)
 
     /*
        Using the following R code to load the data and train the model using glmnet package.
@@ -463,20 +606,46 @@ class LogisticRegressionSuite extends SparkFunSuite with MLlibTestSparkContext {
        data.V4     -0.081203769
        data.V5     -0.142534158
      */
-    val interceptR = 0.0
-    val weightsR = Array(-0.001005743, 0.072577857, -0.081203769, -0.142534158)
+    val interceptR1 = 0.0
+    val weightsR1 = Vectors.dense(-0.001005743, 0.072577857, -0.081203769, -0.142534158)
+
+    assert(model1.intercept ~== interceptR1 relTol 1E-3)
+    assert(model1.weights ~= weightsR1 absTol 1E-2)
+
+    /*
+       Using the following R code to load the data and train the model using glmnet package.
+
+       library("glmnet")
+       data <- read.csv("path", header=FALSE)
+       label = factor(data$V1)
+       features = as.matrix(data.frame(data$V2, data$V3, data$V4, data$V5))
+       weights = coef(glmnet(features,label, family="binomial", alpha = 0.38, lambda = 0.21,
+           intercept=FALSE, standardize=FALSE))
+       weights
 
-    assert(model.intercept ~== interceptR relTol 1E-3)
-    assert(model.weights(0) ~== weightsR(0) absTol 1E-3)
-    assert(model.weights(1) ~== weightsR(1) absTol 1E-2)
-    assert(model.weights(2) ~== weightsR(2) relTol 1E-3)
-    assert(model.weights(3) ~== weightsR(3) relTol 1E-2)
+       5 x 1 sparse Matrix of class "dgCMatrix"
+                            s0
+       (Intercept)   .
+       data.V2       .
+       data.V3      0.03345223
+       data.V4     -0.11304532
+       data.V5       .
+     */
+    val interceptR2 = 0.0
+    val weightsR2 = Vectors.dense(0.0, 0.03345223, -0.11304532, 0.0)
+
+    assert(model2.intercept ~== interceptR2 absTol 1E-3)
+    assert(model2.weights ~= weightsR2 absTol 1E-3)
   }
 
   test("binary logistic regression with intercept with strong L1 regularization") {
-    val trainer = (new LogisticRegression).setFitIntercept(true)
-      .setElasticNetParam(1.0).setRegParam(6.0)
-    val model = trainer.fit(binaryDataset)
+    val trainer1 = (new LogisticRegression).setFitIntercept(true)
+      .setElasticNetParam(1.0).setRegParam(6.0).setStandardization(true)
+    val trainer2 = (new LogisticRegression).setFitIntercept(true)
+      .setElasticNetParam(1.0).setRegParam(6.0).setStandardization(false)
+
+    val model1 = trainer1.fit(binaryDataset)
+    val model2 = trainer2.fit(binaryDataset)
 
     val histogram = binaryDataset.map { case Row(label: Double, features: Vector) => label }
       .treeAggregate(new MultiClassSummarizer)(
@@ -500,13 +669,13 @@ class LogisticRegressionSuite extends SparkFunSuite with MLlibTestSparkContext {
        }}}
      */
     val interceptTheory = math.log(histogram(1).toDouble / histogram(0).toDouble)
-    val weightsTheory = Array(0.0, 0.0, 0.0, 0.0)
+    val weightsTheory = Vectors.dense(0.0, 0.0, 0.0, 0.0)
+
+    assert(model1.intercept ~== interceptTheory relTol 1E-5)
+    assert(model1.weights ~= weightsTheory absTol 1E-6)
 
-    assert(model.intercept ~== interceptTheory relTol 1E-5)
-    assert(model.weights(0) ~== weightsTheory(0) absTol 1E-6)
-    assert(model.weights(1) ~== weightsTheory(1) absTol 1E-6)
-    assert(model.weights(2) ~== weightsTheory(2) absTol 1E-6)
-    assert(model.weights(3) ~== weightsTheory(3) absTol 1E-6)
+    assert(model2.intercept ~== interceptTheory relTol 1E-5)
+    assert(model2.weights ~= weightsTheory absTol 1E-6)
 
     /*
        Using the following R code to load the data and train the model using glmnet package.
@@ -527,12 +696,9 @@ class LogisticRegressionSuite extends SparkFunSuite with MLlibTestSparkContext {
        data.V5       .
      */
     val interceptR = -0.248065
-    val weightsR = Array(0.0, 0.0, 0.0, 0.0)
+    val weightsR = Vectors.dense(0.0, 0.0, 0.0, 0.0)
 
-    assert(model.intercept ~== interceptR relTol 1E-5)
-    assert(model.weights(0) ~== weightsR(0) absTol 1E-6)
-    assert(model.weights(1) ~== weightsR(1) absTol 1E-6)
-    assert(model.weights(2) ~== weightsR(2) absTol 1E-6)
-    assert(model.weights(3) ~== weightsR(3) absTol 1E-6)
+    assert(model1.intercept ~== interceptR relTol 1E-5)
+    assert(model1.weights ~= weightsR absTol 1E-6)
   }
 }
diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/CountVectorizorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/CountVectorizorSuite.scala
new file mode 100644
index 0000000000000..e90d9d4ef21ff
--- /dev/null
+++ b/mllib/src/test/scala/org/apache/spark/ml/feature/CountVectorizorSuite.scala
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.ml.feature
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.ml.param.ParamsSuite
+import org.apache.spark.mllib.linalg.{Vector, Vectors}
+import org.apache.spark.mllib.util.MLlibTestSparkContext
+import org.apache.spark.mllib.util.TestingUtils._
+
+class CountVectorizerSuite extends SparkFunSuite with MLlibTestSparkContext {
+
+  test("params") {
+    ParamsSuite.checkParams(new CountVectorizerModel(Array("empty")))
+  }
+
+  test("CountVectorizerModel common cases") {
+    val df = sqlContext.createDataFrame(Seq(
+      (0, "a b c d".split(" ").toSeq,
+        Vectors.sparse(4, Seq((0, 1.0), (1, 1.0), (2, 1.0), (3, 1.0)))),
+      (1, "a b b c d  a".split(" ").toSeq,
+        Vectors.sparse(4, Seq((0, 2.0), (1, 2.0), (2, 1.0), (3, 1.0)))),
+      (2, "a".split(" ").toSeq, Vectors.sparse(4, Seq((0, 1.0)))),
+      (3, "".split(" ").toSeq, Vectors.sparse(4, Seq())), // empty string
+      (4, "a notInDict d".split(" ").toSeq,
+        Vectors.sparse(4, Seq((0, 1.0), (3, 1.0))))  // with words not in vocabulary
+    )).toDF("id", "words", "expected")
+    val cv = new CountVectorizerModel(Array("a", "b", "c", "d"))
+      .setInputCol("words")
+      .setOutputCol("features")
+    val output = cv.transform(df).collect()
+    output.foreach { p =>
+      val features = p.getAs[Vector]("features")
+      val expected = p.getAs[Vector]("expected")
+      assert(features ~== expected absTol 1e-14)
+    }
+  }
+
+  test("CountVectorizerModel with minTermFreq") {
+    val df = sqlContext.createDataFrame(Seq(
+      (0, "a a a b b c c c d ".split(" ").toSeq, Vectors.sparse(4, Seq((0, 3.0), (2, 3.0)))),
+      (1, "c c c c c c".split(" ").toSeq, Vectors.sparse(4, Seq((2, 6.0)))),
+      (2, "a".split(" ").toSeq, Vectors.sparse(4, Seq())),
+      (3, "e e e e e".split(" ").toSeq, Vectors.sparse(4, Seq())))
+    ).toDF("id", "words", "expected")
+    val cv = new CountVectorizerModel(Array("a", "b", "c", "d"))
+      .setInputCol("words")
+      .setOutputCol("features")
+      .setMinTermFreq(3)
+    val output = cv.transform(df).collect()
+    output.foreach { p =>
+      val features = p.getAs[Vector]("features")
+      val expected = p.getAs[Vector]("expected")
+      assert(features ~== expected absTol 1e-14)
+    }
+  }
+}
+
+
diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/VectorIndexerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/VectorIndexerSuite.scala
index 8c85c96d5c6d8..03120c828ca96 100644
--- a/mllib/src/test/scala/org/apache/spark/ml/feature/VectorIndexerSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/ml/feature/VectorIndexerSuite.scala
@@ -19,7 +19,7 @@ package org.apache.spark.ml.feature
 
 import scala.beans.{BeanInfo, BeanProperty}
 
-import org.apache.spark.{SparkException, SparkFunSuite}
+import org.apache.spark.{Logging, SparkException, SparkFunSuite}
 import org.apache.spark.ml.attribute._
 import org.apache.spark.ml.param.ParamsSuite
 import org.apache.spark.mllib.linalg.{SparseVector, Vector, Vectors}
@@ -27,7 +27,7 @@ import org.apache.spark.mllib.util.MLlibTestSparkContext
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.DataFrame
 
-class VectorIndexerSuite extends SparkFunSuite with MLlibTestSparkContext {
+class VectorIndexerSuite extends SparkFunSuite with MLlibTestSparkContext with Logging {
 
   import VectorIndexerSuite.FeatureData
 
@@ -113,11 +113,11 @@ class VectorIndexerSuite extends SparkFunSuite with MLlibTestSparkContext {
     model.transform(sparsePoints1) // should work
     intercept[SparkException] {
       model.transform(densePoints2).collect()
-      println("Did not throw error when fit, transform were called on vectors of different lengths")
+      logInfo("Did not throw error when fit, transform were called on vectors of different lengths")
     }
     intercept[SparkException] {
       vectorIndexer.fit(badPoints)
-      println("Did not throw error when fitting vectors of different lengths in same RDD.")
+      logInfo("Did not throw error when fitting vectors of different lengths in same RDD.")
     }
   }
 
@@ -196,7 +196,7 @@ class VectorIndexerSuite extends SparkFunSuite with MLlibTestSparkContext {
         }
       } catch {
         case e: org.scalatest.exceptions.TestFailedException =>
-          println(errMsg)
+          logError(errMsg)
           throw e
       }
     }
diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala
index 5f39d44f37352..cf120cf2a4b47 100644
--- a/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala
@@ -18,7 +18,7 @@
 package org.apache.spark.ml.regression
 
 import org.apache.spark.SparkFunSuite
-import org.apache.spark.mllib.linalg.DenseVector
+import org.apache.spark.mllib.linalg.{DenseVector, Vectors}
 import org.apache.spark.mllib.util.{LinearDataGenerator, MLlibTestSparkContext}
 import org.apache.spark.mllib.util.TestingUtils._
 import org.apache.spark.sql.{DataFrame, Row}
@@ -75,11 +75,10 @@ class LinearRegressionSuite extends SparkFunSuite with MLlibTestSparkContext {
        as.numeric.data.V3. 7.198257
      */
     val interceptR = 6.298698
-    val weightsR = Array(4.700706, 7.199082)
+    val weightsR = Vectors.dense(4.700706, 7.199082)
 
     assert(model.intercept ~== interceptR relTol 1E-3)
-    assert(model.weights(0) ~== weightsR(0) relTol 1E-3)
-    assert(model.weights(1) ~== weightsR(1) relTol 1E-3)
+    assert(model.weights ~= weightsR relTol 1E-3)
 
     model.transform(dataset).select("features", "prediction").collect().foreach {
       case Row(features: DenseVector, prediction1: Double) =>
@@ -104,11 +103,10 @@ class LinearRegressionSuite extends SparkFunSuite with MLlibTestSparkContext {
        as.numeric.data.V2. 6.995908
        as.numeric.data.V3. 5.275131
      */
-    val weightsR = Array(6.995908, 5.275131)
+    val weightsR = Vectors.dense(6.995908, 5.275131)
 
-    assert(model.intercept ~== 0 relTol 1E-3)
-    assert(model.weights(0) ~== weightsR(0) relTol 1E-3)
-    assert(model.weights(1) ~== weightsR(1) relTol 1E-3)
+    assert(model.intercept ~== 0 absTol 1E-3)
+    assert(model.weights ~= weightsR relTol 1E-3)
     /*
        Then again with the data with no intercept:
        > weightsWithoutIntercept
@@ -118,11 +116,10 @@ class LinearRegressionSuite extends SparkFunSuite with MLlibTestSparkContext {
        as.numeric.data3.V2. 4.70011
        as.numeric.data3.V3. 7.19943
      */
-    val weightsWithoutInterceptR = Array(4.70011, 7.19943)
+    val weightsWithoutInterceptR = Vectors.dense(4.70011, 7.19943)
 
-    assert(modelWithoutIntercept.intercept ~== 0 relTol 1E-3)
-    assert(modelWithoutIntercept.weights(0) ~== weightsWithoutInterceptR(0) relTol 1E-3)
-    assert(modelWithoutIntercept.weights(1) ~== weightsWithoutInterceptR(1) relTol 1E-3)
+    assert(modelWithoutIntercept.intercept ~== 0 absTol 1E-3)
+    assert(modelWithoutIntercept.weights ~= weightsWithoutInterceptR relTol 1E-3)
   }
 
   test("linear regression with intercept with L1 regularization") {
@@ -139,11 +136,10 @@ class LinearRegressionSuite extends SparkFunSuite with MLlibTestSparkContext {
        as.numeric.data.V3. 6.679841
      */
     val interceptR = 6.24300
-    val weightsR = Array(4.024821, 6.679841)
+    val weightsR = Vectors.dense(4.024821, 6.679841)
 
     assert(model.intercept ~== interceptR relTol 1E-3)
-    assert(model.weights(0) ~== weightsR(0) relTol 1E-3)
-    assert(model.weights(1) ~== weightsR(1) relTol 1E-3)
+    assert(model.weights ~= weightsR relTol 1E-3)
 
     model.transform(dataset).select("features", "prediction").collect().foreach {
       case Row(features: DenseVector, prediction1: Double) =>
@@ -169,11 +165,10 @@ class LinearRegressionSuite extends SparkFunSuite with MLlibTestSparkContext {
        as.numeric.data.V3. 4.772913
      */
     val interceptR = 0.0
-    val weightsR = Array(6.299752, 4.772913)
+    val weightsR = Vectors.dense(6.299752, 4.772913)
 
-    assert(model.intercept ~== interceptR relTol 1E-3)
-    assert(model.weights(0) ~== weightsR(0) relTol 1E-3)
-    assert(model.weights(1) ~== weightsR(1) relTol 1E-3)
+    assert(model.intercept ~== interceptR absTol 1E-5)
+    assert(model.weights ~= weightsR relTol 1E-3)
 
     model.transform(dataset).select("features", "prediction").collect().foreach {
       case Row(features: DenseVector, prediction1: Double) =>
@@ -197,11 +192,10 @@ class LinearRegressionSuite extends SparkFunSuite with MLlibTestSparkContext {
        as.numeric.data.V3. 4.926260
      */
     val interceptR = 5.269376
-    val weightsR = Array(3.736216, 5.712356)
+    val weightsR = Vectors.dense(3.736216, 5.712356)
 
     assert(model.intercept ~== interceptR relTol 1E-3)
-    assert(model.weights(0) ~== weightsR(0) relTol 1E-3)
-    assert(model.weights(1) ~== weightsR(1) relTol 1E-3)
+    assert(model.weights ~= weightsR relTol 1E-3)
 
     model.transform(dataset).select("features", "prediction").collect().foreach {
       case Row(features: DenseVector, prediction1: Double) =>
@@ -227,11 +221,10 @@ class LinearRegressionSuite extends SparkFunSuite with MLlibTestSparkContext {
        as.numeric.data.V3. 4.214502
      */
     val interceptR = 0.0
-    val weightsR = Array(5.522875, 4.214502)
+    val weightsR = Vectors.dense(5.522875, 4.214502)
 
-    assert(model.intercept ~== interceptR relTol 1E-3)
-    assert(model.weights(0) ~== weightsR(0) relTol 1E-3)
-    assert(model.weights(1) ~== weightsR(1) relTol 1E-3)
+    assert(model.intercept ~== interceptR absTol 1E-3)
+    assert(model.weights ~== weightsR relTol 1E-3)
 
     model.transform(dataset).select("features", "prediction").collect().foreach {
       case Row(features: DenseVector, prediction1: Double) =>
@@ -255,11 +248,10 @@ class LinearRegressionSuite extends SparkFunSuite with MLlibTestSparkContext {
        as.numeric.data.V3. 5.200403
      */
     val interceptR = 5.696056
-    val weightsR = Array(3.670489, 6.001122)
+    val weightsR = Vectors.dense(3.670489, 6.001122)
 
     assert(model.intercept ~== interceptR relTol 1E-3)
-    assert(model.weights(0) ~== weightsR(0) relTol 1E-3)
-    assert(model.weights(1) ~== weightsR(1) relTol 1E-3)
+    assert(model.weights ~== weightsR relTol 1E-3)
 
     model.transform(dataset).select("features", "prediction").collect().foreach {
       case Row(features: DenseVector, prediction1: Double) =>
@@ -285,11 +277,10 @@ class LinearRegressionSuite extends SparkFunSuite with MLlibTestSparkContext {
        as.numeric.dataM.V3. 4.322251
      */
     val interceptR = 0.0
-    val weightsR = Array(5.673348, 4.322251)
+    val weightsR = Vectors.dense(5.673348, 4.322251)
 
-    assert(model.intercept ~== interceptR relTol 1E-3)
-    assert(model.weights(0) ~== weightsR(0) relTol 1E-3)
-    assert(model.weights(1) ~== weightsR(1) relTol 1E-3)
+    assert(model.intercept ~== interceptR absTol 1E-3)
+    assert(model.weights ~= weightsR relTol 1E-3)
 
     model.transform(dataset).select("features", "prediction").collect().foreach {
       case Row(features: DenseVector, prediction1: Double) =>
@@ -298,4 +289,63 @@ class LinearRegressionSuite extends SparkFunSuite with MLlibTestSparkContext {
         assert(prediction1 ~== prediction2 relTol 1E-5)
     }
   }
+
+  test("linear regression model training summary") {
+    val trainer = new LinearRegression
+    val model = trainer.fit(dataset)
+
+    // Training results for the model should be available
+    assert(model.hasSummary)
+
+    // Residuals in [[LinearRegressionResults]] should equal those manually computed
+    val expectedResiduals = dataset.select("features", "label")
+      .map { case Row(features: DenseVector, label: Double) =>
+      val prediction =
+        features(0) * model.weights(0) + features(1) * model.weights(1) + model.intercept
+      prediction - label
+    }
+      .zip(model.summary.residuals.map(_.getDouble(0)))
+      .collect()
+      .foreach { case (manualResidual: Double, resultResidual: Double) =>
+      assert(manualResidual ~== resultResidual relTol 1E-5)
+    }
+
+    /*
+       Use the following R code to generate model training results.
+
+       predictions <- predict(fit, newx=features)
+       residuals <- predictions - label
+       > mean(residuals^2) # MSE
+       [1] 0.009720325
+       > mean(abs(residuals)) # MAD
+       [1] 0.07863206
+       > cor(predictions, label)^2# r^2
+               [,1]
+       s0 0.9998749
+     */
+    assert(model.summary.meanSquaredError ~== 0.00972035 relTol 1E-5)
+    assert(model.summary.meanAbsoluteError ~== 0.07863206  relTol 1E-5)
+    assert(model.summary.r2 ~== 0.9998749 relTol 1E-5)
+
+    // Objective function should be monotonically decreasing for linear regression
+    assert(
+      model.summary
+        .objectiveHistory
+        .sliding(2)
+        .forall(x => x(0) >= x(1)))
+  }
+
+  test("linear regression model testset evaluation summary") {
+    val trainer = new LinearRegression
+    val model = trainer.fit(dataset)
+
+    // Evaluating on training dataset should yield results summary equal to training summary
+    val testSummary = model.evaluate(dataset)
+    assert(model.summary.meanSquaredError ~== testSummary.meanSquaredError relTol 1E-5)
+    assert(model.summary.r2 ~== testSummary.r2 relTol 1E-5)
+    model.summary.residuals.select("residuals").collect()
+      .zip(testSummary.residuals.select("residuals").collect())
+      .forall { case (Row(r1: Double), Row(r2: Double)) => r1 ~== r2 relTol 1E-5 }
+  }
+
 }
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala
index f7fc8730606af..cffa1ab700f80 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala
@@ -19,13 +19,14 @@ package org.apache.spark.mllib.classification
 
 import scala.util.Random
 
-import breeze.linalg.{DenseMatrix => BDM, DenseVector => BDV, argmax => brzArgmax, sum => brzSum}
+import breeze.linalg.{DenseMatrix => BDM, DenseVector => BDV, Vector => BV}
 import breeze.stats.distributions.{Multinomial => BrzMultinomial}
 
 import org.apache.spark.{SparkException, SparkFunSuite}
-import org.apache.spark.mllib.linalg.Vectors
+import org.apache.spark.mllib.linalg.{Vector, Vectors}
 import org.apache.spark.mllib.regression.LabeledPoint
 import org.apache.spark.mllib.util.{LocalClusterSparkContext, MLlibTestSparkContext}
+import org.apache.spark.mllib.util.TestingUtils._
 import org.apache.spark.util.Utils
 
 object NaiveBayesSuite {
@@ -154,6 +155,29 @@ class NaiveBayesSuite extends SparkFunSuite with MLlibTestSparkContext {
 
     // Test prediction on Array.
     validatePrediction(validationData.map(row => model.predict(row.features)), validationData)
+
+    // Test posteriors
+    validationData.map(_.features).foreach { features =>
+      val predicted = model.predictProbabilities(features).toArray
+      assert(predicted.sum ~== 1.0 relTol 1.0e-10)
+      val expected = expectedMultinomialProbabilities(model, features)
+      expected.zip(predicted).foreach { case (e, p) => assert(e ~== p relTol 1.0e-10) }
+    }
+  }
+
+  /**
+   * @param model Multinomial Naive Bayes model
+   * @param testData input to compute posterior probabilities for
+   * @return posterior class probabilities (in order of labels) for input
+   */
+  private def expectedMultinomialProbabilities(model: NaiveBayesModel, testData: Vector) = {
+    val piVector = new BDV(model.pi)
+    // model.theta is row-major; treat it as col-major representation of transpose, and transpose:
+    val thetaMatrix = new BDM(model.theta(0).length, model.theta.length, model.theta.flatten).t
+    val logClassProbs: BV[Double] = piVector + (thetaMatrix * testData.toBreeze)
+    val classProbs = logClassProbs.toArray.map(math.exp)
+    val classProbsSum = classProbs.sum
+    classProbs.map(_ / classProbsSum)
   }
 
   test("Naive Bayes Bernoulli") {
@@ -182,6 +206,33 @@ class NaiveBayesSuite extends SparkFunSuite with MLlibTestSparkContext {
 
     // Test prediction on Array.
     validatePrediction(validationData.map(row => model.predict(row.features)), validationData)
+
+    // Test posteriors
+    validationData.map(_.features).foreach { features =>
+      val predicted = model.predictProbabilities(features).toArray
+      assert(predicted.sum ~== 1.0 relTol 1.0e-10)
+      val expected = expectedBernoulliProbabilities(model, features)
+      expected.zip(predicted).foreach { case (e, p) => assert(e ~== p relTol 1.0e-10) }
+    }
+  }
+
+  /**
+   * @param model Bernoulli Naive Bayes model
+   * @param testData input to compute posterior probabilities for
+   * @return posterior class probabilities (in order of labels) for input
+   */
+  private def expectedBernoulliProbabilities(model: NaiveBayesModel, testData: Vector) = {
+    val piVector = new BDV(model.pi)
+    val thetaMatrix = new BDM(model.theta(0).length, model.theta.length, model.theta.flatten).t
+    val negThetaMatrix = new BDM(model.theta(0).length, model.theta.length,
+      model.theta.flatten.map(v => math.log(1.0 - math.exp(v)))).t
+    val testBreeze = testData.toBreeze
+    val negTestBreeze = new BDV(Array.fill(testBreeze.size)(1.0)) - testBreeze
+    val piTheta: BV[Double] = piVector + (thetaMatrix * testBreeze)
+    val logClassProbs: BV[Double] = piTheta + (negThetaMatrix * negTestBreeze)
+    val classProbs = logClassProbs.toArray.map(math.exp)
+    val classProbsSum = classProbs.sum
+    classProbs.map(_ / classProbsSum)
   }
 
   test("detect negative values") {
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala
index 406affa25539d..03a8a2538b464 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala
@@ -99,9 +99,13 @@ class LDASuite extends SparkFunSuite with MLlibTestSparkContext {
 
     // Check: per-doc topic distributions
     val topicDistributions = model.topicDistributions.collect()
+
     //  Ensure all documents are covered.
-    assert(topicDistributions.length === tinyCorpus.length)
-    assert(tinyCorpus.map(_._1).toSet === topicDistributions.map(_._1).toSet)
+    // SPARK-5562. since the topicDistribution returns the distribution of the non empty docs
+    // over topics. Compare it against nonEmptyTinyCorpus instead of tinyCorpus
+    val nonEmptyTinyCorpus = getNonEmptyDoc(tinyCorpus)
+    assert(topicDistributions.length === nonEmptyTinyCorpus.length)
+    assert(nonEmptyTinyCorpus.map(_._1).toSet === topicDistributions.map(_._1).toSet)
     //  Ensure we have proper distributions
     topicDistributions.foreach { case (docId, topicDistribution) =>
       assert(topicDistribution.size === tinyK)
@@ -232,12 +236,17 @@ private[clustering] object LDASuite {
   }
 
   def tinyCorpus: Array[(Long, Vector)] = Array(
+    Vectors.dense(0, 0, 0, 0, 0), // empty doc
     Vectors.dense(1, 3, 0, 2, 8),
     Vectors.dense(0, 2, 1, 0, 4),
     Vectors.dense(2, 3, 12, 3, 1),
+    Vectors.dense(0, 0, 0, 0, 0), // empty doc
     Vectors.dense(0, 3, 1, 9, 8),
     Vectors.dense(1, 1, 4, 2, 6)
   ).zipWithIndex.map { case (wordCounts, docId) => (docId.toLong, wordCounts) }
   assert(tinyCorpus.forall(_._2.size == tinyVocabSize)) // sanity check for test data
 
+  def getNonEmptyDoc(corpus: Array[(Long, Vector)]): Array[(Long, Vector)] = corpus.filter {
+    case (_, wc: Vector) => Vectors.norm(wc, p = 1.0) != 0.0
+  }
 }
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/fpm/AssociationRulesSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/fpm/AssociationRulesSuite.scala
new file mode 100644
index 0000000000000..77a2773c36f56
--- /dev/null
+++ b/mllib/src/test/scala/org/apache/spark/mllib/fpm/AssociationRulesSuite.scala
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.fpm
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.mllib.util.MLlibTestSparkContext
+
+class AssociationRulesSuite extends SparkFunSuite with MLlibTestSparkContext {
+
+  test("association rules using String type") {
+    val freqItemsets = sc.parallelize(Seq(
+      (Set("s"), 3L), (Set("z"), 5L), (Set("x"), 4L), (Set("t"), 3L), (Set("y"), 3L),
+      (Set("r"), 3L),
+      (Set("x", "z"), 3L), (Set("t", "y"), 3L), (Set("t", "x"), 3L), (Set("s", "x"), 3L),
+      (Set("y", "x"), 3L), (Set("y", "z"), 3L), (Set("t", "z"), 3L),
+      (Set("y", "x", "z"), 3L), (Set("t", "x", "z"), 3L), (Set("t", "y", "z"), 3L),
+      (Set("t", "y", "x"), 3L),
+      (Set("t", "y", "x", "z"), 3L)
+    ).map {
+      case (items, freq) => new FPGrowth.FreqItemset(items.toArray, freq)
+    })
+
+    val ar = new AssociationRules()
+
+    val results1 = ar
+      .setMinConfidence(0.9)
+      .run(freqItemsets)
+      .collect()
+
+    /* Verify results using the `R` code:
+       transactions = as(sapply(
+         list("r z h k p",
+              "z y x w v u t s",
+              "s x o n r",
+              "x z y m t s q e",
+              "z",
+              "x z y r q t p"),
+         FUN=function(x) strsplit(x," ",fixed=TRUE)),
+         "transactions")
+       ars = apriori(transactions,
+                     parameter = list(support = 0.0, confidence = 0.5, target="rules", minlen=2))
+       arsDF = as(ars, "data.frame")
+       arsDF$support = arsDF$support * length(transactions)
+       names(arsDF)[names(arsDF) == "support"] = "freq"
+       > nrow(arsDF)
+       [1] 23
+       > sum(arsDF$confidence == 1)
+       [1] 23
+     */
+    assert(results1.size === 23)
+    assert(results1.count(rule => math.abs(rule.confidence - 1.0D) < 1e-6) == 23)
+
+    val results2 = ar
+      .setMinConfidence(0)
+      .run(freqItemsets)
+      .collect()
+
+    /* Verify results using the `R` code:
+       ars = apriori(transactions,
+                  parameter = list(support = 0.5, confidence = 0.5, target="rules", minlen=2))
+       arsDF = as(ars, "data.frame")
+       arsDF$support = arsDF$support * length(transactions)
+       names(arsDF)[names(arsDF) == "support"] = "freq"
+       nrow(arsDF)
+       sum(arsDF$confidence == 1)
+       > nrow(arsDF)
+       [1] 30
+       > sum(arsDF$confidence == 1)
+       [1] 23
+     */
+    assert(results2.size === 30)
+    assert(results2.count(rule => math.abs(rule.confidence - 1.0D) < 1e-6) == 23)
+  }
+}
+
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/fpm/FPGrowthSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/fpm/FPGrowthSuite.scala
index 1a8a1e79f2810..4a9bfdb348d9f 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/fpm/FPGrowthSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/fpm/FPGrowthSuite.scala
@@ -22,7 +22,7 @@ import org.apache.spark.mllib.util.MLlibTestSparkContext
 class FPGrowthSuite extends SparkFunSuite with MLlibTestSparkContext {
 
 
-  test("FP-Growth frequent itemsets using String type") {
+  test("FP-Growth using String type") {
     val transactions = Seq(
       "r z h k p",
       "z y x w v u t s",
@@ -38,18 +38,59 @@ class FPGrowthSuite extends SparkFunSuite with MLlibTestSparkContext {
     val model6 = fpg
       .setMinSupport(0.9)
       .setNumPartitions(1)
-      .setOrdered(false)
       .run(rdd)
+
+    /* Verify results using the `R` code:
+       transactions = as(sapply(
+         list("r z h k p",
+              "z y x w v u t s",
+              "s x o n r",
+              "x z y m t s q e",
+              "z",
+              "x z y r q t p"),
+         FUN=function(x) strsplit(x," ",fixed=TRUE)),
+         "transactions")
+       > eclat(transactions, parameter = list(support = 0.9))
+       ...
+       eclat - zero frequent items
+       set of 0 itemsets
+     */
     assert(model6.freqItemsets.count() === 0)
 
     val model3 = fpg
       .setMinSupport(0.5)
       .setNumPartitions(2)
-      .setOrdered(false)
       .run(rdd)
     val freqItemsets3 = model3.freqItemsets.collect().map { itemset =>
       (itemset.items.toSet, itemset.freq)
     }
+
+    /* Verify results using the `R` code:
+       fp = eclat(transactions, parameter = list(support = 0.5))
+       fpDF = as(sort(fp), "data.frame")
+       fpDF$support = fpDF$support * length(transactions)
+       names(fpDF)[names(fpDF) == "support"] = "freq"
+       > fpDF
+              items freq
+       13       {z}    5
+       14       {x}    4
+       1      {s,x}    3
+       2  {t,x,y,z}    3
+       3    {t,y,z}    3
+       4    {t,x,y}    3
+       5    {x,y,z}    3
+       6      {y,z}    3
+       7      {x,y}    3
+       8      {t,y}    3
+       9    {t,x,z}    3
+       10     {t,z}    3
+       11     {t,x}    3
+       12     {x,z}    3
+       15       {t}    3
+       16       {y}    3
+       17       {s}    3
+       18       {r}    3
+     */
     val expected = Set(
       (Set("s"), 3L), (Set("z"), 5L), (Set("x"), 4L), (Set("t"), 3L), (Set("y"), 3L),
       (Set("r"), 3L),
@@ -63,19 +104,35 @@ class FPGrowthSuite extends SparkFunSuite with MLlibTestSparkContext {
     val model2 = fpg
       .setMinSupport(0.3)
       .setNumPartitions(4)
-      .setOrdered(false)
       .run(rdd)
+
+    /* Verify results using the `R` code:
+       fp = eclat(transactions, parameter = list(support = 0.3))
+       fpDF = as(fp, "data.frame")
+       fpDF$support = fpDF$support * length(transactions)
+       names(fpDF)[names(fpDF) == "support"] = "freq"
+       > nrow(fpDF)
+       [1] 54
+     */
     assert(model2.freqItemsets.count() === 54)
 
     val model1 = fpg
       .setMinSupport(0.1)
       .setNumPartitions(8)
-      .setOrdered(false)
       .run(rdd)
+
+    /* Verify results using the `R` code:
+       fp = eclat(transactions, parameter = list(support = 0.1))
+       fpDF = as(fp, "data.frame")
+       fpDF$support = fpDF$support * length(transactions)
+       names(fpDF)[names(fpDF) == "support"] = "freq"
+       > nrow(fpDF)
+       [1] 625
+     */
     assert(model1.freqItemsets.count() === 625)
   }
 
-  test("FP-Growth frequent sequences using String type"){
+  test("FP-Growth String type association rule generation") {
     val transactions = Seq(
       "r z h k p",
       "z y x w v u t s",
@@ -86,36 +143,38 @@ class FPGrowthSuite extends SparkFunSuite with MLlibTestSparkContext {
       .map(_.split(" "))
     val rdd = sc.parallelize(transactions, 2).cache()
 
-    val fpg = new FPGrowth()
-
-    val model1 = fpg
+    /* Verify results using the `R` code:
+       transactions = as(sapply(
+         list("r z h k p",
+              "z y x w v u t s",
+              "s x o n r",
+              "x z y m t s q e",
+              "z",
+              "x z y r q t p"),
+         FUN=function(x) strsplit(x," ",fixed=TRUE)),
+         "transactions")
+       ars = apriori(transactions,
+                     parameter = list(support = 0.0, confidence = 0.5, target="rules", minlen=2))
+       arsDF = as(ars, "data.frame")
+       arsDF$support = arsDF$support * length(transactions)
+       names(arsDF)[names(arsDF) == "support"] = "freq"
+       > nrow(arsDF)
+       [1] 23
+       > sum(arsDF$confidence == 1)
+       [1] 23
+     */
+    val rules = (new FPGrowth())
       .setMinSupport(0.5)
       .setNumPartitions(2)
-      .setOrdered(true)
       .run(rdd)
+      .generateAssociationRules(0.9)
+      .collect()
 
-    /*
-      Use the following R code to verify association rules using arulesSequences package.
-
-      data = read_baskets("path", info = c("sequenceID","eventID","SIZE"))
-      freqItemSeq = cspade(data, parameter = list(support = 0.5))
-      resSeq = as(freqItemSeq, "data.frame")
-      resSeq$support = resSeq$support * length(transactions)
-      names(resSeq)[names(resSeq) == "support"] = "freq"
-      resSeq
-     */
-    val expected = Set(
-      (Seq("r"), 3L), (Seq("s"), 3L), (Seq("t"), 3L), (Seq("x"), 4L), (Seq("y"), 3L),
-      (Seq("z"), 5L), (Seq("z", "y"), 3L), (Seq("x", "t"), 3L), (Seq("y", "t"), 3L),
-      (Seq("z", "t"), 3L), (Seq("z", "y", "t"), 3L)
-    )
-    val freqItemseqs1 = model1.freqItemsets.collect().map { itemset =>
-      (itemset.items.toSeq, itemset.freq)
-    }.toSet
-    assert(freqItemseqs1 == expected)
+    assert(rules.size === 23)
+    assert(rules.count(rule => math.abs(rule.confidence - 1.0D) < 1e-6) == 23)
   }
 
-  test("FP-Growth frequent itemsets using Int type") {
+  test("FP-Growth using Int type") {
     val transactions = Seq(
       "1 2 3",
       "1 2 3 4",
@@ -132,20 +191,53 @@ class FPGrowthSuite extends SparkFunSuite with MLlibTestSparkContext {
     val model6 = fpg
       .setMinSupport(0.9)
       .setNumPartitions(1)
-      .setOrdered(false)
       .run(rdd)
+
+    /* Verify results using the `R` code:
+       transactions = as(sapply(
+         list("1 2 3",
+              "1 2 3 4",
+              "5 4 3 2 1",
+              "6 5 4 3 2 1",
+              "2 4",
+              "1 3",
+              "1 7"),
+         FUN=function(x) strsplit(x," ",fixed=TRUE)),
+         "transactions")
+       > eclat(transactions, parameter = list(support = 0.9))
+       ...
+       eclat - zero frequent items
+       set of 0 itemsets
+     */
     assert(model6.freqItemsets.count() === 0)
 
     val model3 = fpg
       .setMinSupport(0.5)
       .setNumPartitions(2)
-      .setOrdered(false)
       .run(rdd)
     assert(model3.freqItemsets.first().items.getClass === Array(1).getClass,
       "frequent itemsets should use primitive arrays")
     val freqItemsets3 = model3.freqItemsets.collect().map { itemset =>
       (itemset.items.toSet, itemset.freq)
     }
+
+    /* Verify results using the `R` code:
+       fp = eclat(transactions, parameter = list(support = 0.5))
+       fpDF = as(sort(fp), "data.frame")
+       fpDF$support = fpDF$support * length(transactions)
+       names(fpDF)[names(fpDF) == "support"] = "freq"
+       > fpDF
+          items freq
+      6     {1}    6
+      3   {1,3}    5
+      7     {2}    5
+      8     {3}    5
+      1   {2,4}    4
+      2 {1,2,3}    4
+      4   {2,3}    4
+      5   {1,2}    4
+      9     {4}    4
+     */
     val expected = Set(
       (Set(1), 6L), (Set(2), 5L), (Set(3), 5L), (Set(4), 4L),
       (Set(1, 2), 4L), (Set(1, 3), 5L), (Set(2, 3), 4L),
@@ -155,15 +247,31 @@ class FPGrowthSuite extends SparkFunSuite with MLlibTestSparkContext {
     val model2 = fpg
       .setMinSupport(0.3)
       .setNumPartitions(4)
-      .setOrdered(false)
       .run(rdd)
+
+    /* Verify results using the `R` code:
+       fp = eclat(transactions, parameter = list(support = 0.3))
+       fpDF = as(fp, "data.frame")
+       fpDF$support = fpDF$support * length(transactions)
+       names(fpDF)[names(fpDF) == "support"] = "freq"
+       > nrow(fpDF)
+       [1] 15
+     */
     assert(model2.freqItemsets.count() === 15)
 
     val model1 = fpg
       .setMinSupport(0.1)
       .setNumPartitions(8)
-      .setOrdered(false)
       .run(rdd)
+
+    /* Verify results using the `R` code:
+       fp = eclat(transactions, parameter = list(support = 0.1))
+       fpDF = as(fp, "data.frame")
+       fpDF$support = fpDF$support * length(transactions)
+       names(fpDF)[names(fpDF) == "support"] = "freq"
+       > nrow(fpDF)
+       [1] 65
+     */
     assert(model1.freqItemsets.count() === 65)
   }
 }
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 c4ae0a16f7c04..178d95a7b94ec 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
@@ -21,10 +21,10 @@ import scala.util.Random
 
 import breeze.linalg.{DenseMatrix => BDM, squaredDistance => breezeSquaredDistance}
 
-import org.apache.spark.{SparkException, SparkFunSuite}
+import org.apache.spark.{Logging, SparkException, SparkFunSuite}
 import org.apache.spark.mllib.util.TestingUtils._
 
-class VectorsSuite extends SparkFunSuite {
+class VectorsSuite extends SparkFunSuite with Logging {
 
   val arr = Array(0.1, 0.0, 0.3, 0.4)
   val n = 4
@@ -142,7 +142,7 @@ class VectorsSuite extends SparkFunSuite {
     malformatted.foreach { s =>
       intercept[SparkException] {
         Vectors.parse(s)
-        println(s"Didn't detect malformatted string $s.")
+        logInfo(s"Didn't detect malformatted string $s.")
       }
     }
   }
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/stat/CorrelationSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/stat/CorrelationSuite.scala
index c292ced75e870..c3eeda012571c 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/stat/CorrelationSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/stat/CorrelationSuite.scala
@@ -19,13 +19,13 @@ package org.apache.spark.mllib.stat
 
 import breeze.linalg.{DenseMatrix => BDM, Matrix => BM}
 
-import org.apache.spark.SparkFunSuite
+import org.apache.spark.{Logging, SparkFunSuite}
 import org.apache.spark.mllib.linalg.Vectors
 import org.apache.spark.mllib.stat.correlation.{Correlations, PearsonCorrelation,
   SpearmanCorrelation}
 import org.apache.spark.mllib.util.MLlibTestSparkContext
 
-class CorrelationSuite extends SparkFunSuite with MLlibTestSparkContext {
+class CorrelationSuite extends SparkFunSuite with MLlibTestSparkContext with Logging {
 
   // test input data
   val xData = Array(1.0, 0.0, -2.0)
@@ -146,7 +146,7 @@ class CorrelationSuite extends SparkFunSuite with MLlibTestSparkContext {
   def matrixApproxEqual(A: BM[Double], B: BM[Double], threshold: Double = 1e-6): Boolean = {
     for (i <- 0 until A.rows; j <- 0 until A.cols) {
       if (!approxEqual(A(i, j), B(i, j), threshold)) {
-        println("i, j = " + i + ", " + j + " actual: " + A(i, j) + " expected:" + B(i, j))
+        logInfo("i, j = " + i + ", " + j + " actual: " + A(i, j) + " expected:" + B(i, j))
         return false
       }
     }
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/stat/HypothesisTestSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/stat/HypothesisTestSuite.scala
index b084a5fb4313f..142b90e764a7c 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/stat/HypothesisTestSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/stat/HypothesisTestSuite.scala
@@ -19,6 +19,10 @@ package org.apache.spark.mllib.stat
 
 import java.util.Random
 
+import org.apache.commons.math3.distribution.{ExponentialDistribution,
+  NormalDistribution, UniformRealDistribution}
+import org.apache.commons.math3.stat.inference.KolmogorovSmirnovTest
+
 import org.apache.spark.{SparkException, SparkFunSuite}
 import org.apache.spark.mllib.linalg.{DenseVector, Matrices, Vectors}
 import org.apache.spark.mllib.regression.LabeledPoint
@@ -153,4 +157,101 @@ class HypothesisTestSuite extends SparkFunSuite with MLlibTestSparkContext {
       Statistics.chiSqTest(sc.parallelize(continuousFeature, 2))
     }
   }
+
+  test("1 sample Kolmogorov-Smirnov test: apache commons math3 implementation equivalence") {
+    // Create theoretical distributions
+    val stdNormalDist = new NormalDistribution(0, 1)
+    val expDist = new ExponentialDistribution(0.6)
+    val unifDist = new UniformRealDistribution()
+
+    // set seeds
+    val seed = 10L
+    stdNormalDist.reseedRandomGenerator(seed)
+    expDist.reseedRandomGenerator(seed)
+    unifDist.reseedRandomGenerator(seed)
+
+    // Sample data from the distributions and parallelize it
+    val n = 100000
+    val sampledNorm = sc.parallelize(stdNormalDist.sample(n), 10)
+    val sampledExp = sc.parallelize(expDist.sample(n), 10)
+    val sampledUnif = sc.parallelize(unifDist.sample(n), 10)
+
+    // Use a apache math commons local KS test to verify calculations
+    val ksTest = new KolmogorovSmirnovTest()
+    val pThreshold = 0.05
+
+    // Comparing a standard normal sample to a standard normal distribution
+    val result1 = Statistics.kolmogorovSmirnovTest(sampledNorm, "norm", 0, 1)
+    val referenceStat1 = ksTest.kolmogorovSmirnovStatistic(stdNormalDist, sampledNorm.collect())
+    val referencePVal1 = 1 - ksTest.cdf(referenceStat1, n)
+    // Verify vs apache math commons ks test
+    assert(result1.statistic ~== referenceStat1 relTol 1e-4)
+    assert(result1.pValue ~== referencePVal1 relTol 1e-4)
+    // Cannot reject null hypothesis
+    assert(result1.pValue > pThreshold)
+
+    // Comparing an exponential sample to a standard normal distribution
+    val result2 = Statistics.kolmogorovSmirnovTest(sampledExp, "norm", 0, 1)
+    val referenceStat2 = ksTest.kolmogorovSmirnovStatistic(stdNormalDist, sampledExp.collect())
+    val referencePVal2 = 1 - ksTest.cdf(referenceStat2, n)
+    // verify vs apache math commons ks test
+    assert(result2.statistic ~== referenceStat2 relTol 1e-4)
+    assert(result2.pValue ~== referencePVal2 relTol 1e-4)
+    // reject null hypothesis
+    assert(result2.pValue < pThreshold)
+
+    // Testing the use of a user provided CDF function
+    // Distribution is not serializable, so will have to create in the lambda
+    val expCDF = (x: Double) => new ExponentialDistribution(0.2).cumulativeProbability(x)
+
+    // Comparing an exponential sample with mean X to an exponential distribution with mean Y
+    // Where X != Y
+    val result3 = Statistics.kolmogorovSmirnovTest(sampledExp, expCDF)
+    val referenceStat3 = ksTest.kolmogorovSmirnovStatistic(new ExponentialDistribution(0.2),
+      sampledExp.collect())
+    val referencePVal3 = 1 - ksTest.cdf(referenceStat3, sampledNorm.count().toInt)
+    // verify vs apache math commons ks test
+    assert(result3.statistic ~== referenceStat3 relTol 1e-4)
+    assert(result3.pValue ~== referencePVal3 relTol 1e-4)
+    // reject null hypothesis
+    assert(result3.pValue < pThreshold)
+  }
+
+  test("1 sample Kolmogorov-Smirnov test: R implementation equivalence") {
+    /*
+      Comparing results with R's implementation of Kolmogorov-Smirnov for 1 sample
+      > sessionInfo()
+      R version 3.2.0 (2015-04-16)
+      Platform: x86_64-apple-darwin13.4.0 (64-bit)
+      > set.seed(20)
+      > v <- rnorm(20)
+      > v
+       [1]  1.16268529 -0.58592447  1.78546500 -1.33259371 -0.44656677  0.56960612
+       [7] -2.88971761 -0.86901834 -0.46170268 -0.55554091 -0.02013537 -0.15038222
+      [13] -0.62812676  1.32322085 -1.52135057 -0.43742787  0.97057758  0.02822264
+      [19] -0.08578219  0.38921440
+      > ks.test(v, pnorm, alternative = "two.sided")
+
+               One-sample Kolmogorov-Smirnov test
+
+      data:  v
+      D = 0.18874, p-value = 0.4223
+      alternative hypothesis: two-sided
+    */
+
+    val rKSStat = 0.18874
+    val rKSPVal = 0.4223
+    val rData = sc.parallelize(
+      Array(
+        1.1626852897838, -0.585924465893051, 1.78546500331661, -1.33259371048501,
+        -0.446566766553219, 0.569606122374976, -2.88971761441412, -0.869018343326555,
+        -0.461702683149641, -0.555540910137444, -0.0201353678515895, -0.150382224136063,
+        -0.628126755843964, 1.32322085193283, -1.52135057001199, -0.437427868856691,
+        0.970577579543399, 0.0282226444247749, -0.0857821886527593, 0.389214404984942
+      )
+    )
+    val rCompResult = Statistics.kolmogorovSmirnovTest(rData, "norm", 0, 1)
+    assert(rCompResult.statistic ~== rKSStat relTol 1e-4)
+    assert(rCompResult.pValue ~== rKSPVal relTol 1e-4)
+  }
 }
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/GradientBoostedTreesSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/GradientBoostedTreesSuite.scala
index 84dd3b342d4c0..2521b3342181a 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/tree/GradientBoostedTreesSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/GradientBoostedTreesSuite.scala
@@ -17,7 +17,7 @@
 
 package org.apache.spark.mllib.tree
 
-import org.apache.spark.SparkFunSuite
+import org.apache.spark.{Logging, SparkFunSuite}
 import org.apache.spark.mllib.regression.LabeledPoint
 import org.apache.spark.mllib.tree.configuration.Algo._
 import org.apache.spark.mllib.tree.configuration.{BoostingStrategy, Strategy}
@@ -31,7 +31,7 @@ import org.apache.spark.util.Utils
 /**
  * Test suite for [[GradientBoostedTrees]].
  */
-class GradientBoostedTreesSuite extends SparkFunSuite with MLlibTestSparkContext {
+class GradientBoostedTreesSuite extends SparkFunSuite with MLlibTestSparkContext with Logging {
 
   test("Regression with continuous features: SquaredError") {
     GradientBoostedTreesSuite.testCombinations.foreach {
@@ -50,7 +50,7 @@ class GradientBoostedTreesSuite extends SparkFunSuite with MLlibTestSparkContext
           EnsembleTestHelper.validateRegressor(gbt, GradientBoostedTreesSuite.data, 0.06)
         } catch {
           case e: java.lang.AssertionError =>
-            println(s"FAILED for numIterations=$numIterations, learningRate=$learningRate," +
+            logError(s"FAILED for numIterations=$numIterations, learningRate=$learningRate," +
               s" subsamplingRate=$subsamplingRate")
             throw e
         }
@@ -80,7 +80,7 @@ class GradientBoostedTreesSuite extends SparkFunSuite with MLlibTestSparkContext
           EnsembleTestHelper.validateRegressor(gbt, GradientBoostedTreesSuite.data, 0.85, "mae")
         } catch {
           case e: java.lang.AssertionError =>
-            println(s"FAILED for numIterations=$numIterations, learningRate=$learningRate," +
+            logError(s"FAILED for numIterations=$numIterations, learningRate=$learningRate," +
               s" subsamplingRate=$subsamplingRate")
             throw e
         }
@@ -111,7 +111,7 @@ class GradientBoostedTreesSuite extends SparkFunSuite with MLlibTestSparkContext
           EnsembleTestHelper.validateClassifier(gbt, GradientBoostedTreesSuite.data, 0.9)
         } catch {
           case e: java.lang.AssertionError =>
-            println(s"FAILED for numIterations=$numIterations, learningRate=$learningRate," +
+            logError(s"FAILED for numIterations=$numIterations, learningRate=$learningRate," +
               s" subsamplingRate=$subsamplingRate")
             throw e
         }
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/NumericParserSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/NumericParserSuite.scala
index fa4f74d71b7e7..16d7c3ab39b03 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/util/NumericParserSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/util/NumericParserSuite.scala
@@ -33,7 +33,7 @@ class NumericParserSuite extends SparkFunSuite {
     malformatted.foreach { s =>
       intercept[SparkException] {
         NumericParser.parse(s)
-        println(s"Didn't detect malformatted string $s.")
+        throw new RuntimeException(s"Didn't detect malformatted string $s.")
       }
     }
   }
diff --git a/pom.xml b/pom.xml
index ffa96128a3d61..370c95dd03632 100644
--- a/pom.xml
+++ b/pom.xml
@@ -128,7 +128,7 @@
     ${hadoop.version}
     0.98.7-hadoop2
     hbase
-    1.4.0
+    1.6.0
     3.4.5
     2.4.0
     org.spark-project.hive
@@ -161,6 +161,8 @@
     2.4.4
     1.1.1.7
     1.1.2
+    
+    false
 
     ${java.home}
 
@@ -177,6 +179,7 @@
     compile
     compile
     compile
+    test
 
     
+    
+      twttr-repo
+      Twttr Repository
+      http://maven.twttr.com
+      
+        true
+      
+      
+        false
+      
+    
     
     
       spark-1.4-staging
@@ -326,11 +341,6 @@
   
   
     
-      
-        ${jline.groupid}
-        jline
-        ${jline.version}
-      
       
         com.twitter
         chill_${scala.binary.version}
@@ -1099,6 +1109,12 @@
         ${parquet.version}
         ${parquet.deps.scope}
       
+      
+        org.apache.parquet
+        parquet-avro
+        ${parquet.version}
+        ${parquet.test.deps.scope}
+      
       
         org.apache.flume
         flume-ng-core
@@ -1109,6 +1125,10 @@
             io.netty
             netty
           
+          
+            org.apache.flume
+            flume-ng-auth
+          
           
             org.apache.thrift
             libthrift
@@ -1304,6 +1324,7 @@
               false
               false
               true
+              true
             
           
           
@@ -1440,6 +1461,8 @@
         2.3
         
           false
+          
+          ${create.dependency.reduced.pom}
           
             
               
@@ -1713,7 +1736,6 @@
       
         2.3.0
         0.9.3
-        3.1.1
       
     
 
@@ -1722,7 +1744,6 @@
       
         2.4.0
         0.9.3
-        3.1.1
       
     
 
@@ -1731,7 +1752,6 @@
       
         2.6.0
         0.9.3
-        3.1.1
         3.4.6
         2.6.0
       
@@ -1801,6 +1821,15 @@
         ${scala.version}
         org.scala-lang
       
+      
+        
+          
+            ${jline.groupid}
+            jline
+            ${jline.version}
+          
+        
+      
     
 
     
@@ -1819,10 +1848,28 @@
         scala-2.11
       
       
-        2.11.6
+        2.11.7
         2.11
-        2.12.1
-        jline
+      
+    
+
+    
+      
+      release
+      
+        
+        true
       
     
 
diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala
index 680b699e9e4a1..4e4e810ec36e3 100644
--- a/project/MimaExcludes.scala
+++ b/project/MimaExcludes.scala
@@ -58,30 +58,45 @@ object MimaExcludes {
               "org.apache.spark.ml.regression.LeastSquaresAggregator.this"),
             ProblemFilters.exclude[MissingMethodProblem](
               "org.apache.spark.ml.regression.LeastSquaresCostFun.this"),
+            ProblemFilters.exclude[MissingMethodProblem](
+              "org.apache.spark.ml.classification.LogisticCostFun.this"),
             // SQL execution is considered private.
             excludePackage("org.apache.spark.sql.execution"),
-            // NanoTime and CatalystTimestampConverter is only used inside catalyst,
-            // not needed anymore
-            ProblemFilters.exclude[MissingClassProblem](
-              "org.apache.spark.sql.parquet.timestamp.NanoTime"),
-              ProblemFilters.exclude[MissingClassProblem](
-              "org.apache.spark.sql.parquet.timestamp.NanoTime$"),
-            ProblemFilters.exclude[MissingClassProblem](
-              "org.apache.spark.sql.parquet.CatalystTimestampConverter"),
-            ProblemFilters.exclude[MissingClassProblem](
-              "org.apache.spark.sql.parquet.CatalystTimestampConverter$"),
-            // SPARK-6777 Implements backwards compatibility rules in CatalystSchemaConverter
-            ProblemFilters.exclude[MissingClassProblem](
-              "org.apache.spark.sql.parquet.ParquetTypeInfo"),
-            ProblemFilters.exclude[MissingClassProblem](
-              "org.apache.spark.sql.parquet.ParquetTypeInfo$")
+            // Parquet support is considered private.
+            excludePackage("org.apache.spark.sql.parquet"),
+            // local function inside a method
+            ProblemFilters.exclude[MissingMethodProblem](
+              "org.apache.spark.sql.SQLContext.org$apache$spark$sql$SQLContext$$needsConversion$1")
           ) ++ Seq(
             // SPARK-8479 Add numNonzeros and numActives to Matrix.
             ProblemFilters.exclude[MissingMethodProblem](
               "org.apache.spark.mllib.linalg.Matrix.numNonzeros"),
             ProblemFilters.exclude[MissingMethodProblem](
               "org.apache.spark.mllib.linalg.Matrix.numActives")
+          ) ++ Seq(
+            // SPARK-8914 Remove RDDApi
+            ProblemFilters.exclude[MissingClassProblem](
+            "org.apache.spark.sql.RDDApi")
+          ) ++ Seq(
+            // SPARK-8701 Add input metadata in the batch page.
+            ProblemFilters.exclude[MissingClassProblem](
+              "org.apache.spark.streaming.scheduler.InputInfo$"),
+            ProblemFilters.exclude[MissingClassProblem](
+              "org.apache.spark.streaming.scheduler.InputInfo")
+          ) ++ Seq(
+            // SPARK-6797 Support YARN modes for SparkR
+            ProblemFilters.exclude[MissingMethodProblem](
+              "org.apache.spark.api.r.PairwiseRRDD.this"),
+            ProblemFilters.exclude[MissingMethodProblem](
+              "org.apache.spark.api.r.RRDD.createRWorker"),
+            ProblemFilters.exclude[MissingMethodProblem](
+              "org.apache.spark.api.r.RRDD.this"),
+            ProblemFilters.exclude[MissingMethodProblem](
+              "org.apache.spark.api.r.StringRRDD.this"),
+            ProblemFilters.exclude[MissingMethodProblem](
+              "org.apache.spark.api.r.BaseRRDD.this")
           )
+
         case v if v.startsWith("1.4") =>
           Seq(
             MimaBuild.excludeSparkPackage("deploy"),
diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
index 3408c6d51ed4c..4291b0be2a616 100644
--- a/project/SparkBuild.scala
+++ b/project/SparkBuild.scala
@@ -69,6 +69,7 @@ object SparkBuild extends PomBuild {
     import scala.collection.mutable
     var isAlphaYarn = false
     var profiles: mutable.Seq[String] = mutable.Seq("sbt")
+    // scalastyle:off println
     if (Properties.envOrNone("SPARK_GANGLIA_LGPL").isDefined) {
       println("NOTE: SPARK_GANGLIA_LGPL is deprecated, please use -Pspark-ganglia-lgpl flag.")
       profiles ++= Seq("spark-ganglia-lgpl")
@@ -88,6 +89,7 @@ object SparkBuild extends PomBuild {
       println("NOTE: SPARK_YARN is deprecated, please use -Pyarn flag.")
       profiles ++= Seq("yarn")
     }
+    // scalastyle:on println
     profiles
   }
 
@@ -96,8 +98,10 @@ object SparkBuild extends PomBuild {
     case None => backwardCompatibility
     case Some(v) =>
       if (backwardCompatibility.nonEmpty)
+        // scalastyle:off println
         println("Note: We ignore environment variables, when use of profile is detected in " +
           "conjunction with environment variable.")
+        // scalastyle:on println
       v.split("(\\s+|,)").filterNot(_.isEmpty).map(_.trim.replaceAll("-P", "")).toSeq
     }
 
diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py
index 7abbde8b260eb..89117e492846b 100644
--- a/python/pyspark/ml/classification.py
+++ b/python/pyspark/ml/classification.py
@@ -18,7 +18,8 @@
 from pyspark.ml.util import keyword_only
 from pyspark.ml.wrapper import JavaEstimator, JavaModel
 from pyspark.ml.param.shared import *
-from pyspark.ml.regression import RandomForestParams
+from pyspark.ml.regression import (
+    RandomForestParams, DecisionTreeModel, TreeEnsembleModels)
 from pyspark.mllib.common import inherit_doc
 
 
@@ -202,6 +203,10 @@ class DecisionTreeClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPred
     >>> td = si_model.transform(df)
     >>> dt = DecisionTreeClassifier(maxDepth=2, labelCol="indexed")
     >>> model = dt.fit(td)
+    >>> model.numNodes
+    3
+    >>> model.depth
+    1
     >>> test0 = sqlContext.createDataFrame([(Vectors.dense(-1.0),)], ["features"])
     >>> model.transform(test0).head().prediction
     0.0
@@ -269,7 +274,8 @@ def getImpurity(self):
         return self.getOrDefault(self.impurity)
 
 
-class DecisionTreeClassificationModel(JavaModel):
+@inherit_doc
+class DecisionTreeClassificationModel(DecisionTreeModel):
     """
     Model fitted by DecisionTreeClassifier.
     """
@@ -284,6 +290,7 @@ class RandomForestClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPred
     It supports both binary and multiclass labels, as well as both continuous and categorical
     features.
 
+    >>> from numpy import allclose
     >>> from pyspark.mllib.linalg import Vectors
     >>> from pyspark.ml.feature import StringIndexer
     >>> df = sqlContext.createDataFrame([
@@ -294,6 +301,8 @@ class RandomForestClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPred
     >>> td = si_model.transform(df)
     >>> rf = RandomForestClassifier(numTrees=2, maxDepth=2, labelCol="indexed", seed=42)
     >>> model = rf.fit(td)
+    >>> allclose(model.treeWeights, [1.0, 1.0])
+    True
     >>> test0 = sqlContext.createDataFrame([(Vectors.dense(-1.0),)], ["features"])
     >>> model.transform(test0).head().prediction
     0.0
@@ -423,7 +432,7 @@ def getFeatureSubsetStrategy(self):
         return self.getOrDefault(self.featureSubsetStrategy)
 
 
-class RandomForestClassificationModel(JavaModel):
+class RandomForestClassificationModel(TreeEnsembleModels):
     """
     Model fitted by RandomForestClassifier.
     """
@@ -438,6 +447,7 @@ class GBTClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol
     It supports binary labels, as well as both continuous and categorical features.
     Note: Multiclass labels are not currently supported.
 
+    >>> from numpy import allclose
     >>> from pyspark.mllib.linalg import Vectors
     >>> from pyspark.ml.feature import StringIndexer
     >>> df = sqlContext.createDataFrame([
@@ -448,6 +458,8 @@ class GBTClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol
     >>> td = si_model.transform(df)
     >>> gbt = GBTClassifier(maxIter=5, maxDepth=2, labelCol="indexed")
     >>> model = gbt.fit(td)
+    >>> allclose(model.treeWeights, [1.0, 0.1, 0.1, 0.1, 0.1])
+    True
     >>> test0 = sqlContext.createDataFrame([(Vectors.dense(-1.0),)], ["features"])
     >>> model.transform(test0).head().prediction
     0.0
@@ -558,7 +570,7 @@ def getStepSize(self):
         return self.getOrDefault(self.stepSize)
 
 
-class GBTClassificationModel(JavaModel):
+class GBTClassificationModel(TreeEnsembleModels):
     """
     Model fitted by GBTClassifier.
     """
diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py
index 8804dace849b3..9bca7cc000aa5 100644
--- a/python/pyspark/ml/feature.py
+++ b/python/pyspark/ml/feature.py
@@ -627,6 +627,10 @@ class StandardScaler(JavaEstimator, HasInputCol, HasOutputCol):
     >>> df = sqlContext.createDataFrame([(Vectors.dense([0.0]),), (Vectors.dense([2.0]),)], ["a"])
     >>> standardScaler = StandardScaler(inputCol="a", outputCol="scaled")
     >>> model = standardScaler.fit(df)
+    >>> model.mean
+    DenseVector([1.0])
+    >>> model.std
+    DenseVector([1.4142])
     >>> model.transform(df).collect()[1].scaled
     DenseVector([1.4142])
     """
@@ -692,6 +696,20 @@ class StandardScalerModel(JavaModel):
     Model fitted by StandardScaler.
     """
 
+    @property
+    def std(self):
+        """
+        Standard deviation of the StandardScalerModel.
+        """
+        return self._call_java("std")
+
+    @property
+    def mean(self):
+        """
+        Mean of the StandardScalerModel.
+        """
+        return self._call_java("mean")
+
 
 @inherit_doc
 class StringIndexer(JavaEstimator, HasInputCol, HasOutputCol):
diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py
index b139e27372d80..44f60a769566d 100644
--- a/python/pyspark/ml/regression.py
+++ b/python/pyspark/ml/regression.py
@@ -172,6 +172,10 @@ class DecisionTreeRegressor(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredi
     ...     (0.0, Vectors.sparse(1, [], []))], ["label", "features"])
     >>> dt = DecisionTreeRegressor(maxDepth=2)
     >>> model = dt.fit(df)
+    >>> model.depth
+    1
+    >>> model.numNodes
+    3
     >>> test0 = sqlContext.createDataFrame([(Vectors.dense(-1.0),)], ["features"])
     >>> model.transform(test0).head().prediction
     0.0
@@ -239,7 +243,37 @@ def getImpurity(self):
         return self.getOrDefault(self.impurity)
 
 
-class DecisionTreeRegressionModel(JavaModel):
+@inherit_doc
+class DecisionTreeModel(JavaModel):
+
+    @property
+    def numNodes(self):
+        """Return number of nodes of the decision tree."""
+        return self._call_java("numNodes")
+
+    @property
+    def depth(self):
+        """Return depth of the decision tree."""
+        return self._call_java("depth")
+
+    def __repr__(self):
+        return self._call_java("toString")
+
+
+@inherit_doc
+class TreeEnsembleModels(JavaModel):
+
+    @property
+    def treeWeights(self):
+        """Return the weights for each tree"""
+        return list(self._call_java("javaTreeWeights"))
+
+    def __repr__(self):
+        return self._call_java("toString")
+
+
+@inherit_doc
+class DecisionTreeRegressionModel(DecisionTreeModel):
     """
     Model fitted by DecisionTreeRegressor.
     """
@@ -253,12 +287,15 @@ class RandomForestRegressor(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredi
     learning algorithm for regression.
     It supports both continuous and categorical features.
 
+    >>> from numpy import allclose
     >>> from pyspark.mllib.linalg import Vectors
     >>> df = sqlContext.createDataFrame([
     ...     (1.0, Vectors.dense(1.0)),
     ...     (0.0, Vectors.sparse(1, [], []))], ["label", "features"])
     >>> rf = RandomForestRegressor(numTrees=2, maxDepth=2, seed=42)
     >>> model = rf.fit(df)
+    >>> allclose(model.treeWeights, [1.0, 1.0])
+    True
     >>> test0 = sqlContext.createDataFrame([(Vectors.dense(-1.0),)], ["features"])
     >>> model.transform(test0).head().prediction
     0.0
@@ -389,7 +426,7 @@ def getFeatureSubsetStrategy(self):
         return self.getOrDefault(self.featureSubsetStrategy)
 
 
-class RandomForestRegressionModel(JavaModel):
+class RandomForestRegressionModel(TreeEnsembleModels):
     """
     Model fitted by RandomForestRegressor.
     """
@@ -403,12 +440,15 @@ class GBTRegressor(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol,
     learning algorithm for regression.
     It supports both continuous and categorical features.
 
+    >>> from numpy import allclose
     >>> from pyspark.mllib.linalg import Vectors
     >>> df = sqlContext.createDataFrame([
     ...     (1.0, Vectors.dense(1.0)),
     ...     (0.0, Vectors.sparse(1, [], []))], ["label", "features"])
     >>> gbt = GBTRegressor(maxIter=5, maxDepth=2)
     >>> model = gbt.fit(df)
+    >>> allclose(model.treeWeights, [1.0, 0.1, 0.1, 0.1, 0.1])
+    True
     >>> test0 = sqlContext.createDataFrame([(Vectors.dense(-1.0),)], ["features"])
     >>> model.transform(test0).head().prediction
     0.0
@@ -518,7 +558,7 @@ def getStepSize(self):
         return self.getOrDefault(self.stepSize)
 
 
-class GBTRegressionModel(JavaModel):
+class GBTRegressionModel(TreeEnsembleModels):
     """
     Model fitted by GBTRegressor.
     """
diff --git a/python/pyspark/mllib/clustering.py b/python/pyspark/mllib/clustering.py
index a3eab635282f6..ed4d78a2c6788 100644
--- a/python/pyspark/mllib/clustering.py
+++ b/python/pyspark/mllib/clustering.py
@@ -282,18 +282,30 @@ class PowerIterationClusteringModel(JavaModelWrapper, JavaSaveable, JavaLoader):
 
     Model produced by [[PowerIterationClustering]].
 
-    >>> data = [(0, 1, 1.0), (0, 2, 1.0), (1, 3, 1.0), (2, 3, 1.0),
-    ...     (0, 3, 1.0), (1, 2, 1.0), (0, 4, 0.1)]
+    >>> data = [(0, 1, 1.0), (0, 2, 1.0), (0, 3, 1.0), (1, 2, 1.0), (1, 3, 1.0),
+    ... (2, 3, 1.0), (3, 4, 0.1), (4, 5, 1.0), (4, 15, 1.0), (5, 6, 1.0),
+    ... (6, 7, 1.0), (7, 8, 1.0), (8, 9, 1.0), (9, 10, 1.0), (10, 11, 1.0),
+    ... (11, 12, 1.0), (12, 13, 1.0), (13, 14, 1.0), (14, 15, 1.0)]
     >>> rdd = sc.parallelize(data, 2)
     >>> model = PowerIterationClustering.train(rdd, 2, 100)
     >>> model.k
     2
+    >>> result = sorted(model.assignments().collect(), key=lambda x: x.id)
+    >>> result[0].cluster == result[1].cluster == result[2].cluster == result[3].cluster
+    True
+    >>> result[4].cluster == result[5].cluster == result[6].cluster == result[7].cluster
+    True
     >>> import os, tempfile
     >>> path = tempfile.mkdtemp()
     >>> model.save(sc, path)
     >>> sameModel = PowerIterationClusteringModel.load(sc, path)
     >>> sameModel.k
     2
+    >>> result = sorted(model.assignments().collect(), key=lambda x: x.id)
+    >>> result[0].cluster == result[1].cluster == result[2].cluster == result[3].cluster
+    True
+    >>> result[4].cluster == result[5].cluster == result[6].cluster == result[7].cluster
+    True
     >>> from shutil import rmtree
     >>> try:
     ...     rmtree(path)
diff --git a/python/pyspark/mllib/evaluation.py b/python/pyspark/mllib/evaluation.py
index c5cf3a4e7ff22..f21403707e12a 100644
--- a/python/pyspark/mllib/evaluation.py
+++ b/python/pyspark/mllib/evaluation.py
@@ -152,6 +152,10 @@ class MulticlassMetrics(JavaModelWrapper):
     >>> predictionAndLabels = sc.parallelize([(0.0, 0.0), (0.0, 1.0), (0.0, 0.0),
     ...     (1.0, 0.0), (1.0, 1.0), (1.0, 1.0), (1.0, 1.0), (2.0, 2.0), (2.0, 0.0)])
     >>> metrics = MulticlassMetrics(predictionAndLabels)
+    >>> metrics.confusionMatrix().toArray()
+    array([[ 2.,  1.,  1.],
+           [ 1.,  3.,  0.],
+           [ 0.,  0.,  1.]])
     >>> metrics.falsePositiveRate(0.0)
     0.2...
     >>> metrics.precision(1.0)
@@ -186,6 +190,13 @@ def __init__(self, predictionAndLabels):
         java_model = java_class(df._jdf)
         super(MulticlassMetrics, self).__init__(java_model)
 
+    def confusionMatrix(self):
+        """
+        Returns confusion matrix: predicted classes are in columns,
+        they are ordered by class label ascending, as in "labels".
+        """
+        return self.call("confusionMatrix")
+
     def truePositiveRate(self, label):
         """
         Returns true positive rate for a given label (category).
diff --git a/python/pyspark/mllib/fpm.py b/python/pyspark/mllib/fpm.py
index b7f00d60069e6..bdc4a132b1b18 100644
--- a/python/pyspark/mllib/fpm.py
+++ b/python/pyspark/mllib/fpm.py
@@ -39,8 +39,8 @@ class FPGrowthModel(JavaModelWrapper):
     >>> data = [["a", "b", "c"], ["a", "b", "d", "e"], ["a", "c", "e"], ["a", "c", "f"]]
     >>> rdd = sc.parallelize(data, 2)
     >>> model = FPGrowth.train(rdd, 0.6, 2)
-    >>> sorted(model.freqItemsets().collect(), key=lambda x: x.items)
-    [FreqItemset(items=[u'a'], freq=4), FreqItemset(items=[u'a', u'c'], freq=3), ...
+    >>> sorted(model.freqItemsets().collect())
+    [FreqItemset(items=[u'a'], freq=4), FreqItemset(items=[u'c'], freq=3), ...
     """
 
     def freqItemsets(self):
diff --git a/python/pyspark/mllib/linalg.py b/python/pyspark/mllib/linalg.py
index 9959a01cce7e0..040886f71775b 100644
--- a/python/pyspark/mllib/linalg.py
+++ b/python/pyspark/mllib/linalg.py
@@ -31,6 +31,7 @@
     xrange = range
     import copyreg as copy_reg
 else:
+    from itertools import izip as zip
     import copy_reg
 
 import numpy as np
@@ -116,6 +117,10 @@ def _format_float(f, digits=4):
     return s
 
 
+def _format_float_list(l):
+    return [_format_float(x) for x in l]
+
+
 class VectorUDT(UserDefinedType):
     """
     SQL user-defined type (UDT) for Vector.
@@ -440,8 +445,10 @@ def __init__(self, size, *args):
         values (sorted by index).
 
         :param size: Size of the vector.
-        :param args: Non-zero entries, as a dictionary, list of tupes,
-               or two sorted lists containing indices and values.
+        :param args: Active entries, as a dictionary {index: value, ...},
+          a list of tuples [(index, value), ...], or a list of strictly i
+          ncreasing indices and a list of corresponding values [index, ...],
+          [value, ...]. Inactive entries are treated as zeros.
 
         >>> SparseVector(4, {1: 1.0, 3: 5.5})
         SparseVector(4, {1: 1.0, 3: 5.5})
@@ -451,6 +458,7 @@ def __init__(self, size, *args):
         SparseVector(4, {1: 1.0, 3: 5.5})
         """
         self.size = int(size)
+        """ Size of the vector. """
         assert 1 <= len(args) <= 2, "must pass either 2 or 3 arguments"
         if len(args) == 1:
             pairs = args[0]
@@ -458,7 +466,9 @@ def __init__(self, size, *args):
                 pairs = pairs.items()
             pairs = sorted(pairs)
             self.indices = np.array([p[0] for p in pairs], dtype=np.int32)
+            """ A list of indices corresponding to active entries. """
             self.values = np.array([p[1] for p in pairs], dtype=np.float64)
+            """ A list of values corresponding to active entries. """
         else:
             if isinstance(args[0], bytes):
                 assert isinstance(args[1], bytes), "values should be string too"
@@ -590,18 +600,14 @@ def dot(self, other):
             return np.dot(other.array[self.indices], self.values)
 
         elif isinstance(other, SparseVector):
-            result = 0.0
-            i, j = 0, 0
-            while i < len(self.indices) and j < len(other.indices):
-                if self.indices[i] == other.indices[j]:
-                    result += self.values[i] * other.values[j]
-                    i += 1
-                    j += 1
-                elif self.indices[i] < other.indices[j]:
-                    i += 1
-                else:
-                    j += 1
-            return result
+            # Find out common indices.
+            self_cmind = np.in1d(self.indices, other.indices, assume_unique=True)
+            self_values = self.values[self_cmind]
+            if self_values.size == 0:
+                return 0.0
+            else:
+                other_cmind = np.in1d(other.indices, self.indices, assume_unique=True)
+                return np.dot(self_values, other.values[other_cmind])
 
         else:
             return self.dot(_convert_to_vector(other))
@@ -874,6 +880,50 @@ def __reduce__(self):
             self.numRows, self.numCols, self.values.tostring(),
             int(self.isTransposed))
 
+    def __str__(self):
+        """
+        Pretty printing of a DenseMatrix
+
+        >>> dm = DenseMatrix(2, 2, range(4))
+        >>> print(dm)
+        DenseMatrix([[ 0.,  2.],
+                     [ 1.,  3.]])
+        >>> dm = DenseMatrix(2, 2, range(4), isTransposed=True)
+        >>> print(dm)
+        DenseMatrix([[ 0.,  1.],
+                     [ 2.,  3.]])
+        """
+        # Inspired by __repr__ in scipy matrices.
+        array_lines = repr(self.toArray()).splitlines()
+
+        # We need to adjust six spaces which is the difference in number
+        # of letters between "DenseMatrix" and "array"
+        x = '\n'.join([(" " * 6 + line) for line in array_lines[1:]])
+        return array_lines[0].replace("array", "DenseMatrix") + "\n" + x
+
+    def __repr__(self):
+        """
+        Representation of a DenseMatrix
+
+        >>> dm = DenseMatrix(2, 2, range(4))
+        >>> dm
+        DenseMatrix(2, 2, [0.0, 1.0, 2.0, 3.0], False)
+        """
+        # If the number of values are less than seventeen then return as it is.
+        # Else return first eight values and last eight values.
+        if len(self.values) < 17:
+            entries = _format_float_list(self.values)
+        else:
+            entries = (
+                _format_float_list(self.values[:8]) +
+                ["..."] +
+                _format_float_list(self.values[-8:])
+            )
+
+        entries = ", ".join(entries)
+        return "DenseMatrix({0}, {1}, [{2}], {3})".format(
+            self.numRows, self.numCols, entries, self.isTransposed)
+
     def toArray(self):
         """
         Return an numpy.ndarray
@@ -950,6 +1000,84 @@ def __init__(self, numRows, numCols, colPtrs, rowIndices, values,
             raise ValueError("Expected rowIndices of length %d, got %d."
                              % (self.rowIndices.size, self.values.size))
 
+    def __str__(self):
+        """
+        Pretty printing of a SparseMatrix
+
+        >>> sm1 = SparseMatrix(2, 2, [0, 2, 3], [0, 1, 1], [2, 3, 4])
+        >>> print(sm1)
+        2 X 2 CSCMatrix
+        (0,0) 2.0
+        (1,0) 3.0
+        (1,1) 4.0
+        >>> sm1 = SparseMatrix(2, 2, [0, 2, 3], [0, 1, 1], [2, 3, 4], True)
+        >>> print(sm1)
+        2 X 2 CSRMatrix
+        (0,0) 2.0
+        (0,1) 3.0
+        (1,1) 4.0
+        """
+        spstr = "{0} X {1} ".format(self.numRows, self.numCols)
+        if self.isTransposed:
+            spstr += "CSRMatrix\n"
+        else:
+            spstr += "CSCMatrix\n"
+
+        cur_col = 0
+        smlist = []
+
+        # Display first 16 values.
+        if len(self.values) <= 16:
+            zipindval = zip(self.rowIndices, self.values)
+        else:
+            zipindval = zip(self.rowIndices[:16], self.values[:16])
+        for i, (rowInd, value) in enumerate(zipindval):
+            if self.colPtrs[cur_col + 1] <= i:
+                cur_col += 1
+            if self.isTransposed:
+                smlist.append('({0},{1}) {2}'.format(
+                    cur_col, rowInd, _format_float(value)))
+            else:
+                smlist.append('({0},{1}) {2}'.format(
+                    rowInd, cur_col, _format_float(value)))
+        spstr += "\n".join(smlist)
+
+        if len(self.values) > 16:
+            spstr += "\n.." * 2
+        return spstr
+
+    def __repr__(self):
+        """
+        Representation of a SparseMatrix
+
+        >>> sm1 = SparseMatrix(2, 2, [0, 2, 3], [0, 1, 1], [2, 3, 4])
+        >>> sm1
+        SparseMatrix(2, 2, [0, 2, 3], [0, 1, 1], [2.0, 3.0, 4.0], False)
+        """
+        rowIndices = list(self.rowIndices)
+        colPtrs = list(self.colPtrs)
+
+        if len(self.values) <= 16:
+            values = _format_float_list(self.values)
+
+        else:
+            values = (
+                _format_float_list(self.values[:8]) +
+                ["..."] +
+                _format_float_list(self.values[-8:])
+            )
+            rowIndices = rowIndices[:8] + ["..."] + rowIndices[-8:]
+
+        if len(self.colPtrs) > 16:
+            colPtrs = colPtrs[:8] + ["..."] + colPtrs[-8:]
+
+        values = ", ".join(values)
+        rowIndices = ", ".join([str(ind) for ind in rowIndices])
+        colPtrs = ", ".join([str(ptr) for ptr in colPtrs])
+        return "SparseMatrix({0}, {1}, [{2}], [{3}], [{4}], {5})".format(
+            self.numRows, self.numCols, colPtrs, rowIndices,
+            values, self.isTransposed)
+
     def __reduce__(self):
         return SparseMatrix, (
             self.numRows, self.numCols, self.colPtrs.tostring(),
diff --git a/python/pyspark/mllib/tests.py b/python/pyspark/mllib/tests.py
index d9f9874d50c1a..f2eab5b18f077 100644
--- a/python/pyspark/mllib/tests.py
+++ b/python/pyspark/mllib/tests.py
@@ -27,7 +27,7 @@
 from shutil import rmtree
 
 from numpy import (
-    array, array_equal, zeros, inf, random, exp, dot, all, mean, abs)
+    array, array_equal, zeros, inf, random, exp, dot, all, mean, abs, arange, tile, ones)
 from numpy import sum as array_sum
 
 from py4j.protocol import Py4JJavaError
@@ -189,6 +189,53 @@ def test_matrix_indexing(self):
             for j in range(2):
                 self.assertEquals(mat[i, j], expected[i][j])
 
+    def test_repr_dense_matrix(self):
+        mat = DenseMatrix(3, 2, [0, 1, 4, 6, 8, 10])
+        self.assertTrue(
+            repr(mat),
+            'DenseMatrix(3, 2, [0.0, 1.0, 4.0, 6.0, 8.0, 10.0], False)')
+
+        mat = DenseMatrix(3, 2, [0, 1, 4, 6, 8, 10], True)
+        self.assertTrue(
+            repr(mat),
+            'DenseMatrix(3, 2, [0.0, 1.0, 4.0, 6.0, 8.0, 10.0], False)')
+
+        mat = DenseMatrix(6, 3, zeros(18))
+        self.assertTrue(
+            repr(mat),
+            'DenseMatrix(6, 3, [0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, ..., \
+                0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0], False)')
+
+    def test_repr_sparse_matrix(self):
+        sm1t = SparseMatrix(
+            3, 4, [0, 2, 3, 5], [0, 1, 2, 0, 2], [3.0, 2.0, 4.0, 9.0, 8.0],
+            isTransposed=True)
+        self.assertTrue(
+            repr(sm1t),
+            'SparseMatrix(3, 4, [0, 2, 3, 5], [0, 1, 2, 0, 2], [3.0, 2.0, 4.0, 9.0, 8.0], True)')
+
+        indices = tile(arange(6), 3)
+        values = ones(18)
+        sm = SparseMatrix(6, 3, [0, 6, 12, 18], indices, values)
+        self.assertTrue(
+            repr(sm), "SparseMatrix(6, 3, [0, 6, 12, 18], \
+                [0, 1, 2, 3, 4, 5, 0, 1, ..., 4, 5, 0, 1, 2, 3, 4, 5], \
+                [1.0, 1.0, 1.0, 1.0, 1.0, 1.0, 1.0, 1.0, ..., \
+                1.0, 1.0, 1.0, 1.0, 1.0, 1.0, 1.0, 1.0], False)")
+
+        self.assertTrue(
+            str(sm),
+            "6 X 3 CSCMatrix\n\
+            (0,0) 1.0\n(1,0) 1.0\n(2,0) 1.0\n(3,0) 1.0\n(4,0) 1.0\n(5,0) 1.0\n\
+            (0,1) 1.0\n(1,1) 1.0\n(2,1) 1.0\n(3,1) 1.0\n(4,1) 1.0\n(5,1) 1.0\n\
+            (0,2) 1.0\n(1,2) 1.0\n(2,2) 1.0\n(3,2) 1.0\n..\n..")
+
+        sm = SparseMatrix(1, 18, zeros(19), [], [])
+        self.assertTrue(
+            repr(sm),
+            'SparseMatrix(1, 18, \
+                [0, 0, 0, 0, 0, 0, 0, 0, ..., 0, 0, 0, 0, 0, 0, 0, 0], [], [], False)')
+
     def test_sparse_matrix(self):
         # Test sparse matrix creation.
         sm1 = SparseMatrix(
@@ -198,6 +245,9 @@ def test_sparse_matrix(self):
         self.assertEquals(sm1.colPtrs.tolist(), [0, 2, 2, 4, 4])
         self.assertEquals(sm1.rowIndices.tolist(), [1, 2, 1, 2])
         self.assertEquals(sm1.values.tolist(), [1.0, 2.0, 4.0, 5.0])
+        self.assertTrue(
+            repr(sm1),
+            'SparseMatrix(3, 4, [0, 2, 2, 4, 4], [1, 2, 1, 2], [1.0, 2.0, 4.0, 5.0], False)')
 
         # Test indexing
         expected = [
diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py
index 79dafb0a4ef27..3218bed5c74fc 100644
--- a/python/pyspark/rdd.py
+++ b/python/pyspark/rdd.py
@@ -700,12 +700,14 @@ def groupBy(self, f, numPartitions=None):
         return self.map(lambda x: (f(x), x)).groupByKey(numPartitions)
 
     @ignore_unicode_prefix
-    def pipe(self, command, env={}):
+    def pipe(self, command, env={}, checkCode=False):
         """
         Return an RDD created by piping elements to a forked external process.
 
         >>> sc.parallelize(['1', '2', '', '3']).pipe('cat').collect()
         [u'1', u'2', u'', u'3']
+
+        :param checkCode: whether or not to check the return value of the shell command.
         """
         def func(iterator):
             pipe = Popen(
@@ -717,7 +719,17 @@ def pipe_objs(out):
                     out.write(s.encode('utf-8'))
                 out.close()
             Thread(target=pipe_objs, args=[pipe.stdin]).start()
-            return (x.rstrip(b'\n').decode('utf-8') for x in iter(pipe.stdout.readline, b''))
+
+            def check_return_code():
+                pipe.wait()
+                if checkCode and pipe.returncode:
+                    raise Exception("Pipe function `%s' exited "
+                                    "with error code %d" % (command, pipe.returncode))
+                else:
+                    for i in range(0):
+                        yield i
+            return (x.rstrip(b'\n').decode('utf-8') for x in
+                    chain(iter(pipe.stdout.readline, b''), check_return_code()))
         return self.mapPartitions(func)
 
     def foreach(self, f):
diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py
index 309c11faf9319..c93a15badae29 100644
--- a/python/pyspark/sql/context.py
+++ b/python/pyspark/sql/context.py
@@ -30,7 +30,7 @@
 from pyspark.serializers import AutoBatchedSerializer, PickleSerializer
 from pyspark.sql import since
 from pyspark.sql.types import Row, StringType, StructType, _verify_type, \
-    _infer_schema, _has_nulltype, _merge_type, _create_converter, _python_to_sql_converter
+    _infer_schema, _has_nulltype, _merge_type, _create_converter
 from pyspark.sql.dataframe import DataFrame
 from pyspark.sql.readwriter import DataFrameReader
 from pyspark.sql.utils import install_exception_handler
@@ -388,8 +388,7 @@ def createDataFrame(self, data, schema=None, samplingRatio=None):
             raise TypeError("schema should be StructType or list or None")
 
         # convert python objects to sql data
-        converter = _python_to_sql_converter(schema)
-        rdd = rdd.map(converter)
+        rdd = rdd.map(schema.toInternal)
 
         jrdd = self._jvm.SerDeUtil.toJavaArray(rdd._to_java_object_rdd())
         df = self._ssql_ctx.applySchemaToPythonRDD(jrdd.rdd(), schema.json())
diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py
index 1e9c657cf81b3..83e02b85f06f1 100644
--- a/python/pyspark/sql/dataframe.py
+++ b/python/pyspark/sql/dataframe.py
@@ -31,7 +31,7 @@
 from pyspark.storagelevel import StorageLevel
 from pyspark.traceback_utils import SCCallSiteSync
 from pyspark.sql import since
-from pyspark.sql.types import _create_cls, _parse_datatype_json_string
+from pyspark.sql.types import _parse_datatype_json_string
 from pyspark.sql.column import Column, _to_seq, _to_java_column
 from pyspark.sql.readwriter import DataFrameWriter
 from pyspark.sql.types import *
@@ -83,15 +83,7 @@ def rdd(self):
         """
         if self._lazy_rdd is None:
             jrdd = self._jdf.javaToPython()
-            rdd = RDD(jrdd, self.sql_ctx._sc, BatchedSerializer(PickleSerializer()))
-            schema = self.schema
-
-            def applySchema(it):
-                cls = _create_cls(schema)
-                return map(cls, it)
-
-            self._lazy_rdd = rdd.mapPartitions(applySchema)
-
+            self._lazy_rdd = RDD(jrdd, self.sql_ctx._sc, BatchedSerializer(PickleSerializer()))
         return self._lazy_rdd
 
     @property
@@ -287,9 +279,7 @@ def collect(self):
         """
         with SCCallSiteSync(self._sc) as css:
             port = self._sc._jvm.PythonRDD.collectAndServe(self._jdf.javaToPython().rdd())
-        rs = list(_load_from_socket(port, BatchedSerializer(PickleSerializer())))
-        cls = _create_cls(self.schema)
-        return [cls(r) for r in rs]
+        return list(_load_from_socket(port, BatchedSerializer(PickleSerializer())))
 
     @ignore_unicode_prefix
     @since(1.3)
diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py
index 49dd0332afe74..dca39fa833435 100644
--- a/python/pyspark/sql/functions.py
+++ b/python/pyspark/sql/functions.py
@@ -395,6 +395,34 @@ def randn(seed=None):
     return Column(jc)
 
 
+@ignore_unicode_prefix
+@since(1.5)
+def hex(col):
+    """Computes hex value of the given column, which could be StringType,
+    BinaryType, IntegerType or LongType.
+
+    >>> sqlContext.createDataFrame([('ABC', 3)], ['a', 'b']).select(hex('a'), hex('b')).collect()
+    [Row(hex(a)=u'414243', hex(b)=u'3')]
+    """
+    sc = SparkContext._active_spark_context
+    jc = sc._jvm.functions.hex(_to_java_column(col))
+    return Column(jc)
+
+
+@ignore_unicode_prefix
+@since(1.5)
+def unhex(col):
+    """Inverse of hex. Interprets each pair of characters as a hexadecimal number
+    and converts to the byte representation of number.
+
+    >>> sqlContext.createDataFrame([('414243',)], ['a']).select(unhex('a')).collect()
+    [Row(unhex(a)=bytearray(b'ABC'))]
+    """
+    sc = SparkContext._active_spark_context
+    jc = sc._jvm.functions.unhex(_to_java_column(col))
+    return Column(jc)
+
+
 @ignore_unicode_prefix
 @since(1.5)
 def sha1(col):
diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py
index 333378c7f1854..241eac45cfe36 100644
--- a/python/pyspark/sql/tests.py
+++ b/python/pyspark/sql/tests.py
@@ -151,6 +151,17 @@ def test_range(self):
         self.assertEqual(self.sqlCtx.range(-2).count(), 0)
         self.assertEqual(self.sqlCtx.range(3).count(), 3)
 
+    def test_duplicated_column_names(self):
+        df = self.sqlCtx.createDataFrame([(1, 2)], ["c", "c"])
+        row = df.select('*').first()
+        self.assertEqual(1, row[0])
+        self.assertEqual(2, row[1])
+        self.assertEqual("Row(c=1, c=2)", str(row))
+        # Cannot access columns
+        self.assertRaises(AnalysisException, lambda: df.select(df[0]).first())
+        self.assertRaises(AnalysisException, lambda: df.select(df.c).first())
+        self.assertRaises(AnalysisException, lambda: df.select(df["c"]).first())
+
     def test_explode(self):
         from pyspark.sql.functions import explode
         d = [Row(a=1, intlist=[1, 2, 3], mapfield={"a": "b"})]
@@ -401,6 +412,14 @@ def test_apply_schema_with_udt(self):
         point = df.head().point
         self.assertEquals(point, ExamplePoint(1.0, 2.0))
 
+    def test_udf_with_udt(self):
+        from pyspark.sql.tests import ExamplePoint
+        row = Row(label=1.0, point=ExamplePoint(1.0, 2.0))
+        df = self.sc.parallelize([row]).toDF()
+        self.assertEqual(1.0, df.map(lambda r: r.point.x).first())
+        udf = UserDefinedFunction(lambda p: p.y, DoubleType())
+        self.assertEqual(2.0, df.select(udf(df.point)).first()[0])
+
     def test_parquet_with_udt(self):
         from pyspark.sql.tests import ExamplePoint
         row = Row(label=1.0, point=ExamplePoint(1.0, 2.0))
@@ -686,19 +705,31 @@ def test_filter_with_datetime(self):
     def test_time_with_timezone(self):
         day = datetime.date.today()
         now = datetime.datetime.now()
-        ts = time.mktime(now.timetuple()) + now.microsecond / 1e6
+        ts = time.mktime(now.timetuple())
         # class in __main__ is not serializable
         from pyspark.sql.tests import UTC
         utc = UTC()
-        utcnow = datetime.datetime.fromtimestamp(ts, utc)
+        utcnow = datetime.datetime.utcfromtimestamp(ts)  # without microseconds
+        # add microseconds to utcnow (keeping year,month,day,hour,minute,second)
+        utcnow = datetime.datetime(*(utcnow.timetuple()[:6] + (now.microsecond, utc)))
         df = self.sqlCtx.createDataFrame([(day, now, utcnow)])
         day1, now1, utcnow1 = df.first()
-        # Pyrolite serialize java.sql.Date as datetime, will be fixed in new version
-        self.assertEqual(day1.date(), day)
-        # Pyrolite does not support microsecond, the error should be
-        # less than 1 millisecond
-        self.assertTrue(now - now1 < datetime.timedelta(0.001))
-        self.assertTrue(now - utcnow1 < datetime.timedelta(0.001))
+        self.assertEqual(day1, day)
+        self.assertEqual(now, now1)
+        self.assertEqual(now, utcnow1)
+
+    def test_decimal(self):
+        from decimal import Decimal
+        schema = StructType([StructField("decimal", DecimalType(10, 5))])
+        df = self.sqlCtx.createDataFrame([(Decimal("3.14159"),)], schema)
+        row = df.select(df.decimal + 1).first()
+        self.assertEqual(row[0], Decimal("4.14159"))
+        tmpPath = tempfile.mkdtemp()
+        shutil.rmtree(tmpPath)
+        df.write.parquet(tmpPath)
+        df2 = self.sqlCtx.read.parquet(tmpPath)
+        row = df2.first()
+        self.assertEqual(row[0], Decimal("3.14159"))
 
     def test_dropna(self):
         schema = StructType([
diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py
index 160df40d65cc1..f75791fad1612 100644
--- a/python/pyspark/sql/types.py
+++ b/python/pyspark/sql/types.py
@@ -20,13 +20,9 @@
 import time
 import datetime
 import calendar
-import keyword
-import warnings
 import json
 import re
-import weakref
 from array import array
-from operator import itemgetter
 
 if sys.version >= "3":
     long = int
@@ -71,6 +67,26 @@ def json(self):
                           separators=(',', ':'),
                           sort_keys=True)
 
+    def needConversion(self):
+        """
+        Does this type need to conversion between Python object and internal SQL object.
+
+        This is used to avoid the unnecessary conversion for ArrayType/MapType/StructType.
+        """
+        return False
+
+    def toInternal(self, obj):
+        """
+        Converts a Python object into an internal SQL object.
+        """
+        return obj
+
+    def fromInternal(self, obj):
+        """
+        Converts an internal SQL object into a native Python object.
+        """
+        return obj
+
 
 # This singleton pattern does not work with pickle, you will get
 # another object after pickle and unpickle
@@ -143,6 +159,17 @@ class DateType(AtomicType):
 
     __metaclass__ = DataTypeSingleton
 
+    EPOCH_ORDINAL = datetime.datetime(1970, 1, 1).toordinal()
+
+    def needConversion(self):
+        return True
+
+    def toInternal(self, d):
+        return d and d.toordinal() - self.EPOCH_ORDINAL
+
+    def fromInternal(self, v):
+        return v and datetime.date.fromordinal(v + self.EPOCH_ORDINAL)
+
 
 class TimestampType(AtomicType):
     """Timestamp (datetime.datetime) data type.
@@ -150,6 +177,20 @@ class TimestampType(AtomicType):
 
     __metaclass__ = DataTypeSingleton
 
+    def needConversion(self):
+        return True
+
+    def toInternal(self, dt):
+        if dt is not None:
+            seconds = (calendar.timegm(dt.utctimetuple()) if dt.tzinfo
+                       else time.mktime(dt.timetuple()))
+            return int(seconds * 1e6 + dt.microsecond)
+
+    def fromInternal(self, ts):
+        if ts is not None:
+            # using int to avoid precision loss in float
+            return datetime.datetime.fromtimestamp(ts // 1000000).replace(microsecond=ts % 1000000)
+
 
 class DecimalType(FractionalType):
     """Decimal (decimal.Decimal) data type.
@@ -259,6 +300,19 @@ def fromJson(cls, json):
         return ArrayType(_parse_datatype_json_value(json["elementType"]),
                          json["containsNull"])
 
+    def needConversion(self):
+        return self.elementType.needConversion()
+
+    def toInternal(self, obj):
+        if not self.needConversion():
+            return obj
+        return obj and [self.elementType.toInternal(v) for v in obj]
+
+    def fromInternal(self, obj):
+        if not self.needConversion():
+            return obj
+        return obj and [self.elementType.fromInternal(v) for v in obj]
+
 
 class MapType(DataType):
     """Map data type.
@@ -304,6 +358,21 @@ def fromJson(cls, json):
                        _parse_datatype_json_value(json["valueType"]),
                        json["valueContainsNull"])
 
+    def needConversion(self):
+        return self.keyType.needConversion() or self.valueType.needConversion()
+
+    def toInternal(self, obj):
+        if not self.needConversion():
+            return obj
+        return obj and dict((self.keyType.toInternal(k), self.valueType.toInternal(v))
+                            for k, v in obj.items())
+
+    def fromInternal(self, obj):
+        if not self.needConversion():
+            return obj
+        return obj and dict((self.keyType.fromInternal(k), self.valueType.fromInternal(v))
+                            for k, v in obj.items())
+
 
 class StructField(DataType):
     """A field in :class:`StructType`.
@@ -311,7 +380,7 @@ class StructField(DataType):
     :param name: string, name of the field.
     :param dataType: :class:`DataType` of the field.
     :param nullable: boolean, whether the field can be null (None) or not.
-    :param metadata: a dict from string to simple type that can be serialized to JSON automatically
+    :param metadata: a dict from string to simple type that can be toInternald to JSON automatically
     """
 
     def __init__(self, name, dataType, nullable=True, metadata=None):
@@ -351,6 +420,15 @@ def fromJson(cls, json):
                            json["nullable"],
                            json["metadata"])
 
+    def needConversion(self):
+        return self.dataType.needConversion()
+
+    def toInternal(self, obj):
+        return self.dataType.toInternal(obj)
+
+    def fromInternal(self, obj):
+        return self.dataType.fromInternal(obj)
+
 
 class StructType(DataType):
     """Struct type, consisting of a list of :class:`StructField`.
@@ -371,10 +449,13 @@ def __init__(self, fields=None):
         """
         if not fields:
             self.fields = []
+            self.names = []
         else:
             self.fields = fields
+            self.names = [f.name for f in fields]
             assert all(isinstance(f, StructField) for f in fields),\
                 "fields should be a list of StructField"
+        self._needSerializeFields = None
 
     def add(self, field, data_type=None, nullable=True, metadata=None):
         """
@@ -406,6 +487,7 @@ def add(self, field, data_type=None, nullable=True, metadata=None):
         """
         if isinstance(field, StructField):
             self.fields.append(field)
+            self.names.append(field.name)
         else:
             if isinstance(field, str) and data_type is None:
                 raise ValueError("Must specify DataType if passing name of struct_field to create.")
@@ -415,6 +497,7 @@ def add(self, field, data_type=None, nullable=True, metadata=None):
             else:
                 data_type_f = data_type
             self.fields.append(StructField(field, data_type_f, nullable, metadata))
+            self.names.append(field)
         return self
 
     def simpleString(self):
@@ -432,6 +515,41 @@ def jsonValue(self):
     def fromJson(cls, json):
         return StructType([StructField.fromJson(f) for f in json["fields"]])
 
+    def needConversion(self):
+        # We need convert Row()/namedtuple into tuple()
+        return True
+
+    def toInternal(self, obj):
+        if obj is None:
+            return
+
+        if self._needSerializeFields is None:
+            self._needSerializeFields = any(f.needConversion() for f in self.fields)
+
+        if self._needSerializeFields:
+            if isinstance(obj, dict):
+                return tuple(f.toInternal(obj.get(n)) for n, f in zip(names, self.fields))
+            elif isinstance(obj, (tuple, list)):
+                return tuple(f.toInternal(v) for f, v in zip(self.fields, obj))
+            else:
+                raise ValueError("Unexpected tuple %r with StructType" % obj)
+        else:
+            if isinstance(obj, dict):
+                return tuple(obj.get(n) for n in self.names)
+            elif isinstance(obj, (list, tuple)):
+                return tuple(obj)
+            else:
+                raise ValueError("Unexpected tuple %r with StructType" % obj)
+
+    def fromInternal(self, obj):
+        if obj is None:
+            return
+        if isinstance(obj, Row):
+            # it's already converted by pickler
+            return obj
+        values = [f.dataType.fromInternal(v) for f, v in zip(self.fields, obj)]
+        return _create_row(self.names, values)
+
 
 class UserDefinedType(DataType):
     """User-defined type (UDT).
@@ -464,17 +582,35 @@ def scalaUDT(cls):
         """
         raise NotImplementedError("UDT must have a paired Scala UDT.")
 
+    def needConversion(self):
+        return True
+
+    @classmethod
+    def _cachedSqlType(cls):
+        """
+        Cache the sqlType() into class, because it's heavy used in `toInternal`.
+        """
+        if not hasattr(cls, "_cached_sql_type"):
+            cls._cached_sql_type = cls.sqlType()
+        return cls._cached_sql_type
+
+    def toInternal(self, obj):
+        return self._cachedSqlType().toInternal(self.serialize(obj))
+
+    def fromInternal(self, obj):
+        return self.deserialize(self._cachedSqlType().fromInternal(obj))
+
     def serialize(self, obj):
         """
         Converts the a user-type object into a SQL datum.
         """
-        raise NotImplementedError("UDT must implement serialize().")
+        raise NotImplementedError("UDT must implement toInternal().")
 
     def deserialize(self, datum):
         """
         Converts a SQL datum into a user-type object.
         """
-        raise NotImplementedError("UDT must implement deserialize().")
+        raise NotImplementedError("UDT must implement fromInternal().")
 
     def simpleString(self):
         return 'udt'
@@ -671,117 +807,6 @@ def _infer_schema(row):
     return StructType(fields)
 
 
-def _need_python_to_sql_conversion(dataType):
-    """
-    Checks whether we need python to sql conversion for the given type.
-    For now, only UDTs need this conversion.
-
-    >>> _need_python_to_sql_conversion(DoubleType())
-    False
-    >>> schema0 = StructType([StructField("indices", ArrayType(IntegerType(), False), False),
-    ...                       StructField("values", ArrayType(DoubleType(), False), False)])
-    >>> _need_python_to_sql_conversion(schema0)
-    True
-    >>> _need_python_to_sql_conversion(ExamplePointUDT())
-    True
-    >>> schema1 = ArrayType(ExamplePointUDT(), False)
-    >>> _need_python_to_sql_conversion(schema1)
-    True
-    >>> schema2 = StructType([StructField("label", DoubleType(), False),
-    ...                       StructField("point", ExamplePointUDT(), False)])
-    >>> _need_python_to_sql_conversion(schema2)
-    True
-    """
-    if isinstance(dataType, StructType):
-        # convert namedtuple or Row into tuple
-        return True
-    elif isinstance(dataType, ArrayType):
-        return _need_python_to_sql_conversion(dataType.elementType)
-    elif isinstance(dataType, MapType):
-        return _need_python_to_sql_conversion(dataType.keyType) or \
-            _need_python_to_sql_conversion(dataType.valueType)
-    elif isinstance(dataType, UserDefinedType):
-        return True
-    elif isinstance(dataType, (DateType, TimestampType)):
-        return True
-    else:
-        return False
-
-
-EPOCH_ORDINAL = datetime.datetime(1970, 1, 1).toordinal()
-
-
-def _python_to_sql_converter(dataType):
-    """
-    Returns a converter that converts a Python object into a SQL datum for the given type.
-
-    >>> conv = _python_to_sql_converter(DoubleType())
-    >>> conv(1.0)
-    1.0
-    >>> conv = _python_to_sql_converter(ArrayType(DoubleType(), False))
-    >>> conv([1.0, 2.0])
-    [1.0, 2.0]
-    >>> conv = _python_to_sql_converter(ExamplePointUDT())
-    >>> conv(ExamplePoint(1.0, 2.0))
-    [1.0, 2.0]
-    >>> schema = StructType([StructField("label", DoubleType(), False),
-    ...                      StructField("point", ExamplePointUDT(), False)])
-    >>> conv = _python_to_sql_converter(schema)
-    >>> conv((1.0, ExamplePoint(1.0, 2.0)))
-    (1.0, [1.0, 2.0])
-    """
-    if not _need_python_to_sql_conversion(dataType):
-        return lambda x: x
-
-    if isinstance(dataType, StructType):
-        names, types = zip(*[(f.name, f.dataType) for f in dataType.fields])
-        if any(_need_python_to_sql_conversion(t) for t in types):
-            converters = [_python_to_sql_converter(t) for t in types]
-
-            def converter(obj):
-                if isinstance(obj, dict):
-                    return tuple(c(obj.get(n)) for n, c in zip(names, converters))
-                elif isinstance(obj, tuple):
-                    if hasattr(obj, "__fields__") or hasattr(obj, "_fields"):
-                        return tuple(c(v) for c, v in zip(converters, obj))
-                    else:
-                        return tuple(c(v) for c, v in zip(converters, obj))
-                elif obj is not None:
-                    raise ValueError("Unexpected tuple %r with type %r" % (obj, dataType))
-        else:
-            def converter(obj):
-                if isinstance(obj, dict):
-                    return tuple(obj.get(n) for n in names)
-                else:
-                    return tuple(obj)
-        return converter
-    elif isinstance(dataType, ArrayType):
-        element_converter = _python_to_sql_converter(dataType.elementType)
-        return lambda a: a and [element_converter(v) for v in a]
-    elif isinstance(dataType, MapType):
-        key_converter = _python_to_sql_converter(dataType.keyType)
-        value_converter = _python_to_sql_converter(dataType.valueType)
-        return lambda m: m and dict([(key_converter(k), value_converter(v)) for k, v in m.items()])
-
-    elif isinstance(dataType, UserDefinedType):
-        return lambda obj: obj and dataType.serialize(obj)
-
-    elif isinstance(dataType, DateType):
-        return lambda d: d and d.toordinal() - EPOCH_ORDINAL
-
-    elif isinstance(dataType, TimestampType):
-
-        def to_posix_timstamp(dt):
-            if dt:
-                seconds = (calendar.timegm(dt.utctimetuple()) if dt.tzinfo
-                           else time.mktime(dt.timetuple()))
-                return int(seconds * 1e7 + dt.microsecond * 10)
-        return to_posix_timstamp
-
-    else:
-        raise ValueError("Unexpected type %r" % dataType)
-
-
 def _has_nulltype(dt):
     """ Return whether there is NullType in `dt` or not """
     if isinstance(dt, StructType):
@@ -1069,10 +1094,14 @@ def _verify_type(obj, dataType):
     if obj is None:
         return
 
+    # StringType can work with any types
+    if isinstance(dataType, StringType):
+        return
+
     if isinstance(dataType, UserDefinedType):
         if not (hasattr(obj, '__UDT__') and obj.__UDT__ == dataType):
             raise ValueError("%r is not an instance of type %r" % (obj, dataType))
-        _verify_type(dataType.serialize(obj), dataType.sqlType())
+        _verify_type(dataType.toInternal(obj), dataType.sqlType())
         return
 
     _type = type(dataType)
@@ -1082,7 +1111,7 @@ def _verify_type(obj, dataType):
         if not isinstance(obj, (tuple, list)):
             raise TypeError("StructType can not accept object in type %s" % type(obj))
     else:
-        # subclass of them can not be deserialized in JVM
+        # subclass of them can not be fromInternald in JVM
         if type(obj) not in _acceptable_types[_type]:
             raise TypeError("%s can not accept object in type %s" % (dataType, type(obj)))
 
@@ -1102,159 +1131,10 @@ def _verify_type(obj, dataType):
         for v, f in zip(obj, dataType.fields):
             _verify_type(v, f.dataType)
 
-_cached_cls = weakref.WeakValueDictionary()
-
-
-def _restore_object(dataType, obj):
-    """ Restore object during unpickling. """
-    # use id(dataType) as key to speed up lookup in dict
-    # Because of batched pickling, dataType will be the
-    # same object in most cases.
-    k = id(dataType)
-    cls = _cached_cls.get(k)
-    if cls is None or cls.__datatype is not dataType:
-        # use dataType as key to avoid create multiple class
-        cls = _cached_cls.get(dataType)
-        if cls is None:
-            cls = _create_cls(dataType)
-            _cached_cls[dataType] = cls
-        cls.__datatype = dataType
-        _cached_cls[k] = cls
-    return cls(obj)
-
-
-def _create_object(cls, v):
-    """ Create an customized object with class `cls`. """
-    # datetime.date would be deserialized as datetime.datetime
-    # from java type, so we need to set it back.
-    if cls is datetime.date and isinstance(v, datetime.datetime):
-        return v.date()
-    return cls(v) if v is not None else v
-
-
-def _create_getter(dt, i):
-    """ Create a getter for item `i` with schema """
-    cls = _create_cls(dt)
-
-    def getter(self):
-        return _create_object(cls, self[i])
-
-    return getter
-
-
-def _has_struct_or_date(dt):
-    """Return whether `dt` is or has StructType/DateType in it"""
-    if isinstance(dt, StructType):
-        return True
-    elif isinstance(dt, ArrayType):
-        return _has_struct_or_date(dt.elementType)
-    elif isinstance(dt, MapType):
-        return _has_struct_or_date(dt.keyType) or _has_struct_or_date(dt.valueType)
-    elif isinstance(dt, DateType):
-        return True
-    elif isinstance(dt, UserDefinedType):
-        return True
-    return False
-
-
-def _create_properties(fields):
-    """Create properties according to fields"""
-    ps = {}
-    for i, f in enumerate(fields):
-        name = f.name
-        if (name.startswith("__") and name.endswith("__")
-                or keyword.iskeyword(name)):
-            warnings.warn("field name %s can not be accessed in Python,"
-                          "use position to access it instead" % name)
-        if _has_struct_or_date(f.dataType):
-            # delay creating object until accessing it
-            getter = _create_getter(f.dataType, i)
-        else:
-            getter = itemgetter(i)
-        ps[name] = property(getter)
-    return ps
-
-
-def _create_cls(dataType):
-    """
-    Create an class by dataType
-
-    The created class is similar to namedtuple, but can have nested schema.
-
-    >>> schema = _parse_schema_abstract("a b c")
-    >>> row = (1, 1.0, "str")
-    >>> schema = _infer_schema_type(row, schema)
-    >>> obj = _create_cls(schema)(row)
-    >>> import pickle
-    >>> pickle.loads(pickle.dumps(obj))
-    Row(a=1, b=1.0, c='str')
-
-    >>> row = [[1], {"key": (1, 2.0)}]
-    >>> schema = _parse_schema_abstract("a[] b{c d}")
-    >>> schema = _infer_schema_type(row, schema)
-    >>> obj = _create_cls(schema)(row)
-    >>> pickle.loads(pickle.dumps(obj))
-    Row(a=[1], b={'key': Row(c=1, d=2.0)})
-    >>> pickle.loads(pickle.dumps(obj.a))
-    [1]
-    >>> pickle.loads(pickle.dumps(obj.b))
-    {'key': Row(c=1, d=2.0)}
-    """
-
-    if isinstance(dataType, ArrayType):
-        cls = _create_cls(dataType.elementType)
-
-        def List(l):
-            if l is None:
-                return
-            return [_create_object(cls, v) for v in l]
-
-        return List
-
-    elif isinstance(dataType, MapType):
-        kcls = _create_cls(dataType.keyType)
-        vcls = _create_cls(dataType.valueType)
-
-        def Dict(d):
-            if d is None:
-                return
-            return dict((_create_object(kcls, k), _create_object(vcls, v)) for k, v in d.items())
-
-        return Dict
-
-    elif isinstance(dataType, DateType):
-        return datetime.date
-
-    elif isinstance(dataType, UserDefinedType):
-        return lambda datum: dataType.deserialize(datum)
-
-    elif not isinstance(dataType, StructType):
-        # no wrapper for atomic types
-        return lambda x: x
-
-    class Row(tuple):
-
-        """ Row in DataFrame """
-        __datatype = dataType
-        __fields__ = tuple(f.name for f in dataType.fields)
-        __slots__ = ()
-
-        # create property for fast access
-        locals().update(_create_properties(dataType.fields))
-
-        def asDict(self):
-            """ Return as a dict """
-            return dict((n, getattr(self, n)) for n in self.__fields__)
-
-        def __repr__(self):
-            # call collect __repr__ for nested objects
-            return ("Row(%s)" % ", ".join("%s=%r" % (n, getattr(self, n))
-                                          for n in self.__fields__))
-
-        def __reduce__(self):
-            return (_restore_object, (self.__datatype, tuple(self)))
 
-    return Row
+# This is used to unpickle a Row from JVM
+def _create_row_inbound_converter(dataType):
+    return lambda *a: dataType.fromInternal(a)
 
 
 def _create_row(fields, values):
diff --git a/python/pyspark/streaming/kafka.py b/python/pyspark/streaming/kafka.py
index 10a859a532e28..33dd596335b47 100644
--- a/python/pyspark/streaming/kafka.py
+++ b/python/pyspark/streaming/kafka.py
@@ -21,6 +21,8 @@
 from pyspark.storagelevel import StorageLevel
 from pyspark.serializers import PairDeserializer, NoOpSerializer
 from pyspark.streaming import DStream
+from pyspark.streaming.dstream import TransformedDStream
+from pyspark.streaming.util import TransformFunction
 
 __all__ = ['Broker', 'KafkaUtils', 'OffsetRange', 'TopicAndPartition', 'utf8_decoder']
 
@@ -122,8 +124,9 @@ def createDirectStream(ssc, topics, kafkaParams, fromOffsets={},
             raise e
 
         ser = PairDeserializer(NoOpSerializer(), NoOpSerializer())
-        stream = DStream(jstream, ssc, ser)
-        return stream.map(lambda k_v: (keyDecoder(k_v[0]), valueDecoder(k_v[1])))
+        stream = DStream(jstream, ssc, ser) \
+            .map(lambda k_v: (keyDecoder(k_v[0]), valueDecoder(k_v[1])))
+        return KafkaDStream(stream._jdstream, ssc, stream._jrdd_deserializer)
 
     @staticmethod
     def createRDD(sc, kafkaParams, offsetRanges, leaders={},
@@ -161,8 +164,8 @@ def createRDD(sc, kafkaParams, offsetRanges, leaders={},
             raise e
 
         ser = PairDeserializer(NoOpSerializer(), NoOpSerializer())
-        rdd = RDD(jrdd, sc, ser)
-        return rdd.map(lambda k_v: (keyDecoder(k_v[0]), valueDecoder(k_v[1])))
+        rdd = RDD(jrdd, sc, ser).map(lambda k_v: (keyDecoder(k_v[0]), valueDecoder(k_v[1])))
+        return KafkaRDD(rdd._jrdd, rdd.ctx, rdd._jrdd_deserializer)
 
     @staticmethod
     def _printErrorMsg(sc):
@@ -200,14 +203,30 @@ def __init__(self, topic, partition, fromOffset, untilOffset):
         :param fromOffset: Inclusive starting offset.
         :param untilOffset: Exclusive ending offset.
         """
-        self._topic = topic
-        self._partition = partition
-        self._fromOffset = fromOffset
-        self._untilOffset = untilOffset
+        self.topic = topic
+        self.partition = partition
+        self.fromOffset = fromOffset
+        self.untilOffset = untilOffset
+
+    def __eq__(self, other):
+        if isinstance(other, self.__class__):
+            return (self.topic == other.topic
+                    and self.partition == other.partition
+                    and self.fromOffset == other.fromOffset
+                    and self.untilOffset == other.untilOffset)
+        else:
+            return False
+
+    def __ne__(self, other):
+        return not self.__eq__(other)
+
+    def __str__(self):
+        return "OffsetRange(topic: %s, partition: %d, range: [%d -> %d]" \
+               % (self.topic, self.partition, self.fromOffset, self.untilOffset)
 
     def _jOffsetRange(self, helper):
-        return helper.createOffsetRange(self._topic, self._partition, self._fromOffset,
-                                        self._untilOffset)
+        return helper.createOffsetRange(self.topic, self.partition, self.fromOffset,
+                                        self.untilOffset)
 
 
 class TopicAndPartition(object):
@@ -244,3 +263,87 @@ def __init__(self, host, port):
 
     def _jBroker(self, helper):
         return helper.createBroker(self._host, self._port)
+
+
+class KafkaRDD(RDD):
+    """
+    A Python wrapper of KafkaRDD, to provide additional information on normal RDD.
+    """
+
+    def __init__(self, jrdd, ctx, jrdd_deserializer):
+        RDD.__init__(self, jrdd, ctx, jrdd_deserializer)
+
+    def offsetRanges(self):
+        """
+        Get the OffsetRange of specific KafkaRDD.
+        :return: A list of OffsetRange
+        """
+        try:
+            helperClass = self.ctx._jvm.java.lang.Thread.currentThread().getContextClassLoader() \
+                .loadClass("org.apache.spark.streaming.kafka.KafkaUtilsPythonHelper")
+            helper = helperClass.newInstance()
+            joffsetRanges = helper.offsetRangesOfKafkaRDD(self._jrdd.rdd())
+        except Py4JJavaError as e:
+            if 'ClassNotFoundException' in str(e.java_exception):
+                KafkaUtils._printErrorMsg(self.ctx)
+            raise e
+
+        ranges = [OffsetRange(o.topic(), o.partition(), o.fromOffset(), o.untilOffset())
+                  for o in joffsetRanges]
+        return ranges
+
+
+class KafkaDStream(DStream):
+    """
+    A Python wrapper of KafkaDStream
+    """
+
+    def __init__(self, jdstream, ssc, jrdd_deserializer):
+        DStream.__init__(self, jdstream, ssc, jrdd_deserializer)
+
+    def foreachRDD(self, func):
+        """
+        Apply a function to each RDD in this DStream.
+        """
+        if func.__code__.co_argcount == 1:
+            old_func = func
+            func = lambda r, rdd: old_func(rdd)
+        jfunc = TransformFunction(self._sc, func, self._jrdd_deserializer) \
+            .rdd_wrapper(lambda jrdd, ctx, ser: KafkaRDD(jrdd, ctx, ser))
+        api = self._ssc._jvm.PythonDStream
+        api.callForeachRDD(self._jdstream, jfunc)
+
+    def transform(self, func):
+        """
+        Return a new DStream in which each RDD is generated by applying a function
+        on each RDD of this DStream.
+
+        `func` can have one argument of `rdd`, or have two arguments of
+        (`time`, `rdd`)
+        """
+        if func.__code__.co_argcount == 1:
+            oldfunc = func
+            func = lambda t, rdd: oldfunc(rdd)
+        assert func.__code__.co_argcount == 2, "func should take one or two arguments"
+
+        return KafkaTransformedDStream(self, func)
+
+
+class KafkaTransformedDStream(TransformedDStream):
+    """
+    Kafka specific wrapper of TransformedDStream to transform on Kafka RDD.
+    """
+
+    def __init__(self, prev, func):
+        TransformedDStream.__init__(self, prev, func)
+
+    @property
+    def _jdstream(self):
+        if self._jdstream_val is not None:
+            return self._jdstream_val
+
+        jfunc = TransformFunction(self._sc, self.func, self.prev._jrdd_deserializer) \
+            .rdd_wrapper(lambda jrdd, ctx, ser: KafkaRDD(jrdd, ctx, ser))
+        dstream = self._sc._jvm.PythonTransformedDStream(self.prev._jdstream.dstream(), jfunc)
+        self._jdstream_val = dstream.asJavaDStream()
+        return self._jdstream_val
diff --git a/python/pyspark/streaming/tests.py b/python/pyspark/streaming/tests.py
index 188c8ff12067e..4ecae1e4bf282 100644
--- a/python/pyspark/streaming/tests.py
+++ b/python/pyspark/streaming/tests.py
@@ -678,6 +678,70 @@ def test_kafka_rdd_with_leaders(self):
         rdd = KafkaUtils.createRDD(self.sc, kafkaParams, offsetRanges, leaders)
         self._validateRddResult(sendData, rdd)
 
+    @unittest.skipIf(sys.version >= "3", "long type not support")
+    def test_kafka_rdd_get_offsetRanges(self):
+        """Test Python direct Kafka RDD get OffsetRanges."""
+        topic = self._randomTopic()
+        sendData = {"a": 3, "b": 4, "c": 5}
+        offsetRanges = [OffsetRange(topic, 0, long(0), long(sum(sendData.values())))]
+        kafkaParams = {"metadata.broker.list": self._kafkaTestUtils.brokerAddress()}
+
+        self._kafkaTestUtils.createTopic(topic)
+        self._kafkaTestUtils.sendMessages(topic, sendData)
+        rdd = KafkaUtils.createRDD(self.sc, kafkaParams, offsetRanges)
+        self.assertEqual(offsetRanges, rdd.offsetRanges())
+
+    @unittest.skipIf(sys.version >= "3", "long type not support")
+    def test_kafka_direct_stream_foreach_get_offsetRanges(self):
+        """Test the Python direct Kafka stream foreachRDD get offsetRanges."""
+        topic = self._randomTopic()
+        sendData = {"a": 1, "b": 2, "c": 3}
+        kafkaParams = {"metadata.broker.list": self._kafkaTestUtils.brokerAddress(),
+                       "auto.offset.reset": "smallest"}
+
+        self._kafkaTestUtils.createTopic(topic)
+        self._kafkaTestUtils.sendMessages(topic, sendData)
+
+        stream = KafkaUtils.createDirectStream(self.ssc, [topic], kafkaParams)
+
+        offsetRanges = []
+
+        def getOffsetRanges(_, rdd):
+            for o in rdd.offsetRanges():
+                offsetRanges.append(o)
+
+        stream.foreachRDD(getOffsetRanges)
+        self.ssc.start()
+        self.wait_for(offsetRanges, 1)
+
+        self.assertEqual(offsetRanges, [OffsetRange(topic, 0, long(0), long(6))])
+
+    @unittest.skipIf(sys.version >= "3", "long type not support")
+    def test_kafka_direct_stream_transform_get_offsetRanges(self):
+        """Test the Python direct Kafka stream transform get offsetRanges."""
+        topic = self._randomTopic()
+        sendData = {"a": 1, "b": 2, "c": 3}
+        kafkaParams = {"metadata.broker.list": self._kafkaTestUtils.brokerAddress(),
+                       "auto.offset.reset": "smallest"}
+
+        self._kafkaTestUtils.createTopic(topic)
+        self._kafkaTestUtils.sendMessages(topic, sendData)
+
+        stream = KafkaUtils.createDirectStream(self.ssc, [topic], kafkaParams)
+
+        offsetRanges = []
+
+        def transformWithOffsetRanges(rdd):
+            for o in rdd.offsetRanges():
+                offsetRanges.append(o)
+            return rdd
+
+        stream.transform(transformWithOffsetRanges).foreachRDD(lambda rdd: rdd.count())
+        self.ssc.start()
+        self.wait_for(offsetRanges, 1)
+
+        self.assertEqual(offsetRanges, [OffsetRange(topic, 0, long(0), long(6))])
+
 
 class FlumeStreamTests(PySparkStreamingTestCase):
     timeout = 20  # seconds
diff --git a/python/pyspark/streaming/util.py b/python/pyspark/streaming/util.py
index a9bfec2aab8fc..b20613b1283bd 100644
--- a/python/pyspark/streaming/util.py
+++ b/python/pyspark/streaming/util.py
@@ -37,6 +37,11 @@ def __init__(self, ctx, func, *deserializers):
         self.ctx = ctx
         self.func = func
         self.deserializers = deserializers
+        self._rdd_wrapper = lambda jrdd, ctx, ser: RDD(jrdd, ctx, ser)
+
+    def rdd_wrapper(self, func):
+        self._rdd_wrapper = func
+        return self
 
     def call(self, milliseconds, jrdds):
         try:
@@ -51,7 +56,7 @@ def call(self, milliseconds, jrdds):
             if len(sers) < len(jrdds):
                 sers += (sers[0],) * (len(jrdds) - len(sers))
 
-            rdds = [RDD(jrdd, self.ctx, ser) if jrdd else None
+            rdds = [self._rdd_wrapper(jrdd, self.ctx, ser) if jrdd else None
                     for jrdd, ser in zip(jrdds, sers)]
             t = datetime.fromtimestamp(milliseconds / 1000.0)
             r = self.func(t, *rdds)
diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py
index 17256dfc95744..c5c0add49d02c 100644
--- a/python/pyspark/tests.py
+++ b/python/pyspark/tests.py
@@ -885,6 +885,18 @@ def test_sortByKey_uses_all_partitions_not_only_first_and_last(self):
             for size in sizes:
                 self.assertGreater(size, 0)
 
+    def test_pipe_functions(self):
+        data = ['1', '2', '3']
+        rdd = self.sc.parallelize(data)
+        with QuietTest(self.sc):
+            self.assertEqual([], rdd.pipe('cc').collect())
+            self.assertRaises(Py4JJavaError, rdd.pipe('cc', checkCode=True).collect)
+        result = rdd.pipe('cat').collect()
+        result.sort()
+        [self.assertEqual(x, y) for x, y in zip(data, result)]
+        self.assertRaises(Py4JJavaError, rdd.pipe('grep 4', checkCode=True).collect)
+        self.assertEqual([], rdd.pipe('grep 4').collect())
+
 
 class ProfilerTests(PySparkTestCase):
 
diff --git a/python/run-tests.py b/python/run-tests.py
index 7638854def2e8..cc560779373b3 100755
--- a/python/run-tests.py
+++ b/python/run-tests.py
@@ -72,7 +72,8 @@ def print_red(text):
 
 
 def run_individual_python_test(test_name, pyspark_python):
-    env = {'SPARK_TESTING': '1', 'PYSPARK_PYTHON': which(pyspark_python)}
+    env = dict(os.environ)
+    env.update({'SPARK_TESTING': '1', 'PYSPARK_PYTHON': which(pyspark_python)})
     LOGGER.debug("Starting test(%s): %s", pyspark_python, test_name)
     start_time = time.time()
     try:
diff --git a/repl/pom.xml b/repl/pom.xml
index 370b2bc2fa8ed..70c9bd7c01296 100644
--- a/repl/pom.xml
+++ b/repl/pom.xml
@@ -38,11 +38,6 @@
   
 
   
-    
-      ${jline.groupid}
-      jline
-      ${jline.version}
-    
     
       org.apache.spark
       spark-core_${scala.binary.version}
@@ -161,6 +156,20 @@
     
   
   
+    
+      scala-2.10
+      
+        !scala-2.11
+      
+      
+        
+          ${jline.groupid}
+          jline
+          ${jline.version}
+        
+      
+    
+
     
       scala-2.11
       
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 6480e2d24e044..24fbbc12c08da 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
@@ -39,6 +39,8 @@ class SparkCommandLine(args: List[String], override val settings: Settings)
   }
 
   def this(args: List[String]) {
+    // scalastyle:off println
     this(args, str => Console.println("Error: " + str))
+    // scalastyle:on println
   }
 }
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 2b235525250c2..8f7f9074d3f03 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
@@ -1101,7 +1101,9 @@ object SparkILoop extends Logging {
             val s = super.readLine()
             // helping out by printing the line being interpreted.
             if (s != null)
+              // scalastyle:off println
               output.println(s)
+              // scalastyle:on println
             s
           }
         }
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 05faef8786d2c..bd3314d94eed6 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
@@ -80,11 +80,13 @@ private[repl] trait SparkILoopInit {
     if (!initIsComplete)
       withLock { while (!initIsComplete) initLoopCondition.await() }
     if (initError != null) {
+      // scalastyle:off println
       println("""
         |Failed to initialize the REPL due to an unexpected error.
         |This is a bug, please, report it along with the error diagnostics printed below.
         |%s.""".stripMargin.format(initError)
       )
+      // scalastyle:on println
       false
     } else true
   }
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 35fb625645022..4ee605fd7f11e 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
@@ -1079,8 +1079,10 @@ import org.apache.spark.annotation.DeveloperApi
       throw new EvalException("Failed to load '" + path + "': " + ex.getMessage, ex)
 
     private def load(path: String): Class[_] = {
+      // scalastyle:off classforname
       try Class.forName(path, true, classLoader)
       catch { case ex: Throwable => evalError(path, unwrap(ex)) }
+      // scalastyle:on classforname
     }
 
     lazy val evalClass = load(evalPath)
@@ -1761,7 +1763,9 @@ object SparkIMain {
         if (intp.totalSilence) ()
         else super.printMessage(msg)
       }
+      // scalastyle:off println
       else Console.println(msg)
+      // scalastyle:on println
     }
   }
 }
diff --git a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala
index f4f4b626988e9..eed4a379afa60 100644
--- a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala
+++ b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala
@@ -17,13 +17,14 @@
 
 package org.apache.spark.repl
 
+import java.io.File
+
+import scala.tools.nsc.Settings
+
 import org.apache.spark.util.Utils
 import org.apache.spark._
 import org.apache.spark.sql.SQLContext
 
-import scala.tools.nsc.Settings
-import scala.tools.nsc.interpreter.SparkILoop
-
 object Main extends Logging {
 
   val conf = new SparkConf()
@@ -32,7 +33,8 @@ object Main extends Logging {
   val outputDir = Utils.createTempDir(rootDir)
   val s = new Settings()
   s.processArguments(List("-Yrepl-class-based",
-    "-Yrepl-outdir", s"${outputDir.getAbsolutePath}", "-Yrepl-sync"), true)
+    "-Yrepl-outdir", s"${outputDir.getAbsolutePath}",
+    "-classpath", getAddedJars.mkString(File.pathSeparator)), true)
   val classServer = new HttpServer(conf, outputDir, new SecurityManager(conf))
   var sparkContext: SparkContext = _
   var sqlContext: SQLContext = _
@@ -48,7 +50,6 @@ object Main extends Logging {
     Option(sparkContext).map(_.stop)
   }
 
-
   def getAddedJars: Array[String] = {
     val envJars = sys.env.get("ADD_JARS")
     if (envJars.isDefined) {
@@ -84,10 +85,9 @@ object Main extends Logging {
     val loader = Utils.getContextOrSparkClassLoader
     try {
       sqlContext = loader.loadClass(name).getConstructor(classOf[SparkContext])
-        .newInstance(sparkContext).asInstanceOf[SQLContext] 
+        .newInstance(sparkContext).asInstanceOf[SQLContext]
       logInfo("Created sql context (with Hive support)..")
-    }
-    catch {
+    } catch {
       case _: java.lang.ClassNotFoundException | _: java.lang.NoClassDefFoundError =>
         sqlContext = new SQLContext(sparkContext)
         logInfo("Created sql context..")
diff --git a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala
deleted file mode 100644
index 8e519fa67f649..0000000000000
--- a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala
+++ /dev/null
@@ -1,86 +0,0 @@
-/* NSC -- new Scala compiler
- * Copyright 2005-2013 LAMP/EPFL
- * @author  Paul Phillips
- */
-
-package scala.tools.nsc
-package interpreter
-
-import scala.tools.nsc.ast.parser.Tokens.EOF
-
-trait SparkExprTyper {
-  val repl: SparkIMain
-
-  import repl._
-  import global.{ reporter => _, Import => _, _ }
-  import naming.freshInternalVarName
-
-  def symbolOfLine(code: String): Symbol = {
-    def asExpr(): Symbol = {
-      val name  = freshInternalVarName()
-      // Typing it with a lazy val would give us the right type, but runs
-      // into compiler bugs with things like existentials, so we compile it
-      // behind a def and strip the NullaryMethodType which wraps the expr.
-      val line = "def " + name + " = " + code
-
-      interpretSynthetic(line) match {
-        case IR.Success =>
-          val sym0 = symbolOfTerm(name)
-          // drop NullaryMethodType
-          sym0.cloneSymbol setInfo exitingTyper(sym0.tpe_*.finalResultType)
-        case _          => NoSymbol
-      }
-    }
-    def asDefn(): Symbol = {
-      val old = repl.definedSymbolList.toSet
-
-      interpretSynthetic(code) match {
-        case IR.Success =>
-          repl.definedSymbolList filterNot old match {
-            case Nil        => NoSymbol
-            case sym :: Nil => sym
-            case syms       => NoSymbol.newOverloaded(NoPrefix, syms)
-          }
-        case _ => NoSymbol
-      }
-    }
-    def asError(): Symbol = {
-      interpretSynthetic(code)
-      NoSymbol
-    }
-    beSilentDuring(asExpr()) orElse beSilentDuring(asDefn()) orElse asError()
-  }
-
-  private var typeOfExpressionDepth = 0
-  def typeOfExpression(expr: String, silent: Boolean = true): Type = {
-    if (typeOfExpressionDepth > 2) {
-      repldbg("Terminating typeOfExpression recursion for expression: " + expr)
-      return NoType
-    }
-    typeOfExpressionDepth += 1
-    // Don't presently have a good way to suppress undesirable success output
-    // while letting errors through, so it is first trying it silently: if there
-    // is an error, and errors are desired, then it re-evaluates non-silently
-    // to induce the error message.
-    try beSilentDuring(symbolOfLine(expr).tpe) match {
-      case NoType if !silent => symbolOfLine(expr).tpe // generate error
-      case tpe               => tpe
-    }
-    finally typeOfExpressionDepth -= 1
-  }
-
-  // This only works for proper types.
-  def typeOfTypeString(typeString: String): Type = {
-    def asProperType(): Option[Type] = {
-      val name = freshInternalVarName()
-      val line = "def %s: %s = ???" format (name, typeString)
-      interpretSynthetic(line) match {
-        case IR.Success =>
-          val sym0 = symbolOfTerm(name)
-          Some(sym0.asMethod.returnType)
-        case _          => None
-      }
-    }
-    beSilentDuring(asProperType()) getOrElse NoType
-  }
-}
diff --git a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkILoop.scala
index 7a5e94da5cbf3..bf609ff0f65fc 100644
--- a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkILoop.scala
+++ b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkILoop.scala
@@ -1,88 +1,64 @@
-/* NSC -- new Scala compiler
- * Copyright 2005-2013 LAMP/EPFL
- * @author Alexander Spoon
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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 scala
-package tools.nsc
-package interpreter
+package org.apache.spark.repl
 
-import scala.language.{ implicitConversions, existentials }
-import scala.annotation.tailrec
-import Predef.{ println => _, _ }
-import interpreter.session._
-import StdReplTags._
-import scala.reflect.api.{Mirror, Universe, TypeCreator}
-import scala.util.Properties.{ jdkHome, javaVersion, versionString, javaVmName }
-import scala.tools.nsc.util.{ ClassPath, Exceptional, stringFromWriter, stringFromStream }
-import scala.reflect.{ClassTag, classTag}
-import scala.reflect.internal.util.{ BatchSourceFile, ScalaClassLoader }
-import ScalaClassLoader._
-import scala.reflect.io.{ File, Directory }
-import scala.tools.util._
-import scala.collection.generic.Clearable
-import scala.concurrent.{ ExecutionContext, Await, Future, future }
-import ExecutionContext.Implicits._
-import java.io.{ BufferedReader, FileReader }
+import java.io.{BufferedReader, FileReader}
 
-/** The Scala interactive shell.  It provides a read-eval-print loop
-  *  around the Interpreter class.
-  *  After instantiation, clients should call the main() method.
-  *
-  *  If no in0 is specified, then input will come from the console, and
-  *  the class will attempt to provide input editing feature such as
-  *  input history.
-  *
-  *  @author Moez A. Abdel-Gawad
-  *  @author  Lex Spoon
-  *  @version 1.2
-  */
-class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter)
-  extends AnyRef
-  with LoopCommands
-{
-  def this(in0: BufferedReader, out: JPrintWriter) = this(Some(in0), out)
-  def this() = this(None, new JPrintWriter(Console.out, true))
-//
-//  @deprecated("Use `intp` instead.", "2.9.0") def interpreter = intp
-//  @deprecated("Use `intp` instead.", "2.9.0") def interpreter_= (i: Interpreter): Unit = intp = i
-
-  var in: InteractiveReader = _   // the input stream from which commands come
-  var settings: Settings = _
-  var intp: SparkIMain = _
+import Predef.{println => _, _}
+import scala.util.Properties.{jdkHome, javaVersion, versionString, javaVmName}
 
-  var globalFuture: Future[Boolean] = _
+import scala.tools.nsc.interpreter.{JPrintWriter, ILoop}
+import scala.tools.nsc.Settings
+import scala.tools.nsc.util.stringFromStream
 
-  protected def asyncMessage(msg: String) {
-    if (isReplInfo || isReplPower)
-      echoAndRefresh(msg)
-  }
+/**
+ *  A Spark-specific interactive shell.
+ */
+class SparkILoop(in0: Option[BufferedReader], out: JPrintWriter)
+    extends ILoop(in0, out) {
+  def this(in0: BufferedReader, out: JPrintWriter) = this(Some(in0), out)
+  def this() = this(None, new JPrintWriter(Console.out, true))
 
   def initializeSpark() {
     intp.beQuietDuring {
-      command( """
+      processLine("""
          @transient val sc = {
            val _sc = org.apache.spark.repl.Main.createSparkContext()
            println("Spark context available as sc.")
            _sc
          }
         """)
-      command( """
+      processLine("""
          @transient val sqlContext = {
            val _sqlContext = org.apache.spark.repl.Main.createSQLContext()
            println("SQL context available as sqlContext.")
            _sqlContext
          }
         """)
-      command("import org.apache.spark.SparkContext._")
-      command("import sqlContext.implicits._")
-      command("import sqlContext.sql")
-      command("import org.apache.spark.sql.functions._")
+      processLine("import org.apache.spark.SparkContext._")
+      processLine("import sqlContext.implicits._")
+      processLine("import sqlContext.sql")
+      processLine("import org.apache.spark.sql.functions._")
     }
   }
 
   /** Print a welcome message */
-  def printWelcome() {
+  override def printWelcome() {
     import org.apache.spark.SPARK_VERSION
     echo("""Welcome to
       ____              __
@@ -98,875 +74,42 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter)
     echo("Type :help for more information.")
   }
 
-  override def echoCommandMessage(msg: String) {
-    intp.reporter printUntruncatedMessage msg
-  }
-
-  // lazy val power = new Power(intp, new StdReplVals(this))(tagOfStdReplVals, classTag[StdReplVals])
-  def history = in.history
-
-  // classpath entries added via :cp
-  var addedClasspath: String = ""
-
-  /** A reverse list of commands to replay if the user requests a :replay */
-  var replayCommandStack: List[String] = Nil
-
-  /** A list of commands to replay if the user requests a :replay */
-  def replayCommands = replayCommandStack.reverse
-
-  /** Record a command for replay should the user request a :replay */
-  def addReplay(cmd: String) = replayCommandStack ::= cmd
-
-  def savingReplayStack[T](body: => T): T = {
-    val saved = replayCommandStack
-    try body
-    finally replayCommandStack = saved
-  }
-  def savingReader[T](body: => T): T = {
-    val saved = in
-    try body
-    finally in = saved
-  }
-
-  /** Close the interpreter and set the var to null. */
-  def closeInterpreter() {
-    if (intp ne null) {
-      intp.close()
-      intp = null
-    }
-  }
-
-  class SparkILoopInterpreter extends SparkIMain(settings, out) {
-    outer =>
-
-    override lazy val formatting = new Formatting {
-      def prompt = SparkILoop.this.prompt
-    }
-    override protected def parentClassLoader =
-      settings.explicitParentLoader.getOrElse( classOf[SparkILoop].getClassLoader )
-  }
-
-  /** Create a new interpreter. */
-  def createInterpreter() {
-    if (addedClasspath != "")
-      settings.classpath append addedClasspath
-
-    intp = new SparkILoopInterpreter
-  }
-
-  /** print a friendly help message */
-  def helpCommand(line: String): Result = {
-    if (line == "") helpSummary()
-    else uniqueCommand(line) match {
-      case Some(lc) => echo("\n" + lc.help)
-      case _        => ambiguousError(line)
-    }
-  }
-  private def helpSummary() = {
-    val usageWidth  = commands map (_.usageMsg.length) max
-    val formatStr   = "%-" + usageWidth + "s %s"
-
-    echo("All commands can be abbreviated, e.g. :he instead of :help.")
-
-    commands foreach { cmd =>
-      echo(formatStr.format(cmd.usageMsg, cmd.help))
-    }
-  }
-  private def ambiguousError(cmd: String): Result = {
-    matchingCommands(cmd) match {
-      case Nil  => echo(cmd + ": no such command.  Type :help for help.")
-      case xs   => echo(cmd + " is ambiguous: did you mean " + xs.map(":" + _.name).mkString(" or ") + "?")
-    }
-    Result(keepRunning = true, None)
-  }
-  private def matchingCommands(cmd: String) = commands filter (_.name startsWith cmd)
-  private def uniqueCommand(cmd: String): Option[LoopCommand] = {
-    // this lets us add commands willy-nilly and only requires enough command to disambiguate
-    matchingCommands(cmd) match {
-      case List(x)  => Some(x)
-      // exact match OK even if otherwise appears ambiguous
-      case xs       => xs find (_.name == cmd)
-    }
-  }
-
-  /** Show the history */
-  lazy val historyCommand = new LoopCommand("history", "show the history (optional num is commands to show)") {
-    override def usage = "[num]"
-    def defaultLines = 20
-
-    def apply(line: String): Result = {
-      if (history eq NoHistory)
-        return "No history available."
-
-      val xs      = words(line)
-      val current = history.index
-      val count   = try xs.head.toInt catch { case _: Exception => defaultLines }
-      val lines   = history.asStrings takeRight count
-      val offset  = current - lines.size + 1
-
-      for ((line, index) <- lines.zipWithIndex)
-        echo("%3d  %s".format(index + offset, line))
-    }
-  }
-
-  // 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) = {
-    echo("\n" + msg)
-    in.redrawLine()
-  }
-  protected def echo(msg: String) = {
-    out println msg
-    out.flush()
-  }
-
-  /** Search the history */
-  def searchHistory(_cmdline: String) {
-    val cmdline = _cmdline.toLowerCase
-    val offset  = history.index - history.size + 1
-
-    for ((line, index) <- history.asStrings.zipWithIndex ; if line.toLowerCase contains cmdline)
-      echo("%d %s".format(index + offset, line))
-  }
-
-  private val currentPrompt = Properties.shellPromptString
-
-  /** Prompt to print when awaiting input */
-  def prompt = currentPrompt
-
   import LoopCommand.{ cmd, nullary }
 
-  /** Standard commands **/
-  lazy val standardCommands = List(
-    cmd("cp", "", "add a jar or directory to the classpath", addClasspath),
-    cmd("edit", "|", "edit history", editCommand),
-    cmd("help", "[command]", "print this summary or command-specific help", helpCommand),
-    historyCommand,
-    cmd("h?", "", "search the history", searchHistory),
-    cmd("imports", "[name name ...]", "show import history, identifying sources of names", importsCommand),
-    //cmd("implicits", "[-v]", "show the implicits in scope", intp.implicitsCommand),
-    cmd("javap", "", "disassemble a file or class name", javapCommand),
-    cmd("line", "|", "place line(s) at the end of history", lineCommand),
-    cmd("load", "", "interpret lines in a file", loadCommand),
-    cmd("paste", "[-raw] [path]", "enter paste mode or paste a file", pasteCommand),
-    // nullary("power", "enable power user mode", powerCmd),
-    nullary("quit", "exit the interpreter", () => Result(keepRunning = false, None)),
-    nullary("replay", "reset execution and replay all previous commands", replay),
-    nullary("reset", "reset the repl to its initial state, forgetting all session entries", resetCommand),
-    cmd("save", "", "save replayable session to a file", saveCommand),
-    shCommand,
-    cmd("settings", "[+|-]", "+enable/-disable flags, set compiler options", changeSettings),
-    nullary("silent", "disable/enable automatic printing of results", verbosity),
-//    cmd("type", "[-v] ", "display the type of an expression without evaluating it", typeCommand),
-//    cmd("kind", "[-v] ", "display the kind of expression's type", kindCommand),
-    nullary("warnings", "show the suppressed warnings from the most recent line which had any", warningsCommand)
-  )
-
-  /** Power user commands */
-//  lazy val powerCommands: List[LoopCommand] = List(
-//    cmd("phase", "", "set the implicit phase for power commands", phaseCommand)
-//  )
-
-  private def importsCommand(line: String): Result = {
-    val tokens    = words(line)
-    val handlers  = intp.languageWildcardHandlers ++ intp.importHandlers
-
-    handlers.filterNot(_.importedSymbols.isEmpty).zipWithIndex foreach {
-      case (handler, idx) =>
-        val (types, terms) = handler.importedSymbols partition (_.name.isTypeName)
-        val imps           = handler.implicitSymbols
-        val found          = tokens filter (handler importsSymbolNamed _)
-        val typeMsg        = if (types.isEmpty) "" else types.size + " types"
-        val termMsg        = if (terms.isEmpty) "" else terms.size + " terms"
-        val implicitMsg    = if (imps.isEmpty) "" else imps.size + " are implicit"
-        val foundMsg       = if (found.isEmpty) "" else found.mkString(" // imports: ", ", ", "")
-        val statsMsg       = List(typeMsg, termMsg, implicitMsg) filterNot (_ == "") mkString ("(", ", ", ")")
-
-        intp.reporter.printMessage("%2d) %-30s %s%s".format(
-          idx + 1,
-          handler.importString,
-          statsMsg,
-          foundMsg
-        ))
-    }
-  }
-
-  private def findToolsJar() = PathResolver.SupplementalLocations.platformTools
+  private val blockedCommands = Set("implicits", "javap", "power", "type", "kind")
 
-  private def addToolsJarToLoader() = {
-    val cl = findToolsJar() match {
-      case Some(tools) => ScalaClassLoader.fromURLs(Seq(tools.toURL), intp.classLoader)
-      case _           => intp.classLoader
-    }
-    if (Javap.isAvailable(cl)) {
-      repldbg(":javap available.")
-      cl
-    }
-    else {
-      repldbg(":javap unavailable: no tools.jar at " + jdkHome)
-      intp.classLoader
-    }
-  }
-//
-//  protected def newJavap() =
-//    JavapClass(addToolsJarToLoader(), new IMain.ReplStrippingWriter(intp), Some(intp))
-//
-//  private lazy val javap = substituteAndLog[Javap]("javap", NoJavap)(newJavap())
-
-  // Still todo: modules.
-//  private def typeCommand(line0: String): Result = {
-//    line0.trim match {
-//      case "" => ":type [-v] "
-//      case s  => intp.typeCommandInternal(s stripPrefix "-v " trim, verbose = s startsWith "-v ")
-//    }
-//  }
-
-//  private def kindCommand(expr: String): Result = {
-//    expr.trim match {
-//      case "" => ":kind [-v] "
-//      case s  => intp.kindCommandInternal(s stripPrefix "-v " trim, verbose = s startsWith "-v ")
-//    }
-//  }
-
-  private def warningsCommand(): Result = {
-    if (intp.lastWarnings.isEmpty)
-      "Can't find any cached warnings."
-    else
-      intp.lastWarnings foreach { case (pos, msg) => intp.reporter.warning(pos, msg) }
-  }
-
-  private def changeSettings(args: String): Result = {
-    def showSettings() = {
-      for (s <- settings.userSetSettings.toSeq.sorted) echo(s.toString)
-    }
-    def updateSettings() = {
-      // put aside +flag options
-      val (pluses, rest) = (args split "\\s+").toList partition (_.startsWith("+"))
-      val tmps = new Settings
-      val (ok, leftover) = tmps.processArguments(rest, processAll = true)
-      if (!ok) echo("Bad settings request.")
-      else if (leftover.nonEmpty) echo("Unprocessed settings.")
-      else {
-        // boolean flags set-by-user on tmp copy should be off, not on
-        val offs = tmps.userSetSettings filter (_.isInstanceOf[Settings#BooleanSetting])
-        val (minuses, nonbools) = rest partition (arg => offs exists (_ respondsTo arg))
-        // update non-flags
-        settings.processArguments(nonbools, processAll = true)
-        // also snag multi-value options for clearing, e.g. -Ylog: and -language:
-        for {
-          s <- settings.userSetSettings
-          if s.isInstanceOf[Settings#MultiStringSetting] || s.isInstanceOf[Settings#PhasesSetting]
-          if nonbools exists (arg => arg.head == '-' && arg.last == ':' && (s respondsTo arg.init))
-        } s match {
-          case c: Clearable => c.clear()
-          case _ =>
-        }
-        def update(bs: Seq[String], name: String=>String, setter: Settings#Setting=>Unit) = {
-          for (b <- bs)
-            settings.lookupSetting(name(b)) match {
-              case Some(s) =>
-                if (s.isInstanceOf[Settings#BooleanSetting]) setter(s)
-                else echo(s"Not a boolean flag: $b")
-              case _ =>
-                echo(s"Not an option: $b")
-            }
-        }
-        update(minuses, identity, _.tryToSetFromPropertyValue("false"))  // turn off
-        update(pluses, "-" + _.drop(1), _.tryToSet(Nil))                 // turn on
-      }
-    }
-    if (args.isEmpty) showSettings() else updateSettings()
-  }
-
-  private def javapCommand(line: String): Result = {
-//    if (javap == null)
-//      ":javap unavailable, no tools.jar at %s.  Set JDK_HOME.".format(jdkHome)
-//    else if (line == "")
-//      ":javap [-lcsvp] [path1 path2 ...]"
-//    else
-//      javap(words(line)) foreach { res =>
-//        if (res.isError) return "Failed: " + res.value
-//        else res.show()
-//      }
-  }
-
-  private def pathToPhaseWrapper = intp.originalPath("$r") + ".phased.atCurrent"
-
-  private def phaseCommand(name: String): Result = {
-//    val phased: Phased = power.phased
-//    import phased.NoPhaseName
-//
-//    if (name == "clear") {
-//      phased.set(NoPhaseName)
-//      intp.clearExecutionWrapper()
-//      "Cleared active phase."
-//    }
-//    else if (name == "") phased.get match {
-//      case NoPhaseName => "Usage: :phase  (e.g. typer, erasure.next, erasure+3)"
-//      case ph          => "Active phase is '%s'.  (To clear, :phase clear)".format(phased.get)
-//    }
-//    else {
-//      val what = phased.parse(name)
-//      if (what.isEmpty || !phased.set(what))
-//        "'" + name + "' does not appear to represent a valid phase."
-//      else {
-//        intp.setExecutionWrapper(pathToPhaseWrapper)
-//        val activeMessage =
-//          if (what.toString.length == name.length) "" + what
-//          else "%s (%s)".format(what, name)
-//
-//        "Active phase is now: " + activeMessage
-//      }
-//    }
-  }
+  /** Standard commands **/
+  lazy val sparkStandardCommands: List[SparkILoop.this.LoopCommand] =
+    standardCommands.filter(cmd => !blockedCommands(cmd.name))
 
   /** Available commands */
-  def commands: List[LoopCommand] = standardCommands ++ (
-    // if (isReplPower)
-    //  powerCommands
-    // else
-      Nil
-    )
-
-  val replayQuestionMessage =
-    """|That entry seems to have slain the compiler.  Shall I replay
-      |your session? I can re-run each line except the last one.
-      |[y/n]
-    """.trim.stripMargin
-
-  private val crashRecovery: PartialFunction[Throwable, Boolean] = {
-    case ex: Throwable =>
-      val (err, explain) = (
-        if (intp.isInitializeComplete)
-          (intp.global.throwableAsString(ex), "")
-        else
-          (ex.getMessage, "The compiler did not initialize.\n")
-        )
-      echo(err)
-
-      ex match {
-        case _: NoSuchMethodError | _: NoClassDefFoundError =>
-          echo("\nUnrecoverable error.")
-          throw ex
-        case _  =>
-          def fn(): Boolean =
-            try in.readYesOrNo(explain + replayQuestionMessage, { echo("\nYou must enter y or n.") ; fn() })
-            catch { case _: RuntimeException => false }
-
-          if (fn()) replay()
-          else echo("\nAbandoning crashed session.")
-      }
-      true
-  }
-
-  // return false if repl should exit
-  def processLine(line: String): Boolean = {
-    import scala.concurrent.duration._
-    Await.ready(globalFuture, 60.seconds)
-
-    (line ne null) && (command(line) match {
-      case Result(false, _)      => false
-      case Result(_, Some(line)) => addReplay(line) ; true
-      case _                     => true
-    })
-  }
-
-  private def readOneLine() = {
-    out.flush()
-    in readLine prompt
-  }
-
-  /** The main read-eval-print loop for the repl.  It calls
-    *  command() for each line of input, and stops when
-    *  command() returns false.
-    */
-  @tailrec final def loop() {
-    if ( try processLine(readOneLine()) catch crashRecovery )
-      loop()
-  }
-
-  /** interpret all lines from a specified file */
-  def interpretAllFrom(file: File) {
-    savingReader {
-      savingReplayStack {
-        file applyReader { reader =>
-          in = SimpleReader(reader, out, interactive = false)
-          echo("Loading " + file + "...")
-          loop()
-        }
-      }
-    }
-  }
-
-  /** create a new interpreter and replay the given commands */
-  def replay() {
-    reset()
-    if (replayCommandStack.isEmpty)
-      echo("Nothing to replay.")
-    else for (cmd <- replayCommands) {
-      echo("Replaying: " + cmd)  // flush because maybe cmd will have its own output
-      command(cmd)
-      echo("")
-    }
-  }
-  def resetCommand() {
-    echo("Resetting interpreter state.")
-    if (replayCommandStack.nonEmpty) {
-      echo("Forgetting this session history:\n")
-      replayCommands foreach echo
-      echo("")
-      replayCommandStack = Nil
-    }
-    if (intp.namedDefinedTerms.nonEmpty)
-      echo("Forgetting all expression results and named terms: " + intp.namedDefinedTerms.mkString(", "))
-    if (intp.definedTypes.nonEmpty)
-      echo("Forgetting defined types: " + intp.definedTypes.mkString(", "))
-
-    reset()
-  }
-  def reset() {
-    intp.reset()
-    unleashAndSetPhase()
-  }
-
-  def lineCommand(what: String): Result = editCommand(what, None)
-
-  // :edit id or :edit line
-  def editCommand(what: String): Result = editCommand(what, Properties.envOrNone("EDITOR"))
-
-  def editCommand(what: String, editor: Option[String]): Result = {
-    def diagnose(code: String) = {
-      echo("The edited code is incomplete!\n")
-      val errless = intp compileSources new BatchSourceFile("", s"object pastel {\n$code\n}")
-      if (errless) echo("The compiler reports no errors.")
-    }
-    def historicize(text: String) = history match {
-      case jlh: JLineHistory => text.lines foreach jlh.add ; jlh.moveToEnd() ; true
-      case _ => false
-    }
-    def edit(text: String): Result = editor match {
-      case Some(ed) =>
-        val tmp = File.makeTemp()
-        tmp.writeAll(text)
-        try {
-          val pr = new ProcessResult(s"$ed ${tmp.path}")
-          pr.exitCode match {
-            case 0 =>
-              tmp.safeSlurp() match {
-                case Some(edited) if edited.trim.isEmpty => echo("Edited text is empty.")
-                case Some(edited) =>
-                  echo(edited.lines map ("+" + _) mkString "\n")
-                  val res = intp interpret edited
-                  if (res == IR.Incomplete) diagnose(edited)
-                  else {
-                    historicize(edited)
-                    Result(lineToRecord = Some(edited), keepRunning = true)
-                  }
-                case None => echo("Can't read edited text. Did you delete it?")
-              }
-            case x => echo(s"Error exit from $ed ($x), ignoring")
-          }
-        } finally {
-          tmp.delete()
-        }
-      case None =>
-        if (historicize(text)) echo("Placing text in recent history.")
-        else echo(f"No EDITOR defined and you can't change history, echoing your text:%n$text")
-    }
-
-    // if what is a number, use it as a line number or range in history
-    def isNum = what forall (c => c.isDigit || c == '-' || c == '+')
-    // except that "-" means last value
-    def isLast = (what == "-")
-    if (isLast || !isNum) {
-      val name = if (isLast) intp.mostRecentVar else what
-      val sym = intp.symbolOfIdent(name)
-      intp.prevRequestList collectFirst { case r if r.defines contains sym => r } match {
-        case Some(req) => edit(req.line)
-        case None      => echo(s"No symbol in scope: $what")
-      }
-    } else try {
-      val s = what
-      // line 123, 120+3, -3, 120-123, 120-, note -3 is not 0-3 but (cur-3,cur)
-      val (start, len) =
-        if ((s indexOf '+') > 0) {
-          val (a,b) = s splitAt (s indexOf '+')
-          (a.toInt, b.drop(1).toInt)
-        } else {
-          (s indexOf '-') match {
-            case -1 => (s.toInt, 1)
-            case 0  => val n = s.drop(1).toInt ; (history.index - n, n)
-            case _ if s.last == '-' => val n = s.init.toInt ; (n, history.index - n)
-            case i  => val n = s.take(i).toInt ; (n, s.drop(i+1).toInt - n)
-          }
-        }
-      import scala.collection.JavaConverters._
-      val index = (start - 1) max 0
-      val text = history match {
-        case jlh: JLineHistory => jlh.entries(index).asScala.take(len) map (_.value) mkString "\n"
-        case _ => history.asStrings.slice(index, index + len) mkString "\n"
-      }
-      edit(text)
-    } catch {
-      case _: NumberFormatException => echo(s"Bad range '$what'")
-        echo("Use line 123, 120+3, -3, 120-123, 120-, note -3 is not 0-3 but (cur-3,cur)")
-    }
-  }
-
-  /** fork a shell and run a command */
-  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()
-      case _    =>
-        val toRun = s"new ${classOf[ProcessResult].getName}(${string2codeQuoted(line)})"
-        intp interpret toRun
-        ()
-    }
-  }
-
-  def withFile[A](filename: String)(action: File => A): Option[A] = {
-    val res = Some(File(filename)) filter (_.exists) map action
-    if (res.isEmpty) echo("That file does not exist")  // courtesy side-effect
-    res
-  }
-
-  def loadCommand(arg: String) = {
-    var shouldReplay: Option[String] = None
-    withFile(arg)(f => {
-      interpretAllFrom(f)
-      shouldReplay = Some(":load " + arg)
-    })
-    Result(keepRunning = true, shouldReplay)
-  }
-
-  def saveCommand(filename: String): Result = (
-    if (filename.isEmpty) echo("File name is required.")
-    else if (replayCommandStack.isEmpty) echo("No replay commands in session")
-    else File(filename).printlnAll(replayCommands: _*)
-    )
-
-  def addClasspath(arg: String): Unit = {
-    val f = File(arg).normalize
-    if (f.exists) {
-      addedClasspath = ClassPath.join(addedClasspath, f.path)
-      val totalClasspath = ClassPath.join(settings.classpath.value, addedClasspath)
-      echo("Added '%s'.  Your new classpath is:\n\"%s\"".format(f.path, totalClasspath))
-      replay()
-    }
-    else echo("The path '" + f + "' doesn't seem to exist.")
-  }
-
-  def powerCmd(): Result = {
-    if (isReplPower) "Already in power mode."
-    else enablePowerMode(isDuringInit = false)
-  }
-  def enablePowerMode(isDuringInit: Boolean) = {
-    replProps.power setValue true
-    unleashAndSetPhase()
-    // asyncEcho(isDuringInit, power.banner)
-  }
-  private def unleashAndSetPhase() {
-    if (isReplPower) {
-    //  power.unleash()
-      // Set the phase to "typer"
-      // intp beSilentDuring phaseCommand("typer")
-    }
-  }
-
-  def asyncEcho(async: Boolean, msg: => String) {
-    if (async) asyncMessage(msg)
-    else echo(msg)
-  }
-
-  def verbosity() = {
-    val old = intp.printResults
-    intp.printResults = !old
-    echo("Switched " + (if (old) "off" else "on") + " result printing.")
-  }
-
-  /** 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 = {
-    if (line startsWith ":") {
-      val cmd = line.tail takeWhile (x => !x.isWhitespace)
-      uniqueCommand(cmd) match {
-        case Some(lc) => lc(line.tail stripPrefix cmd dropWhile (_.isWhitespace))
-        case _        => ambiguousError(cmd)
-      }
-    }
-    else if (intp.global == null) Result(keepRunning = false, None)  // Notice failure to create compiler
-    else Result(keepRunning = true, interpretStartingWith(line))
-  }
-
-  private def readWhile(cond: String => Boolean) = {
-    Iterator continually in.readLine("") takeWhile (x => x != null && cond(x))
-  }
-
-  def pasteCommand(arg: String): Result = {
-    var shouldReplay: Option[String] = None
-    def result = Result(keepRunning = true, shouldReplay)
-    val (raw, file) =
-      if (arg.isEmpty) (false, None)
-      else {
-        val r = """(-raw)?(\s+)?([^\-]\S*)?""".r
-        arg match {
-          case r(flag, sep, name) =>
-            if (flag != null && name != null && sep == null)
-              echo(s"""I assume you mean "$flag $name"?""")
-            (flag != null, Option(name))
-          case _ =>
-            echo("usage: :paste -raw file")
-            return result
-        }
-      }
-    val code = file match {
-      case Some(name) =>
-        withFile(name)(f => {
-          shouldReplay = Some(s":paste $arg")
-          val s = f.slurp.trim
-          if (s.isEmpty) echo(s"File contains no code: $f")
-          else echo(s"Pasting file $f...")
-          s
-        }) getOrElse ""
-      case None =>
-        echo("// Entering paste mode (ctrl-D to finish)\n")
-        val text = (readWhile(_ => true) mkString "\n").trim
-        if (text.isEmpty) echo("\n// Nothing pasted, nothing gained.\n")
-        else echo("\n// Exiting paste mode, now interpreting.\n")
-        text
-    }
-    def interpretCode() = {
-      val res = intp interpret code
-      // if input is incomplete, let the compiler try to say why
-      if (res == IR.Incomplete) {
-        echo("The pasted code is incomplete!\n")
-        // Remembrance of Things Pasted in an object
-        val errless = intp compileSources new BatchSourceFile("", s"object pastel {\n$code\n}")
-        if (errless) echo("...but compilation found no error? Good luck with that.")
-      }
-    }
-    def compileCode() = {
-      val errless = intp compileSources new BatchSourceFile("", code)
-      if (!errless) echo("There were compilation errors!")
-    }
-    if (code.nonEmpty) {
-      if (raw) compileCode() else interpretCode()
-    }
-    result
-  }
-
-  private object paste extends Pasted {
-    val ContinueString = "     | "
-    val PromptString   = "scala> "
-
-    def interpret(line: String): Unit = {
-      echo(line.trim)
-      intp interpret line
-      echo("")
-    }
-
-    def transcript(start: String) = {
-      echo("\n// Detected repl transcript paste: ctrl-D to finish.\n")
-      apply(Iterator(start) ++ readWhile(_.trim != PromptString.trim))
-    }
-  }
-  import paste.{ ContinueString, PromptString }
-
-  /** Interpret expressions starting with the first line.
-    * Read lines until a complete compilation unit is available
-    * or until a syntax error has been seen.  If a full unit is
-    * read, go ahead and interpret it.  Return the full string
-    * to be recorded for replay, if any.
-    */
-  def interpretStartingWith(code: String): Option[String] = {
-    // signal completion non-completion input has been received
-    in.completion.resetVerbosity()
-
-    def reallyInterpret = {
-      val reallyResult = intp.interpret(code)
-      (reallyResult, reallyResult match {
-        case IR.Error       => None
-        case IR.Success     => Some(code)
-        case IR.Incomplete  =>
-          if (in.interactive && code.endsWith("\n\n")) {
-            echo("You typed two blank lines.  Starting a new command.")
-            None
-          }
-          else in.readLine(ContinueString) match {
-            case null =>
-              // we know compilation is going to fail since we're at EOF and the
-              // parser thinks the input is still incomplete, but since this is
-              // a file being read non-interactively we want to fail.  So we send
-              // it straight to the compiler for the nice error message.
-              intp.compileString(code)
-              None
-
-            case line => interpretStartingWith(code + "\n" + line)
-          }
-      })
-    }
-
-    /** Here we place ourselves between the user and the interpreter and examine
-      *  the input they are ostensibly submitting.  We intervene in several cases:
-      *
-      *  1) If the line starts with "scala> " it is assumed to be an interpreter paste.
-      *  2) If the line starts with "." (but not ".." or "./") it is treated as an invocation
-      *     on the previous result.
-      *  3) If the Completion object's execute returns Some(_), we inject that value
-      *     and avoid the interpreter, as it's likely not valid scala code.
-      */
-    if (code == "") None
-    else if (!paste.running && code.trim.startsWith(PromptString)) {
-      paste.transcript(code)
-      None
-    }
-    else if (Completion.looksLikeInvocation(code) && intp.mostRecentVar != "") {
-      interpretStartingWith(intp.mostRecentVar + code)
-    }
-    else if (code.trim startsWith "//") {
-      // line comment, do nothing
-      None
-    }
-    else
-      reallyInterpret._2
-  }
-
-  // runs :load `file` on any files passed via -i
-  def loadFiles(settings: Settings) = settings match {
-    case settings: GenericRunnerSettings =>
-      for (filename <- settings.loadfiles.value) {
-        val cmd = ":load " + filename
-        command(cmd)
-        addReplay(cmd)
-        echo("")
-      }
-    case _ =>
-  }
-
-  /** Tries to create a JLineReader, falling back to SimpleReader:
-    *  unless settings or properties are such that it should start
-    *  with SimpleReader.
-    */
-  def chooseReader(settings: Settings): InteractiveReader = {
-    if (settings.Xnojline || Properties.isEmacsShell)
-      SimpleReader()
-    else try new JLineReader(
-      if (settings.noCompletion) NoCompletion
-      else new SparkJLineCompletion(intp)
-    )
-    catch {
-      case ex @ (_: Exception | _: NoClassDefFoundError) =>
-        echo("Failed to created JLineReader: " + ex + "\nFalling back to SimpleReader.")
-        SimpleReader()
-    }
-  }
-  protected def tagOfStaticClass[T: ClassTag]: u.TypeTag[T] =
-    u.TypeTag[T](
-      m,
-      new TypeCreator {
-        def apply[U <: Universe with Singleton](m: Mirror[U]): U # Type =
-          m.staticClass(classTag[T].runtimeClass.getName).toTypeConstructor.asInstanceOf[U # Type]
-      })
-
-  private def loopPostInit() {
-    // Bind intp somewhere out of the regular namespace where
-    // we can get at it in generated code.
-    intp.quietBind(NamedParam[SparkIMain]("$intp", intp)(tagOfStaticClass[SparkIMain], classTag[SparkIMain]))
-    // Auto-run code via some setting.
-    ( replProps.replAutorunCode.option
-      flatMap (f => io.File(f).safeSlurp())
-      foreach (intp quietRun _)
-      )
-    // classloader and power mode setup
-    intp.setContextClassLoader()
-    if (isReplPower) {
-     // replProps.power setValue true
-     // unleashAndSetPhase()
-     // asyncMessage(power.banner)
-    }
-    // SI-7418 Now, and only now, can we enable TAB completion.
-    in match {
-      case x: JLineReader => x.consoleReader.postInit
-      case _              =>
-    }
-  }
-  def process(settings: Settings): Boolean = savingContextLoader {
-    this.settings = settings
-    createInterpreter()
-
-    // sets in to some kind of reader depending on environmental cues
-    in = in0.fold(chooseReader(settings))(r => SimpleReader(r, out, interactive = true))
-    globalFuture = future {
-      intp.initializeSynchronous()
-      loopPostInit()
-      !intp.reporter.hasErrors
-    }
-    import scala.concurrent.duration._
-    Await.ready(globalFuture, 10 seconds)
-    printWelcome()
+  override def commands: List[LoopCommand] = sparkStandardCommands
+
+  /** 
+   * We override `loadFiles` because we need to initialize Spark *before* the REPL
+   * sees any files, so that the Spark context is visible in those files. This is a bit of a
+   * hack, but there isn't another hook available to us at this point.
+   */
+  override def loadFiles(settings: Settings): Unit = {
     initializeSpark()
-    loadFiles(settings)
-
-    try loop()
-    catch AbstractOrMissingHandler()
-    finally closeInterpreter()
-
-    true
+    super.loadFiles(settings)
   }
-
-  @deprecated("Use `process` instead", "2.9.0")
-  def main(settings: Settings): Unit = process(settings) //used by sbt
 }
 
 object SparkILoop {
-  implicit def loopToInterpreter(repl: SparkILoop): SparkIMain = repl.intp
 
-  // 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 = {
-    import java.io.{ BufferedReader, StringReader, OutputStreamWriter }
-
-    stringFromStream { ostream =>
-      Console.withOut(ostream) {
-        val output = new JPrintWriter(new OutputStreamWriter(ostream), true) {
-          override def write(str: String) = {
-            // completely skip continuation lines
-            if (str forall (ch => ch.isWhitespace || ch == '|')) ()
-            else super.write(str)
-          }
-        }
-        val input = new BufferedReader(new StringReader(code.trim + "\n")) {
-          override def readLine(): String = {
-            val s = super.readLine()
-            // helping out by printing the line being interpreted.
-            if (s != null)
-              output.println(s)
-            s
-          }
-        }
-        val repl = new SparkILoop(input, output)
-        if (settings.classpath.isDefault)
-          settings.classpath.value = sys.props("java.class.path")
-
-        repl process settings
-      }
-    }
-  }
-
-  /** Creates an interpreter loop with default settings and feeds
-    *  the given code to it as input.
-    */
+  /** 
+   * 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 = {
     import java.io.{ BufferedReader, StringReader, OutputStreamWriter }
 
     stringFromStream { ostream =>
       Console.withOut(ostream) {
-        val input    = new BufferedReader(new StringReader(code))
-        val output   = new JPrintWriter(new OutputStreamWriter(ostream), true)
-        val repl     = new SparkILoop(input, output)
+        val input = new BufferedReader(new StringReader(code))
+        val output = new JPrintWriter(new OutputStreamWriter(ostream), true)
+        val repl = new SparkILoop(input, output)
 
         if (sets.classpath.isDefault)
           sets.classpath.value = sys.props("java.class.path")
diff --git a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkIMain.scala b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkIMain.scala
deleted file mode 100644
index 1cb910f376060..0000000000000
--- a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkIMain.scala
+++ /dev/null
@@ -1,1319 +0,0 @@
-/* NSC -- new Scala compiler
- * Copyright 2005-2013 LAMP/EPFL
- * @author  Martin Odersky
- */
-
-package scala
-package tools.nsc
-package interpreter
-
-import PartialFunction.cond
-import scala.language.implicitConversions
-import scala.beans.BeanProperty
-import scala.collection.mutable
-import scala.concurrent.{ Future, ExecutionContext }
-import scala.reflect.runtime.{ universe => ru }
-import scala.reflect.{ ClassTag, classTag }
-import scala.reflect.internal.util.{ BatchSourceFile, SourceFile }
-import scala.tools.util.PathResolver
-import scala.tools.nsc.io.AbstractFile
-import scala.tools.nsc.typechecker.{ TypeStrings, StructuredTypeStrings }
-import scala.tools.nsc.util.{ ScalaClassLoader, stringFromReader, stringFromWriter, StackTraceOps }
-import scala.tools.nsc.util.Exceptional.unwrap
-import javax.script.{AbstractScriptEngine, Bindings, ScriptContext, ScriptEngine, ScriptEngineFactory, ScriptException, CompiledScript, Compilable}
-
-/** An interpreter for Scala code.
-  *
-  *  The main public entry points are compile(), interpret(), and bind().
-  *  The compile() method loads a complete Scala file.  The interpret() method
-  *  executes one line of Scala code at the request of the user.  The bind()
-  *  method binds an object to a variable that can then be used by later
-  *  interpreted code.
-  *
-  *  The overall approach is based on compiling the requested code and then
-  *  using a Java classloader and Java reflection to run the code
-  *  and access its results.
-  *
-  *  In more detail, a single compiler instance is used
-  *  to accumulate all successfully compiled or interpreted Scala code.  To
-  *  "interpret" a line of code, the compiler generates a fresh object that
-  *  includes the line of code and which has public member(s) to export
-  *  all variables defined by that code.  To extract the result of an
-  *  interpreted line to show the user, a second "result object" is created
-  *  which imports the variables exported by the above object and then
-  *  exports members called "$eval" and "$print". To accomodate user expressions
-  *  that read from variables or methods defined in previous statements, "import"
-  *  statements are used.
-  *
-  *  This interpreter shares the strengths and weaknesses of using the
-  *  full compiler-to-Java.  The main strength is that interpreted code
-  *  behaves exactly as does compiled code, including running at full speed.
-  *  The main weakness is that redefining classes and methods is not handled
-  *  properly, because rebinding at the Java level is technically difficult.
-  *
-  *  @author Moez A. Abdel-Gawad
-  *  @author Lex Spoon
-  */
-class SparkIMain(@BeanProperty val factory: ScriptEngineFactory, initialSettings: Settings,
-  protected val out: JPrintWriter) extends AbstractScriptEngine with Compilable with SparkImports {
-  imain =>
-
-  setBindings(createBindings, ScriptContext.ENGINE_SCOPE)
-  object replOutput extends ReplOutput(settings.Yreploutdir) { }
-
-  @deprecated("Use replOutput.dir instead", "2.11.0")
-  def virtualDirectory = replOutput.dir
-  // Used in a test case.
-  def showDirectory() = replOutput.show(out)
-
-  private[nsc] var printResults               = true      // whether to print result lines
-  private[nsc] 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
-  private var _executionWrapper               = ""        // code to be wrapped around all lines
-
-  /** 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
-    *  through the roof if we wait for it.  So we initialize it with a future and
-    *  use a lazy val to ensure that any attempt to use the compiler object waits
-    *  on the future.
-    */
-  private var _classLoader: util.AbstractFileClassLoader = null                              // active classloader
-  private val _compiler: ReplGlobal                 = newCompiler(settings, reporter)   // our private compiler
-
-  def compilerClasspath: Seq[java.net.URL] = (
-    if (isInitializeComplete) global.classPath.asURLs
-    else new PathResolver(settings).result.asURLs  // the compiler's classpath
-    )
-  def settings = initialSettings
-  // Run the code body with the given boolean settings flipped to true.
-  def withoutWarnings[T](body: => T): T = beQuietDuring {
-    val saved = settings.nowarn.value
-    if (!saved)
-      settings.nowarn.value = true
-
-    try body
-    finally if (!saved) settings.nowarn.value = false
-  }
-
-  /** construct an interpreter that reports to Console */
-  def this(settings: Settings, out: JPrintWriter) = this(null, settings, out)
-  def this(factory: ScriptEngineFactory, settings: Settings) = this(factory, settings, new NewLinePrintWriter(new ConsoleWriter, true))
-  def this(settings: Settings) = this(settings, new NewLinePrintWriter(new ConsoleWriter, true))
-  def this(factory: ScriptEngineFactory) = this(factory, new Settings())
-  def this() = this(new Settings())
-
-  lazy val formatting: Formatting = new Formatting {
-    val prompt = Properties.shellPromptString
-  }
-  lazy val reporter: SparkReplReporter = new SparkReplReporter(this)
-
-  import formatting._
-  import reporter.{ printMessage, printUntruncatedMessage }
-
-  // This exists mostly because using the reporter too early leads to deadlock.
-  private def echo(msg: String) { Console println msg }
-  private def _initSources = List(new BatchSourceFile("", "class $repl_$init { }"))
-  private def _initialize() = {
-    try {
-      // if this crashes, REPL will hang its head in shame
-      val run = new _compiler.Run()
-      assert(run.typerPhase != NoPhase, "REPL requires a typer phase.")
-      run compileSources _initSources
-      _initializeComplete = true
-      true
-    }
-    catch AbstractOrMissingHandler()
-  }
-  private def tquoted(s: String) = "\"\"\"" + s + "\"\"\""
-  private val logScope = scala.sys.props contains "scala.repl.scope"
-  private def scopelog(msg: String) = if (logScope) Console.err.println(msg)
-
-  // argument is a thunk to execute after init is done
-  def initialize(postInitSignal: => Unit) {
-    synchronized {
-      if (_isInitialized == null) {
-        _isInitialized =
-          Future(try _initialize() finally postInitSignal)(ExecutionContext.global)
-      }
-    }
-  }
-  def initializeSynchronous(): Unit = {
-    if (!isInitializeComplete) {
-      _initialize()
-      assert(global != null, global)
-    }
-  }
-  def isInitializeComplete = _initializeComplete
-
-  lazy val global: Global = {
-    if (!isInitializeComplete) _initialize()
-    _compiler
-  }
-
-  import global._
-  import definitions.{ ObjectClass, termMember, dropNullaryMethod}
-
-  lazy val runtimeMirror = ru.runtimeMirror(classLoader)
-
-  private def noFatal(body: => Symbol): Symbol = try body catch { case _: FatalError => NoSymbol }
-
-  def getClassIfDefined(path: String)  = (
-    noFatal(runtimeMirror staticClass path)
-      orElse noFatal(rootMirror staticClass path)
-    )
-  def getModuleIfDefined(path: String) = (
-    noFatal(runtimeMirror staticModule path)
-      orElse noFatal(rootMirror staticModule path)
-    )
-
-  implicit class ReplTypeOps(tp: Type) {
-    def andAlso(fn: Type => Type): Type = if (tp eq NoType) tp else fn(tp)
-  }
-
-  // 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 {
-    val global: imain.global.type = imain.global
-  } with Naming {
-    // make sure we don't overwrite their unwisely named res3 etc.
-    def freshUserTermName(): TermName = {
-      val name = newTermName(freshUserVarName())
-      if (replScope containsName name) freshUserTermName()
-      else name
-    }
-    def isInternalTermName(name: Name) = isInternalVarName("" + name)
-  }
-  import naming._
-
-  object deconstruct extends {
-    val global: imain.global.type = imain.global
-  } with StructuredTypeStrings
-
-  lazy val memberHandlers = new {
-    val intp: imain.type = imain
-  } with SparkMemberHandlers
-  import memberHandlers._
-
-  /** Temporarily be quiet */
-  def beQuietDuring[T](body: => T): T = {
-    val saved = printResults
-    printResults = false
-    try body
-    finally printResults = saved
-  }
-  def beSilentDuring[T](operation: => T): T = {
-    val saved = totalSilence
-    totalSilence = true
-    try operation
-    finally totalSilence = saved
-  }
-
-  def quietRun[T](code: String) = beQuietDuring(interpret(code))
-
-  /** takes AnyRef because it may be binding a Throwable or an Exceptional */
-  private def withLastExceptionLock[T](body: => T, alt: => T): T = {
-    assert(bindExceptions, "withLastExceptionLock called incorrectly.")
-    bindExceptions = false
-
-    try     beQuietDuring(body)
-    catch   logAndDiscard("withLastExceptionLock", alt)
-    finally bindExceptions = true
-  }
-
-  def executionWrapper = _executionWrapper
-  def setExecutionWrapper(code: String) = _executionWrapper = code
-  def clearExecutionWrapper() = _executionWrapper = ""
-
-  /** interpreter settings */
-  lazy val isettings = new SparkISettings(this)
-
-  /** Instantiate a compiler.  Overridable. */
-  protected def newCompiler(settings: Settings, reporter: reporters.Reporter): ReplGlobal = {
-    settings.outputDirs setSingleOutput replOutput.dir
-    settings.exposeEmptyPackage.value = true
-    new Global(settings, reporter) with ReplGlobal { override def toString: String = "" }
-  }
-
-  /** Parent classloader.  Overridable. */
-  protected def parentClassLoader: ClassLoader =
-    settings.explicitParentLoader.getOrElse( this.getClass.getClassLoader() )
-
-  /* A single class loader is used for all commands interpreted by this Interpreter.
-     It would also be possible to create a new class loader for each command
-     to interpret.  The advantages of the current approach are:
-
-       - Expressions are only evaluated one time.  This is especially
-         significant for I/O, e.g. "val x = Console.readLine"
-
-     The main disadvantage is:
-
-       - Objects, classes, and methods cannot be rebound.  Instead, definitions
-         shadow the old ones, and old code objects refer to the old
-         definitions.
-  */
-  def resetClassLoader() = {
-    repldbg("Setting new classloader: was " + _classLoader)
-    _classLoader = null
-    ensureClassLoader()
-  }
-  final def ensureClassLoader() {
-    if (_classLoader == null)
-      _classLoader = makeClassLoader()
-  }
-  def classLoader: util.AbstractFileClassLoader = {
-    ensureClassLoader()
-    _classLoader
-  }
-
-  def backticked(s: String): String = (
-    (s split '.').toList map {
-      case "_"                               => "_"
-      case s if nme.keywords(newTermName(s)) => s"`$s`"
-      case s                                 => s
-    } mkString "."
-    )
-  def readRootPath(readPath: String) = getModuleIfDefined(readPath)
-
-  abstract class PhaseDependentOps {
-    def shift[T](op: => T): T
-
-    def path(name: => Name): String = shift(path(symbolOfName(name)))
-    def path(sym: Symbol): String = backticked(shift(sym.fullName))
-    def sig(sym: Symbol): String  = shift(sym.defString)
-  }
-  object typerOp extends PhaseDependentOps {
-    def shift[T](op: => T): T = exitingTyper(op)
-  }
-  object flatOp extends PhaseDependentOps {
-    def shift[T](op: => T): T = exitingFlatten(op)
-  }
-
-  def originalPath(name: String): String = originalPath(name: TermName)
-  def originalPath(name: Name): String   = typerOp path name
-  def originalPath(sym: Symbol): String  = typerOp path sym
-  def flatPath(sym: Symbol): String      = flatOp shift sym.javaClassName
-  def translatePath(path: String) = {
-    val sym = if (path endsWith "$") symbolOfTerm(path.init) else symbolOfIdent(path)
-    sym.toOption map flatPath
-  }
-  def translateEnclosingClass(n: String) = symbolOfTerm(n).enclClass.toOption map flatPath
-
-  private class TranslatingClassLoader(parent: ClassLoader) extends util.AbstractFileClassLoader(replOutput.dir, parent) {
-    /** Overridden here to try translating a simple name to the generated
-      *  class name if the original attempt fails.  This method is used by
-      *  getResourceAsStream as well as findClass.
-      */
-    override protected def findAbstractFile(name: String): AbstractFile =
-      super.findAbstractFile(name) match {
-        case null if _initializeComplete => translatePath(name) map (super.findAbstractFile(_)) orNull
-        case file => file
-      }
-  }
-  private def makeClassLoader(): util.AbstractFileClassLoader =
-    new TranslatingClassLoader(parentClassLoader match {
-      case null   => ScalaClassLoader fromURLs compilerClasspath
-      case p      => new ScalaClassLoader.URLClassLoader(compilerClasspath, p)
-    })
-
-  // Set the current Java "context" class loader to this interpreter's class loader
-  def setContextClassLoader() = classLoader.setAsContext()
-
-  def allDefinedNames: List[Name]  = exitingTyper(replScope.toList.map(_.name).sorted)
-  def unqualifiedIds: List[String] = allDefinedNames map (_.decode) sorted
-
-  /** Most recent tree handled which wasn't wholly synthetic. */
-  private def mostRecentlyHandledTree: Option[Tree] = {
-    prevRequests.reverse foreach { req =>
-      req.handlers.reverse foreach {
-        case x: MemberDefHandler if x.definesValue && !isInternalTermName(x.name) => return Some(x.member)
-        case _ => ()
-      }
-    }
-    None
-  }
-
-  private def updateReplScope(sym: Symbol, isDefined: Boolean) {
-    def log(what: String) {
-      val mark = if (sym.isType) "t " else "v "
-      val name = exitingTyper(sym.nameString)
-      val info = cleanTypeAfterTyper(sym)
-      val defn = sym defStringSeenAs info
-
-      scopelog(f"[$mark$what%6s] $name%-25s $defn%s")
-    }
-    if (ObjectClass isSubClass sym.owner) return
-    // unlink previous
-    replScope lookupAll sym.name foreach { sym =>
-      log("unlink")
-      replScope unlink sym
-    }
-    val what = if (isDefined) "define" else "import"
-    log(what)
-    replScope enter sym
-  }
-
-  def recordRequest(req: Request) {
-    if (req == null)
-      return
-
-    prevRequests += req
-
-    // warning about serially defining companions.  It'd be easy
-    // enough to just redefine them together but that may not always
-    // be what people want so I'm waiting until I can do it better.
-    exitingTyper {
-      req.defines filterNot (s => req.defines contains s.companionSymbol) foreach { newSym =>
-        val oldSym = replScope lookup newSym.name.companionName
-        if (Seq(oldSym, newSym).permutations exists { case Seq(s1, s2) => s1.isClass && s2.isModule }) {
-          replwarn(s"warning: previously defined $oldSym is not a companion to $newSym.")
-          replwarn("Companions must be defined together; you may wish to use :paste mode for this.")
-        }
-      }
-    }
-    exitingTyper {
-      req.imports foreach (sym => updateReplScope(sym, isDefined = false))
-      req.defines foreach (sym => updateReplScope(sym, isDefined = true))
-    }
-  }
-
-  private[nsc] def replwarn(msg: => String) {
-    if (!settings.nowarnings)
-      printMessage(msg)
-  }
-
-  def compileSourcesKeepingRun(sources: SourceFile*) = {
-    val run = new Run()
-    assert(run.typerPhase != NoPhase, "REPL requires a typer phase.")
-    reporter.reset()
-    run compileSources sources.toList
-    (!reporter.hasErrors, run)
-  }
-
-  /** Compile an nsc SourceFile.  Returns true if there are
-    *  no compilation errors, or false otherwise.
-    */
-  def compileSources(sources: SourceFile*): Boolean =
-    compileSourcesKeepingRun(sources: _*)._1
-
-  /** Compile a string.  Returns true if there are no
-    *  compilation errors, or false otherwise.
-    */
-  def compileString(code: String): Boolean =
-    compileSources(new BatchSourceFile("