diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE
index 5834813319bfd..ff116cb1fbde2 100644
--- a/R/pkg/NAMESPACE
+++ b/R/pkg/NAMESPACE
@@ -12,7 +12,8 @@ export("print.jobj")
# MLlib integration
exportMethods("glm",
- "predict")
+ "predict",
+ "summary")
# Job group lifecycle management methods
export("setJobGroup",
@@ -26,7 +27,9 @@ exportMethods("arrange",
"collect",
"columns",
"count",
+ "crosstab",
"describe",
+ "dim",
"distinct",
"dropna",
"dtypes",
@@ -43,11 +46,15 @@ exportMethods("arrange",
"isLocal",
"join",
"limit",
+ "names",
+ "ncol",
+ "nrow",
"orderBy",
"mutate",
"names",
"persist",
"printSchema",
+ "rbind",
"registerTempTable",
"rename",
"repartition",
@@ -64,6 +71,7 @@ exportMethods("arrange",
"summarize",
"take",
"unionAll",
+ "unique",
"unpersist",
"where",
"withColumn",
diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R
index a58433df3c8c1..b4065d2944bdc 100644
--- a/R/pkg/R/DataFrame.R
+++ b/R/pkg/R/DataFrame.R
@@ -255,6 +255,16 @@ setMethod("names",
columns(x)
})
+#' @rdname columns
+setMethod("names<-",
+ signature(x = "DataFrame"),
+ function(x, value) {
+ if (!is.null(value)) {
+ sdf <- callJMethod(x@sdf, "toDF", listToSeq(as.list(value)))
+ dataFrame(sdf)
+ }
+ })
+
#' Register Temporary Table
#'
#' Registers a DataFrame as a Temporary Table in the SQLContext
@@ -473,6 +483,18 @@ setMethod("distinct",
dataFrame(sdf)
})
+#' @title Distinct rows in a DataFrame
+#
+#' @description Returns a new DataFrame containing distinct rows in this DataFrame
+#'
+#' @rdname unique
+#' @aliases unique
+setMethod("unique",
+ signature(x = "DataFrame"),
+ function(x) {
+ distinct(x)
+ })
+
#' Sample
#'
#' Return a sampled subset of this DataFrame using a random seed.
@@ -534,6 +556,58 @@ setMethod("count",
callJMethod(x@sdf, "count")
})
+#' @title Number of rows for a DataFrame
+#' @description Returns number of rows in a DataFrames
+#'
+#' @name nrow
+#'
+#' @rdname nrow
+#' @aliases count
+setMethod("nrow",
+ signature(x = "DataFrame"),
+ function(x) {
+ count(x)
+ })
+
+#' Returns the number of columns in a DataFrame
+#'
+#' @param x a SparkSQL DataFrame
+#'
+#' @rdname ncol
+#' @export
+#' @examples
+#'\dontrun{
+#' sc <- sparkR.init()
+#' sqlContext <- sparkRSQL.init(sc)
+#' path <- "path/to/file.json"
+#' df <- jsonFile(sqlContext, path)
+#' ncol(df)
+#' }
+setMethod("ncol",
+ signature(x = "DataFrame"),
+ function(x) {
+ length(columns(x))
+ })
+
+#' Returns the dimentions (number of rows and columns) of a DataFrame
+#' @param x a SparkSQL DataFrame
+#'
+#' @rdname dim
+#' @export
+#' @examples
+#'\dontrun{
+#' sc <- sparkR.init()
+#' sqlContext <- sparkRSQL.init(sc)
+#' path <- "path/to/file.json"
+#' df <- jsonFile(sqlContext, path)
+#' dim(df)
+#' }
+setMethod("dim",
+ signature(x = "DataFrame"),
+ function(x) {
+ c(count(x), ncol(x))
+ })
+
#' Collects all the elements of a Spark DataFrame and coerces them into an R data.frame.
#'
#' @param x A SparkSQL DataFrame
@@ -1231,6 +1305,22 @@ setMethod("unionAll",
dataFrame(unioned)
})
+#' @title Union two or more DataFrames
+#
+#' @description Returns a new DataFrame containing rows of all parameters.
+#
+#' @rdname rbind
+#' @aliases unionAll
+setMethod("rbind",
+ signature(... = "DataFrame"),
+ function(x, ..., deparse.level = 1) {
+ if (nargs() == 3) {
+ unionAll(x, ...)
+ } else {
+ unionAll(x, Recall(..., deparse.level = 1))
+ }
+ })
+
#' Intersect
#'
#' Return a new DataFrame containing rows only in both this DataFrame
@@ -1322,9 +1412,11 @@ setMethod("write.df",
"org.apache.spark.sql.parquet")
}
allModes <- c("append", "overwrite", "error", "ignore")
+ # nolint start
if (!(mode %in% allModes)) {
stop('mode should be one of "append", "overwrite", "error", "ignore"')
}
+ # nolint end
jmode <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "saveMode", mode)
options <- varargsToEnv(...)
if (!is.null(path)) {
@@ -1384,9 +1476,11 @@ setMethod("saveAsTable",
"org.apache.spark.sql.parquet")
}
allModes <- c("append", "overwrite", "error", "ignore")
+ # nolint start
if (!(mode %in% allModes)) {
stop('mode should be one of "append", "overwrite", "error", "ignore"')
}
+ # nolint end
jmode <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "saveMode", mode)
options <- varargsToEnv(...)
callJMethod(df@sdf, "saveAsTable", tableName, source, jmode, options)
@@ -1554,3 +1648,31 @@ setMethod("fillna",
}
dataFrame(sdf)
})
+
+#' crosstab
+#'
+#' Computes a pair-wise frequency table of the given columns. Also known as a contingency
+#' table. The number of distinct values for each column should be less than 1e4. At most 1e6
+#' non-zero pair frequencies will be returned.
+#'
+#' @param col1 name of the first column. Distinct items will make the first item of each row.
+#' @param col2 name of the second column. Distinct items will make the column names of the output.
+#' @return a local R data.frame representing the contingency table. The first column of each row
+#' will be the distinct values of `col1` and the column names will be the distinct values
+#' of `col2`. The name of the first column will be `$col1_$col2`. Pairs that have no
+#' occurrences will have zero as their counts.
+#'
+#' @rdname statfunctions
+#' @export
+#' @examples
+#' \dontrun{
+#' df <- jsonFile(sqlCtx, "/path/to/file.json")
+#' ct = crosstab(df, "title", "gender")
+#' }
+setMethod("crosstab",
+ signature(x = "DataFrame", col1 = "character", col2 = "character"),
+ function(x, col1, col2) {
+ statFunctions <- callJMethod(x@sdf, "stat")
+ sct <- callJMethod(statFunctions, "crosstab", col1, col2)
+ collect(dataFrame(sct))
+ })
diff --git a/R/pkg/R/RDD.R b/R/pkg/R/RDD.R
index d2d096709245d..051e441d4e063 100644
--- a/R/pkg/R/RDD.R
+++ b/R/pkg/R/RDD.R
@@ -85,7 +85,9 @@ setMethod("initialize", "PipelinedRDD", function(.Object, prev, func, jrdd_val)
isPipelinable <- function(rdd) {
e <- rdd@env
+ # nolint start
!(e$isCached || e$isCheckpointed)
+ # nolint end
}
if (!inherits(prev, "PipelinedRDD") || !isPipelinable(prev)) {
@@ -97,7 +99,8 @@ setMethod("initialize", "PipelinedRDD", function(.Object, prev, func, jrdd_val)
# prev_serializedMode is used during the delayed computation of JRDD in getJRDD
} else {
pipelinedFunc <- function(partIndex, part) {
- func(partIndex, prev@func(partIndex, part))
+ f <- prev@func
+ func(partIndex, f(partIndex, part))
}
.Object@func <- cleanClosure(pipelinedFunc)
.Object@prev_jrdd <- prev@prev_jrdd # maintain the pipeline
@@ -841,7 +844,7 @@ setMethod("sampleRDD",
if (withReplacement) {
count <- rpois(1, fraction)
if (count > 0) {
- res[(len + 1):(len + count)] <- rep(list(elem), count)
+ res[ (len + 1) : (len + count) ] <- rep(list(elem), count)
len <- len + count
}
} else {
@@ -1261,12 +1264,12 @@ setMethod("pipeRDD",
signature(x = "RDD", command = "character"),
function(x, command, env = list()) {
func <- function(part) {
- trim.trailing.func <- function(x) {
+ trim_trailing_func <- function(x) {
sub("[\r\n]*$", "", toString(x))
}
- input <- unlist(lapply(part, trim.trailing.func))
+ input <- unlist(lapply(part, trim_trailing_func))
res <- system2(command, stdout = TRUE, input = input, env = env)
- lapply(res, trim.trailing.func)
+ lapply(res, trim_trailing_func)
}
lapplyPartition(x, func)
})
diff --git a/R/pkg/R/backend.R b/R/pkg/R/backend.R
index 2fb6fae55f28c..49162838b8d1a 100644
--- a/R/pkg/R/backend.R
+++ b/R/pkg/R/backend.R
@@ -110,6 +110,8 @@ invokeJava <- function(isStatic, objId, methodName, ...) {
# TODO: check the status code to output error information
returnStatus <- readInt(conn)
- stopifnot(returnStatus == 0)
+ if (returnStatus != 0) {
+ stop(readString(conn))
+ }
readObject(conn)
}
diff --git a/R/pkg/R/client.R b/R/pkg/R/client.R
index 6f772158ddfe8..c811d1dac3bd5 100644
--- a/R/pkg/R/client.R
+++ b/R/pkg/R/client.R
@@ -48,7 +48,7 @@ generateSparkSubmitArgs <- function(args, sparkHome, jars, sparkSubmitOpts, pack
jars <- paste("--jars", jars)
}
- if (packages != "") {
+ if (!identical(packages, "")) {
packages <- paste("--packages", packages)
}
diff --git a/R/pkg/R/column.R b/R/pkg/R/column.R
index 2892e1416cc65..eeaf9f193b728 100644
--- a/R/pkg/R/column.R
+++ b/R/pkg/R/column.R
@@ -65,7 +65,7 @@ functions <- c("min", "max", "sum", "avg", "mean", "count", "abs", "sqrt",
"acos", "asin", "atan", "cbrt", "ceiling", "cos", "cosh", "exp",
"expm1", "floor", "log", "log10", "log1p", "rint", "sign",
"sin", "sinh", "tan", "tanh", "toDegrees", "toRadians")
-binary_mathfunctions<- c("atan2", "hypot")
+binary_mathfunctions <- c("atan2", "hypot")
createOperator <- function(op) {
setMethod(op,
diff --git a/R/pkg/R/context.R b/R/pkg/R/context.R
index 43be9c904fdf6..720990e1c6087 100644
--- a/R/pkg/R/context.R
+++ b/R/pkg/R/context.R
@@ -121,7 +121,7 @@ parallelize <- function(sc, coll, numSlices = 1) {
numSlices <- length(coll)
sliceLen <- ceiling(length(coll) / numSlices)
- slices <- split(coll, rep(1:(numSlices + 1), each = sliceLen)[1:length(coll)])
+ slices <- split(coll, rep(1: (numSlices + 1), each = sliceLen)[1:length(coll)])
# Serialize each slice: obtain a list of raws, or a list of lists (slices) of
# 2-tuples of raws
diff --git a/R/pkg/R/deserialize.R b/R/pkg/R/deserialize.R
index 7d1f6b0819ed0..6d364f77be7ee 100644
--- a/R/pkg/R/deserialize.R
+++ b/R/pkg/R/deserialize.R
@@ -102,11 +102,11 @@ readList <- function(con) {
readRaw <- function(con) {
dataLen <- readInt(con)
- data <- readBin(con, raw(), as.integer(dataLen), endian = "big")
+ readBin(con, raw(), as.integer(dataLen), endian = "big")
}
readRawLen <- function(con, dataLen) {
- data <- readBin(con, raw(), as.integer(dataLen), endian = "big")
+ readBin(con, raw(), as.integer(dataLen), endian = "big")
}
readDeserialize <- function(con) {
diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R
index 39b5586f7c90e..71d1e348c4efb 100644
--- a/R/pkg/R/generics.R
+++ b/R/pkg/R/generics.R
@@ -59,6 +59,10 @@ setGeneric("count", function(x) { standardGeneric("count") })
# @export
setGeneric("countByValue", function(x) { standardGeneric("countByValue") })
+# @rdname statfunctions
+# @export
+setGeneric("crosstab", function(x, col1, col2) { standardGeneric("crosstab") })
+
# @rdname distinct
# @export
setGeneric("distinct", function(x, numPartitions = 1) { standardGeneric("distinct") })
@@ -250,8 +254,10 @@ setGeneric("flatMapValues", function(X, FUN) { standardGeneric("flatMapValues")
# @rdname intersection
# @export
-setGeneric("intersection", function(x, other, numPartitions = 1) {
- standardGeneric("intersection") })
+setGeneric("intersection",
+ function(x, other, numPartitions = 1) {
+ standardGeneric("intersection")
+ })
# @rdname keys
# @export
@@ -485,9 +491,7 @@ setGeneric("sample",
#' @rdname sample
#' @export
setGeneric("sample_frac",
- function(x, withReplacement, fraction, seed) {
- standardGeneric("sample_frac")
- })
+ function(x, withReplacement, fraction, seed) { standardGeneric("sample_frac") })
#' @rdname saveAsParquetFile
#' @export
@@ -549,8 +553,8 @@ setGeneric("withColumn", function(x, colName, col) { standardGeneric("withColumn
#' @rdname withColumnRenamed
#' @export
-setGeneric("withColumnRenamed", function(x, existingCol, newCol) {
- standardGeneric("withColumnRenamed") })
+setGeneric("withColumnRenamed",
+ function(x, existingCol, newCol) { standardGeneric("withColumnRenamed") })
###################### Column Methods ##########################
@@ -665,3 +669,7 @@ setGeneric("upper", function(x) { standardGeneric("upper") })
#' @rdname glm
#' @export
setGeneric("glm")
+
+#' @rdname rbind
+#' @export
+setGeneric("rbind", signature = "...")
diff --git a/R/pkg/R/mllib.R b/R/pkg/R/mllib.R
index 258e354081fc1..efddcc1d8d71c 100644
--- a/R/pkg/R/mllib.R
+++ b/R/pkg/R/mllib.R
@@ -27,7 +27,7 @@ setClass("PipelineModel", representation(model = "jobj"))
#' Fits a generalized linear model, similarly to R's glm(). Also see the glmnet package.
#'
#' @param formula A symbolic description of the model to be fitted. Currently only a few formula
-#' operators are supported, including '~' and '+'.
+#' operators are supported, including '~', '+', '-', and '.'.
#' @param data DataFrame for training
#' @param family Error distribution. "gaussian" -> linear regression, "binomial" -> logistic reg.
#' @param lambda Regularization parameter
@@ -71,3 +71,29 @@ setMethod("predict", signature(object = "PipelineModel"),
function(object, newData) {
return(dataFrame(callJMethod(object@model, "transform", newData@sdf)))
})
+
+#' Get the summary of a model
+#'
+#' Returns the summary of a model produced by glm(), similarly to R's summary().
+#'
+#' @param model A fitted MLlib model
+#' @return a list with a 'coefficient' component, which is the matrix of coefficients. See
+#' summary.glm for more information.
+#' @rdname glm
+#' @export
+#' @examples
+#'\dontrun{
+#' model <- glm(y ~ x, trainingData)
+#' summary(model)
+#'}
+setMethod("summary", signature(object = "PipelineModel"),
+ function(object) {
+ features <- callJStatic("org.apache.spark.ml.api.r.SparkRWrappers",
+ "getModelFeatures", object@model)
+ weights <- callJStatic("org.apache.spark.ml.api.r.SparkRWrappers",
+ "getModelWeights", object@model)
+ coefficients <- as.matrix(unlist(weights))
+ colnames(coefficients) <- c("Estimate")
+ rownames(coefficients) <- unlist(features)
+ return(list(coefficients = coefficients))
+ })
diff --git a/R/pkg/R/pairRDD.R b/R/pkg/R/pairRDD.R
index ebc6ff65e9d0f..199c3fd6ab1b2 100644
--- a/R/pkg/R/pairRDD.R
+++ b/R/pkg/R/pairRDD.R
@@ -202,8 +202,8 @@ setMethod("partitionBy",
packageNamesArr <- serialize(.sparkREnv$.packages,
connection = NULL)
- broadcastArr <- lapply(ls(.broadcastNames), function(name) {
- get(name, .broadcastNames) })
+ broadcastArr <- lapply(ls(.broadcastNames),
+ function(name) { get(name, .broadcastNames) })
jrdd <- getJRDD(x)
# We create a PairwiseRRDD that extends RDD[(Int, Array[Byte])],
@@ -879,7 +879,7 @@ setMethod("sampleByKey",
if (withReplacement) {
count <- rpois(1, frac)
if (count > 0) {
- res[(len + 1):(len + count)] <- rep(list(elem), count)
+ res[ (len + 1) : (len + count) ] <- rep(list(elem), count)
len <- len + count
}
} else {
diff --git a/R/pkg/R/sparkR.R b/R/pkg/R/sparkR.R
index 79b79d70943cb..e83104f116422 100644
--- a/R/pkg/R/sparkR.R
+++ b/R/pkg/R/sparkR.R
@@ -22,7 +22,8 @@
connExists <- function(env) {
tryCatch({
exists(".sparkRCon", envir = env) && isOpen(env[[".sparkRCon"]])
- }, error = function(err) {
+ },
+ error = function(err) {
return(FALSE)
})
}
@@ -104,16 +105,13 @@ sparkR.init <- function(
return(get(".sparkRjsc", envir = .sparkREnv))
}
- sparkMem <- Sys.getenv("SPARK_MEM", "1024m")
jars <- suppressWarnings(normalizePath(as.character(sparkJars)))
# Classpath separator is ";" on Windows
# URI needs four /// as from http://stackoverflow.com/a/18522792
if (.Platform$OS.type == "unix") {
- collapseChar <- ":"
uriSep <- "//"
} else {
- collapseChar <- ";"
uriSep <- "////"
}
@@ -156,7 +154,8 @@ sparkR.init <- function(
.sparkREnv$backendPort <- backendPort
tryCatch({
connectBackend("localhost", backendPort)
- }, error = function(err) {
+ },
+ error = function(err) {
stop("Failed to connect JVM\n")
})
@@ -267,7 +266,8 @@ sparkRHive.init <- function(jsc = NULL) {
ssc <- callJMethod(sc, "sc")
hiveCtx <- tryCatch({
newJObject("org.apache.spark.sql.hive.HiveContext", ssc)
- }, error = function(err) {
+ },
+ error = function(err) {
stop("Spark SQL is not built with Hive support")
})
diff --git a/R/pkg/R/utils.R b/R/pkg/R/utils.R
index 3f45589a50443..4f9f4d9cad2a8 100644
--- a/R/pkg/R/utils.R
+++ b/R/pkg/R/utils.R
@@ -32,7 +32,7 @@ convertJListToRList <- function(jList, flatten, logicalUpperBound = NULL,
}
results <- if (arrSize > 0) {
- lapply(0:(arrSize - 1),
+ lapply(0 : (arrSize - 1),
function(index) {
obj <- callJMethod(jList, "get", as.integer(index))
@@ -572,7 +572,7 @@ mergePartitions <- function(rdd, zip) {
keys <- list()
}
if (lengthOfValues > 1) {
- values <- part[(lengthOfKeys + 1) : (len - 1)]
+ values <- part[ (lengthOfKeys + 1) : (len - 1) ]
} else {
values <- list()
}
diff --git a/R/pkg/inst/tests/test_binary_function.R b/R/pkg/inst/tests/test_binary_function.R
index dca0657c57e0d..f054ac9a87d61 100644
--- a/R/pkg/inst/tests/test_binary_function.R
+++ b/R/pkg/inst/tests/test_binary_function.R
@@ -40,7 +40,7 @@ test_that("union on two RDDs", {
expect_equal(actual, c(as.list(nums), mockFile))
expect_equal(getSerializedMode(union.rdd), "byte")
- rdd<- map(text.rdd, function(x) {x})
+ rdd <- map(text.rdd, function(x) {x})
union.rdd <- unionRDD(rdd, text.rdd)
actual <- collect(union.rdd)
expect_equal(actual, as.list(c(mockFile, mockFile)))
diff --git a/R/pkg/inst/tests/test_client.R b/R/pkg/inst/tests/test_client.R
index 30b05c1a2afcd..8a20991f89af8 100644
--- a/R/pkg/inst/tests/test_client.R
+++ b/R/pkg/inst/tests/test_client.R
@@ -30,3 +30,7 @@ test_that("no package specified doesn't add packages flag", {
expect_equal(gsub("[[:space:]]", "", args),
"")
})
+
+test_that("multiple packages don't produce a warning", {
+ expect_that(generateSparkSubmitArgs("", "", "", "", c("A", "B")), not(gives_warning()))
+})
diff --git a/R/pkg/inst/tests/test_mllib.R b/R/pkg/inst/tests/test_mllib.R
index a492763344ae6..f272de78ad4a6 100644
--- a/R/pkg/inst/tests/test_mllib.R
+++ b/R/pkg/inst/tests/test_mllib.R
@@ -35,8 +35,27 @@ test_that("glm and predict", {
test_that("predictions match with native glm", {
training <- createDataFrame(sqlContext, iris)
- model <- glm(Sepal_Width ~ Sepal_Length, data = training)
+ model <- glm(Sepal_Width ~ Sepal_Length + Species, data = training)
vals <- collect(select(predict(model, training), "prediction"))
- rVals <- predict(glm(Sepal.Width ~ Sepal.Length, data = iris), iris)
- expect_true(all(abs(rVals - vals) < 1e-9), rVals - vals)
+ rVals <- predict(glm(Sepal.Width ~ Sepal.Length + Species, data = iris), iris)
+ expect_true(all(abs(rVals - vals) < 1e-6), rVals - vals)
+})
+
+test_that("dot minus and intercept vs native glm", {
+ training <- createDataFrame(sqlContext, iris)
+ model <- glm(Sepal_Width ~ . - Species + 0, data = training)
+ vals <- collect(select(predict(model, training), "prediction"))
+ rVals <- predict(glm(Sepal.Width ~ . - Species + 0, data = iris), iris)
+ expect_true(all(abs(rVals - vals) < 1e-6), rVals - vals)
+})
+
+test_that("summary coefficients match with native glm", {
+ training <- createDataFrame(sqlContext, iris)
+ stats <- summary(glm(Sepal_Width ~ Sepal_Length + Species, data = training))
+ coefs <- as.vector(stats$coefficients)
+ rCoefs <- as.vector(coef(glm(Sepal.Width ~ Sepal.Length + Species, data = iris)))
+ expect_true(all(abs(rCoefs - coefs) < 1e-6))
+ expect_true(all(
+ as.character(stats$features) ==
+ c("(Intercept)", "Sepal_Length", "Species__versicolor", "Species__virginica")))
})
diff --git a/R/pkg/inst/tests/test_rdd.R b/R/pkg/inst/tests/test_rdd.R
index 6c3aaab8c711e..71aed2bb9d6a8 100644
--- a/R/pkg/inst/tests/test_rdd.R
+++ b/R/pkg/inst/tests/test_rdd.R
@@ -250,7 +250,7 @@ test_that("flatMapValues() on pairwise RDDs", {
expect_equal(actual, list(list(1,1), list(1,2), list(2,3), list(2,4)))
# Generate x to x+1 for every value
- actual <- collect(flatMapValues(intRdd, function(x) { x:(x + 1) }))
+ actual <- collect(flatMapValues(intRdd, function(x) { x: (x + 1) }))
expect_equal(actual,
list(list(1L, -1), list(1L, 0), list(2L, 100), list(2L, 101),
list(2L, 1), list(2L, 2), list(1L, 200), list(1L, 201)))
@@ -293,7 +293,7 @@ test_that("sumRDD() on RDDs", {
})
test_that("keyBy on RDDs", {
- func <- function(x) { x*x }
+ func <- function(x) { x * x }
keys <- keyBy(rdd, func)
actual <- collect(keys)
expect_equal(actual, lapply(nums, function(x) { list(func(x), x) }))
@@ -311,7 +311,7 @@ test_that("repartition/coalesce on RDDs", {
r2 <- repartition(rdd, 6)
expect_equal(numPartitions(r2), 6L)
count <- length(collectPartition(r2, 0L))
- expect_true(count >=0 && count <= 4)
+ expect_true(count >= 0 && count <= 4)
# coalesce
r3 <- coalesce(rdd, 1)
diff --git a/R/pkg/inst/tests/test_sparkSQL.R b/R/pkg/inst/tests/test_sparkSQL.R
index a3039d36c9402..9faee8d59c3af 100644
--- a/R/pkg/inst/tests/test_sparkSQL.R
+++ b/R/pkg/inst/tests/test_sparkSQL.R
@@ -88,6 +88,9 @@ test_that("create DataFrame from RDD", {
df <- createDataFrame(sqlContext, rdd, list("a", "b"))
expect_is(df, "DataFrame")
expect_equal(count(df), 10)
+ expect_equal(nrow(df), 10)
+ expect_equal(ncol(df), 2)
+ expect_equal(dim(df), c(10, 2))
expect_equal(columns(df), c("a", "b"))
expect_equal(dtypes(df), list(c("a", "int"), c("b", "string")))
@@ -112,7 +115,8 @@ test_that("create DataFrame from RDD", {
df <- jsonFile(sqlContext, jsonPathNa)
hiveCtx <- tryCatch({
newJObject("org.apache.spark.sql.hive.test.TestHiveContext", ssc)
- }, error = function(err) {
+ },
+ error = function(err) {
skip("Hive is not build with SparkSQL, skipped")
})
sql(hiveCtx, "CREATE TABLE people (name string, age double, height float)")
@@ -127,7 +131,9 @@ test_that("create DataFrame from RDD", {
expect_equal(dtypes(df2), list(c("name", "string"), c("age", "int"), c("height", "float")))
expect_equal(collect(where(df2, df2$name == "Bob")), c("Bob", 16, 176.5))
- localDF <- data.frame(name=c("John", "Smith", "Sarah"), age=c(19, 23, 18), height=c(164.10, 181.4, 173.7))
+ localDF <- data.frame(name=c("John", "Smith", "Sarah"),
+ age=c(19, 23, 18),
+ height=c(164.10, 181.4, 173.7))
df <- createDataFrame(sqlContext, localDF, schema)
expect_is(df, "DataFrame")
expect_equal(count(df), 3)
@@ -488,7 +494,7 @@ test_that("head() and first() return the correct data", {
expect_equal(nrow(testFirst), 1)
})
-test_that("distinct() on DataFrames", {
+test_that("distinct() and unique on DataFrames", {
lines <- c("{\"name\":\"Michael\"}",
"{\"name\":\"Andy\", \"age\":30}",
"{\"name\":\"Justin\", \"age\":19}",
@@ -500,6 +506,10 @@ test_that("distinct() on DataFrames", {
uniques <- distinct(df)
expect_is(uniques, "DataFrame")
expect_equal(count(uniques), 3)
+
+ uniques2 <- unique(df)
+ expect_is(uniques2, "DataFrame")
+ expect_equal(count(uniques2), 3)
})
test_that("sample on a DataFrame", {
@@ -602,7 +612,8 @@ test_that("write.df() as parquet file", {
test_that("test HiveContext", {
hiveCtx <- tryCatch({
newJObject("org.apache.spark.sql.hive.test.TestHiveContext", ssc)
- }, error = function(err) {
+ },
+ error = function(err) {
skip("Hive is not build with SparkSQL, skipped")
})
df <- createExternalTable(hiveCtx, "json", jsonPath, "json")
@@ -664,10 +675,12 @@ test_that("column binary mathfunctions", {
expect_equal(collect(select(df, atan2(df$a, df$b)))[2, "ATAN2(a, b)"], atan2(2, 6))
expect_equal(collect(select(df, atan2(df$a, df$b)))[3, "ATAN2(a, b)"], atan2(3, 7))
expect_equal(collect(select(df, atan2(df$a, df$b)))[4, "ATAN2(a, b)"], atan2(4, 8))
+ ## nolint start
expect_equal(collect(select(df, hypot(df$a, df$b)))[1, "HYPOT(a, b)"], sqrt(1^2 + 5^2))
expect_equal(collect(select(df, hypot(df$a, df$b)))[2, "HYPOT(a, b)"], sqrt(2^2 + 6^2))
expect_equal(collect(select(df, hypot(df$a, df$b)))[3, "HYPOT(a, b)"], sqrt(3^2 + 7^2))
expect_equal(collect(select(df, hypot(df$a, df$b)))[4, "HYPOT(a, b)"], sqrt(4^2 + 8^2))
+ ## nolint end
})
test_that("string operators", {
@@ -809,7 +822,7 @@ test_that("isLocal()", {
expect_false(isLocal(df))
})
-test_that("unionAll(), except(), and intersect() on a DataFrame", {
+test_that("unionAll(), rbind(), except(), and intersect() on a DataFrame", {
df <- jsonFile(sqlContext, jsonPath)
lines <- c("{\"name\":\"Bob\", \"age\":24}",
@@ -824,6 +837,11 @@ test_that("unionAll(), except(), and intersect() on a DataFrame", {
expect_equal(count(unioned), 6)
expect_equal(first(unioned)$name, "Michael")
+ unioned2 <- arrange(rbind(unioned, df, df2), df$age)
+ expect_is(unioned2, "DataFrame")
+ expect_equal(count(unioned2), 12)
+ expect_equal(first(unioned2)$name, "Michael")
+
excepted <- arrange(except(df, df2), desc(df$age))
expect_is(unioned, "DataFrame")
expect_equal(count(excepted), 2)
@@ -847,7 +865,7 @@ test_that("withColumn() and withColumnRenamed()", {
expect_equal(columns(newDF2)[1], "newerAge")
})
-test_that("mutate() and rename()", {
+test_that("mutate(), rename() and names()", {
df <- jsonFile(sqlContext, jsonPath)
newDF <- mutate(df, newAge = df$age + 2)
expect_equal(length(columns(newDF)), 3)
@@ -857,6 +875,10 @@ test_that("mutate() and rename()", {
newDF2 <- rename(df, newerAge = df$age)
expect_equal(length(columns(newDF2)), 2)
expect_equal(columns(newDF2)[1], "newerAge")
+
+ names(newDF2) <- c("newerName", "evenNewerAge")
+ expect_equal(length(names(newDF2)), 2)
+ expect_equal(names(newDF2)[1], "newerName")
})
test_that("write.df() on DataFrame and works with parquetFile", {
@@ -874,7 +896,7 @@ test_that("parquetFile works with multiple input paths", {
write.df(df, parquetPath2, "parquet", mode="overwrite")
parquetDF <- parquetFile(sqlContext, parquetPath, parquetPath2)
expect_is(parquetDF, "DataFrame")
- expect_equal(count(parquetDF), count(df)*2)
+ expect_equal(count(parquetDF), count(df) * 2)
})
test_that("describe() on a DataFrame", {
@@ -987,6 +1009,24 @@ test_that("fillna() on a DataFrame", {
expect_identical(expected, actual)
})
+test_that("crosstab() on a DataFrame", {
+ rdd <- lapply(parallelize(sc, 0:3), function(x) {
+ list(paste0("a", x %% 3), paste0("b", x %% 2))
+ })
+ df <- toDF(rdd, list("a", "b"))
+ ct <- crosstab(df, "a", "b")
+ ordered <- ct[order(ct$a_b),]
+ row.names(ordered) <- NULL
+ expected <- data.frame("a_b" = c("a0", "a1", "a2"), "b0" = c(1, 0, 1), "b1" = c(1, 1, 0),
+ stringsAsFactors = FALSE, row.names = NULL)
+ expect_identical(expected, ordered)
+})
+
+test_that("SQL error message is returned from JVM", {
+ retError <- tryCatch(sql(sqlContext, "select * from blah"), error = function(e) e)
+ expect_equal(grepl("Table Not Found: blah", retError), TRUE)
+})
+
unlink(parquetPath)
unlink(jsonPath)
unlink(jsonPathNa)
diff --git a/R/run-tests.sh b/R/run-tests.sh
index e82ad0ba2cd06..18a1e13bdc655 100755
--- a/R/run-tests.sh
+++ b/R/run-tests.sh
@@ -23,7 +23,7 @@ FAILED=0
LOGFILE=$FWDIR/unit-tests.out
rm -f $LOGFILE
-SPARK_TESTING=1 $FWDIR/../bin/sparkR --driver-java-options "-Dlog4j.configuration=file:$FWDIR/log4j.properties" $FWDIR/pkg/tests/run-all.R 2>&1 | tee -a $LOGFILE
+SPARK_TESTING=1 $FWDIR/../bin/sparkR --conf spark.buffer.pageSize=4m --driver-java-options "-Dlog4j.configuration=file:$FWDIR/log4j.properties" $FWDIR/pkg/tests/run-all.R 2>&1 | tee -a $LOGFILE
FAILED=$((PIPESTATUS[0]||$FAILED))
if [[ $FAILED != 0 ]]; then
diff --git a/bin/pyspark b/bin/pyspark
index f9dbddfa53560..8f2a3b5a7717b 100755
--- a/bin/pyspark
+++ b/bin/pyspark
@@ -82,4 +82,4 @@ fi
export PYSPARK_DRIVER_PYTHON
export PYSPARK_DRIVER_PYTHON_OPTS
-exec "$SPARK_HOME"/bin/spark-submit pyspark-shell-main "$@"
+exec "$SPARK_HOME"/bin/spark-submit pyspark-shell-main --name "PySparkShell" "$@"
diff --git a/bin/pyspark2.cmd b/bin/pyspark2.cmd
index 45e9e3def5121..3c6169983e76b 100644
--- a/bin/pyspark2.cmd
+++ b/bin/pyspark2.cmd
@@ -35,4 +35,4 @@ set PYTHONPATH=%SPARK_HOME%\python\lib\py4j-0.8.2.1-src.zip;%PYTHONPATH%
set OLD_PYTHONSTARTUP=%PYTHONSTARTUP%
set PYTHONSTARTUP=%SPARK_HOME%\python\pyspark\shell.py
-call %SPARK_HOME%\bin\spark-submit2.cmd pyspark-shell-main %*
+call %SPARK_HOME%\bin\spark-submit2.cmd pyspark-shell-main --name "PySparkShell" %*
diff --git a/bin/spark-shell b/bin/spark-shell
index a6dc863d83fc6..00ab7afd118b5 100755
--- a/bin/spark-shell
+++ b/bin/spark-shell
@@ -47,11 +47,11 @@ function main() {
# (see https://github.com/sbt/sbt/issues/562).
stty -icanon min 1 -echo > /dev/null 2>&1
export SPARK_SUBMIT_OPTS="$SPARK_SUBMIT_OPTS -Djline.terminal=unix"
- "$FWDIR"/bin/spark-submit --class org.apache.spark.repl.Main "$@"
+ "$FWDIR"/bin/spark-submit --class org.apache.spark.repl.Main --name "Spark shell" "$@"
stty icanon echo > /dev/null 2>&1
else
export SPARK_SUBMIT_OPTS
- "$FWDIR"/bin/spark-submit --class org.apache.spark.repl.Main "$@"
+ "$FWDIR"/bin/spark-submit --class org.apache.spark.repl.Main --name "Spark shell" "$@"
fi
}
diff --git a/bin/spark-shell2.cmd b/bin/spark-shell2.cmd
index 251309d67f860..b9b0f510d7f5d 100644
--- a/bin/spark-shell2.cmd
+++ b/bin/spark-shell2.cmd
@@ -32,4 +32,4 @@ if "x%SPARK_SUBMIT_OPTS%"=="x" (
set SPARK_SUBMIT_OPTS="%SPARK_SUBMIT_OPTS% -Dscala.usejavacp=true"
:run_shell
-%SPARK_HOME%\bin\spark-submit2.cmd --class org.apache.spark.repl.Main %*
+%SPARK_HOME%\bin\spark-submit2.cmd --class org.apache.spark.repl.Main --name "Spark shell" %*
diff --git a/build/sbt-launch-lib.bash b/build/sbt-launch-lib.bash
index 504be48b358fa..7930a38b9674a 100755
--- a/build/sbt-launch-lib.bash
+++ b/build/sbt-launch-lib.bash
@@ -51,9 +51,13 @@ acquire_sbt_jar () {
printf "Attempting to fetch sbt\n"
JAR_DL="${JAR}.part"
if [ $(command -v curl) ]; then
- (curl --silent ${URL1} > "${JAR_DL}" || curl --silent ${URL2} > "${JAR_DL}") && mv "${JAR_DL}" "${JAR}"
+ (curl --fail --location --silent ${URL1} > "${JAR_DL}" ||\
+ (rm -f "${JAR_DL}" && curl --fail --location --silent ${URL2} > "${JAR_DL}")) &&\
+ mv "${JAR_DL}" "${JAR}"
elif [ $(command -v wget) ]; then
- (wget --quiet ${URL1} -O "${JAR_DL}" || wget --quiet ${URL2} -O "${JAR_DL}") && mv "${JAR_DL}" "${JAR}"
+ (wget --quiet ${URL1} -O "${JAR_DL}" ||\
+ (rm -f "${JAR_DL}" && wget --quiet ${URL2} -O "${JAR_DL}")) &&\
+ mv "${JAR_DL}" "${JAR}"
else
printf "You do not have curl or wget installed, please install sbt manually from http://www.scala-sbt.org/\n"
exit -1
diff --git a/conf/log4j.properties.template b/conf/log4j.properties.template
index 3a2a88219818f..27006e45e932b 100644
--- a/conf/log4j.properties.template
+++ b/conf/log4j.properties.template
@@ -10,3 +10,7 @@ log4j.logger.org.spark-project.jetty=WARN
log4j.logger.org.spark-project.jetty.util.component.AbstractLifeCycle=ERROR
log4j.logger.org.apache.spark.repl.SparkIMain$exprTyper=INFO
log4j.logger.org.apache.spark.repl.SparkILoop$SparkILoopInterpreter=INFO
+
+# SPARK-9183: Settings to avoid annoying messages when looking up nonexistent UDFs in SparkSQL with Hive support
+log4j.logger.org.apache.hadoop.hive.metastore.RetryingHMSHandler=FATAL
+log4j.logger.org.apache.hadoop.hive.ql.exec.FunctionRegistry=ERROR
diff --git a/core/pom.xml b/core/pom.xml
index 95f36eb348698..202678779150b 100644
--- a/core/pom.xml
+++ b/core/pom.xml
@@ -34,6 +34,11 @@
Spark Project Corehttp://spark.apache.org/
+
+ org.apache.avro
+ avro-mapred
+ ${avro.mapred.classifier}
+ com.google.guavaguava
@@ -281,7 +286,7 @@
org.tachyonprojecttachyon-client
- 0.6.4
+ 0.7.0org.apache.hadoop
@@ -292,36 +297,12 @@
curator-recipes
- org.eclipse.jetty
- jetty-jsp
-
-
- org.eclipse.jetty
- jetty-webapp
-
-
- org.eclipse.jetty
- jetty-server
-
-
- org.eclipse.jetty
- jetty-servlet
+ org.tachyonproject
+ tachyon-underfs-glusterfs
- junit
- junit
-
-
- org.powermock
- powermock-module-junit4
-
-
- org.powermock
- powermock-api-mockito
-
-
- org.apache.curator
- curator-test
+ org.tachyonproject
+ tachyon-underfs-s3
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 1d460432be9ff..1aa6ba4201261 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
@@ -59,14 +59,14 @@ final class UnsafeShuffleExternalSorter {
private final Logger logger = LoggerFactory.getLogger(UnsafeShuffleExternalSorter.class);
- private static final int PAGE_SIZE = PackedRecordPointer.MAXIMUM_PAGE_SIZE_BYTES;
@VisibleForTesting
static final int DISK_WRITE_BUFFER_SIZE = 1024 * 1024;
- @VisibleForTesting
- static final int MAX_RECORD_SIZE = PAGE_SIZE - 4;
private final int initialSize;
private final int numPartitions;
+ private final int pageSizeBytes;
+ @VisibleForTesting
+ final int maxRecordSizeBytes;
private final TaskMemoryManager memoryManager;
private final ShuffleMemoryManager shuffleMemoryManager;
private final BlockManager blockManager;
@@ -109,7 +109,10 @@ public UnsafeShuffleExternalSorter(
this.numPartitions = numPartitions;
// Use getSizeAsKb (not bytes) to maintain backwards compatibility if no units are provided
this.fileBufferSizeBytes = (int) conf.getSizeAsKb("spark.shuffle.file.buffer", "32k") * 1024;
-
+ this.pageSizeBytes = (int) Math.min(
+ PackedRecordPointer.MAXIMUM_PAGE_SIZE_BYTES,
+ conf.getSizeAsBytes("spark.buffer.pageSize", "64m"));
+ this.maxRecordSizeBytes = pageSizeBytes - 4;
this.writeMetrics = writeMetrics;
initializeForWriting();
}
@@ -272,7 +275,11 @@ void spill() throws IOException {
}
private long getMemoryUsage() {
- return sorter.getMemoryUsage() + (allocatedPages.size() * (long) PAGE_SIZE);
+ long totalPageSize = 0;
+ for (MemoryBlock page : allocatedPages) {
+ totalPageSize += page.size();
+ }
+ return sorter.getMemoryUsage() + totalPageSize;
}
private long freeMemory() {
@@ -346,23 +353,23 @@ private void allocateSpaceForRecord(int requiredSpace) throws IOException {
// 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) {
+ if (requiredSpace > pageSizeBytes) {
throw new IOException("Required space " + requiredSpace + " is greater than page size (" +
- PAGE_SIZE + ")");
+ pageSizeBytes + ")");
} else {
- final long memoryAcquired = shuffleMemoryManager.tryToAcquire(PAGE_SIZE);
- if (memoryAcquired < PAGE_SIZE) {
+ final long memoryAcquired = shuffleMemoryManager.tryToAcquire(pageSizeBytes);
+ if (memoryAcquired < pageSizeBytes) {
shuffleMemoryManager.release(memoryAcquired);
spill();
- final long memoryAcquiredAfterSpilling = shuffleMemoryManager.tryToAcquire(PAGE_SIZE);
- if (memoryAcquiredAfterSpilling != PAGE_SIZE) {
+ final long memoryAcquiredAfterSpilling = shuffleMemoryManager.tryToAcquire(pageSizeBytes);
+ if (memoryAcquiredAfterSpilling != pageSizeBytes) {
shuffleMemoryManager.release(memoryAcquiredAfterSpilling);
- throw new IOException("Unable to acquire " + PAGE_SIZE + " bytes of memory");
+ throw new IOException("Unable to acquire " + pageSizeBytes + " bytes of memory");
}
}
- currentPage = memoryManager.allocatePage(PAGE_SIZE);
+ currentPage = memoryManager.allocatePage(pageSizeBytes);
currentPagePosition = currentPage.getBaseOffset();
- freeSpaceInCurrentPage = PAGE_SIZE;
+ freeSpaceInCurrentPage = pageSizeBytes;
allocatedPages.add(currentPage);
}
}
diff --git a/core/src/main/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriter.java
index 764578b181422..d47d6fc9c2ac4 100644
--- a/core/src/main/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriter.java
+++ b/core/src/main/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriter.java
@@ -129,6 +129,11 @@ public UnsafeShuffleWriter(
open();
}
+ @VisibleForTesting
+ public int maxRecordSizeBytes() {
+ return sorter.maxRecordSizeBytes;
+ }
+
/**
* This convenience method should only be called in test code.
*/
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
index bf1bc5dffba78..4d7e5b3dfba6e 100644
--- 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
@@ -17,9 +17,7 @@
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 com.google.common.primitives.UnsignedLongs;
import org.apache.spark.annotation.Private;
import org.apache.spark.unsafe.types.UTF8String;
@@ -30,81 +28,67 @@ 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 StringPrefixComparatorDesc STRING_DESC = new StringPrefixComparatorDesc();
+ public static final LongPrefixComparator LONG = new LongPrefixComparator();
+ public static final LongPrefixComparatorDesc LONG_DESC = new LongPrefixComparatorDesc();
public static final DoublePrefixComparator DOUBLE = new DoublePrefixComparator();
+ public static final DoublePrefixComparatorDesc DOUBLE_DESC = new DoublePrefixComparatorDesc();
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;
+ return UnsignedLongs.compare(aPrefix, bPrefix);
}
- 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 static long computePrefix(UTF8String value) {
+ return value == null ? 0L : value.getPrefix();
}
+ }
- public long computePrefix(UTF8String value) {
- return value == null ? 0L : computePrefix(value.getBytes());
+ public static final class StringPrefixComparatorDesc extends PrefixComparator {
+ @Override
+ public int compare(long bPrefix, long aPrefix) {
+ return UnsignedLongs.compare(aPrefix, bPrefix);
}
}
- /**
- * Prefix comparator for all integral types (boolean, byte, short, int, long).
- */
- public static final class IntegralPrefixComparator extends PrefixComparator {
+ public static final class LongPrefixComparator 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 LongPrefixComparatorDesc extends PrefixComparator {
+ @Override
+ public int compare(long b, long a) {
+ return (a < b) ? -1 : (a > b) ? 1 : 0;
+ }
}
- public static final class FloatPrefixComparator extends PrefixComparator {
+ public static final class DoublePrefixComparator extends PrefixComparator {
@Override
public int compare(long aPrefix, long bPrefix) {
- float a = Float.intBitsToFloat((int) aPrefix);
- float b = Float.intBitsToFloat((int) bPrefix);
- return Utils.nanSafeCompareFloats(a, b);
+ double a = Double.longBitsToDouble(aPrefix);
+ double b = Double.longBitsToDouble(bPrefix);
+ return Utils.nanSafeCompareDoubles(a, b);
}
- public long computePrefix(float value) {
- return Float.floatToIntBits(value) & 0xffffffffL;
+ public static long computePrefix(double value) {
+ return Double.doubleToLongBits(value);
}
-
- public final long NULL_PREFIX = computePrefix(Float.NEGATIVE_INFINITY);
}
- public static final class DoublePrefixComparator extends PrefixComparator {
+ public static final class DoublePrefixComparatorDesc extends PrefixComparator {
@Override
- public int compare(long aPrefix, long bPrefix) {
+ public int compare(long bPrefix, long aPrefix) {
double a = Double.longBitsToDouble(aPrefix);
double b = Double.longBitsToDouble(bPrefix);
return Utils.nanSafeCompareDoubles(a, b);
}
- public long computePrefix(double value) {
+ public static 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/UnsafeExternalSorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java
index 4d6731ee60af3..866e0b4151577 100644
--- 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
@@ -20,6 +20,9 @@
import java.io.IOException;
import java.util.LinkedList;
+import scala.runtime.AbstractFunction0;
+import scala.runtime.BoxedUnit;
+
import com.google.common.annotations.VisibleForTesting;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -41,10 +44,7 @@ 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 long pageSizeBytes;
private final PrefixComparator prefixComparator;
private final RecordComparator recordComparator;
private final int initialSize;
@@ -91,7 +91,19 @@ public UnsafeExternalSorter(
this.initialSize = initialSize;
// Use getSizeAsKb (not bytes) to maintain backwards compatibility for units
this.fileBufferSizeBytes = (int) conf.getSizeAsKb("spark.shuffle.file.buffer", "32k") * 1024;
+ this.pageSizeBytes = conf.getSizeAsBytes("spark.buffer.pageSize", "64m");
initializeForWriting();
+
+ // Register a cleanup task with TaskContext to ensure that memory is guaranteed to be freed at
+ // the end of the task. This is necessary to avoid memory leaks in when the downstream operator
+ // does not fully consume the sorter's output (e.g. sort followed by limit).
+ taskContext.addOnCompleteCallback(new AbstractFunction0() {
+ @Override
+ public BoxedUnit apply() {
+ freeMemory();
+ return null;
+ }
+ });
}
// TODO: metrics tracking + integration with shuffle write metrics
@@ -147,7 +159,16 @@ public void spill() throws IOException {
}
private long getMemoryUsage() {
- return sorter.getMemoryUsage() + (allocatedPages.size() * (long) PAGE_SIZE);
+ long totalPageSize = 0;
+ for (MemoryBlock page : allocatedPages) {
+ totalPageSize += page.size();
+ }
+ return sorter.getMemoryUsage() + totalPageSize;
+ }
+
+ @VisibleForTesting
+ public int getNumberOfAllocatedPages() {
+ return allocatedPages.size();
}
public long freeMemory() {
@@ -209,23 +230,23 @@ private void allocateSpaceForRecord(int requiredSpace) throws IOException {
// 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) {
+ if (requiredSpace > pageSizeBytes) {
throw new IOException("Required space " + requiredSpace + " is greater than page size (" +
- PAGE_SIZE + ")");
+ pageSizeBytes + ")");
} else {
- final long memoryAcquired = shuffleMemoryManager.tryToAcquire(PAGE_SIZE);
- if (memoryAcquired < PAGE_SIZE) {
+ final long memoryAcquired = shuffleMemoryManager.tryToAcquire(pageSizeBytes);
+ if (memoryAcquired < pageSizeBytes) {
shuffleMemoryManager.release(memoryAcquired);
spill();
- final long memoryAcquiredAfterSpilling = shuffleMemoryManager.tryToAcquire(PAGE_SIZE);
- if (memoryAcquiredAfterSpilling != PAGE_SIZE) {
+ final long memoryAcquiredAfterSpilling = shuffleMemoryManager.tryToAcquire(pageSizeBytes);
+ if (memoryAcquiredAfterSpilling != pageSizeBytes) {
shuffleMemoryManager.release(memoryAcquiredAfterSpilling);
- throw new IOException("Unable to acquire " + PAGE_SIZE + " bytes of memory");
+ throw new IOException("Unable to acquire " + pageSizeBytes + " bytes of memory");
}
}
- currentPage = memoryManager.allocatePage(PAGE_SIZE);
+ currentPage = memoryManager.allocatePage(pageSizeBytes);
currentPagePosition = currentPage.getBaseOffset();
- freeSpaceInCurrentPage = PAGE_SIZE;
+ freeSpaceInCurrentPage = pageSizeBytes;
allocatedPages.add(currentPage);
}
}
@@ -257,7 +278,7 @@ public void insertRecord(
currentPagePosition,
lengthInBytes);
currentPagePosition += lengthInBytes;
-
+ freeSpaceInCurrentPage -= totalSpaceRequired;
sorter.insertRecord(recordAddress, prefix);
}
diff --git a/core/src/main/resources/org/apache/spark/log4j-defaults-repl.properties b/core/src/main/resources/org/apache/spark/log4j-defaults-repl.properties
index b146f8a784127..689afea64f8db 100644
--- a/core/src/main/resources/org/apache/spark/log4j-defaults-repl.properties
+++ b/core/src/main/resources/org/apache/spark/log4j-defaults-repl.properties
@@ -10,3 +10,7 @@ log4j.logger.org.spark-project.jetty=WARN
log4j.logger.org.spark-project.jetty.util.component.AbstractLifeCycle=ERROR
log4j.logger.org.apache.spark.repl.SparkIMain$exprTyper=INFO
log4j.logger.org.apache.spark.repl.SparkILoop$SparkILoopInterpreter=INFO
+
+# SPARK-9183: Settings to avoid annoying messages when looking up nonexistent UDFs in SparkSQL with Hive support
+log4j.logger.org.apache.hadoop.hive.metastore.RetryingHMSHandler=FATAL
+log4j.logger.org.apache.hadoop.hive.ql.exec.FunctionRegistry=ERROR
diff --git a/core/src/main/resources/org/apache/spark/log4j-defaults.properties b/core/src/main/resources/org/apache/spark/log4j-defaults.properties
index 3a2a88219818f..27006e45e932b 100644
--- a/core/src/main/resources/org/apache/spark/log4j-defaults.properties
+++ b/core/src/main/resources/org/apache/spark/log4j-defaults.properties
@@ -10,3 +10,7 @@ log4j.logger.org.spark-project.jetty=WARN
log4j.logger.org.spark-project.jetty.util.component.AbstractLifeCycle=ERROR
log4j.logger.org.apache.spark.repl.SparkIMain$exprTyper=INFO
log4j.logger.org.apache.spark.repl.SparkILoop$SparkILoopInterpreter=INFO
+
+# SPARK-9183: Settings to avoid annoying messages when looking up nonexistent UDFs in SparkSQL with Hive support
+log4j.logger.org.apache.hadoop.hive.metastore.RetryingHMSHandler=FATAL
+log4j.logger.org.apache.hadoop.hive.ql.exec.FunctionRegistry=ERROR
diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala
index 2f4fcac890eef..eb75f26718e19 100644
--- a/core/src/main/scala/org/apache/spark/Accumulators.scala
+++ b/core/src/main/scala/org/apache/spark/Accumulators.scala
@@ -341,7 +341,4 @@ private[spark] object Accumulators extends Logging {
}
}
- def stringifyPartialValue(partialValue: Any): String = "%s".format(partialValue)
-
- def stringifyValue(value: Any): String = "%s".format(value)
}
diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala
index 443830f8d03b6..842bfdbadc948 100644
--- a/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala
+++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala
@@ -24,11 +24,23 @@ package org.apache.spark
private[spark] trait ExecutorAllocationClient {
/**
- * Express a preference to the cluster manager for a given total number of executors.
- * This can result in canceling pending requests or filing additional requests.
+ * Update the cluster manager on our scheduling needs. Three bits of information are included
+ * to help it make decisions.
+ * @param numExecutors The total number of executors we'd like to have. The cluster manager
+ * shouldn't kill any running executor to reach this number, but,
+ * if all existing executors were to die, this is the number of executors
+ * we'd want to be allocated.
+ * @param localityAwareTasks The number of tasks in all active stages that have a locality
+ * preferences. This includes running, pending, and completed tasks.
+ * @param hostToLocalTaskCount A map of hosts to the number of tasks from all active stages
+ * that would like to like to run on that host.
+ * This includes running, pending, and completed tasks.
* @return whether the request is acknowledged by the cluster manager.
*/
- private[spark] def requestTotalExecutors(numExecutors: Int): Boolean
+ private[spark] def requestTotalExecutors(
+ numExecutors: Int,
+ localityAwareTasks: Int,
+ hostToLocalTaskCount: Map[String, Int]): Boolean
/**
* Request an additional number of executors from the cluster manager.
diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala
index 648bcfe28cad2..1877aaf2cac55 100644
--- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala
+++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala
@@ -161,6 +161,12 @@ private[spark] class ExecutorAllocationManager(
// (2) an executor idle timeout has elapsed.
@volatile private var initializing: Boolean = true
+ // Number of locality aware tasks, used for executor placement.
+ private var localityAwareTasks = 0
+
+ // Host to possible task running on it, used for executor placement.
+ private var hostToLocalTaskCount: Map[String, Int] = Map.empty
+
/**
* Verify that the settings specified through the config are valid.
* If not, throw an appropriate exception.
@@ -295,7 +301,7 @@ private[spark] class ExecutorAllocationManager(
// If the new target has not changed, avoid sending a message to the cluster manager
if (numExecutorsTarget < oldNumExecutorsTarget) {
- client.requestTotalExecutors(numExecutorsTarget)
+ client.requestTotalExecutors(numExecutorsTarget, localityAwareTasks, hostToLocalTaskCount)
logDebug(s"Lowering target number of executors to $numExecutorsTarget (previously " +
s"$oldNumExecutorsTarget) because not all requested executors are actually needed")
}
@@ -349,7 +355,8 @@ private[spark] class ExecutorAllocationManager(
return 0
}
- val addRequestAcknowledged = testing || client.requestTotalExecutors(numExecutorsTarget)
+ val addRequestAcknowledged = testing ||
+ client.requestTotalExecutors(numExecutorsTarget, localityAwareTasks, hostToLocalTaskCount)
if (addRequestAcknowledged) {
val executorsString = "executor" + { if (delta > 1) "s" else "" }
logInfo(s"Requesting $delta new $executorsString because tasks are backlogged" +
@@ -519,6 +526,12 @@ private[spark] class ExecutorAllocationManager(
// Number of tasks currently running on the cluster. Should be 0 when no stages are active.
private var numRunningTasks: Int = _
+ // stageId to tuple (the number of task with locality preferences, a map where each pair is a
+ // node and the number of tasks that would like to be scheduled on that node) map,
+ // maintain the executor placement hints for each stage Id used by resource framework to better
+ // place the executors.
+ private val stageIdToExecutorPlacementHints = new mutable.HashMap[Int, (Int, Map[String, Int])]
+
override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted): Unit = {
initializing = false
val stageId = stageSubmitted.stageInfo.stageId
@@ -526,6 +539,24 @@ private[spark] class ExecutorAllocationManager(
allocationManager.synchronized {
stageIdToNumTasks(stageId) = numTasks
allocationManager.onSchedulerBacklogged()
+
+ // Compute the number of tasks requested by the stage on each host
+ var numTasksPending = 0
+ val hostToLocalTaskCountPerStage = new mutable.HashMap[String, Int]()
+ stageSubmitted.stageInfo.taskLocalityPreferences.foreach { locality =>
+ if (!locality.isEmpty) {
+ numTasksPending += 1
+ locality.foreach { location =>
+ val count = hostToLocalTaskCountPerStage.getOrElse(location.host, 0) + 1
+ hostToLocalTaskCountPerStage(location.host) = count
+ }
+ }
+ }
+ stageIdToExecutorPlacementHints.put(stageId,
+ (numTasksPending, hostToLocalTaskCountPerStage.toMap))
+
+ // Update the executor placement hints
+ updateExecutorPlacementHints()
}
}
@@ -534,6 +565,10 @@ private[spark] class ExecutorAllocationManager(
allocationManager.synchronized {
stageIdToNumTasks -= stageId
stageIdToTaskIndices -= stageId
+ stageIdToExecutorPlacementHints -= stageId
+
+ // Update the executor placement hints
+ updateExecutorPlacementHints()
// If this is the last stage with pending tasks, mark the scheduler queue as empty
// This is needed in case the stage is aborted for any reason
@@ -637,6 +672,29 @@ private[spark] class ExecutorAllocationManager(
def isExecutorIdle(executorId: String): Boolean = {
!executorIdToTaskIds.contains(executorId)
}
+
+ /**
+ * Update the Executor placement hints (the number of tasks with locality preferences,
+ * a map where each pair is a node and the number of tasks that would like to be scheduled
+ * on that node).
+ *
+ * These hints are updated when stages arrive and complete, so are not up-to-date at task
+ * granularity within stages.
+ */
+ def updateExecutorPlacementHints(): Unit = {
+ var localityAwareTasks = 0
+ val localityToCount = new mutable.HashMap[String, Int]()
+ stageIdToExecutorPlacementHints.values.foreach { case (numTasksPending, localities) =>
+ localityAwareTasks += numTasksPending
+ localities.foreach { case (hostname, count) =>
+ val updatedCount = localityToCount.getOrElse(hostname, 0) + count
+ localityToCount(hostname) = updatedCount
+ }
+ }
+
+ allocationManager.localityAwareTasks = localityAwareTasks
+ allocationManager.hostToLocalTaskCount = localityToCount.toMap
+ }
}
/**
diff --git a/core/src/main/scala/org/apache/spark/Partitioner.scala b/core/src/main/scala/org/apache/spark/Partitioner.scala
index ad68512dccb79..4b9d59975bdc2 100644
--- a/core/src/main/scala/org/apache/spark/Partitioner.scala
+++ b/core/src/main/scala/org/apache/spark/Partitioner.scala
@@ -56,7 +56,7 @@ object Partitioner {
*/
def defaultPartitioner(rdd: RDD[_], others: RDD[_]*): Partitioner = {
val bySize = (Seq(rdd) ++ others).sortBy(_.partitions.size).reverse
- for (r <- bySize if r.partitioner.isDefined) {
+ for (r <- bySize if r.partitioner.isDefined && r.partitioner.get.numPartitions > 0) {
return r.partitioner.get
}
if (rdd.context.conf.contains("spark.default.parallelism")) {
diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala
index 6cf36fbbd6254..08bab4bf2739f 100644
--- a/core/src/main/scala/org/apache/spark/SparkConf.scala
+++ b/core/src/main/scala/org/apache/spark/SparkConf.scala
@@ -18,11 +18,12 @@
package org.apache.spark
import java.util.concurrent.ConcurrentHashMap
-import java.util.concurrent.atomic.AtomicBoolean
import scala.collection.JavaConverters._
import scala.collection.mutable.LinkedHashSet
+import org.apache.avro.{SchemaNormalization, Schema}
+
import org.apache.spark.serializer.KryoSerializer
import org.apache.spark.util.Utils
@@ -161,6 +162,26 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging {
this
}
+ private final val avroNamespace = "avro.schema."
+
+ /**
+ * Use Kryo serialization and register the given set of Avro schemas so that the generic
+ * record serializer can decrease network IO
+ */
+ def registerAvroSchemas(schemas: Schema*): SparkConf = {
+ for (schema <- schemas) {
+ set(avroNamespace + SchemaNormalization.parsingFingerprint64(schema), schema.toString)
+ }
+ this
+ }
+
+ /** Gets all the avro schemas in the configuration used in the generic Avro record serializer */
+ def getAvroSchema: Map[Long, String] = {
+ getAll.filter { case (k, v) => k.startsWith(avroNamespace) }
+ .map { case (k, v) => (k.substring(avroNamespace.length).toLong, v) }
+ .toMap
+ }
+
/** Remove a parameter from the configuration */
def remove(key: String): SparkConf = {
settings.remove(key)
@@ -527,7 +548,9 @@ private[spark] object SparkConf extends Logging {
"spark.rpc.askTimeout" -> Seq(
AlternateConfig("spark.akka.askTimeout", "1.4")),
"spark.rpc.lookupTimeout" -> Seq(
- AlternateConfig("spark.akka.lookupTimeout", "1.4"))
+ AlternateConfig("spark.akka.lookupTimeout", "1.4")),
+ "spark.streaming.fileStream.minRememberDuration" -> Seq(
+ AlternateConfig("spark.streaming.minRememberDuration", "1.5"))
)
/**
diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala
index d00c012d80560..2d8aa25d81daa 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -471,7 +471,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
.orElse(Option(System.getenv("SPARK_MEM"))
.map(warnSparkMem))
.map(Utils.memoryStringToMb)
- .getOrElse(512)
+ .getOrElse(1024)
// Convert java options to env vars as a work around
// since we can't set env vars directly in sbt.
@@ -1382,16 +1382,29 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
}
/**
- * Express a preference to the cluster manager for a given total number of executors.
- * This can result in canceling pending requests or filing additional requests.
- * This is currently only supported in YARN mode. Return whether the request is received.
- */
- private[spark] override def requestTotalExecutors(numExecutors: Int): Boolean = {
+ * Update the cluster manager on our scheduling needs. Three bits of information are included
+ * to help it make decisions.
+ * @param numExecutors The total number of executors we'd like to have. The cluster manager
+ * shouldn't kill any running executor to reach this number, but,
+ * if all existing executors were to die, this is the number of executors
+ * we'd want to be allocated.
+ * @param localityAwareTasks The number of tasks in all active stages that have a locality
+ * preferences. This includes running, pending, and completed tasks.
+ * @param hostToLocalTaskCount A map of hosts to the number of tasks from all active stages
+ * that would like to like to run on that host.
+ * This includes running, pending, and completed tasks.
+ * @return whether the request is acknowledged by the cluster manager.
+ */
+ private[spark] override def requestTotalExecutors(
+ numExecutors: Int,
+ localityAwareTasks: Int,
+ hostToLocalTaskCount: scala.collection.immutable.Map[String, Int]
+ ): Boolean = {
assert(supportDynamicAllocation,
"Requesting executors is currently only supported in YARN and Mesos modes")
schedulerBackend match {
case b: CoarseGrainedSchedulerBackend =>
- b.requestTotalExecutors(numExecutors)
+ b.requestTotalExecutors(numExecutors, localityAwareTasks, hostToLocalTaskCount)
case _ =>
logWarning("Requesting executors is only supported in coarse-grained mode")
false
@@ -1676,33 +1689,57 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
Utils.removeShutdownHook(_shutdownHookRef)
}
- postApplicationEnd()
- _ui.foreach(_.stop())
+ Utils.tryLogNonFatalError {
+ postApplicationEnd()
+ }
+ Utils.tryLogNonFatalError {
+ _ui.foreach(_.stop())
+ }
if (env != null) {
- env.metricsSystem.report()
+ Utils.tryLogNonFatalError {
+ env.metricsSystem.report()
+ }
}
if (metadataCleaner != null) {
- metadataCleaner.cancel()
+ Utils.tryLogNonFatalError {
+ metadataCleaner.cancel()
+ }
+ }
+ Utils.tryLogNonFatalError {
+ _cleaner.foreach(_.stop())
+ }
+ Utils.tryLogNonFatalError {
+ _executorAllocationManager.foreach(_.stop())
}
- _cleaner.foreach(_.stop())
- _executorAllocationManager.foreach(_.stop())
if (_dagScheduler != null) {
- _dagScheduler.stop()
+ Utils.tryLogNonFatalError {
+ _dagScheduler.stop()
+ }
_dagScheduler = null
}
if (_listenerBusStarted) {
- listenerBus.stop()
- _listenerBusStarted = false
+ Utils.tryLogNonFatalError {
+ listenerBus.stop()
+ _listenerBusStarted = false
+ }
+ }
+ Utils.tryLogNonFatalError {
+ _eventLogger.foreach(_.stop())
}
- _eventLogger.foreach(_.stop())
if (env != null && _heartbeatReceiver != null) {
- env.rpcEnv.stop(_heartbeatReceiver)
+ Utils.tryLogNonFatalError {
+ env.rpcEnv.stop(_heartbeatReceiver)
+ }
+ }
+ Utils.tryLogNonFatalError {
+ _progressBar.foreach(_.stop())
}
- _progressBar.foreach(_.stop())
_taskScheduler = null
// TODO: Cache.stop()?
if (_env != null) {
- _env.stop()
+ Utils.tryLogNonFatalError {
+ _env.stop()
+ }
SparkEnv.set(null)
}
SparkContext.clearActiveContext()
@@ -1758,16 +1795,13 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
/**
* Run a function on a given set of partitions in an RDD and pass the results to the given
- * handler function. This is the main entry point for all actions in Spark. The allowLocal
- * flag specifies whether the scheduler can run the computation on the driver rather than
- * shipping it out to the cluster, for short actions like first().
+ * handler function. This is the main entry point for all actions in Spark.
*/
def runJob[T, U: ClassTag](
rdd: RDD[T],
func: (TaskContext, Iterator[T]) => U,
partitions: Seq[Int],
- allowLocal: Boolean,
- resultHandler: (Int, U) => Unit) {
+ resultHandler: (Int, U) => Unit): Unit = {
if (stopped.get()) {
throw new IllegalStateException("SparkContext has been shutdown")
}
@@ -1777,54 +1811,104 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
if (conf.getBoolean("spark.logLineage", false)) {
logInfo("RDD's recursive dependencies:\n" + rdd.toDebugString)
}
- dagScheduler.runJob(rdd, cleanedFunc, partitions, callSite, allowLocal,
- resultHandler, localProperties.get)
+ dagScheduler.runJob(rdd, cleanedFunc, partitions, callSite, resultHandler, localProperties.get)
progressBar.foreach(_.finishAll())
rdd.doCheckpoint()
}
/**
- * Run a function on a given set of partitions in an RDD and return the results as an array. The
- * allowLocal flag specifies whether the scheduler can run the computation on the driver rather
- * than shipping it out to the cluster, for short actions like first().
+ * Run a function on a given set of partitions in an RDD and return the results as an array.
+ */
+ def runJob[T, U: ClassTag](
+ rdd: RDD[T],
+ func: (TaskContext, Iterator[T]) => U,
+ partitions: Seq[Int]): Array[U] = {
+ val results = new Array[U](partitions.size)
+ runJob[T, U](rdd, func, partitions, (index, res) => results(index) = res)
+ results
+ }
+
+ /**
+ * Run a job on a given set of partitions of an RDD, but take a function of type
+ * `Iterator[T] => U` instead of `(TaskContext, Iterator[T]) => U`.
+ */
+ def runJob[T, U: ClassTag](
+ rdd: RDD[T],
+ func: Iterator[T] => U,
+ partitions: Seq[Int]): Array[U] = {
+ val cleanedFunc = clean(func)
+ runJob(rdd, (ctx: TaskContext, it: Iterator[T]) => cleanedFunc(it), partitions)
+ }
+
+
+ /**
+ * Run a function on a given set of partitions in an RDD and pass the results to the given
+ * handler function. This is the main entry point for all actions in Spark.
+ *
+ * The allowLocal flag is deprecated as of Spark 1.5.0+.
*/
+ @deprecated("use the version of runJob without the allowLocal parameter", "1.5.0")
+ def runJob[T, U: ClassTag](
+ rdd: RDD[T],
+ func: (TaskContext, Iterator[T]) => U,
+ partitions: Seq[Int],
+ allowLocal: Boolean,
+ resultHandler: (Int, U) => Unit): Unit = {
+ if (allowLocal) {
+ logWarning("sc.runJob with allowLocal=true is deprecated in Spark 1.5.0+")
+ }
+ runJob(rdd, func, partitions, resultHandler)
+ }
+
+ /**
+ * Run a function on a given set of partitions in an RDD and return the results as an array.
+ *
+ * The allowLocal flag is deprecated as of Spark 1.5.0+.
+ */
+ @deprecated("use the version of runJob without the allowLocal parameter", "1.5.0")
def runJob[T, U: ClassTag](
rdd: RDD[T],
func: (TaskContext, Iterator[T]) => U,
partitions: Seq[Int],
allowLocal: Boolean
): Array[U] = {
- val results = new Array[U](partitions.size)
- runJob[T, U](rdd, func, partitions, allowLocal, (index, res) => results(index) = res)
- results
+ if (allowLocal) {
+ logWarning("sc.runJob with allowLocal=true is deprecated in Spark 1.5.0+")
+ }
+ runJob(rdd, func, partitions)
}
/**
* Run a job on a given set of partitions of an RDD, but take a function of type
* `Iterator[T] => U` instead of `(TaskContext, Iterator[T]) => U`.
+ *
+ * The allowLocal argument is deprecated as of Spark 1.5.0+.
*/
+ @deprecated("use the version of runJob without the allowLocal parameter", "1.5.0")
def runJob[T, U: ClassTag](
rdd: RDD[T],
func: Iterator[T] => U,
partitions: Seq[Int],
allowLocal: Boolean
): Array[U] = {
- val cleanedFunc = clean(func)
- runJob(rdd, (ctx: TaskContext, it: Iterator[T]) => cleanedFunc(it), partitions, allowLocal)
+ if (allowLocal) {
+ logWarning("sc.runJob with allowLocal=true is deprecated in Spark 1.5.0+")
+ }
+ runJob(rdd, func, partitions)
}
/**
* Run a job on all partitions in an RDD and return the results in an array.
*/
def runJob[T, U: ClassTag](rdd: RDD[T], func: (TaskContext, Iterator[T]) => U): Array[U] = {
- runJob(rdd, func, 0 until rdd.partitions.size, false)
+ runJob(rdd, func, 0 until rdd.partitions.length)
}
/**
* Run a job on all partitions in an RDD and return the results in an array.
*/
def runJob[T, U: ClassTag](rdd: RDD[T], func: Iterator[T] => U): Array[U] = {
- runJob(rdd, func, 0 until rdd.partitions.size, false)
+ runJob(rdd, func, 0 until rdd.partitions.length)
}
/**
@@ -1835,7 +1919,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
processPartition: (TaskContext, Iterator[T]) => U,
resultHandler: (Int, U) => Unit)
{
- runJob[T, U](rdd, processPartition, 0 until rdd.partitions.size, false, resultHandler)
+ runJob[T, U](rdd, processPartition, 0 until rdd.partitions.length, resultHandler)
}
/**
@@ -1847,7 +1931,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
resultHandler: (Int, U) => Unit)
{
val processFunc = (context: TaskContext, iter: Iterator[T]) => processPartition(iter)
- runJob[T, U](rdd, processFunc, 0 until rdd.partitions.size, false, resultHandler)
+ runJob[T, U](rdd, processFunc, 0 until rdd.partitions.length, resultHandler)
}
/**
@@ -1892,7 +1976,6 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli
(context: TaskContext, iter: Iterator[T]) => cleanF(iter),
partitions,
callSite,
- allowLocal = false,
resultHandler,
localProperties.get)
new SimpleFutureAction(waiter, resultFunc)
diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala
index c95615a5a9307..829fae1d1d9bf 100644
--- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala
+++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala
@@ -364,7 +364,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
// This is useful for implementing `take` from other language frontends
// like Python where the data is serialized.
import scala.collection.JavaConversions._
- val res = context.runJob(rdd, (it: Iterator[T]) => it.toArray, partitionIds, true)
+ val res = context.runJob(rdd, (it: Iterator[T]) => it.toArray, partitionIds)
res.map(x => new java.util.ArrayList(x.toSeq)).toArray
}
diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
index dc9f62f39e6d5..55e563ee968be 100644
--- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
+++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
@@ -207,6 +207,7 @@ private[spark] class PythonRDD(
override def run(): Unit = Utils.logUncaughtExceptions {
try {
+ TaskContext.setTaskContext(context)
val stream = new BufferedOutputStream(worker.getOutputStream, bufferSize)
val dataOut = new DataOutputStream(stream)
// Partition index
@@ -263,11 +264,6 @@ private[spark] class PythonRDD(
if (!worker.isClosed) {
Utils.tryLog(worker.shutdownOutput())
}
- } finally {
- // Release memory used by this thread for shuffles
- env.shuffleMemoryManager.releaseMemoryForThisThread()
- // Release memory used by this thread for unrolling blocks
- env.blockManager.memoryStore.releaseUnrollMemoryForThisThread()
}
}
}
@@ -358,12 +354,11 @@ private[spark] object PythonRDD extends Logging {
def runJob(
sc: SparkContext,
rdd: JavaRDD[Array[Byte]],
- partitions: JArrayList[Int],
- allowLocal: Boolean): Int = {
+ partitions: JArrayList[Int]): Int = {
type ByteArray = Array[Byte]
type UnrolledPartition = Array[ByteArray]
val allPartitions: Array[UnrolledPartition] =
- sc.runJob(rdd, (x: Iterator[ByteArray]) => x.toArray, partitions, allowLocal)
+ sc.runJob(rdd, (x: Iterator[ByteArray]) => x.toArray, partitions)
val flattenedPartition: UnrolledPartition = Array.concat(allPartitions: _*)
serveIterator(flattenedPartition.iterator,
s"serve RDD ${rdd.id} with partitions ${partitions.mkString(",")}")
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 9658e9a696ffa..14dac4ed28ce3 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
@@ -20,6 +20,7 @@ package org.apache.spark.api.r
import java.io.{ByteArrayInputStream, ByteArrayOutputStream, DataInputStream, DataOutputStream}
import scala.collection.mutable.HashMap
+import scala.language.existentials
import io.netty.channel.ChannelHandler.Sharable
import io.netty.channel.{ChannelHandlerContext, SimpleChannelInboundHandler}
@@ -68,8 +69,11 @@ private[r] class RBackendHandler(server: RBackend)
case e: Exception =>
logError(s"Removing $objId failed", e)
writeInt(dos, -1)
+ writeString(dos, s"Removing $objId failed: ${e.getMessage}")
}
- case _ => dos.writeInt(-1)
+ case _ =>
+ dos.writeInt(-1)
+ writeString(dos, s"Error: unknown method $methodName")
}
} else {
handleMethodCall(isStatic, objId, methodName, numArgs, dis, dos)
@@ -145,8 +149,11 @@ private[r] class RBackendHandler(server: RBackend)
}
} catch {
case e: Exception =>
- logError(s"$methodName on $objId failed", e)
+ logError(s"$methodName on $objId failed")
writeInt(dos, -1)
+ // Writing the error message of the cause for the exception. This will be returned
+ // to user in the R process.
+ writeString(dos, Utils.exceptionString(e.getCause))
}
}
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 23a470d6afcae..1cf2824f862ee 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
@@ -112,6 +112,7 @@ private abstract class BaseRRDD[T: ClassTag, U: ClassTag](
partition: Int): Unit = {
val env = SparkEnv.get
+ val taskContext = TaskContext.get()
val bufferSize = System.getProperty("spark.buffer.size", "65536").toInt
val stream = new BufferedOutputStream(output, bufferSize)
@@ -119,6 +120,7 @@ private abstract class BaseRRDD[T: ClassTag, U: ClassTag](
override def run(): Unit = {
try {
SparkEnv.set(env)
+ TaskContext.setTaskContext(taskContext)
val dataOut = new DataOutputStream(stream)
dataOut.writeInt(partition)
diff --git a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala
index 79b251e7e62fe..a659abf70395d 100644
--- a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala
@@ -27,7 +27,7 @@ import org.apache.spark.deploy.{ApplicationDescription, ExecutorState}
import org.apache.spark.deploy.DeployMessages._
import org.apache.spark.deploy.master.Master
import org.apache.spark.rpc._
-import org.apache.spark.util.{ThreadUtils, Utils}
+import org.apache.spark.util.{RpcUtils, ThreadUtils, Utils}
/**
* Interface allowing applications to speak with a Spark deploy cluster. Takes a master URL,
@@ -248,7 +248,8 @@ private[spark] class AppClient(
def stop() {
if (endpoint != null) {
try {
- endpoint.askWithRetry[Boolean](StopAppClient)
+ val timeout = RpcUtils.askRpcTimeout(conf)
+ timeout.awaitResult(endpoint.ask[Boolean](StopAppClient))
} catch {
case e: TimeoutException =>
logInfo("Stop request to Master timed out; it may already be shut down.")
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 4615febf17d24..51b3f0dead73e 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
@@ -541,6 +541,7 @@ private[master] class Master(
/**
* Schedule executors to be launched on the workers.
+ * Returns an array containing number of cores assigned to each worker.
*
* There are two modes of launching executors. The first attempts to spread out an application's
* executors on as many workers as possible, while the second does the opposite (i.e. launch them
@@ -551,39 +552,77 @@ private[master] class Master(
* multiple executors from the same application may be launched on the same worker if the worker
* has enough cores and memory. Otherwise, each executor grabs all the cores available on the
* worker by default, in which case only one executor may be launched on each worker.
+ *
+ * It is important to allocate coresPerExecutor on each worker at a time (instead of 1 core
+ * at a time). Consider the following example: cluster has 4 workers with 16 cores each.
+ * User requests 3 executors (spark.cores.max = 48, spark.executor.cores = 16). If 1 core is
+ * allocated at a time, 12 cores from each worker would be assigned to each executor.
+ * Since 12 < 16, no executors would launch [SPARK-8881].
*/
- private def startExecutorsOnWorkers(): Unit = {
- // Right now this is a very simple FIFO scheduler. We keep trying to fit in the first app
- // in the queue, then the second app, etc.
- if (spreadOutApps) {
- // Try to spread out each app among all the workers, until it has all its cores
- for (app <- waitingApps if app.coresLeft > 0) {
- val usableWorkers = workers.toArray.filter(_.state == WorkerState.ALIVE)
- .filter(worker => worker.memoryFree >= app.desc.memoryPerExecutorMB &&
- worker.coresFree >= app.desc.coresPerExecutor.getOrElse(1))
- .sortBy(_.coresFree).reverse
- val numUsable = usableWorkers.length
- val assigned = new Array[Int](numUsable) // Number of cores to give on each node
- var toAssign = math.min(app.coresLeft, usableWorkers.map(_.coresFree).sum)
- var pos = 0
- while (toAssign > 0) {
- if (usableWorkers(pos).coresFree - assigned(pos) > 0) {
- toAssign -= 1
- assigned(pos) += 1
+ private def scheduleExecutorsOnWorkers(
+ app: ApplicationInfo,
+ usableWorkers: Array[WorkerInfo],
+ spreadOutApps: Boolean): Array[Int] = {
+ // If the number of cores per executor is not specified, then we can just schedule
+ // 1 core at a time since we expect a single executor to be launched on each worker
+ val coresPerExecutor = app.desc.coresPerExecutor.getOrElse(1)
+ val memoryPerExecutor = app.desc.memoryPerExecutorMB
+ val numUsable = usableWorkers.length
+ val assignedCores = new Array[Int](numUsable) // Number of cores to give to each worker
+ val assignedMemory = new Array[Int](numUsable) // Amount of memory to give to each worker
+ var coresToAssign = math.min(app.coresLeft, usableWorkers.map(_.coresFree).sum)
+ var freeWorkers = (0 until numUsable).toIndexedSeq
+
+ def canLaunchExecutor(pos: Int): Boolean = {
+ usableWorkers(pos).coresFree - assignedCores(pos) >= coresPerExecutor &&
+ usableWorkers(pos).memoryFree - assignedMemory(pos) >= memoryPerExecutor
+ }
+
+ while (coresToAssign >= coresPerExecutor && freeWorkers.nonEmpty) {
+ freeWorkers = freeWorkers.filter(canLaunchExecutor)
+ freeWorkers.foreach { pos =>
+ var keepScheduling = true
+ while (keepScheduling && canLaunchExecutor(pos) && coresToAssign >= coresPerExecutor) {
+ coresToAssign -= coresPerExecutor
+ assignedCores(pos) += coresPerExecutor
+ // If cores per executor is not set, we are assigning 1 core at a time
+ // without actually meaning to launch 1 executor for each core assigned
+ if (app.desc.coresPerExecutor.isDefined) {
+ assignedMemory(pos) += memoryPerExecutor
+ }
+
+ // Spreading out an application means spreading out its executors across as
+ // many workers as possible. If we are not spreading out, then we should keep
+ // scheduling executors on this worker until we use all of its resources.
+ // Otherwise, just move on to the next worker.
+ if (spreadOutApps) {
+ keepScheduling = false
}
- pos = (pos + 1) % numUsable
- }
- // Now that we've decided how many cores to give on each node, let's actually give them
- for (pos <- 0 until numUsable if assigned(pos) > 0) {
- allocateWorkerResourceToExecutors(app, assigned(pos), usableWorkers(pos))
}
}
- } else {
- // Pack each app into as few workers as possible until we've assigned all its cores
- for (worker <- workers if worker.coresFree > 0 && worker.state == WorkerState.ALIVE) {
- for (app <- waitingApps if app.coresLeft > 0) {
- allocateWorkerResourceToExecutors(app, app.coresLeft, worker)
- }
+ }
+ assignedCores
+ }
+
+ /**
+ * Schedule and launch executors on workers
+ */
+ private def startExecutorsOnWorkers(): Unit = {
+ // Right now this is a very simple FIFO scheduler. We keep trying to fit in the first app
+ // in the queue, then the second app, etc.
+ for (app <- waitingApps if app.coresLeft > 0) {
+ val coresPerExecutor: Option[Int] = app.desc.coresPerExecutor
+ // Filter out workers that don't have enough resources to launch an executor
+ val usableWorkers = workers.toArray.filter(_.state == WorkerState.ALIVE)
+ .filter(worker => worker.memoryFree >= app.desc.memoryPerExecutorMB &&
+ worker.coresFree >= coresPerExecutor.getOrElse(1))
+ .sortBy(_.coresFree).reverse
+ val assignedCores = scheduleExecutorsOnWorkers(app, usableWorkers, spreadOutApps)
+
+ // Now that we've decided how many cores to allocate on each worker, let's allocate them
+ for (pos <- 0 until usableWorkers.length if assignedCores(pos) > 0) {
+ allocateWorkerResourceToExecutors(
+ app, assignedCores(pos), coresPerExecutor, usableWorkers(pos))
}
}
}
@@ -591,19 +630,22 @@ private[master] class Master(
/**
* Allocate a worker's resources to one or more executors.
* @param app the info of the application which the executors belong to
- * @param coresToAllocate cores on this worker to be allocated to this application
+ * @param assignedCores number of cores on this worker for this application
+ * @param coresPerExecutor number of cores per executor
* @param worker the worker info
*/
private def allocateWorkerResourceToExecutors(
app: ApplicationInfo,
- coresToAllocate: Int,
+ assignedCores: Int,
+ coresPerExecutor: Option[Int],
worker: WorkerInfo): Unit = {
- val memoryPerExecutor = app.desc.memoryPerExecutorMB
- val coresPerExecutor = app.desc.coresPerExecutor.getOrElse(coresToAllocate)
- var coresLeft = coresToAllocate
- while (coresLeft >= coresPerExecutor && worker.memoryFree >= memoryPerExecutor) {
- val exec = app.addExecutor(worker, coresPerExecutor)
- coresLeft -= coresPerExecutor
+ // If the number of cores per executor is specified, we divide the cores assigned
+ // to this worker evenly among the executors with no remainder.
+ // Otherwise, we launch a single executor that grabs all the assignedCores on this worker.
+ val numExecutors = coresPerExecutor.map { assignedCores / _ }.getOrElse(1)
+ val coresToAssign = coresPerExecutor.getOrElse(assignedCores)
+ for (i <- 1 to numExecutors) {
+ val exec = app.addExecutor(worker, coresToAssign)
launchExecutor(worker, exec)
app.state = ApplicationState.RUNNING
}
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
index 82e9578bbcba5..0276c24f85368 100755
--- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
@@ -25,7 +25,7 @@ import java.util.concurrent._
import java.util.concurrent.{Future => JFuture, ScheduledFuture => JScheduledFuture}
import scala.collection.JavaConversions._
-import scala.collection.mutable.{HashMap, HashSet}
+import scala.collection.mutable.{HashMap, HashSet, LinkedHashMap}
import scala.concurrent.ExecutionContext
import scala.util.Random
import scala.util.control.NonFatal
@@ -115,13 +115,18 @@ private[worker] class Worker(
}
var workDir: File = null
- val finishedExecutors = new HashMap[String, ExecutorRunner]
+ val finishedExecutors = new LinkedHashMap[String, ExecutorRunner]
val drivers = new HashMap[String, DriverRunner]
val executors = new HashMap[String, ExecutorRunner]
- val finishedDrivers = new HashMap[String, DriverRunner]
+ val finishedDrivers = new LinkedHashMap[String, DriverRunner]
val appDirectories = new HashMap[String, Seq[String]]
val finishedApps = new HashSet[String]
+ val retainedExecutors = conf.getInt("spark.worker.ui.retainedExecutors",
+ WorkerWebUI.DEFAULT_RETAINED_EXECUTORS)
+ val retainedDrivers = conf.getInt("spark.worker.ui.retainedDrivers",
+ WorkerWebUI.DEFAULT_RETAINED_DRIVERS)
+
// The shuffle service is not actually started unless configured.
private val shuffleService = new ExternalShuffleService(conf, securityMgr)
@@ -461,25 +466,7 @@ private[worker] class Worker(
}
case executorStateChanged @ ExecutorStateChanged(appId, execId, state, message, exitStatus) =>
- sendToMaster(executorStateChanged)
- val fullId = appId + "/" + execId
- if (ExecutorState.isFinished(state)) {
- executors.get(fullId) match {
- case Some(executor) =>
- logInfo("Executor " + fullId + " finished with state " + state +
- message.map(" message " + _).getOrElse("") +
- exitStatus.map(" exitStatus " + _).getOrElse(""))
- executors -= fullId
- finishedExecutors(fullId) = executor
- coresUsed -= executor.cores
- memoryUsed -= executor.memory
- case None =>
- logInfo("Unknown Executor " + fullId + " finished with state " + state +
- message.map(" message " + _).getOrElse("") +
- exitStatus.map(" exitStatus " + _).getOrElse(""))
- }
- maybeCleanupApplication(appId)
- }
+ handleExecutorStateChanged(executorStateChanged)
case KillExecutor(masterUrl, appId, execId) =>
if (masterUrl != activeMasterUrl) {
@@ -523,24 +510,8 @@ private[worker] class Worker(
}
}
- case driverStageChanged @ DriverStateChanged(driverId, state, exception) => {
- state match {
- case DriverState.ERROR =>
- logWarning(s"Driver $driverId failed with unrecoverable exception: ${exception.get}")
- case DriverState.FAILED =>
- logWarning(s"Driver $driverId exited with failure")
- case DriverState.FINISHED =>
- logInfo(s"Driver $driverId exited successfully")
- case DriverState.KILLED =>
- logInfo(s"Driver $driverId was killed by user")
- case _ =>
- logDebug(s"Driver $driverId changed state to $state")
- }
- sendToMaster(driverStageChanged)
- val driver = drivers.remove(driverId).get
- finishedDrivers(driverId) = driver
- memoryUsed -= driver.driverDesc.mem
- coresUsed -= driver.driverDesc.cores
+ case driverStateChanged @ DriverStateChanged(driverId, state, exception) => {
+ handleDriverStateChanged(driverStateChanged)
}
case ReregisterWithMaster =>
@@ -614,6 +585,78 @@ private[worker] class Worker(
webUi.stop()
metricsSystem.stop()
}
+
+ private def trimFinishedExecutorsIfNecessary(): Unit = {
+ // do not need to protect with locks since both WorkerPage and Restful server get data through
+ // thread-safe RpcEndPoint
+ if (finishedExecutors.size > retainedExecutors) {
+ finishedExecutors.take(math.max(finishedExecutors.size / 10, 1)).foreach {
+ case (executorId, _) => finishedExecutors.remove(executorId)
+ }
+ }
+ }
+
+ private def trimFinishedDriversIfNecessary(): Unit = {
+ // do not need to protect with locks since both WorkerPage and Restful server get data through
+ // thread-safe RpcEndPoint
+ if (finishedDrivers.size > retainedDrivers) {
+ finishedDrivers.take(math.max(finishedDrivers.size / 10, 1)).foreach {
+ case (driverId, _) => finishedDrivers.remove(driverId)
+ }
+ }
+ }
+
+ private[worker] def handleDriverStateChanged(driverStateChanged: DriverStateChanged): Unit = {
+ val driverId = driverStateChanged.driverId
+ val exception = driverStateChanged.exception
+ val state = driverStateChanged.state
+ state match {
+ case DriverState.ERROR =>
+ logWarning(s"Driver $driverId failed with unrecoverable exception: ${exception.get}")
+ case DriverState.FAILED =>
+ logWarning(s"Driver $driverId exited with failure")
+ case DriverState.FINISHED =>
+ logInfo(s"Driver $driverId exited successfully")
+ case DriverState.KILLED =>
+ logInfo(s"Driver $driverId was killed by user")
+ case _ =>
+ logDebug(s"Driver $driverId changed state to $state")
+ }
+ sendToMaster(driverStateChanged)
+ val driver = drivers.remove(driverId).get
+ finishedDrivers(driverId) = driver
+ trimFinishedDriversIfNecessary()
+ memoryUsed -= driver.driverDesc.mem
+ coresUsed -= driver.driverDesc.cores
+ }
+
+ private[worker] def handleExecutorStateChanged(executorStateChanged: ExecutorStateChanged):
+ Unit = {
+ sendToMaster(executorStateChanged)
+ val state = executorStateChanged.state
+ if (ExecutorState.isFinished(state)) {
+ val appId = executorStateChanged.appId
+ val fullId = appId + "/" + executorStateChanged.execId
+ val message = executorStateChanged.message
+ val exitStatus = executorStateChanged.exitStatus
+ executors.get(fullId) match {
+ case Some(executor) =>
+ logInfo("Executor " + fullId + " finished with state " + state +
+ message.map(" message " + _).getOrElse("") +
+ exitStatus.map(" exitStatus " + _).getOrElse(""))
+ executors -= fullId
+ finishedExecutors(fullId) = executor
+ trimFinishedExecutorsIfNecessary()
+ coresUsed -= executor.cores
+ memoryUsed -= executor.memory
+ case None =>
+ logInfo("Unknown Executor " + fullId + " finished with state " + state +
+ message.map(" message " + _).getOrElse("") +
+ exitStatus.map(" exitStatus " + _).getOrElse(""))
+ }
+ maybeCleanupApplication(appId)
+ }
+ }
}
private[deploy] object Worker extends Logging {
@@ -669,5 +712,4 @@ private[deploy] object Worker extends Logging {
cmd
}
}
-
}
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala
index 334a5b10142aa..709a27233598c 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala
@@ -53,6 +53,8 @@ class WorkerWebUI(
}
}
-private[ui] object WorkerWebUI {
+private[worker] object WorkerWebUI {
val STATIC_RESOURCE_BASE = SparkUI.STATIC_RESOURCE_DIR
+ val DEFAULT_RETAINED_DRIVERS = 1000
+ val DEFAULT_RETAINED_EXECUTORS = 1000
}
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 66624ffbe4790..7bc7fce7ae8dd 100644
--- a/core/src/main/scala/org/apache/spark/executor/Executor.scala
+++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala
@@ -209,18 +209,19 @@ private[spark] class Executor(
// Run the actual task and measure its runtime.
taskStart = System.currentTimeMillis()
+ var threwException = true
val (value, accumUpdates) = try {
- task.run(
+ val res = task.run(
taskAttemptId = taskId,
attemptNumber = attemptNumber,
metricsSystem = env.metricsSystem)
+ threwException = false
+ res
} finally {
- // Note: this memory freeing logic is duplicated in DAGScheduler.runLocallyWithinThread;
- // when changing this, make sure to update both copies.
val freedMemory = taskMemoryManager.cleanUpAllAllocatedMemory()
if (freedMemory > 0) {
val errMsg = s"Managed memory leak detected; size = $freedMemory bytes, TID = $taskId"
- if (conf.getBoolean("spark.unsafe.exceptionOnMemoryLeak", false)) {
+ if (conf.getBoolean("spark.unsafe.exceptionOnMemoryLeak", false) && !threwException) {
throw new SparkException(errMsg)
} else {
logError(errMsg)
@@ -312,10 +313,6 @@ private[spark] class Executor(
}
} finally {
- // Release memory used by this thread for shuffles
- env.shuffleMemoryManager.releaseMemoryForThisThread()
- // Release memory used by this thread for unrolling blocks
- env.blockManager.memoryStore.releaseUnrollMemoryForThisThread()
runningTasks.remove(taskId)
}
}
diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala
index 658e8c8b89318..130b58882d8ee 100644
--- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala
@@ -94,13 +94,14 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part:
}
override def getDependencies: Seq[Dependency[_]] = {
- rdds.map { rdd: RDD[_ <: Product2[K, _]] =>
+ rdds.map { rdd: RDD[_] =>
if (rdd.partitioner == Some(part)) {
logDebug("Adding one-to-one dependency with " + rdd)
new OneToOneDependency(rdd)
} else {
logDebug("Adding shuffle dependency with " + rdd)
- new ShuffleDependency[K, Any, CoGroupCombiner](rdd, part, serializer)
+ new ShuffleDependency[K, Any, CoGroupCombiner](
+ rdd.asInstanceOf[RDD[_ <: Product2[K, _]]], part, serializer)
}
}
}
@@ -133,7 +134,7 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part:
// A list of (rdd iterator, dependency number) pairs
val rddIterators = new ArrayBuffer[(Iterator[Product2[K, Any]], Int)]
for ((dep, depNum) <- dependencies.zipWithIndex) dep match {
- case oneToOneDependency: OneToOneDependency[Product2[K, Any]] =>
+ case oneToOneDependency: OneToOneDependency[Product2[K, Any]] @unchecked =>
val dependencyPartition = split.narrowDeps(depNum).get.split
// Read them from the parent
val it = oneToOneDependency.rdd.iterator(dependencyPartition, context)
diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala
index f827270ee6a44..f83a051f5da11 100644
--- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala
@@ -128,7 +128,7 @@ class NewHadoopRDD[K, V](
configurable.setConf(conf)
case _ =>
}
- val reader = format.createRecordReader(
+ private var reader = format.createRecordReader(
split.serializableHadoopSplit.value, hadoopAttemptContext)
reader.initialize(split.serializableHadoopSplit.value, hadoopAttemptContext)
@@ -141,6 +141,12 @@ class NewHadoopRDD[K, V](
override def hasNext: Boolean = {
if (!finished && !havePair) {
finished = !reader.nextKeyValue
+ if (finished) {
+ // Close and release the reader here; close() will also be called when the task
+ // completes, but for tasks that read from many files, it helps to release the
+ // resources early.
+ close()
+ }
havePair = !finished
}
!finished
@@ -159,18 +165,23 @@ class NewHadoopRDD[K, V](
private def close() {
try {
- reader.close()
- if (bytesReadCallback.isDefined) {
- inputMetrics.updateBytesRead()
- } else if (split.serializableHadoopSplit.value.isInstanceOf[FileSplit] ||
- split.serializableHadoopSplit.value.isInstanceOf[CombineFileSplit]) {
- // If we can't get the bytes read from the FS stats, fall back to the split size,
- // which may be inaccurate.
- try {
- inputMetrics.incBytesRead(split.serializableHadoopSplit.value.getLength)
- } catch {
- case e: java.io.IOException =>
- logWarning("Unable to get input size to set InputMetrics for task", e)
+ if (reader != null) {
+ // Close reader and release it
+ reader.close()
+ reader = null
+
+ if (bytesReadCallback.isDefined) {
+ inputMetrics.updateBytesRead()
+ } else if (split.serializableHadoopSplit.value.isInstanceOf[FileSplit] ||
+ split.serializableHadoopSplit.value.isInstanceOf[CombineFileSplit]) {
+ // If we can't get the bytes read from the FS stats, fall back to the split size,
+ // which may be inaccurate.
+ try {
+ inputMetrics.incBytesRead(split.serializableHadoopSplit.value.getLength)
+ } catch {
+ case e: java.io.IOException =>
+ logWarning("Unable to get input size to set InputMetrics for task", e)
+ }
}
}
} catch {
diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala
index 91a6a2d039852..326fafb230a40 100644
--- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala
@@ -881,7 +881,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)])
}
buf
} : Seq[V]
- val res = self.context.runJob(self, process, Array(index), false)
+ val res = self.context.runJob(self, process, Array(index))
res(0)
case None =>
self.filter(_._1 == key).map(_._2).collect()
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 defdabf95ac4b..3bb9998e1db44 100644
--- a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala
@@ -133,6 +133,7 @@ private[spark] class PipedRDD[T: ClassTag](
// Start a thread to feed the process input from our parent's iterator
new Thread("stdin writer for " + command) {
override def run() {
+ TaskContext.setTaskContext(context)
val out = new PrintWriter(proc.getOutputStream)
// scalastyle:off println
diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala
index 9f7ebae3e9af3..6d61d227382d7 100644
--- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala
@@ -897,7 +897,7 @@ abstract class RDD[T: ClassTag](
*/
def toLocalIterator: Iterator[T] = withScope {
def collectPartition(p: Int): Array[T] = {
- sc.runJob(this, (iter: Iterator[T]) => iter.toArray, Seq(p), allowLocal = false).head
+ sc.runJob(this, (iter: Iterator[T]) => iter.toArray, Seq(p)).head
}
(0 until partitions.length).iterator.flatMap(i => collectPartition(i))
}
@@ -1082,7 +1082,9 @@ abstract class RDD[T: ClassTag](
val scale = math.max(math.ceil(math.pow(numPartitions, 1.0 / depth)).toInt, 2)
// If creating an extra level doesn't help reduce
// the wall-clock time, we stop tree aggregation.
- while (numPartitions > scale + numPartitions / scale) {
+
+ // Don't trigger TreeAggregation when it doesn't save wall-clock time
+ while (numPartitions > scale + math.ceil(numPartitions.toDouble / scale)) {
numPartitions /= scale
val curNumPartitions = numPartitions
partiallyAggregated = partiallyAggregated.mapPartitionsWithIndex {
@@ -1273,7 +1275,7 @@ abstract class RDD[T: ClassTag](
val left = num - buf.size
val p = partsScanned until math.min(partsScanned + numPartsToTry, totalParts)
- val res = sc.runJob(this, (it: Iterator[T]) => it.take(left).toArray, p, allowLocal = true)
+ val res = sc.runJob(this, (it: Iterator[T]) => it.take(left).toArray, p)
res.foreach(buf ++= _.take(num - buf.size))
partsScanned += numPartsToTry
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SqlNewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/SqlNewHadoopRDD.scala
similarity index 81%
rename from sql/core/src/main/scala/org/apache/spark/sql/execution/SqlNewHadoopRDD.scala
rename to core/src/main/scala/org/apache/spark/rdd/SqlNewHadoopRDD.scala
index e1c1a6c06268f..35e44cb59c1be 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SqlNewHadoopRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/SqlNewHadoopRDD.scala
@@ -15,12 +15,13 @@
* limitations under the License.
*/
-package org.apache.spark.sql.execution
+package org.apache.spark.rdd
import java.text.SimpleDateFormat
import java.util.Date
-import org.apache.spark.{Partition => SparkPartition, _}
+import scala.reflect.ClassTag
+
import org.apache.hadoop.conf.{Configurable, Configuration}
import org.apache.hadoop.io.Writable
import org.apache.hadoop.mapreduce._
@@ -30,12 +31,12 @@ import org.apache.spark.broadcast.Broadcast
import org.apache.spark.deploy.SparkHadoopUtil
import org.apache.spark.executor.DataReadMethod
import org.apache.spark.mapreduce.SparkHadoopMapReduceUtil
+import org.apache.spark.unsafe.types.UTF8String
+import org.apache.spark.{Partition => SparkPartition, _}
import org.apache.spark.rdd.NewHadoopRDD.NewHadoopMapPartitionsWithSplitRDD
-import org.apache.spark.rdd.{HadoopRDD, RDD}
import org.apache.spark.storage.StorageLevel
import org.apache.spark.util.{SerializableConfiguration, Utils}
-import scala.reflect.ClassTag
private[spark] class SqlNewHadoopPartition(
rddId: Int,
@@ -62,7 +63,7 @@ private[spark] class SqlNewHadoopPartition(
* changes based on [[org.apache.spark.rdd.HadoopRDD]]. In future, this functionality will be
* folded into core.
*/
-private[sql] class SqlNewHadoopRDD[K, V](
+private[spark] class SqlNewHadoopRDD[K, V](
@transient sc : SparkContext,
broadcastedConf: Broadcast[SerializableConfiguration],
@transient initDriverSideJobFuncOpt: Option[Job => Unit],
@@ -128,6 +129,12 @@ private[sql] class SqlNewHadoopRDD[K, V](
val inputMetrics = context.taskMetrics
.getInputMetricsForReadMethod(DataReadMethod.Hadoop)
+ // Sets the thread local variable for the file's name
+ split.serializableHadoopSplit.value match {
+ case fs: FileSplit => SqlNewHadoopRDD.setInputFileName(fs.getPath.toString)
+ case _ => SqlNewHadoopRDD.unsetInputFileName()
+ }
+
// Find a function that will return the FileSystem bytes read by this thread. Do this before
// creating RecordReader, because RecordReader's constructor might read some bytes
val bytesReadCallback = inputMetrics.bytesReadCallback.orElse {
@@ -147,7 +154,7 @@ private[sql] class SqlNewHadoopRDD[K, V](
configurable.setConf(conf)
case _ =>
}
- val reader = format.createRecordReader(
+ private var reader = format.createRecordReader(
split.serializableHadoopSplit.value, hadoopAttemptContext)
reader.initialize(split.serializableHadoopSplit.value, hadoopAttemptContext)
@@ -160,6 +167,12 @@ private[sql] class SqlNewHadoopRDD[K, V](
override def hasNext: Boolean = {
if (!finished && !havePair) {
finished = !reader.nextKeyValue
+ if (finished) {
+ // Close and release the reader here; close() will also be called when the task
+ // completes, but for tasks that read from many files, it helps to release the
+ // resources early.
+ close()
+ }
havePair = !finished
}
!finished
@@ -178,18 +191,24 @@ private[sql] class SqlNewHadoopRDD[K, V](
private def close() {
try {
- reader.close()
- if (bytesReadCallback.isDefined) {
- inputMetrics.updateBytesRead()
- } else if (split.serializableHadoopSplit.value.isInstanceOf[FileSplit] ||
- split.serializableHadoopSplit.value.isInstanceOf[CombineFileSplit]) {
- // If we can't get the bytes read from the FS stats, fall back to the split size,
- // which may be inaccurate.
- try {
- inputMetrics.incBytesRead(split.serializableHadoopSplit.value.getLength)
- } catch {
- case e: java.io.IOException =>
- logWarning("Unable to get input size to set InputMetrics for task", e)
+ if (reader != null) {
+ reader.close()
+ reader = null
+
+ SqlNewHadoopRDD.unsetInputFileName()
+
+ if (bytesReadCallback.isDefined) {
+ inputMetrics.updateBytesRead()
+ } else if (split.serializableHadoopSplit.value.isInstanceOf[FileSplit] ||
+ split.serializableHadoopSplit.value.isInstanceOf[CombineFileSplit]) {
+ // If we can't get the bytes read from the FS stats, fall back to the split size,
+ // which may be inaccurate.
+ try {
+ inputMetrics.incBytesRead(split.serializableHadoopSplit.value.getLength)
+ } catch {
+ case e: java.io.IOException =>
+ logWarning("Unable to get input size to set InputMetrics for task", e)
+ }
}
}
} catch {
@@ -240,6 +259,21 @@ private[sql] class SqlNewHadoopRDD[K, V](
}
private[spark] object SqlNewHadoopRDD {
+
+ /**
+ * The thread variable for the name of the current file being read. This is used by
+ * the InputFileName function in Spark SQL.
+ */
+ private[this] val inputFileName: ThreadLocal[UTF8String] = new ThreadLocal[UTF8String] {
+ override protected def initialValue(): UTF8String = UTF8String.fromString("")
+ }
+
+ def getInputFileName(): UTF8String = inputFileName.get()
+
+ private[spark] def setInputFileName(file: String) = inputFileName.set(UTF8String.fromString(file))
+
+ private[spark] def unsetInputFileName(): Unit = inputFileName.remove()
+
/**
* Analogous to [[org.apache.spark.rdd.MapPartitionsRDD]], but passes in an InputSplit to
* the given function rather than the index of the partition.
diff --git a/core/src/main/scala/org/apache/spark/rdd/ZippedWithIndexRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ZippedWithIndexRDD.scala
index 523aaf2b860b5..e277ae28d588f 100644
--- a/core/src/main/scala/org/apache/spark/rdd/ZippedWithIndexRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/ZippedWithIndexRDD.scala
@@ -50,8 +50,7 @@ class ZippedWithIndexRDD[T: ClassTag](@transient prev: RDD[T]) extends RDD[(T, L
prev.context.runJob(
prev,
Utils.getIteratorSize _,
- 0 until n - 1, // do not need to count the last partition
- allowLocal = false
+ 0 until n - 1 // do not need to count the last partition
).scanLeft(0L)(_ + _)
}
}
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 b829d06923404..c4fa277c21254 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
@@ -38,7 +38,6 @@ import org.apache.spark.partial.{ApproximateActionListener, ApproximateEvaluator
import org.apache.spark.rdd.RDD
import org.apache.spark.rpc.RpcTimeout
import org.apache.spark.storage._
-import org.apache.spark.unsafe.memory.TaskMemoryManager
import org.apache.spark.util._
import org.apache.spark.storage.BlockManagerMessages.BlockManagerHeartbeat
@@ -128,10 +127,6 @@ class DAGScheduler(
// This is only safe because DAGScheduler runs in a single thread.
private val closureSerializer = SparkEnv.get.closureSerializer.newInstance()
-
- /** If enabled, we may run certain actions like take() and first() locally. */
- private val localExecutionEnabled = sc.getConf.getBoolean("spark.localExecution.enabled", false)
-
/** If enabled, FetchFailed will not cause stage retry, in order to surface the problem. */
private val disallowStageRetryForTest = sc.getConf.getBoolean("spark.test.noStageRetry", false)
@@ -515,7 +510,6 @@ class DAGScheduler(
func: (TaskContext, Iterator[T]) => U,
partitions: Seq[Int],
callSite: CallSite,
- allowLocal: Boolean,
resultHandler: (Int, U) => Unit,
properties: Properties): JobWaiter[U] = {
// Check to make sure we are not launching a task on a partition that does not exist.
@@ -535,7 +529,7 @@ class DAGScheduler(
val func2 = func.asInstanceOf[(TaskContext, Iterator[_]) => _]
val waiter = new JobWaiter(this, jobId, partitions.size, resultHandler)
eventProcessLoop.post(JobSubmitted(
- jobId, rdd, func2, partitions.toArray, allowLocal, callSite, waiter,
+ jobId, rdd, func2, partitions.toArray, callSite, waiter,
SerializationUtils.clone(properties)))
waiter
}
@@ -545,11 +539,10 @@ class DAGScheduler(
func: (TaskContext, Iterator[T]) => U,
partitions: Seq[Int],
callSite: CallSite,
- allowLocal: Boolean,
resultHandler: (Int, U) => Unit,
properties: Properties): Unit = {
val start = System.nanoTime
- val waiter = submitJob(rdd, func, partitions, callSite, allowLocal, resultHandler, properties)
+ val waiter = submitJob(rdd, func, partitions, callSite, resultHandler, properties)
waiter.awaitResult() match {
case JobSucceeded =>
logInfo("Job %d finished: %s, took %f s".format
@@ -576,8 +569,7 @@ class DAGScheduler(
val partitions = (0 until rdd.partitions.size).toArray
val jobId = nextJobId.getAndIncrement()
eventProcessLoop.post(JobSubmitted(
- jobId, rdd, func2, partitions, allowLocal = false, callSite, listener,
- SerializationUtils.clone(properties)))
+ jobId, rdd, func2, partitions, callSite, listener, SerializationUtils.clone(properties)))
listener.awaitResult() // Will throw an exception if the job fails
}
@@ -654,74 +646,6 @@ class DAGScheduler(
}
}
- /**
- * Run a job on an RDD locally, assuming it has only a single partition and no dependencies.
- * We run the operation in a separate thread just in case it takes a bunch of time, so that we
- * don't block the DAGScheduler event loop or other concurrent jobs.
- */
- protected def runLocally(job: ActiveJob) {
- logInfo("Computing the requested partition locally")
- new Thread("Local computation of job " + job.jobId) {
- override def run() {
- runLocallyWithinThread(job)
- }
- }.start()
- }
-
- // Broken out for easier testing in DAGSchedulerSuite.
- protected def runLocallyWithinThread(job: ActiveJob) {
- var jobResult: JobResult = JobSucceeded
- try {
- val rdd = job.finalStage.rdd
- val split = rdd.partitions(job.partitions(0))
- val taskMemoryManager = new TaskMemoryManager(env.executorMemoryManager)
- val taskContext =
- new TaskContextImpl(
- job.finalStage.id,
- job.partitions(0),
- taskAttemptId = 0,
- attemptNumber = 0,
- taskMemoryManager = taskMemoryManager,
- metricsSystem = env.metricsSystem,
- runningLocally = true)
- TaskContext.setTaskContext(taskContext)
- try {
- val result = job.func(taskContext, rdd.iterator(split, taskContext))
- job.listener.taskSucceeded(0, result)
- } finally {
- taskContext.markTaskCompleted()
- TaskContext.unset()
- // Note: this memory freeing logic is duplicated in Executor.run(); when changing this,
- // make sure to update both copies.
- val freedMemory = taskMemoryManager.cleanUpAllAllocatedMemory()
- if (freedMemory > 0) {
- if (sc.getConf.getBoolean("spark.unsafe.exceptionOnMemoryLeak", false)) {
- throw new SparkException(s"Managed memory leak detected; size = $freedMemory bytes")
- } else {
- logError(s"Managed memory leak detected; size = $freedMemory bytes")
- }
- }
- }
- } catch {
- case e: Exception =>
- val exception = new SparkDriverExecutionException(e)
- jobResult = JobFailed(exception)
- job.listener.jobFailed(exception)
- case oom: OutOfMemoryError =>
- val exception = new SparkException("Local job aborted due to out of memory error", oom)
- jobResult = JobFailed(exception)
- job.listener.jobFailed(exception)
- } finally {
- val s = job.finalStage
- // clean up data structures that were populated for a local job,
- // but that won't get cleaned up via the normal paths through
- // completion events or stage abort
- stageIdToStage -= s.id
- jobIdToStageIds -= job.jobId
- listenerBus.post(SparkListenerJobEnd(job.jobId, clock.getTimeMillis(), jobResult))
- }
- }
-
/** Finds the earliest-created active job that needs the stage */
// TODO: Probably should actually find among the active jobs that need this
// stage the one with the highest priority (highest-priority pool, earliest created).
@@ -784,7 +708,6 @@ class DAGScheduler(
finalRDD: RDD[_],
func: (TaskContext, Iterator[_]) => _,
partitions: Array[Int],
- allowLocal: Boolean,
callSite: CallSite,
listener: JobListener,
properties: Properties) {
@@ -802,29 +725,20 @@ class DAGScheduler(
if (finalStage != null) {
val job = new ActiveJob(jobId, finalStage, func, partitions, callSite, listener, properties)
clearCacheLocs()
- logInfo("Got job %s (%s) with %d output partitions (allowLocal=%s)".format(
- job.jobId, callSite.shortForm, partitions.length, allowLocal))
+ logInfo("Got job %s (%s) with %d output partitions".format(
+ job.jobId, callSite.shortForm, partitions.length))
logInfo("Final stage: " + finalStage + "(" + finalStage.name + ")")
logInfo("Parents of final stage: " + finalStage.parents)
logInfo("Missing parents: " + getMissingParentStages(finalStage))
- val shouldRunLocally =
- localExecutionEnabled && allowLocal && finalStage.parents.isEmpty && partitions.length == 1
val jobSubmissionTime = clock.getTimeMillis()
- if (shouldRunLocally) {
- // Compute very short actions like first() or take() with no parent stages locally.
- listenerBus.post(
- SparkListenerJobStart(job.jobId, jobSubmissionTime, Seq.empty, properties))
- runLocally(job)
- } else {
- jobIdToActiveJob(jobId) = job
- activeJobs += job
- finalStage.resultOfJob = Some(job)
- val stageIds = jobIdToStageIds(jobId).toArray
- val stageInfos = stageIds.flatMap(id => stageIdToStage.get(id).map(_.latestInfo))
- listenerBus.post(
- SparkListenerJobStart(job.jobId, jobSubmissionTime, stageInfos, properties))
- submitStage(finalStage)
- }
+ jobIdToActiveJob(jobId) = job
+ activeJobs += job
+ finalStage.resultOfJob = Some(job)
+ val stageIds = jobIdToStageIds(jobId).toArray
+ val stageInfos = stageIds.flatMap(id => stageIdToStage.get(id).map(_.latestInfo))
+ listenerBus.post(
+ SparkListenerJobStart(job.jobId, jobSubmissionTime, stageInfos, properties))
+ submitStage(finalStage)
}
submitWaitingStages()
}
@@ -876,8 +790,28 @@ class DAGScheduler(
// serializable. If tasks are not serializable, a SparkListenerStageCompleted event
// will be posted, which should always come after a corresponding SparkListenerStageSubmitted
// event.
- stage.makeNewStageAttempt(partitionsToCompute.size)
outputCommitCoordinator.stageStart(stage.id)
+ val taskIdToLocations = try {
+ stage match {
+ case s: ShuffleMapStage =>
+ partitionsToCompute.map { id => (id, getPreferredLocs(stage.rdd, id))}.toMap
+ case s: ResultStage =>
+ val job = s.resultOfJob.get
+ partitionsToCompute.map { id =>
+ val p = job.partitions(id)
+ (id, getPreferredLocs(stage.rdd, p))
+ }.toMap
+ }
+ } catch {
+ case NonFatal(e) =>
+ stage.makeNewStageAttempt(partitionsToCompute.size)
+ listenerBus.post(SparkListenerStageSubmitted(stage.latestInfo, properties))
+ abortStage(stage, s"Task creation failed: $e\n${e.getStackTraceString}")
+ runningStages -= stage
+ return
+ }
+
+ stage.makeNewStageAttempt(partitionsToCompute.size, taskIdToLocations.values.toSeq)
listenerBus.post(SparkListenerStageSubmitted(stage.latestInfo, properties))
// TODO: Maybe we can keep the taskBinary in Stage to avoid serializing it multiple times.
@@ -916,7 +850,7 @@ class DAGScheduler(
stage match {
case stage: ShuffleMapStage =>
partitionsToCompute.map { id =>
- val locs = getPreferredLocs(stage.rdd, id)
+ val locs = taskIdToLocations(id)
val part = stage.rdd.partitions(id)
new ShuffleMapTask(stage.id, stage.latestInfo.attemptId, taskBinary, part, locs)
}
@@ -926,7 +860,7 @@ class DAGScheduler(
partitionsToCompute.map { id =>
val p: Int = job.partitions(id)
val part = stage.rdd.partitions(p)
- val locs = getPreferredLocs(stage.rdd, p)
+ val locs = taskIdToLocations(id)
new ResultTask(stage.id, stage.latestInfo.attemptId, taskBinary, part, locs, id)
}
}
@@ -982,11 +916,9 @@ class DAGScheduler(
// To avoid UI cruft, ignore cases where value wasn't updated
if (acc.name.isDefined && partialValue != acc.zero) {
val name = acc.name.get
- val stringPartialValue = Accumulators.stringifyPartialValue(partialValue)
- val stringValue = Accumulators.stringifyValue(acc.value)
- stage.latestInfo.accumulables(id) = AccumulableInfo(id, name, stringValue)
+ stage.latestInfo.accumulables(id) = AccumulableInfo(id, name, s"${acc.value}")
event.taskInfo.accumulables +=
- AccumulableInfo(id, name, Some(stringPartialValue), stringValue)
+ AccumulableInfo(id, name, Some(s"$partialValue"), s"${acc.value}")
}
}
} catch {
@@ -1013,7 +945,7 @@ class DAGScheduler(
// The success case is dealt with separately below, since we need to compute accumulator
// updates before posting.
if (event.reason != Success) {
- val attemptId = stageIdToStage.get(task.stageId).map(_.latestInfo.attemptId).getOrElse(-1)
+ val attemptId = task.stageAttemptId
listenerBus.post(SparkListenerTaskEnd(stageId, attemptId, taskType, event.reason,
event.taskInfo, event.taskMetrics))
}
@@ -1486,9 +1418,8 @@ private[scheduler] class DAGSchedulerEventProcessLoop(dagScheduler: DAGScheduler
}
private def doOnReceive(event: DAGSchedulerEvent): Unit = event match {
- case JobSubmitted(jobId, rdd, func, partitions, allowLocal, callSite, listener, properties) =>
- dagScheduler.handleJobSubmitted(jobId, rdd, func, partitions, allowLocal, callSite,
- listener, properties)
+ case JobSubmitted(jobId, rdd, func, partitions, callSite, listener, properties) =>
+ dagScheduler.handleJobSubmitted(jobId, rdd, func, partitions, callSite, listener, properties)
case StageCancelled(stageId) =>
dagScheduler.handleStageCancellation(stageId)
diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala
index a927eae2b04be..a213d419cf033 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala
@@ -40,7 +40,6 @@ private[scheduler] case class JobSubmitted(
finalRDD: RDD[_],
func: (TaskContext, Iterator[_]) => _,
partitions: Array[Int],
- allowLocal: Boolean,
callSite: CallSite,
listener: JobListener,
properties: Properties = null)
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 b86724de2cb73..40a333a3e06b2 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala
@@ -77,8 +77,11 @@ private[spark] abstract class Stage(
private var _latestInfo: StageInfo = StageInfo.fromStage(this, 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))
+ def makeNewStageAttempt(
+ numPartitionsToCompute: Int,
+ taskLocalityPreferences: Seq[Seq[TaskLocation]] = Seq.empty): Unit = {
+ _latestInfo = StageInfo.fromStage(
+ this, nextAttemptId, Some(numPartitionsToCompute), taskLocalityPreferences)
nextAttemptId += 1
}
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 5d2abbc67e9d9..24796c14300b1 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala
@@ -34,7 +34,8 @@ class StageInfo(
val numTasks: Int,
val rddInfos: Seq[RDDInfo],
val parentIds: Seq[Int],
- val details: String) {
+ val details: String,
+ private[spark] val taskLocalityPreferences: Seq[Seq[TaskLocation]] = Seq.empty) {
/** When this stage was submitted from the DAGScheduler to a TaskScheduler. */
var submissionTime: Option[Long] = None
/** Time when all tasks in the stage completed or when the stage was cancelled. */
@@ -70,7 +71,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, attemptId: Int, numTasks: Option[Int] = None): StageInfo = {
+ def fromStage(
+ stage: Stage,
+ attemptId: Int,
+ numTasks: Option[Int] = None,
+ taskLocalityPreferences: Seq[Seq[TaskLocation]] = Seq.empty
+ ): StageInfo = {
val ancestorRddInfos = stage.rdd.getNarrowAncestors.map(RDDInfo.fromRdd)
val rddInfos = Seq(RDDInfo.fromRdd(stage.rdd)) ++ ancestorRddInfos
new StageInfo(
@@ -80,6 +86,7 @@ private[spark] object StageInfo {
numTasks.getOrElse(stage.numTasks),
rddInfos,
stage.parents.map(_.id),
- stage.details)
+ stage.details,
+ taskLocalityPreferences)
}
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala
index d11a00956a9a9..1978305cfefbd 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala
@@ -23,7 +23,7 @@ import java.nio.ByteBuffer
import scala.collection.mutable.HashMap
import org.apache.spark.metrics.MetricsSystem
-import org.apache.spark.{TaskContextImpl, TaskContext}
+import org.apache.spark.{SparkEnv, TaskContextImpl, TaskContext}
import org.apache.spark.executor.TaskMetrics
import org.apache.spark.serializer.SerializerInstance
import org.apache.spark.unsafe.memory.TaskMemoryManager
@@ -86,7 +86,18 @@ private[spark] abstract class Task[T](
(runTask(context), context.collectAccumulators())
} finally {
context.markTaskCompleted()
- TaskContext.unset()
+ try {
+ Utils.tryLogNonFatalError {
+ // Release memory used by this thread for shuffles
+ SparkEnv.get.shuffleMemoryManager.releaseMemoryForThisTask()
+ }
+ Utils.tryLogNonFatalError {
+ // Release memory used by this thread for unrolling blocks
+ SparkEnv.get.blockManager.memoryStore.releaseUnrollMemoryForThisTask()
+ }
+ } finally {
+ TaskContext.unset()
+ }
}
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala
index 4be1eda2e9291..06f5438433b6e 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala
@@ -86,7 +86,11 @@ private[spark] object CoarseGrainedClusterMessages {
// Request executors by specifying the new total number of executors desired
// This includes executors already pending or running
- case class RequestExecutors(requestedTotal: Int) extends CoarseGrainedClusterMessage
+ case class RequestExecutors(
+ requestedTotal: Int,
+ localityAwareTasks: Int,
+ hostToLocalTaskCount: Map[String, Int])
+ extends CoarseGrainedClusterMessage
case class KillExecutors(executorIds: Seq[String]) extends CoarseGrainedClusterMessage
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
index c65b3e517773e..bd89160af4ffa 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
@@ -66,6 +66,12 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
// Executors we have requested the cluster manager to kill that have not died yet
private val executorsPendingToRemove = new HashSet[String]
+ // A map to store hostname with its possible task number running on it
+ protected var hostToLocalTaskCount: Map[String, Int] = Map.empty
+
+ // The number of pending tasks which is locality required
+ protected var localityAwareTasks = 0
+
class DriverEndpoint(override val rpcEnv: RpcEnv, sparkProperties: Seq[(String, String)])
extends ThreadSafeRpcEndpoint with Logging {
@@ -235,7 +241,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
scheduler.executorLost(executorId, SlaveLost(reason))
listenerBus.post(
SparkListenerExecutorRemoved(System.currentTimeMillis(), executorId, reason))
- case None => logError(s"Asked to remove non-existent executor $executorId")
+ case None => logInfo(s"Asked to remove non-existent executor $executorId")
}
}
@@ -339,6 +345,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
}
logInfo(s"Requesting $numAdditionalExecutors additional executor(s) from the cluster manager")
logDebug(s"Number of pending executors is now $numPendingExecutors")
+
numPendingExecutors += numAdditionalExecutors
// Account for executors pending to be added or removed
val newTotal = numExistingExecutors + numPendingExecutors - executorsPendingToRemove.size
@@ -346,16 +353,33 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
}
/**
- * Express a preference to the cluster manager for a given total number of executors. This can
- * result in canceling pending requests or filing additional requests.
- * @return whether the request is acknowledged.
+ * Update the cluster manager on our scheduling needs. Three bits of information are included
+ * to help it make decisions.
+ * @param numExecutors The total number of executors we'd like to have. The cluster manager
+ * shouldn't kill any running executor to reach this number, but,
+ * if all existing executors were to die, this is the number of executors
+ * we'd want to be allocated.
+ * @param localityAwareTasks The number of tasks in all active stages that have a locality
+ * preferences. This includes running, pending, and completed tasks.
+ * @param hostToLocalTaskCount A map of hosts to the number of tasks from all active stages
+ * that would like to like to run on that host.
+ * This includes running, pending, and completed tasks.
+ * @return whether the request is acknowledged by the cluster manager.
*/
- final override def requestTotalExecutors(numExecutors: Int): Boolean = synchronized {
+ final override def requestTotalExecutors(
+ numExecutors: Int,
+ localityAwareTasks: Int,
+ hostToLocalTaskCount: Map[String, Int]
+ ): Boolean = synchronized {
if (numExecutors < 0) {
throw new IllegalArgumentException(
"Attempted to request a negative number of executor(s) " +
s"$numExecutors from the cluster manager. Please specify a positive number!")
}
+
+ this.localityAwareTasks = localityAwareTasks
+ this.hostToLocalTaskCount = hostToLocalTaskCount
+
numPendingExecutors =
math.max(numExecutors - numExistingExecutors + executorsPendingToRemove.size, 0)
doRequestTotalExecutors(numExecutors)
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala
index bc67abb5df446..044f6288fabdd 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala
@@ -53,7 +53,8 @@ private[spark] abstract class YarnSchedulerBackend(
* This includes executors already pending or running.
*/
override def doRequestTotalExecutors(requestedTotal: Int): Boolean = {
- yarnSchedulerEndpoint.askWithRetry[Boolean](RequestExecutors(requestedTotal))
+ yarnSchedulerEndpoint.askWithRetry[Boolean](
+ RequestExecutors(requestedTotal, localityAwareTasks, hostToLocalTaskCount))
}
/**
@@ -108,6 +109,8 @@ private[spark] abstract class YarnSchedulerBackend(
case AddWebUIFilter(filterName, filterParams, proxyBase) =>
addWebUIFilter(filterName, filterParams, proxyBase)
+ case RemoveExecutor(executorId, reason) =>
+ removeExecutor(executorId, reason)
}
override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = {
diff --git a/core/src/main/scala/org/apache/spark/serializer/GenericAvroSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/GenericAvroSerializer.scala
new file mode 100644
index 0000000000000..62f8aae7f2126
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/serializer/GenericAvroSerializer.scala
@@ -0,0 +1,150 @@
+/*
+ * 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.serializer
+
+import java.io.{ByteArrayInputStream, ByteArrayOutputStream}
+import java.nio.ByteBuffer
+
+import scala.collection.mutable
+
+import com.esotericsoftware.kryo.{Kryo, Serializer => KSerializer}
+import com.esotericsoftware.kryo.io.{Input => KryoInput, Output => KryoOutput}
+import org.apache.avro.{Schema, SchemaNormalization}
+import org.apache.avro.generic.{GenericData, GenericRecord}
+import org.apache.avro.io._
+import org.apache.commons.io.IOUtils
+
+import org.apache.spark.{SparkException, SparkEnv}
+import org.apache.spark.io.CompressionCodec
+
+/**
+ * Custom serializer used for generic Avro records. If the user registers the schemas
+ * ahead of time, then the schema's fingerprint will be sent with each message instead of the actual
+ * schema, as to reduce network IO.
+ * Actions like parsing or compressing schemas are computationally expensive so the serializer
+ * caches all previously seen values as to reduce the amount of work needed to do.
+ * @param schemas a map where the keys are unique IDs for Avro schemas and the values are the
+ * string representation of the Avro schema, used to decrease the amount of data
+ * that needs to be serialized.
+ */
+private[serializer] class GenericAvroSerializer(schemas: Map[Long, String])
+ extends KSerializer[GenericRecord] {
+
+ /** Used to reduce the amount of effort to compress the schema */
+ private val compressCache = new mutable.HashMap[Schema, Array[Byte]]()
+ private val decompressCache = new mutable.HashMap[ByteBuffer, Schema]()
+
+ /** Reuses the same datum reader/writer since the same schema will be used many times */
+ private val writerCache = new mutable.HashMap[Schema, DatumWriter[_]]()
+ private val readerCache = new mutable.HashMap[Schema, DatumReader[_]]()
+
+ /** Fingerprinting is very expensive so this alleviates most of the work */
+ private val fingerprintCache = new mutable.HashMap[Schema, Long]()
+ private val schemaCache = new mutable.HashMap[Long, Schema]()
+
+ // GenericAvroSerializer can't take a SparkConf in the constructor b/c then it would become
+ // a member of KryoSerializer, which would make KryoSerializer not Serializable. We make
+ // the codec lazy here just b/c in some unit tests, we use a KryoSerializer w/out having
+ // the SparkEnv set (note those tests would fail if they tried to serialize avro data).
+ private lazy val codec = CompressionCodec.createCodec(SparkEnv.get.conf)
+
+ /**
+ * Used to compress Schemas when they are being sent over the wire.
+ * The compression results are memoized to reduce the compression time since the
+ * same schema is compressed many times over
+ */
+ def compress(schema: Schema): Array[Byte] = compressCache.getOrElseUpdate(schema, {
+ val bos = new ByteArrayOutputStream()
+ val out = codec.compressedOutputStream(bos)
+ out.write(schema.toString.getBytes("UTF-8"))
+ out.close()
+ bos.toByteArray
+ })
+
+ /**
+ * Decompresses the schema into the actual in-memory object. Keeps an internal cache of already
+ * seen values so to limit the number of times that decompression has to be done.
+ */
+ def decompress(schemaBytes: ByteBuffer): Schema = decompressCache.getOrElseUpdate(schemaBytes, {
+ val bis = new ByteArrayInputStream(schemaBytes.array())
+ val bytes = IOUtils.toByteArray(codec.compressedInputStream(bis))
+ new Schema.Parser().parse(new String(bytes, "UTF-8"))
+ })
+
+ /**
+ * Serializes a record to the given output stream. It caches a lot of the internal data as
+ * to not redo work
+ */
+ def serializeDatum[R <: GenericRecord](datum: R, output: KryoOutput): Unit = {
+ val encoder = EncoderFactory.get.binaryEncoder(output, null)
+ val schema = datum.getSchema
+ val fingerprint = fingerprintCache.getOrElseUpdate(schema, {
+ SchemaNormalization.parsingFingerprint64(schema)
+ })
+ schemas.get(fingerprint) match {
+ case Some(_) =>
+ output.writeBoolean(true)
+ output.writeLong(fingerprint)
+ case None =>
+ output.writeBoolean(false)
+ val compressedSchema = compress(schema)
+ output.writeInt(compressedSchema.length)
+ output.writeBytes(compressedSchema)
+ }
+
+ writerCache.getOrElseUpdate(schema, GenericData.get.createDatumWriter(schema))
+ .asInstanceOf[DatumWriter[R]]
+ .write(datum, encoder)
+ encoder.flush()
+ }
+
+ /**
+ * Deserializes generic records into their in-memory form. There is internal
+ * state to keep a cache of already seen schemas and datum readers.
+ */
+ def deserializeDatum(input: KryoInput): GenericRecord = {
+ val schema = {
+ if (input.readBoolean()) {
+ val fingerprint = input.readLong()
+ schemaCache.getOrElseUpdate(fingerprint, {
+ schemas.get(fingerprint) match {
+ case Some(s) => new Schema.Parser().parse(s)
+ case None =>
+ throw new SparkException(
+ "Error reading attempting to read avro data -- encountered an unknown " +
+ s"fingerprint: $fingerprint, not sure what schema to use. This could happen " +
+ "if you registered additional schemas after starting your spark context.")
+ }
+ })
+ } else {
+ val length = input.readInt()
+ decompress(ByteBuffer.wrap(input.readBytes(length)))
+ }
+ }
+ val decoder = DecoderFactory.get.directBinaryDecoder(input, null)
+ readerCache.getOrElseUpdate(schema, GenericData.get.createDatumReader(schema))
+ .asInstanceOf[DatumReader[GenericRecord]]
+ .read(null, decoder)
+ }
+
+ override def write(kryo: Kryo, output: KryoOutput, datum: GenericRecord): Unit =
+ serializeDatum(datum, output)
+
+ override def read(kryo: Kryo, input: KryoInput, datumClass: Class[GenericRecord]): GenericRecord =
+ deserializeDatum(input)
+}
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 7cb6e080533ad..0ff7562e912ca 100644
--- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
+++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala
@@ -27,6 +27,7 @@ import com.esotericsoftware.kryo.{Kryo, KryoException}
import com.esotericsoftware.kryo.io.{Input => KryoInput, Output => KryoOutput}
import com.esotericsoftware.kryo.serializers.{JavaSerializer => KryoJavaSerializer}
import com.twitter.chill.{AllScalaRegistrar, EmptyScalaKryoInstantiator}
+import org.apache.avro.generic.{GenericData, GenericRecord}
import org.roaringbitmap.{ArrayContainer, BitmapContainer, RoaringArray, RoaringBitmap}
import org.apache.spark._
@@ -73,6 +74,8 @@ class KryoSerializer(conf: SparkConf)
.split(',')
.filter(!_.isEmpty)
+ private val avroSchemas = conf.getAvroSchema
+
def newKryoOutput(): KryoOutput = new KryoOutput(bufferSize, math.max(bufferSize, maxBufferSize))
def newKryo(): Kryo = {
@@ -101,6 +104,9 @@ class KryoSerializer(conf: SparkConf)
kryo.register(classOf[HttpBroadcast[_]], new KryoJavaSerializer())
kryo.register(classOf[PythonBroadcast], new KryoJavaSerializer())
+ kryo.register(classOf[GenericRecord], new GenericAvroSerializer(avroSchemas))
+ kryo.register(classOf[GenericData.Record], new GenericAvroSerializer(avroSchemas))
+
try {
// scalastyle:off classforname
// Use the default classloader when calling the user registrator.
diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleMemoryManager.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleMemoryManager.scala
index 3bcc7178a3d8b..f038b722957b8 100644
--- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleMemoryManager.scala
+++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleMemoryManager.scala
@@ -19,95 +19,101 @@ package org.apache.spark.shuffle
import scala.collection.mutable
-import org.apache.spark.{Logging, SparkException, SparkConf}
+import org.apache.spark.{Logging, SparkException, SparkConf, TaskContext}
/**
- * Allocates a pool of memory to task threads for use in shuffle operations. Each disk-spilling
+ * Allocates a pool of memory to tasks for use in shuffle operations. Each disk-spilling
* collection (ExternalAppendOnlyMap or ExternalSorter) used by these tasks can acquire memory
* from this pool and release it as it spills data out. When a task ends, all its memory will be
* released by the Executor.
*
- * This class tries to ensure that each thread gets a reasonable share of memory, instead of some
- * thread ramping up to a large amount first and then causing others to spill to disk repeatedly.
- * If there are N threads, it ensures that each thread can acquire at least 1 / 2N of the memory
+ * This class tries to ensure that each task gets a reasonable share of memory, instead of some
+ * task ramping up to a large amount first and then causing others to spill to disk repeatedly.
+ * If there are N tasks, it ensures that each tasks can acquire at least 1 / 2N of the memory
* before it has to spill, and at most 1 / N. Because N varies dynamically, we keep track of the
- * set of active threads and redo the calculations of 1 / 2N and 1 / N in waiting threads whenever
+ * set of active tasks and redo the calculations of 1 / 2N and 1 / N in waiting tasks whenever
* this set changes. This is all done by synchronizing access on "this" to mutate state and using
* wait() and notifyAll() to signal changes.
*/
private[spark] class ShuffleMemoryManager(maxMemory: Long) extends Logging {
- private val threadMemory = new mutable.HashMap[Long, Long]() // threadId -> memory bytes
+ private val taskMemory = new mutable.HashMap[Long, Long]() // taskAttemptId -> memory bytes
def this(conf: SparkConf) = this(ShuffleMemoryManager.getMaxMemory(conf))
+ private def currentTaskAttemptId(): Long = {
+ // In case this is called on the driver, return an invalid task attempt id.
+ Option(TaskContext.get()).map(_.taskAttemptId()).getOrElse(-1L)
+ }
+
/**
- * Try to acquire up to numBytes memory for the current thread, and return the number of bytes
+ * Try to acquire up to numBytes memory for the current task, and return the number of bytes
* obtained, or 0 if none can be allocated. This call may block until there is enough free memory
- * in some situations, to make sure each thread has a chance to ramp up to at least 1 / 2N of the
- * total memory pool (where N is the # of active threads) before it is forced to spill. This can
- * happen if the number of threads increases but an older thread had a lot of memory already.
+ * in some situations, to make sure each task has a chance to ramp up to at least 1 / 2N of the
+ * total memory pool (where N is the # of active tasks) before it is forced to spill. This can
+ * happen if the number of tasks increases but an older task had a lot of memory already.
*/
def tryToAcquire(numBytes: Long): Long = synchronized {
- val threadId = Thread.currentThread().getId
+ val taskAttemptId = currentTaskAttemptId()
assert(numBytes > 0, "invalid number of bytes requested: " + numBytes)
- // Add this thread to the threadMemory map just so we can keep an accurate count of the number
- // of active threads, to let other threads ramp down their memory in calls to tryToAcquire
- if (!threadMemory.contains(threadId)) {
- threadMemory(threadId) = 0L
- notifyAll() // Will later cause waiting threads to wake up and check numThreads again
+ // Add this task to the taskMemory map just so we can keep an accurate count of the number
+ // of active tasks, to let other tasks ramp down their memory in calls to tryToAcquire
+ if (!taskMemory.contains(taskAttemptId)) {
+ taskMemory(taskAttemptId) = 0L
+ notifyAll() // Will later cause waiting tasks to wake up and check numThreads again
}
// Keep looping until we're either sure that we don't want to grant this request (because this
- // thread would have more than 1 / numActiveThreads of the memory) or we have enough free
- // memory to give it (we always let each thread get at least 1 / (2 * numActiveThreads)).
+ // task would have more than 1 / numActiveTasks of the memory) or we have enough free
+ // memory to give it (we always let each task get at least 1 / (2 * numActiveTasks)).
while (true) {
- val numActiveThreads = threadMemory.keys.size
- val curMem = threadMemory(threadId)
- val freeMemory = maxMemory - threadMemory.values.sum
+ val numActiveTasks = taskMemory.keys.size
+ val curMem = taskMemory(taskAttemptId)
+ val freeMemory = maxMemory - taskMemory.values.sum
- // How much we can grant this thread; don't let it grow to more than 1 / numActiveThreads;
+ // How much we can grant this task; don't let it grow to more than 1 / numActiveTasks;
// don't let it be negative
- val maxToGrant = math.min(numBytes, math.max(0, (maxMemory / numActiveThreads) - curMem))
+ val maxToGrant = math.min(numBytes, math.max(0, (maxMemory / numActiveTasks) - curMem))
- if (curMem < maxMemory / (2 * numActiveThreads)) {
- // We want to let each thread get at least 1 / (2 * numActiveThreads) before blocking;
- // if we can't give it this much now, wait for other threads to free up memory
- // (this happens if older threads allocated lots of memory before N grew)
- if (freeMemory >= math.min(maxToGrant, maxMemory / (2 * numActiveThreads) - curMem)) {
+ if (curMem < maxMemory / (2 * numActiveTasks)) {
+ // We want to let each task get at least 1 / (2 * numActiveTasks) before blocking;
+ // if we can't give it this much now, wait for other tasks to free up memory
+ // (this happens if older tasks allocated lots of memory before N grew)
+ if (freeMemory >= math.min(maxToGrant, maxMemory / (2 * numActiveTasks) - curMem)) {
val toGrant = math.min(maxToGrant, freeMemory)
- threadMemory(threadId) += toGrant
+ taskMemory(taskAttemptId) += toGrant
return toGrant
} else {
- logInfo(s"Thread $threadId waiting for at least 1/2N of shuffle memory pool to be free")
+ logInfo(
+ s"Thread $taskAttemptId waiting for at least 1/2N of shuffle memory pool to be free")
wait()
}
} else {
// Only give it as much memory as is free, which might be none if it reached 1 / numThreads
val toGrant = math.min(maxToGrant, freeMemory)
- threadMemory(threadId) += toGrant
+ taskMemory(taskAttemptId) += toGrant
return toGrant
}
}
0L // Never reached
}
- /** Release numBytes bytes for the current thread. */
+ /** Release numBytes bytes for the current task. */
def release(numBytes: Long): Unit = synchronized {
- val threadId = Thread.currentThread().getId
- val curMem = threadMemory.getOrElse(threadId, 0L)
+ val taskAttemptId = currentTaskAttemptId()
+ val curMem = taskMemory.getOrElse(taskAttemptId, 0L)
if (curMem < numBytes) {
throw new SparkException(
- s"Internal error: release called on ${numBytes} bytes but thread only has ${curMem}")
+ s"Internal error: release called on ${numBytes} bytes but task only has ${curMem}")
}
- threadMemory(threadId) -= numBytes
+ taskMemory(taskAttemptId) -= numBytes
notifyAll() // Notify waiters who locked "this" in tryToAcquire that memory has been freed
}
- /** Release all memory for the current thread and mark it as inactive (e.g. when a task ends). */
- def releaseMemoryForThisThread(): Unit = synchronized {
- val threadId = Thread.currentThread().getId
- threadMemory.remove(threadId)
+ /** Release all memory for the current task and mark it as inactive (e.g. when a task ends). */
+ def releaseMemoryForThisTask(): Unit = synchronized {
+ val taskAttemptId = currentTaskAttemptId()
+ taskMemory.remove(taskAttemptId)
notifyAll() // Notify waiters who locked "this" in tryToAcquire that memory has been freed
}
}
diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala
index ed609772e6979..6f27f00307f8c 100644
--- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala
+++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala
@@ -23,6 +23,7 @@ import java.util.LinkedHashMap
import scala.collection.mutable
import scala.collection.mutable.ArrayBuffer
+import org.apache.spark.TaskContext
import org.apache.spark.util.{SizeEstimator, Utils}
import org.apache.spark.util.collection.SizeTrackingVector
@@ -43,11 +44,11 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long)
// Ensure only one thread is putting, and if necessary, dropping blocks at any given time
private val accountingLock = new Object
- // A mapping from thread ID to amount of memory used for unrolling a block (in bytes)
+ // A mapping from taskAttemptId to amount of memory used for unrolling a block (in bytes)
// All accesses of this map are assumed to have manually synchronized on `accountingLock`
private val unrollMemoryMap = mutable.HashMap[Long, Long]()
// Same as `unrollMemoryMap`, but for pending unroll memory as defined below.
- // Pending unroll memory refers to the intermediate memory occupied by a thread
+ // Pending unroll memory refers to the intermediate memory occupied by a task
// after the unroll but before the actual putting of the block in the cache.
// This chunk of memory is expected to be released *as soon as* we finish
// caching the corresponding block as opposed to until after the task finishes.
@@ -250,21 +251,21 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long)
var elementsUnrolled = 0
// Whether there is still enough memory for us to continue unrolling this block
var keepUnrolling = true
- // Initial per-thread memory to request for unrolling blocks (bytes). Exposed for testing.
+ // Initial per-task memory to request for unrolling blocks (bytes). Exposed for testing.
val initialMemoryThreshold = unrollMemoryThreshold
// How often to check whether we need to request more memory
val memoryCheckPeriod = 16
- // Memory currently reserved by this thread for this particular unrolling operation
+ // Memory currently reserved by this task for this particular unrolling operation
var memoryThreshold = initialMemoryThreshold
// Memory to request as a multiple of current vector size
val memoryGrowthFactor = 1.5
- // Previous unroll memory held by this thread, for releasing later (only at the very end)
- val previousMemoryReserved = currentUnrollMemoryForThisThread
+ // Previous unroll memory held by this task, for releasing later (only at the very end)
+ val previousMemoryReserved = currentUnrollMemoryForThisTask
// Underlying vector for unrolling the block
var vector = new SizeTrackingVector[Any]
// Request enough memory to begin unrolling
- keepUnrolling = reserveUnrollMemoryForThisThread(initialMemoryThreshold)
+ keepUnrolling = reserveUnrollMemoryForThisTask(initialMemoryThreshold)
if (!keepUnrolling) {
logWarning(s"Failed to reserve initial memory threshold of " +
@@ -283,7 +284,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long)
// Hold the accounting lock, in case another thread concurrently puts a block that
// takes up the unrolling space we just ensured here
accountingLock.synchronized {
- if (!reserveUnrollMemoryForThisThread(amountToRequest)) {
+ if (!reserveUnrollMemoryForThisTask(amountToRequest)) {
// If the first request is not granted, try again after ensuring free space
// If there is still not enough space, give up and drop the partition
val spaceToEnsure = maxUnrollMemory - currentUnrollMemory
@@ -291,7 +292,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long)
val result = ensureFreeSpace(blockId, spaceToEnsure)
droppedBlocks ++= result.droppedBlocks
}
- keepUnrolling = reserveUnrollMemoryForThisThread(amountToRequest)
+ keepUnrolling = reserveUnrollMemoryForThisTask(amountToRequest)
}
}
// New threshold is currentSize * memoryGrowthFactor
@@ -317,9 +318,9 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long)
// later when the task finishes.
if (keepUnrolling) {
accountingLock.synchronized {
- val amountToRelease = currentUnrollMemoryForThisThread - previousMemoryReserved
- releaseUnrollMemoryForThisThread(amountToRelease)
- reservePendingUnrollMemoryForThisThread(amountToRelease)
+ val amountToRelease = currentUnrollMemoryForThisTask - previousMemoryReserved
+ releaseUnrollMemoryForThisTask(amountToRelease)
+ reservePendingUnrollMemoryForThisTask(amountToRelease)
}
}
}
@@ -397,7 +398,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long)
droppedBlockStatus.foreach { status => droppedBlocks += ((blockId, status)) }
}
// Release the unroll memory used because we no longer need the underlying Array
- releasePendingUnrollMemoryForThisThread()
+ releasePendingUnrollMemoryForThisTask()
}
ResultWithDroppedBlocks(putSuccess, droppedBlocks)
}
@@ -427,9 +428,9 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long)
// Take into account the amount of memory currently occupied by unrolling blocks
// and minus the pending unroll memory for that block on current thread.
- val threadId = Thread.currentThread().getId
+ val taskAttemptId = currentTaskAttemptId()
val actualFreeMemory = freeMemory - currentUnrollMemory +
- pendingUnrollMemoryMap.getOrElse(threadId, 0L)
+ pendingUnrollMemoryMap.getOrElse(taskAttemptId, 0L)
if (actualFreeMemory < space) {
val rddToAdd = getRddId(blockIdToAdd)
@@ -455,7 +456,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long)
logInfo(s"${selectedBlocks.size} blocks selected for dropping")
for (blockId <- selectedBlocks) {
val entry = entries.synchronized { entries.get(blockId) }
- // This should never be null as only one thread should be dropping
+ // This should never be null as only one task should be dropping
// blocks and removing entries. However the check is still here for
// future safety.
if (entry != null) {
@@ -482,79 +483,85 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long)
entries.synchronized { entries.containsKey(blockId) }
}
+ private def currentTaskAttemptId(): Long = {
+ // In case this is called on the driver, return an invalid task attempt id.
+ Option(TaskContext.get()).map(_.taskAttemptId()).getOrElse(-1L)
+ }
+
/**
- * Reserve additional memory for unrolling blocks used by this thread.
+ * Reserve additional memory for unrolling blocks used by this task.
* Return whether the request is granted.
*/
- def reserveUnrollMemoryForThisThread(memory: Long): Boolean = {
+ def reserveUnrollMemoryForThisTask(memory: Long): Boolean = {
accountingLock.synchronized {
val granted = freeMemory > currentUnrollMemory + memory
if (granted) {
- val threadId = Thread.currentThread().getId
- unrollMemoryMap(threadId) = unrollMemoryMap.getOrElse(threadId, 0L) + memory
+ val taskAttemptId = currentTaskAttemptId()
+ unrollMemoryMap(taskAttemptId) = unrollMemoryMap.getOrElse(taskAttemptId, 0L) + memory
}
granted
}
}
/**
- * Release memory used by this thread for unrolling blocks.
- * If the amount is not specified, remove the current thread's allocation altogether.
+ * Release memory used by this task for unrolling blocks.
+ * If the amount is not specified, remove the current task's allocation altogether.
*/
- def releaseUnrollMemoryForThisThread(memory: Long = -1L): Unit = {
- val threadId = Thread.currentThread().getId
+ def releaseUnrollMemoryForThisTask(memory: Long = -1L): Unit = {
+ val taskAttemptId = currentTaskAttemptId()
accountingLock.synchronized {
if (memory < 0) {
- unrollMemoryMap.remove(threadId)
+ unrollMemoryMap.remove(taskAttemptId)
} else {
- unrollMemoryMap(threadId) = unrollMemoryMap.getOrElse(threadId, memory) - memory
- // If this thread claims no more unroll memory, release it completely
- if (unrollMemoryMap(threadId) <= 0) {
- unrollMemoryMap.remove(threadId)
+ unrollMemoryMap(taskAttemptId) = unrollMemoryMap.getOrElse(taskAttemptId, memory) - memory
+ // If this task claims no more unroll memory, release it completely
+ if (unrollMemoryMap(taskAttemptId) <= 0) {
+ unrollMemoryMap.remove(taskAttemptId)
}
}
}
}
/**
- * Reserve the unroll memory of current unroll successful block used by this thread
+ * Reserve the unroll memory of current unroll successful block used by this task
* until actually put the block into memory entry.
*/
- def reservePendingUnrollMemoryForThisThread(memory: Long): Unit = {
- val threadId = Thread.currentThread().getId
+ def reservePendingUnrollMemoryForThisTask(memory: Long): Unit = {
+ val taskAttemptId = currentTaskAttemptId()
accountingLock.synchronized {
- pendingUnrollMemoryMap(threadId) = pendingUnrollMemoryMap.getOrElse(threadId, 0L) + memory
+ pendingUnrollMemoryMap(taskAttemptId) =
+ pendingUnrollMemoryMap.getOrElse(taskAttemptId, 0L) + memory
}
}
/**
- * Release pending unroll memory of current unroll successful block used by this thread
+ * Release pending unroll memory of current unroll successful block used by this task
*/
- def releasePendingUnrollMemoryForThisThread(): Unit = {
- val threadId = Thread.currentThread().getId
+ def releasePendingUnrollMemoryForThisTask(): Unit = {
+ val taskAttemptId = currentTaskAttemptId()
accountingLock.synchronized {
- pendingUnrollMemoryMap.remove(threadId)
+ pendingUnrollMemoryMap.remove(taskAttemptId)
}
}
/**
- * Return the amount of memory currently occupied for unrolling blocks across all threads.
+ * Return the amount of memory currently occupied for unrolling blocks across all tasks.
*/
def currentUnrollMemory: Long = accountingLock.synchronized {
unrollMemoryMap.values.sum + pendingUnrollMemoryMap.values.sum
}
/**
- * Return the amount of memory currently occupied for unrolling blocks by this thread.
+ * Return the amount of memory currently occupied for unrolling blocks by this task.
*/
- def currentUnrollMemoryForThisThread: Long = accountingLock.synchronized {
- unrollMemoryMap.getOrElse(Thread.currentThread().getId, 0L)
+ def currentUnrollMemoryForThisTask: Long = accountingLock.synchronized {
+ unrollMemoryMap.getOrElse(currentTaskAttemptId(), 0L)
}
/**
- * Return the number of threads currently unrolling blocks.
+ * Return the number of tasks currently unrolling blocks.
*/
- def numThreadsUnrolling: Int = accountingLock.synchronized { unrollMemoryMap.keys.size }
+ def numTasksUnrolling: Int = accountingLock.synchronized { unrollMemoryMap.keys.size }
/**
* Log information about current memory usage.
@@ -566,7 +573,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long)
logInfo(
s"Memory use = ${Utils.bytesToString(blocksMemory)} (blocks) + " +
s"${Utils.bytesToString(unrollMemory)} (scratch space shared across " +
- s"$numThreadsUnrolling thread(s)) = ${Utils.bytesToString(totalMemory)}. " +
+ s"$numTasksUnrolling tasks(s)) = ${Utils.bytesToString(totalMemory)}. " +
s"Storage limit = ${Utils.bytesToString(maxMemory)}."
)
}
diff --git a/core/src/main/scala/org/apache/spark/ui/WebUI.scala b/core/src/main/scala/org/apache/spark/ui/WebUI.scala
index 2c84e4485996e..61449847add3d 100644
--- a/core/src/main/scala/org/apache/spark/ui/WebUI.scala
+++ b/core/src/main/scala/org/apache/spark/ui/WebUI.scala
@@ -107,6 +107,25 @@ private[spark] abstract class WebUI(
}
}
+ /**
+ * Add a handler for static content.
+ *
+ * @param resourceBase Root of where to find resources to serve.
+ * @param path Path in UI where to mount the resources.
+ */
+ def addStaticHandler(resourceBase: String, path: String): Unit = {
+ attachHandler(JettyUtils.createStaticHandler(resourceBase, path))
+ }
+
+ /**
+ * Remove a static content handler.
+ *
+ * @param path Path in UI to unmount.
+ */
+ def removeStaticHandler(path: String): Unit = {
+ handlers.find(_.getContextPath() == path).foreach(detachHandler)
+ }
+
/** Initialize all components of the server. */
def initialize()
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala
index 2ce670ad02e97..e72547df7254b 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala
@@ -79,6 +79,7 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") {
case JobExecutionStatus.SUCCEEDED => "succeeded"
case JobExecutionStatus.FAILED => "failed"
case JobExecutionStatus.RUNNING => "running"
+ case JobExecutionStatus.UNKNOWN => "unknown"
}
// The timeline library treats contents as HTML, so we have to escape them; for the
diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala
index c179833e5b06a..78e7ddc27d1c7 100644
--- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala
+++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala
@@ -128,7 +128,7 @@ private[spark] object AkkaUtils extends Logging {
/** Returns the configured max frame size for Akka messages in bytes. */
def maxFrameSizeBytes(conf: SparkConf): Int = {
- val frameSizeInMB = conf.getInt("spark.akka.frameSize", 10)
+ val frameSizeInMB = conf.getInt("spark.akka.frameSize", 128)
if (frameSizeInMB > AKKA_MAX_FRAME_SIZE_IN_MB) {
throw new IllegalArgumentException(
s"spark.akka.frameSize should not be greater than $AKKA_MAX_FRAME_SIZE_IN_MB MB")
diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala
index a078f14af52a1..c600319d9ddb4 100644
--- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala
+++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala
@@ -94,6 +94,8 @@ private[spark] object JsonProtocol {
logStartToJson(logStart)
case metricsUpdate: SparkListenerExecutorMetricsUpdate =>
executorMetricsUpdateToJson(metricsUpdate)
+ case blockUpdated: SparkListenerBlockUpdated =>
+ throw new MatchError(blockUpdated) // TODO(ekl) implement this
}
}
diff --git a/core/src/main/scala/org/apache/spark/util/SerializableConfiguration.scala b/core/src/main/scala/org/apache/spark/util/SerializableConfiguration.scala
index 30bcf1d2f24d5..3354a923273ff 100644
--- a/core/src/main/scala/org/apache/spark/util/SerializableConfiguration.scala
+++ b/core/src/main/scala/org/apache/spark/util/SerializableConfiguration.scala
@@ -20,8 +20,6 @@ import java.io.{ObjectInputStream, ObjectOutputStream}
import org.apache.hadoop.conf.Configuration
-import org.apache.spark.util.Utils
-
private[spark]
class SerializableConfiguration(@transient var value: Configuration) extends Serializable {
private def writeObject(out: ObjectOutputStream): Unit = Utils.tryOrIOException {
diff --git a/core/src/main/scala/org/apache/spark/util/SerializableJobConf.scala b/core/src/main/scala/org/apache/spark/util/SerializableJobConf.scala
index afbcc6efc850c..cadae472b3f85 100644
--- a/core/src/main/scala/org/apache/spark/util/SerializableJobConf.scala
+++ b/core/src/main/scala/org/apache/spark/util/SerializableJobConf.scala
@@ -21,8 +21,6 @@ import java.io.{ObjectInputStream, ObjectOutputStream}
import org.apache.hadoop.mapred.JobConf
-import org.apache.spark.util.Utils
-
private[spark]
class SerializableJobConf(@transient var value: JobConf) extends Serializable {
private def writeObject(out: ObjectOutputStream): Unit = Utils.tryOrIOException {
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 7d84468f62ab1..14b1f2a17e707 100644
--- a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala
+++ b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala
@@ -217,10 +217,10 @@ object SizeEstimator extends Logging {
var arrSize: Long = alignSize(objectSize + INT_SIZE)
if (elementClass.isPrimitive) {
- arrSize += alignSize(length * primitiveSize(elementClass))
+ arrSize += alignSize(length.toLong * primitiveSize(elementClass))
state.size += arrSize
} else {
- arrSize += alignSize(length * pointerSize)
+ arrSize += alignSize(length.toLong * pointerSize)
state.size += arrSize
if (length <= ARRAY_SIZE_FOR_SAMPLING) {
@@ -336,7 +336,7 @@ object SizeEstimator extends Logging {
// hg.openjdk.java.net/jdk8/jdk8/hotspot/file/tip/src/share/vm/classfile/classFileParser.cpp
var alignedSize = shellSize
for (size <- fieldSizes if sizeCount(size) > 0) {
- val count = sizeCount(size)
+ val count = sizeCount(size).toLong
// If there are internal gaps, smaller field can fit in.
alignedSize = math.max(alignedSize, alignSizeUp(shellSize, size) + size * count)
shellSize += size * count
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 c5816949cd360..c4012d0e83f7d 100644
--- a/core/src/main/scala/org/apache/spark/util/Utils.scala
+++ b/core/src/main/scala/org/apache/spark/util/Utils.scala
@@ -443,11 +443,11 @@ private[spark] object Utils extends Logging {
val lockFileName = s"${url.hashCode}${timestamp}_lock"
val localDir = new File(getLocalDir(conf))
val lockFile = new File(localDir, lockFileName)
- val raf = new RandomAccessFile(lockFile, "rw")
+ val lockFileChannel = new RandomAccessFile(lockFile, "rw").getChannel()
// Only one executor entry.
// The FileLock is only used to control synchronization for executors download file,
// it's always safe regardless of lock type (mandatory or advisory).
- val lock = raf.getChannel().lock()
+ val lock = lockFileChannel.lock()
val cachedFile = new File(localDir, cachedFileName)
try {
if (!cachedFile.exists()) {
@@ -455,6 +455,7 @@ private[spark] object Utils extends Logging {
}
} finally {
lock.release()
+ lockFileChannel.close()
}
copyFile(
url,
diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java
index 1b04a3b1cff0e..e948ca33471a4 100644
--- a/core/src/test/java/org/apache/spark/JavaAPISuite.java
+++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java
@@ -1783,7 +1783,7 @@ public void testGuavaOptional() {
// Stop the context created in setUp() and start a local-cluster one, to force usage of the
// assembly.
sc.stop();
- JavaSparkContext localCluster = new JavaSparkContext("local-cluster[1,1,512]", "JavaAPISuite");
+ JavaSparkContext localCluster = new JavaSparkContext("local-cluster[1,1,1024]", "JavaAPISuite");
try {
JavaRDD rdd1 = localCluster.parallelize(Arrays.asList(1, 2, null), 3);
JavaRDD> rdd2 = rdd1.map(
diff --git a/core/src/test/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriterSuite.java b/core/src/test/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriterSuite.java
index 10c3eedbf4b46..04fc09b323dbb 100644
--- a/core/src/test/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriterSuite.java
+++ b/core/src/test/java/org/apache/spark/shuffle/unsafe/UnsafeShuffleWriterSuite.java
@@ -111,7 +111,7 @@ public void setUp() throws IOException {
mergedOutputFile = File.createTempFile("mergedoutput", "", tempDir);
partitionSizesInMergedFile = null;
spillFilesCreated.clear();
- conf = new SparkConf();
+ conf = new SparkConf().set("spark.buffer.pageSize", "128m");
taskMetrics = new TaskMetrics();
when(shuffleMemoryManager.tryToAcquire(anyLong())).then(returnsFirstArg());
@@ -512,12 +512,12 @@ public void close() { }
writer.insertRecordIntoSorter(new Tuple2